From 465d46bf17f26356b7fb3e14a3a01bd6e9eb71ce Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Tue, 2 Feb 2021 21:27:10 -0700 Subject: [PATCH 01/19] Basic implementation for loading snapshot --- checkstyle/import-control.xml | 1 + .../apache/kafka/raft/KafkaRaftClient.java | 41 ++++++- .../org/apache/kafka/raft/RaftClient.java | 6 + .../apache/kafka/raft/ReplicatedCounter.java | 35 +++++- .../apache/kafka/snapshot/SnapshotReader.java | 112 ++++++++++++++++++ .../raft/KafkaRaftClientSnapshotTest.java | 46 ++++++- .../java/org/apache/kafka/raft/MockLog.java | 58 ++++----- .../kafka/raft/RaftClientTestContext.java | 30 ++++- 8 files changed, 284 insertions(+), 45 deletions(-) create mode 100644 raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 9cc432efd1892..0df45e7399435 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -352,6 +352,7 @@ + diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 09f8672d287cd..9bc8a66a03096 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -71,6 +71,7 @@ import org.apache.kafka.raft.internals.ThresholdPurgatory; import org.apache.kafka.snapshot.RawSnapshotReader; import org.apache.kafka.snapshot.RawSnapshotWriter; +import org.apache.kafka.snapshot.SnapshotReader; import org.apache.kafka.snapshot.SnapshotWriter; import org.slf4j.Logger; @@ -130,6 +131,8 @@ * some additional metadata on responses (i.e. current leader and epoch). Unlike partition replication, * we also piggyback truncation detection on this API rather than through a separate truncation state. * + * 5) TODO: Talk about FetchSnapshotRequestData + * */ public class KafkaRaftClient implements RaftClient { private static final int RETRY_BACKOFF_BASE_MS = 100; @@ -300,7 +303,16 @@ private void updateListenersProgress(List listenerContexts, lon for (ListenerContext listenerContext : listenerContexts) { listenerContext.nextExpectedOffset().ifPresent(nextExpectedOffset -> { if (nextExpectedOffset < log.startOffset()) { - listenerContext.fireHandleSnapshot(log.startOffset()); + SnapshotReader snapshot = oldestSnapshot().orElseThrow(() -> { + return new IllegalStateException( + String.format( + "Snapshot expected when next offset is %s and log start offset is %s", + nextExpectedOffset, + log.startOffset() + ) + ); + }); + listenerContext.fireHandleSnapshot(snapshot); } }); @@ -314,6 +326,23 @@ private void updateListenersProgress(List listenerContexts, lon } } + private Optional> oldestSnapshot() { + if (log.oldestSnapshotId().isPresent()) { + try { + return log + .readSnapshot(log.oldestSnapshotId().get()) + .map(reader -> new SnapshotReader<>(reader, serde)); + } catch (IOException e) { + logger.error( + String.format("Unable to read snapshot: %s", log.oldestSnapshotId().get()), + e + ); + } + } + + return Optional.empty(); + } + private void maybeFireHandleCommit(long baseOffset, int epoch, List records) { for (ListenerContext listenerContext : listenerContexts) { OptionalLong nextExpectedOffsetOpt = listenerContext.nextExpectedOffset(); @@ -2304,14 +2333,16 @@ public synchronized OptionalLong nextExpectedOffset() { } /** - * This API is used when the Listener needs to be notified of a new Snapshot. This happens - * when the context last acked end offset is less that then log start offset. + * This API is used when the Listener needs to be notified of a new snapshot. This happens + * when the context's next offset is less that then log start offset. */ - public void fireHandleSnapshot(long logStartOffset) { + public void fireHandleSnapshot(SnapshotReader reader) { synchronized (this) { - nextOffset = logStartOffset; + nextOffset = reader.snapshotId().offset; lastSent = null; } + + listener.handleSnapshot(reader); } /** diff --git a/raft/src/main/java/org/apache/kafka/raft/RaftClient.java b/raft/src/main/java/org/apache/kafka/raft/RaftClient.java index 554ce6173df98..15187df175b12 100644 --- a/raft/src/main/java/org/apache/kafka/raft/RaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/RaftClient.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.raft; +import org.apache.kafka.snapshot.SnapshotReader; import org.apache.kafka.snapshot.SnapshotWriter; import java.io.Closeable; @@ -42,6 +43,11 @@ interface Listener { */ void handleCommit(BatchReader reader); + /** + * TOOD: Write documentation + */ + void handleSnapshot(SnapshotReader reader); + /** * Invoked after this node has become a leader. This is only called after * all commits up to the start of the leader's epoch have been sent to diff --git a/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java b/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java index 47dae5d8e1372..96763f16ad391 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java +++ b/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java @@ -18,8 +18,12 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.snapshot.SnapshotReader; import org.slf4j.Logger; +import java.io.IOException; +import java.util.Iterator; +import java.util.List; import java.util.Optional; import static java.util.Collections.singletonList; @@ -75,8 +79,14 @@ public synchronized void handleCommit(BatchReader reader) { batch.records(), batch.baseOffset()); for (Integer value : batch.records()) { if (value != this.committed + 1) { - throw new AssertionError("Expected next committed value to be " + - (this.committed + 1) + ", but instead found " + value + " on node " + nodeId); + throw new AssertionError( + String.format( + "Expected next committed value to be %s, but instead found %s on node %s", + this.committed + 1, + value, + nodeId + ) + ); } this.committed = value; } @@ -87,6 +97,26 @@ public synchronized void handleCommit(BatchReader reader) { } } + @Override + public synchronized void handleSnapshot(SnapshotReader reader) { + try { + try (SnapshotReader snapshot = reader) { + log.debug("Loading snapshot {}", snapshot.snapshotId()); + Iterator> batches = snapshot.iterator(); + while (batches.hasNext()) { + for (Integer value : batches.next()) { + log.debug("Setting value: {}", value); + this.committed = value; + this.uncommitted = value; + } + } + log.debug("Finished loading snapshot. Set value: {}", this.committed); + } + } catch (IOException e) { + log.error(String.format("Unable to read snapshot %s", reader.snapshotId()), e); + } + } + @Override public synchronized void handleClaim(int epoch) { log.debug("Counter uncommitted value initialized to {} after claiming leadership in epoch {}", @@ -101,5 +131,4 @@ public synchronized void handleResign() { this.uncommitted = -1; this.claimedEpoch = Optional.empty(); } - } diff --git a/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java new file mode 100644 index 0000000000000..03b40050c25bc --- /dev/null +++ b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.snapshot; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import java.util.ArrayList; +import org.apache.kafka.common.protocol.ByteBufferAccessor; +import org.apache.kafka.common.record.BufferSupplier.GrowableBufferSupplier; +import org.apache.kafka.common.record.Record; +import org.apache.kafka.common.record.RecordBatch; +import org.apache.kafka.raft.OffsetAndEpoch; +import org.apache.kafka.raft.RecordSerde; + +final public class SnapshotReader implements Closeable, Iterable> { + final private RawSnapshotReader snapshot; + final private RecordSerde serde; + + /** + * TODO: write documentation + */ + public SnapshotReader( + RawSnapshotReader snapshot, + RecordSerde serde + ) { + this.snapshot = snapshot; + this.serde = serde; + } + + /** + * Returns the end offset and epoch for the snapshot. + */ + public OffsetAndEpoch snapshotId() { + return snapshot.snapshotId(); + } + + @Override + public Iterator> iterator() { + return new SnapshotReaderIterator<>(snapshot.iterator(), serde); + } + + /** + * Closes the snapshot reader. + * + * @throws IOException for any IO error during close + */ + public void close() throws IOException { + snapshot.close(); + } + + + final static class SnapshotReaderIterator implements Iterator> { + final Iterator iterator; + final RecordSerde serde; + + SnapshotReaderIterator(Iterator iterator, RecordSerde serde) { + this.iterator = iterator; + this.serde = serde; + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public List next() { + RecordBatch batch = iterator.next(); + + if (batch.countOrNull() == null) { + throw new IllegalStateException( + String.format( + "Expected a record count the the batch (%s)", + batch + ) + ); + } + List data = new ArrayList<>(batch.countOrNull()); + + // TODO: make BufferSupplier configurable + Iterator records = batch.streamingIterator(new GrowableBufferSupplier()); + while (records.hasNext()) { + Record record = records.next(); + + // TODO: ignore control records + // TODO: verify hasValue is true + data.add( + serde.read(new ByteBufferAccessor(record.value()), record.value().remaining()) + ); + } + + return data; + } + } +} diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java index 614ff327cffed..985e5cc1c5f4c 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java @@ -47,6 +47,50 @@ import static org.junit.jupiter.api.Assertions.assertTrue; final public class KafkaRaftClientSnapshotTest { + @Test + public void testLeaderListernerNotified() throws Exception { + int localId = 0; + int otherNodeId = localId + 1; + Set voters = Utils.mkSet(localId, otherNodeId); + OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 1); + + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .appendToLog(oldestSnapshotId.offset, oldestSnapshotId.epoch, Arrays.asList("a", "b", "c")) + .withSnapshot(oldestSnapshotId) + .build(); + + context.becomeLeader(); + int epoch = context.currentEpoch(); + context.client.poll(); + context.client.poll(); + + assertEquals(oldestSnapshotId, context.listener.takeSnapshot().get().snapshotId()); + + /* + List appendRecords = Arrays.asList("a", "b", "c"); + context.client.scheduleAppend(epoch, appendRecords); + context.client.poll(); + + // Advance the highWatermark + context.deliverRequest(context.fetchRequest(epoch, otherNodeId, localLogEndOffset, epoch, 0)); + context.pollUntilResponse(); + context.assertSentFetchResponse(Errors.NONE, epoch, OptionalInt.of(localId)); + assertEquals(localLogEndOffset, context.client.highWatermark().getAsLong()); + */ + } + + @Test + public void testFollowerListenerNotified() { + } + + @Test + public void testSecondListenerNotified() { + } + + @Test + public void testListenerRenotified() { + } + @Test public void testFetchRequestOffsetLessThanLogStart() throws Exception { int localId = 0; @@ -85,7 +129,7 @@ public void testFetchRequestOffsetLessThanLogStart() throws Exception { context.client.poll(); assertEquals(snapshotId.offset, context.log.startOffset()); - + // Send Fetch request less than start offset context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 0, epoch, 0)); context.pollUntilResponse(); diff --git a/raft/src/test/java/org/apache/kafka/raft/MockLog.java b/raft/src/test/java/org/apache/kafka/raft/MockLog.java index 3252d54ea01b2..72256609b7863 100644 --- a/raft/src/test/java/org/apache/kafka/raft/MockLog.java +++ b/raft/src/test/java/org/apache/kafka/raft/MockLog.java @@ -431,47 +431,37 @@ public void onSnapshotFrozen(OffsetAndEpoch snapshotId) {} @Override public boolean deleteBeforeSnapshot(OffsetAndEpoch logStartSnapshotId) { - if (logStartOffset() > logStartSnapshotId.offset || - highWatermark.offset < logStartSnapshotId.offset) { - - throw new OffsetOutOfRangeException( - String.format( - "New log start (%s) is less than start offset (%s) or is greater than the high watermark (%s)", - logStartSnapshotId, - logStartOffset(), - highWatermark.offset - ) - ); - } - boolean updated = false; - Optional snapshotIdOpt = latestSnapshotId(); - if (snapshotIdOpt.isPresent()) { - OffsetAndEpoch snapshotId = snapshotIdOpt.get(); - if (logStartOffset() < logStartSnapshotId.offset && - highWatermark.offset >= logStartSnapshotId.offset && - snapshotId.offset >= logStartSnapshotId.offset) { + if (snapshots.containsKey(logStartSnapshotId) && + logStartOffset() < logStartSnapshotId.offset) { + + oldestSnapshotId = Optional.of(logStartSnapshotId); - oldestSnapshotId = Optional.of(logStartSnapshotId); + // Update the high watermark if it is less than the new log start offset + if (logStartSnapshotId.offset > highWatermark.offset) { + updateHighWatermark(new LogOffsetMetadata(logStartSnapshotId.offset)); + } - batches.removeIf(entry -> entry.lastOffset() < logStartSnapshotId.offset); + batches.removeIf(entry -> entry.lastOffset() < logStartSnapshotId.offset); - AtomicReference> last = new AtomicReference<>(Optional.empty()); - epochStartOffsets.removeIf(epochStartOffset -> { - if (epochStartOffset.startOffset <= logStartSnapshotId.offset) { - last.set(Optional.of(epochStartOffset)); - return true; - } + AtomicReference> last = new AtomicReference<>(Optional.empty()); + epochStartOffsets.removeIf(epochStartOffset -> { + if (epochStartOffset.startOffset <= logStartSnapshotId.offset) { + last.set(Optional.of(epochStartOffset)); + return true; + } - return false; - }); + return false; + }); - last.get().ifPresent(epochStartOffset -> { - epochStartOffsets.add(0, new EpochStartOffset(epochStartOffset.epoch, logStartSnapshotId.offset)); - }); + last.get().ifPresent(epochStartOffset -> { + epochStartOffsets.add( + 0, + new EpochStartOffset(epochStartOffset.epoch, logStartSnapshotId.offset) + ); + }); - updated = true; - } + updated = true; } return updated; diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java index efe7c95bfbcbf..6cbe030c6b267 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -20,15 +20,15 @@ import org.apache.kafka.common.memory.MemoryPool; import org.apache.kafka.common.message.BeginQuorumEpochRequestData; import org.apache.kafka.common.message.BeginQuorumEpochResponseData; -import org.apache.kafka.common.message.DescribeQuorumResponseData; import org.apache.kafka.common.message.DescribeQuorumResponseData.ReplicaState; +import org.apache.kafka.common.message.DescribeQuorumResponseData; import org.apache.kafka.common.message.EndQuorumEpochRequestData; import org.apache.kafka.common.message.EndQuorumEpochResponseData; import org.apache.kafka.common.message.FetchRequestData; import org.apache.kafka.common.message.FetchResponseData; import org.apache.kafka.common.message.FetchSnapshotResponseData; -import org.apache.kafka.common.message.LeaderChangeMessage; import org.apache.kafka.common.message.LeaderChangeMessage.Voter; +import org.apache.kafka.common.message.LeaderChangeMessage; import org.apache.kafka.common.message.VoteRequestData; import org.apache.kafka.common.message.VoteResponseData; import org.apache.kafka.common.metrics.Metrics; @@ -54,6 +54,8 @@ import org.apache.kafka.common.utils.Utils; import org.apache.kafka.raft.internals.BatchBuilder; import org.apache.kafka.raft.internals.StringSerde; +import org.apache.kafka.snapshot.RawSnapshotWriter; +import org.apache.kafka.snapshot.SnapshotReader; import org.apache.kafka.test.TestCondition; import org.apache.kafka.test.TestUtils; import org.mockito.Mockito; @@ -78,6 +80,7 @@ import java.util.stream.Collectors; import static org.apache.kafka.raft.RaftUtil.hasValidTopicPartition; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -177,6 +180,13 @@ Builder appendToLog(long baseOffset, int epoch, List records) { return this; } + Builder withSnapshot(OffsetAndEpoch snapshotId) throws IOException { + try (RawSnapshotWriter snapshot = log.createSnapshot(snapshotId)) { + snapshot.freeze(); + } + return this; + } + Builder withElectionTimeoutMs(int electionTimeoutMs) { this.electionTimeoutMs = electionTimeoutMs; return this; @@ -918,6 +928,7 @@ static class MockListener implements RaftClient.Listener { private final List> commits = new ArrayList<>(); private final Map claimedEpochStartOffsets = new HashMap<>(); private OptionalInt currentClaimedEpoch = OptionalInt.empty(); + private Optional> snapshot = Optional.empty(); int numCommittedBatches() { return commits.size(); @@ -963,6 +974,12 @@ List commitWithLastOffset(long lastOffset) { .orElse(null); } + Optional> takeSnapshot() { + Optional> temp = snapshot; + snapshot = Optional.empty(); + return temp; + } + @Override public void handleClaim(int epoch) { // We record the next expected offset as the claimed epoch's start @@ -997,6 +1014,15 @@ public void handleCommit(BatchReader reader) { reader.close(); } } + + @Override + public void handleSnapshot(SnapshotReader reader) { + if (snapshot.isPresent()) { + assertDoesNotThrow(() -> snapshot.get().close()); + } + + snapshot = Optional.of(reader); + } } } From 1ff2ffcea8931893200f33c62bb94e568b306275 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Mon, 8 Feb 2021 17:03:31 -0700 Subject: [PATCH 02/19] Add test for snapshot loading API --- .../scala/kafka/tools/TestRaftServer.scala | 11 +- .../org/apache/kafka/raft/FollowerState.java | 2 + .../apache/kafka/raft/KafkaRaftClient.java | 3 +- .../kafka/snapshot/FileRawSnapshotWriter.java | 10 ++ .../raft/KafkaRaftClientSnapshotTest.java | 170 +++++++++++++++--- .../kafka/raft/KafkaRaftClientTest.java | 32 ++-- .../java/org/apache/kafka/raft/MockLog.java | 10 ++ .../kafka/raft/RaftClientTestContext.java | 5 +- .../kafka/snapshot/SnapshotWriterTest.java | 10 ++ 9 files changed, 207 insertions(+), 46 deletions(-) diff --git a/core/src/main/scala/kafka/tools/TestRaftServer.scala b/core/src/main/scala/kafka/tools/TestRaftServer.scala index 0a83fd9453581..e7fdfea2e3307 100644 --- a/core/src/main/scala/kafka/tools/TestRaftServer.scala +++ b/core/src/main/scala/kafka/tools/TestRaftServer.scala @@ -36,6 +36,7 @@ import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{TopicPartition, Uuid, protocol} import org.apache.kafka.raft.BatchReader.Batch import org.apache.kafka.raft.{BatchReader, RaftClient, RecordSerde} +import org.apache.kafka.snapshot.SnapshotReader import scala.jdk.CollectionConverters._ @@ -143,6 +144,7 @@ class TestRaftServer( case class HandleClaim(epoch: Int) extends RaftEvent case object HandleResign extends RaftEvent case class HandleCommit(reader: BatchReader[Array[Byte]]) extends RaftEvent + case class HandleSnapshot(reader: SnapshotReader[Array[Byte]]) extends RaftEvent case object Shutdown extends RaftEvent private val eventQueue = new LinkedBlockingDeque[RaftEvent]() @@ -168,6 +170,10 @@ class TestRaftServer( eventQueue.offer(HandleCommit(reader)) } + override def handleSnapshot(reader: SnapshotReader[Array[Byte]]): Unit = { + eventQueue.offer(HandleSnapshot(reader)) + } + override def initiateShutdown(): Boolean = { val initiated = super.initiateShutdown() eventQueue.offer(Shutdown) @@ -219,7 +225,10 @@ class TestRaftServer( reader.close() } - case _ => + case HandleSnapshot(reader) => + // TODO: what are we suppose to do here? + + case Shutdown => // Ignore shutdown command } } diff --git a/raft/src/main/java/org/apache/kafka/raft/FollowerState.java b/raft/src/main/java/org/apache/kafka/raft/FollowerState.java index e1ef7aa6d56be..87a6d0e3b4112 100644 --- a/raft/src/main/java/org/apache/kafka/raft/FollowerState.java +++ b/raft/src/main/java/org/apache/kafka/raft/FollowerState.java @@ -146,6 +146,8 @@ public String toString() { ", epoch=" + epoch + ", leaderId=" + leaderId + ", voters=" + voters + + ", highWatermark=" + highWatermark + + ", fetchingSnapshot=" + fetchingSnapshot + ')'; } diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 9bc8a66a03096..f790e46af74d9 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -302,7 +302,7 @@ private void updateListenersProgress(long highWatermark) { private void updateListenersProgress(List listenerContexts, long highWatermark) { for (ListenerContext listenerContext : listenerContexts) { listenerContext.nextExpectedOffset().ifPresent(nextExpectedOffset -> { - if (nextExpectedOffset < log.startOffset()) { + if (nextExpectedOffset < log.startOffset() && nextExpectedOffset < highWatermark) { SnapshotReader snapshot = oldestSnapshot().orElseThrow(() -> { return new IllegalStateException( String.format( @@ -1137,6 +1137,7 @@ private boolean handleFetchResponse( if (records.sizeInBytes() > 0) { appendAsFollower(records); } + OptionalLong highWatermark = partitionResponse.highWatermark() < 0 ? OptionalLong.empty() : OptionalLong.of(partitionResponse.highWatermark()); updateFollowerHighWatermark(state, highWatermark); diff --git a/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotWriter.java b/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotWriter.java index 6d0c17c4e051d..342639487e448 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotWriter.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotWriter.java @@ -115,6 +115,16 @@ public void close() throws IOException { } } + @Override + public String toString() { + return String.format( + "FileRawSnapshotWriter(path=%s, snapshotId=%s, frozen=%s)", + tempSnapshotPath, + snapshotId, + frozen + ); + } + /** * Create a snapshot writer for topic partition log dir and snapshot id. * diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java index 985e5cc1c5f4c..86dbc69e5eab9 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java @@ -31,6 +31,7 @@ import org.apache.kafka.raft.internals.StringSerde; import org.apache.kafka.snapshot.RawSnapshotReader; import org.apache.kafka.snapshot.RawSnapshotWriter; +import org.apache.kafka.snapshot.SnapshotReader; import org.apache.kafka.snapshot.SnapshotWriter; import org.apache.kafka.snapshot.SnapshotWriterTest; import org.junit.jupiter.api.Test; @@ -52,43 +53,148 @@ public void testLeaderListernerNotified() throws Exception { int localId = 0; int otherNodeId = localId + 1; Set voters = Utils.mkSet(localId, otherNodeId); - OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 1); + OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 1); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .appendToLog(oldestSnapshotId.offset, oldestSnapshotId.epoch, Arrays.asList("a", "b", "c")) - .withSnapshot(oldestSnapshotId) + .appendToLog(snapshotId.epoch, Arrays.asList("a", "b", "c")) + .appendToLog(snapshotId.epoch, Arrays.asList("d", "e", "f")) + .withSnapshot(snapshotId) .build(); context.becomeLeader(); int epoch = context.currentEpoch(); - context.client.poll(); - context.client.poll(); - - assertEquals(oldestSnapshotId, context.listener.takeSnapshot().get().snapshotId()); - /* - List appendRecords = Arrays.asList("a", "b", "c"); - context.client.scheduleAppend(epoch, appendRecords); - context.client.poll(); - // Advance the highWatermark + long localLogEndOffset = context.log.endOffset().offset; context.deliverRequest(context.fetchRequest(epoch, otherNodeId, localLogEndOffset, epoch, 0)); context.pollUntilResponse(); context.assertSentFetchResponse(Errors.NONE, epoch, OptionalInt.of(localId)); assertEquals(localLogEndOffset, context.client.highWatermark().getAsLong()); - */ + + // Check that listener was notified of the new snapshot + try (SnapshotReader snapshot = context.listener.takeSnapshot().get()) { + assertEquals(snapshotId, snapshot.snapshotId()); + SnapshotWriterTest.assertSnapshot(Arrays.asList(), snapshot); + } } @Test - public void testFollowerListenerNotified() { + public void testFollowerListenerNotified() throws Exception { + int localId = 0; + int leaderId = localId + 1; + Set voters = Utils.mkSet(localId, leaderId); + int epoch = 2; + OffsetAndEpoch snapshotId = new OffsetAndEpoch(3L, 1); + + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .appendToLog(snapshotId.epoch, Arrays.asList("a", "b", "c")) + .appendToLog(snapshotId.epoch, Arrays.asList("d", "e", "f")) + .withSnapshot(snapshotId) + .withElectedLeader(epoch, leaderId) + .build(); + + // Advance the highWatermark + long localLogEndOffset = context.log.endOffset().offset; + context.pollUntilRequest(); + RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); + context.assertFetchRequestData(fetchRequest, epoch, localLogEndOffset, snapshotId.epoch); + context.deliverResponse( + fetchRequest.correlationId, + fetchRequest.destinationId(), + context.fetchResponse(epoch, leaderId, MemoryRecords.EMPTY, localLogEndOffset, Errors.NONE) + ); + + context.pollUntilRequest(); + context.assertSentFetchRequest(epoch, localLogEndOffset, snapshotId.epoch); + + // Check that listener was notified of the new snapshot + try (SnapshotReader snapshot = context.listener.takeSnapshot().get()) { + assertEquals(snapshotId, snapshot.snapshotId()); + SnapshotWriterTest.assertSnapshot(Arrays.asList(), snapshot); + } } @Test - public void testSecondListenerNotified() { + public void testSecondListenerNotified() throws Exception { + int localId = 0; + int leaderId = localId + 1; + Set voters = Utils.mkSet(localId, leaderId); + int epoch = 2; + OffsetAndEpoch snapshotId = new OffsetAndEpoch(3L, 1); + + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .appendToLog(snapshotId.epoch, Arrays.asList("a", "b", "c")) + .appendToLog(snapshotId.epoch, Arrays.asList("d", "e", "f")) + .withSnapshot(snapshotId) + .withElectedLeader(epoch, leaderId) + .build(); + + // Advance the highWatermark + long localLogEndOffset = context.log.endOffset().offset; + context.pollUntilRequest(); + RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); + context.assertFetchRequestData(fetchRequest, epoch, localLogEndOffset, snapshotId.epoch); + context.deliverResponse( + fetchRequest.correlationId, + fetchRequest.destinationId(), + context.fetchResponse(epoch, leaderId, MemoryRecords.EMPTY, localLogEndOffset, Errors.NONE) + ); + + context.pollUntilRequest(); + context.assertSentFetchRequest(epoch, localLogEndOffset, snapshotId.epoch); + + RaftClientTestContext.MockListener secondListener = new RaftClientTestContext.MockListener(); + context.client.register(secondListener); + context.client.poll(); + + // Check that the second listener was notified of the new snapshot + try (SnapshotReader snapshot = secondListener.takeSnapshot().get()) { + assertEquals(snapshotId, snapshot.snapshotId()); + SnapshotWriterTest.assertSnapshot(Arrays.asList(), snapshot); + } } @Test - public void testListenerRenotified() { + public void testListenerRenotified() throws Exception { + int localId = 0; + int otherNodeId = localId + 1; + Set voters = Utils.mkSet(localId, otherNodeId); + 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")) + .withSnapshot(snapshotId) + .build(); + + context.becomeLeader(); + int epoch = context.currentEpoch(); + + // Advance the highWatermark + long localLogEndOffset = context.log.endOffset().offset; + context.deliverRequest(context.fetchRequest(epoch, otherNodeId, localLogEndOffset, epoch, 0)); + context.pollUntilResponse(); + context.assertSentFetchResponse(Errors.NONE, epoch, OptionalInt.of(localId)); + assertEquals(localLogEndOffset, context.client.highWatermark().getAsLong()); + + // Check that listener was notified of the new snapshot + try (SnapshotReader snapshot = context.listener.takeSnapshot().get()) { + assertEquals(snapshotId, snapshot.snapshotId()); + SnapshotWriterTest.assertSnapshot(Arrays.asList(), snapshot); + } + + // Generate a new snapshot + OffsetAndEpoch secondSnapshot = new OffsetAndEpoch(6, snapshotId.epoch); + try (SnapshotWriter snapshot = context.client.createSnapshot(snapshotId)) { + snapshot.freeze(); + } + + context.client.poll(); + // Check that listener was notified of the second snapshot + try (SnapshotReader snapshot = context.listener.takeSnapshot().get()) { + assertEquals(secondSnapshot, snapshot.snapshotId()); + SnapshotWriterTest.assertSnapshot(Arrays.asList(), snapshot); + } } @Test @@ -150,7 +256,7 @@ public void testFetchRequestWithLargerLastFetchedEpoch() throws Exception { OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .appendToLog(oldestSnapshotId.offset, oldestSnapshotId.epoch, Arrays.asList("a", "b", "c")) + .appendToLog(oldestSnapshotId.epoch, Arrays.asList("a", "b", "c")) .withAppendLingerMs(1) .build(); @@ -190,8 +296,8 @@ public void testFetchRequestTruncateToLogStart() throws Exception { OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .appendToLog(oldestSnapshotId.offset, oldestSnapshotId.epoch, Arrays.asList("a", "b", "c")) - .appendToLog(oldestSnapshotId.offset + 3, oldestSnapshotId.epoch + 2, Arrays.asList("a", "b", "c")) + .appendToLog(oldestSnapshotId.epoch, Arrays.asList("a", "b", "c")) + .appendToLog(oldestSnapshotId.epoch + 2, Arrays.asList("a", "b", "c")) .withAppendLingerMs(1) .build(); @@ -236,8 +342,8 @@ public void testFetchRequestAtLogStartOffsetWithValidEpoch() throws Exception { OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .appendToLog(oldestSnapshotId.offset, oldestSnapshotId.epoch, Arrays.asList("a", "b", "c")) - .appendToLog(oldestSnapshotId.offset + 3, oldestSnapshotId.epoch + 2, Arrays.asList("a", "b", "c")) + .appendToLog(oldestSnapshotId.epoch, Arrays.asList("a", "b", "c")) + .appendToLog(oldestSnapshotId.epoch + 2, Arrays.asList("a", "b", "c")) .withAppendLingerMs(1) .build(); @@ -277,8 +383,8 @@ public void testFetchRequestAtLogStartOffsetWithInvalidEpoch() throws Exception OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .appendToLog(oldestSnapshotId.offset, oldestSnapshotId.epoch, Arrays.asList("a", "b", "c")) - .appendToLog(oldestSnapshotId.offset + 3, oldestSnapshotId.epoch + 2, Arrays.asList("a", "b", "c")) + .appendToLog(oldestSnapshotId.epoch, Arrays.asList("a", "b", "c")) + .appendToLog(oldestSnapshotId.epoch + 2, Arrays.asList("a", "b", "c")) .withAppendLingerMs(1) .build(); @@ -323,8 +429,8 @@ public void testFetchRequestWithLastFetchedEpochLessThanOldestSnapshot() throws OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .appendToLog(oldestSnapshotId.offset, oldestSnapshotId.epoch, Arrays.asList("a", "b", "c")) - .appendToLog(oldestSnapshotId.offset + 3, oldestSnapshotId.epoch + 2, Arrays.asList("a", "b", "c")) + .appendToLog(oldestSnapshotId.epoch, Arrays.asList("a", "b", "c")) + .appendToLog(oldestSnapshotId.epoch + 2, Arrays.asList("a", "b", "c")) .withAppendLingerMs(1) .build(); @@ -786,10 +892,17 @@ public void testFetchResponseWithSnapshotId() throws Exception { fetchRequest = context.assertSentFetchRequest(); context.assertFetchRequestData(fetchRequest, epoch, snapshotId.offset, snapshotId.epoch); + // Check that the snapshot was written to the log try (RawSnapshotReader snapshot = context.log.readSnapshot(snapshotId).get()) { assertEquals(memorySnapshot.buffer().remaining(), snapshot.sizeInBytes()); SnapshotWriterTest.assertSnapshot(Arrays.asList(records), snapshot); } + + // Check that listener was notified of the new snapshot + try (SnapshotReader snapshot = context.listener.takeSnapshot().get()) { + assertEquals(snapshotId, snapshot.snapshotId()); + SnapshotWriterTest.assertSnapshot(Arrays.asList(records), snapshot); + } } @Test @@ -883,10 +996,17 @@ public void testFetchSnapshotResponsePartialData() throws Exception { fetchRequest = context.assertSentFetchRequest(); context.assertFetchRequestData(fetchRequest, epoch, snapshotId.offset, snapshotId.epoch); + // Check that the snapshot was written to the log try (RawSnapshotReader snapshot = context.log.readSnapshot(snapshotId).get()) { assertEquals(memorySnapshot.buffer().remaining(), snapshot.sizeInBytes()); SnapshotWriterTest.assertSnapshot(Arrays.asList(records), snapshot); } + + // Check that listener was notified of the new snapshot + try (SnapshotReader snapshot = context.listener.takeSnapshot().get()) { + assertEquals(snapshotId, snapshot.snapshotId()); + SnapshotWriterTest.assertSnapshot(Arrays.asList(records), snapshot); + } } @Test diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java index b29f1efaf2ee4..4c8218c9a196f 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -945,7 +945,7 @@ public void testInitializeAsFollowerNonEmptyLog() throws Exception { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, otherNodeId) - .appendToLog(0L, lastEpoch, singletonList("foo")) + .appendToLog(lastEpoch, singletonList("foo")) .build(); context.assertElectedLeader(epoch, otherNodeId); @@ -964,7 +964,7 @@ public void testVoterBecomeCandidateAfterFetchTimeout() throws Exception { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, otherNodeId) - .appendToLog(0L, lastEpoch, singletonList("foo")) + .appendToLog(lastEpoch, singletonList("foo")) .build(); context.assertElectedLeader(epoch, otherNodeId); @@ -1781,8 +1781,8 @@ public void testFollowerLogReconciliation() throws Exception { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, otherNodeId) - .appendToLog(0L, lastEpoch, Arrays.asList("foo", "bar")) - .appendToLog(2L, lastEpoch, Arrays.asList("baz")) + .appendToLog(lastEpoch, Arrays.asList("foo", "bar")) + .appendToLog(lastEpoch, Arrays.asList("baz")) .build(); context.assertElectedLeader(epoch, otherNodeId); @@ -1964,9 +1964,9 @@ public void testHandleClaimCallbackFiresAfterHighWatermarkReachesEpochStartOffse List batch3 = Arrays.asList("7", "8", "9"); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .appendToLog(0L, 1, batch1) - .appendToLog(3L, 1, batch2) - .appendToLog(6L, 2, batch3) + .appendToLog(1, batch1) + .appendToLog(1, batch2) + .appendToLog(2, batch3) .withUnknownLeader(epoch - 1) .build(); @@ -2016,9 +2016,9 @@ public void testLateRegisteredListenerCatchesUp() throws Exception { List batch3 = Arrays.asList("7", "8", "9"); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .appendToLog(0L, 1, batch1) - .appendToLog(3L, 1, batch2) - .appendToLog(6L, 2, batch3) + .appendToLog(1, batch1) + .appendToLog(1, batch2) + .appendToLog(2, batch3) .withUnknownLeader(epoch - 1) .build(); @@ -2105,9 +2105,9 @@ public void testHandleCommitCallbackFiresInVotedState() throws Exception { Set voters = Utils.mkSet(localId, otherNodeId); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .appendToLog(0L, 2, Arrays.asList("a", "b", "c")) - .appendToLog(3L, 4, Arrays.asList("d", "e", "f")) - .appendToLog(6L, 4, Arrays.asList("g", "h", "i")) + .appendToLog(2, Arrays.asList("a", "b", "c")) + .appendToLog(4, Arrays.asList("d", "e", "f")) + .appendToLog(4, Arrays.asList("g", "h", "i")) .withUnknownLeader(epoch - 1) .build(); @@ -2146,9 +2146,9 @@ public void testHandleCommitCallbackFiresInCandidateState() throws Exception { Set voters = Utils.mkSet(localId, otherNodeId); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .appendToLog(0L, 2, Arrays.asList("a", "b", "c")) - .appendToLog(3L, 4, Arrays.asList("d", "e", "f")) - .appendToLog(6L, 4, Arrays.asList("g", "h", "i")) + .appendToLog(2, Arrays.asList("a", "b", "c")) + .appendToLog(4, Arrays.asList("d", "e", "f")) + .appendToLog(4, Arrays.asList("g", "h", "i")) .withUnknownLeader(epoch - 1) .build(); diff --git a/raft/src/test/java/org/apache/kafka/raft/MockLog.java b/raft/src/test/java/org/apache/kafka/raft/MockLog.java index 72256609b7863..fc2910b498d35 100644 --- a/raft/src/test/java/org/apache/kafka/raft/MockLog.java +++ b/raft/src/test/java/org/apache/kafka/raft/MockLog.java @@ -524,6 +524,16 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(metadata, offset, record); } + + @Override + public String toString() { + return String.format( + "LogEntry(metadata=%s, offset=%s, record=%s)", + metadata, + offset, + record + ); + } } static class LogBatch { diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java index 6cbe030c6b267..c6fd167ea3d96 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -174,8 +174,8 @@ Builder withAppendLingerMs(int appendLingerMs) { return this; } - Builder appendToLog(long baseOffset, int epoch, List records) { - MemoryRecords batch = buildBatch(time.milliseconds(), baseOffset, epoch, records); + Builder appendToLog(int epoch, List records) { + MemoryRecords batch = buildBatch(time.milliseconds(), 0, epoch, records); log.appendAsLeader(batch, epoch); return this; } @@ -1024,5 +1024,4 @@ public void handleSnapshot(SnapshotReader reader) { snapshot = Optional.of(reader); } } - } diff --git a/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterTest.java b/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterTest.java index 35652c75306c9..8588196c99457 100644 --- a/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterTest.java +++ b/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterTest.java @@ -113,4 +113,14 @@ public static void assertSnapshot(List> batches, RawSnapshotReader assertEquals(expected, actual); } + + public static void assertSnapshot(List> batches, SnapshotReader reader) { + List expected = new ArrayList<>(); + batches.forEach(expected::addAll); + + List actual = new ArrayList<>(expected.size()); + reader.forEach(actual::addAll); + + assertEquals(expected, actual); + } } From 1ee5b2766f1dd11364f3f8bcf20488df80480d01 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Wed, 10 Feb 2021 14:06:59 -0700 Subject: [PATCH 03/19] Allow for configurable buffer supplier --- .../scala/kafka/tools/TestRaftServer.scala | 3 +- .../apache/kafka/raft/KafkaRaftClient.java | 8 +- .../org/apache/kafka/raft/RaftClient.java | 11 ++- .../apache/kafka/raft/ReplicatedCounter.java | 6 +- .../apache/kafka/snapshot/SnapshotReader.java | 92 +++++++++++++------ .../apache/kafka/snapshot/SnapshotWriter.java | 2 +- .../raft/KafkaRaftClientSnapshotTest.java | 2 +- 7 files changed, 86 insertions(+), 38 deletions(-) diff --git a/core/src/main/scala/kafka/tools/TestRaftServer.scala b/core/src/main/scala/kafka/tools/TestRaftServer.scala index e7fdfea2e3307..d1a2154564033 100644 --- a/core/src/main/scala/kafka/tools/TestRaftServer.scala +++ b/core/src/main/scala/kafka/tools/TestRaftServer.scala @@ -226,7 +226,8 @@ class TestRaftServer( } case HandleSnapshot(reader) => - // TODO: what are we suppose to do here? + // Ignore snapshots; only interested on records appended by this leader + reader.close() case Shutdown => // Ignore shutdown command } diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index f790e46af74d9..936874f249904 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -131,8 +131,10 @@ * some additional metadata on responses (i.e. current leader and epoch). Unlike partition replication, * we also piggyback truncation detection on this API rather than through a separate truncation state. * - * 5) TODO: Talk about FetchSnapshotRequestData - * + * 5) {@link FetchSnapshotRequestData}: Sent by the followers to the epoch leader when fetching a snapshot. + * Followers need to fetch snapshots when the follower's log end offset is less than the leader's log + * start offset. The follower discover this case when the leader replies with a fetch response that + * contains a snapshot id. */ public class KafkaRaftClient implements RaftClient { private static final int RETRY_BACKOFF_BASE_MS = 100; @@ -331,7 +333,7 @@ private Optional> oldestSnapshot() { try { return log .readSnapshot(log.oldestSnapshotId().get()) - .map(reader -> new SnapshotReader<>(reader, serde)); + .map(reader -> new SnapshotReader<>(reader, serde, BufferSupplier.create())); } catch (IOException e) { logger.error( String.format("Unable to read snapshot: %s", log.oldestSnapshotId().get()), diff --git a/raft/src/main/java/org/apache/kafka/raft/RaftClient.java b/raft/src/main/java/org/apache/kafka/raft/RaftClient.java index 15187df175b12..614fe85073004 100644 --- a/raft/src/main/java/org/apache/kafka/raft/RaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/RaftClient.java @@ -29,7 +29,7 @@ public interface RaftClient extends Closeable { interface Listener { /** * Callback which is invoked for all records committed to the log. - * It is the responsibility of the caller to invoke {@link BatchReader#close()} + * It is the responsibility of this implementation to invoke {@link BatchReader#close()} * after consuming the reader. * * Note that there is not a one-to-one correspondence between writes through @@ -44,7 +44,14 @@ interface Listener { void handleCommit(BatchReader reader); /** - * TOOD: Write documentation + * Callback which is invoked when the Listener needs to load a snapshot. + * It is the responsibility of this implementation to invoke {@link BatchReader#close()} + * after consuming the reader. + * + * When handling this call, the implementation must assume that all previous calls + * to {@link #handleCommit} contain invalid data. + * + * @param reader snapshot reader instance which must be iterated and closed */ void handleSnapshot(SnapshotReader reader); diff --git a/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java b/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java index 96763f16ad391..97834e0d94bf6 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java +++ b/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java @@ -22,7 +22,6 @@ import org.slf4j.Logger; import java.io.IOException; -import java.util.Iterator; import java.util.List; import java.util.Optional; @@ -102,9 +101,8 @@ public synchronized void handleSnapshot(SnapshotReader reader) { try { try (SnapshotReader snapshot = reader) { log.debug("Loading snapshot {}", snapshot.snapshotId()); - Iterator> batches = snapshot.iterator(); - while (batches.hasNext()) { - for (Integer value : batches.next()) { + for (List batch : snapshot) { + for (Integer value : batch) { log.debug("Setting value: {}", value); this.committed = value; this.uncommitted = value; diff --git a/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java index 03b40050c25bc..6fa6deb7be229 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java @@ -19,29 +19,37 @@ import java.io.Closeable; import java.io.IOException; +import java.util.ArrayList; import java.util.Iterator; import java.util.List; -import java.util.ArrayList; +import java.util.NoSuchElementException; import org.apache.kafka.common.protocol.ByteBufferAccessor; -import org.apache.kafka.common.record.BufferSupplier.GrowableBufferSupplier; +import org.apache.kafka.common.record.BufferSupplier; import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.raft.RecordSerde; -final public class SnapshotReader implements Closeable, Iterable> { - final private RawSnapshotReader snapshot; - final private RecordSerde serde; +public final class SnapshotReader implements Closeable, Iterable> { + private final RawSnapshotReader snapshot; + private final RecordSerde serde; + private final BufferSupplier bufferSupplier; /** - * TODO: write documentation + * A type for reading an immutable snapshot. + * + * A snapshot reader can be used to scan through all of the objects T in a the snapshot. It + * is assumed that the content of the snapshot represents all of the objects T for the topic + * partition from offset 0 up to but not including the end offset in the snapshot id. */ public SnapshotReader( RawSnapshotReader snapshot, - RecordSerde serde + RecordSerde serde, + BufferSupplier bufferSupplier ) { this.snapshot = snapshot; this.serde = serde; + this.bufferSupplier = bufferSupplier; } /** @@ -53,7 +61,7 @@ public OffsetAndEpoch snapshotId() { @Override public Iterator> iterator() { - return new SnapshotReaderIterator<>(snapshot.iterator(), serde); + return new SnapshotReaderIterator(snapshot.iterator()); } /** @@ -66,44 +74,76 @@ public void close() throws IOException { } - final static class SnapshotReaderIterator implements Iterator> { - final Iterator iterator; - final RecordSerde serde; + final class SnapshotReaderIterator implements Iterator> { + private final Iterator iterator; + private List nextBatch; - SnapshotReaderIterator(Iterator iterator, RecordSerde serde) { + SnapshotReaderIterator(Iterator iterator) { this.iterator = iterator; - this.serde = serde; } @Override public boolean hasNext() { - return iterator.hasNext(); + if (nextBatch == null) { + nextBatch = findNext(); + } + + return nextBatch != null; } @Override public List next() { - RecordBatch batch = iterator.next(); + if (nextBatch == null) { + nextBatch = findNext(); + } + + if (nextBatch == null) { + throw new NoSuchElementException( + String.format("Snapshot (%s) doesn't have any more elements", snapshotId()) + ); + } else { + List result = nextBatch; + nextBatch = null; + return result; + } + } + + private List findNext() { + RecordBatch batch = null; + while (iterator.hasNext()) { + batch = iterator.next(); + if (!batch.isControlBatch()) { + break; + } + } + + if (batch == null) { + return null; + } if (batch.countOrNull() == null) { throw new IllegalStateException( - String.format( - "Expected a record count the the batch (%s)", - batch - ) + String.format("Expected a record count the the batch (%s)", batch) ); } List data = new ArrayList<>(batch.countOrNull()); - // TODO: make BufferSupplier configurable - Iterator records = batch.streamingIterator(new GrowableBufferSupplier()); + Iterator records = batch.streamingIterator(bufferSupplier); while (records.hasNext()) { Record record = records.next(); - // TODO: ignore control records - // TODO: verify hasValue is true - data.add( - serde.read(new ByteBufferAccessor(record.value()), record.value().remaining()) - ); + if (record.hasValue()) { + data.add( + serde.read(new ByteBufferAccessor(record.value()), record.value().remaining()) + ); + } else { + throw new IllegalStateException( + String.format( + "Expected all records in the snapshot (%s) to have a value", + snapshotId() + ) + ); + } } return data; diff --git a/raft/src/main/java/org/apache/kafka/snapshot/SnapshotWriter.java b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotWriter.java index 542e54608bc2c..2789745bd3fca 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/SnapshotWriter.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotWriter.java @@ -30,7 +30,7 @@ import java.util.List; /** - * A type for writing a snapshot fora given end offset and epoch. + * A type for writing a snapshot for a given end offset and epoch. * * A snapshot writer can be used to append objects until freeze is called. When freeze is * called the snapshot is validated and marked as immutable. After freeze is called any diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java index 86dbc69e5eab9..e942d0bed6eaf 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java @@ -235,7 +235,7 @@ public void testFetchRequestOffsetLessThanLogStart() throws Exception { context.client.poll(); assertEquals(snapshotId.offset, context.log.startOffset()); - + // Send Fetch request less than start offset context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 0, epoch, 0)); context.pollUntilResponse(); From abb1441171f88016b87398dbb5c0df51f1fb6b42 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Wed, 10 Feb 2021 14:57:39 -0700 Subject: [PATCH 04/19] Add snapshot reader tests --- .../apache/kafka/raft/KafkaRaftClient.java | 2 +- .../org/apache/kafka/raft/RaftClient.java | 2 +- .../apache/kafka/snapshot/SnapshotReader.java | 7 +++--- .../raft/KafkaRaftClientSnapshotTest.java | 20 ++++++++-------- .../kafka/raft/RaftClientTestContext.java | 8 +++---- ...est.java => SnapshotWriterReaderTest.java} | 23 ++++++++++++++----- 6 files changed, 37 insertions(+), 25 deletions(-) rename raft/src/test/java/org/apache/kafka/snapshot/{SnapshotWriterTest.java => SnapshotWriterReaderTest.java} (87%) diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 936874f249904..09da8aa5318fd 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -2337,7 +2337,7 @@ public synchronized OptionalLong nextExpectedOffset() { /** * This API is used when the Listener needs to be notified of a new snapshot. This happens - * when the context's next offset is less that then log start offset. + * when the context's next offset is less than then log start offset. */ public void fireHandleSnapshot(SnapshotReader reader) { synchronized (this) { diff --git a/raft/src/main/java/org/apache/kafka/raft/RaftClient.java b/raft/src/main/java/org/apache/kafka/raft/RaftClient.java index 614fe85073004..2825ed60d681b 100644 --- a/raft/src/main/java/org/apache/kafka/raft/RaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/RaftClient.java @@ -45,7 +45,7 @@ interface Listener { /** * Callback which is invoked when the Listener needs to load a snapshot. - * It is the responsibility of this implementation to invoke {@link BatchReader#close()} + * It is the responsibility of this implementation to invoke {@link SnapshotReader#close()} * after consuming the reader. * * When handling this call, the implementation must assume that all previous calls diff --git a/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java index 6fa6deb7be229..19275d69f2e18 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java @@ -38,7 +38,7 @@ public final class SnapshotReader implements Closeable, Iterable> { /** * A type for reading an immutable snapshot. * - * A snapshot reader can be used to scan through all of the objects T in a the snapshot. It + * A snapshot reader can be used to scan through all of the objects T in a snapshot. It * is assumed that the content of the snapshot represents all of the objects T for the topic * partition from offset 0 up to but not including the end offset in the snapshot id. */ @@ -111,8 +111,9 @@ public List next() { private List findNext() { RecordBatch batch = null; while (iterator.hasNext()) { - batch = iterator.next(); - if (!batch.isControlBatch()) { + RecordBatch current = iterator.next(); + if (!current.isControlBatch()) { + batch = current; break; } } diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java index e942d0bed6eaf..3b4c2357de76c 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java @@ -33,7 +33,7 @@ import org.apache.kafka.snapshot.RawSnapshotWriter; import org.apache.kafka.snapshot.SnapshotReader; import org.apache.kafka.snapshot.SnapshotWriter; -import org.apache.kafka.snapshot.SnapshotWriterTest; +import org.apache.kafka.snapshot.SnapshotWriterReaderTest; import org.junit.jupiter.api.Test; import java.io.IOException; @@ -74,7 +74,7 @@ public void testLeaderListernerNotified() throws Exception { // Check that listener was notified of the new snapshot try (SnapshotReader snapshot = context.listener.takeSnapshot().get()) { assertEquals(snapshotId, snapshot.snapshotId()); - SnapshotWriterTest.assertSnapshot(Arrays.asList(), snapshot); + SnapshotWriterReaderTest.assertSnapshot(Arrays.asList(), snapshot); } } @@ -110,7 +110,7 @@ public void testFollowerListenerNotified() throws Exception { // Check that listener was notified of the new snapshot try (SnapshotReader snapshot = context.listener.takeSnapshot().get()) { assertEquals(snapshotId, snapshot.snapshotId()); - SnapshotWriterTest.assertSnapshot(Arrays.asList(), snapshot); + SnapshotWriterReaderTest.assertSnapshot(Arrays.asList(), snapshot); } } @@ -150,7 +150,7 @@ public void testSecondListenerNotified() throws Exception { // Check that the second listener was notified of the new snapshot try (SnapshotReader snapshot = secondListener.takeSnapshot().get()) { assertEquals(snapshotId, snapshot.snapshotId()); - SnapshotWriterTest.assertSnapshot(Arrays.asList(), snapshot); + SnapshotWriterReaderTest.assertSnapshot(Arrays.asList(), snapshot); } } @@ -180,7 +180,7 @@ public void testListenerRenotified() throws Exception { // Check that listener was notified of the new snapshot try (SnapshotReader snapshot = context.listener.takeSnapshot().get()) { assertEquals(snapshotId, snapshot.snapshotId()); - SnapshotWriterTest.assertSnapshot(Arrays.asList(), snapshot); + SnapshotWriterReaderTest.assertSnapshot(Arrays.asList(), snapshot); } // Generate a new snapshot @@ -193,7 +193,7 @@ public void testListenerRenotified() throws Exception { // Check that listener was notified of the second snapshot try (SnapshotReader snapshot = context.listener.takeSnapshot().get()) { assertEquals(secondSnapshot, snapshot.snapshotId()); - SnapshotWriterTest.assertSnapshot(Arrays.asList(), snapshot); + SnapshotWriterReaderTest.assertSnapshot(Arrays.asList(), snapshot); } } @@ -895,13 +895,13 @@ public void testFetchResponseWithSnapshotId() throws Exception { // Check that the snapshot was written to the log try (RawSnapshotReader snapshot = context.log.readSnapshot(snapshotId).get()) { assertEquals(memorySnapshot.buffer().remaining(), snapshot.sizeInBytes()); - SnapshotWriterTest.assertSnapshot(Arrays.asList(records), snapshot); + SnapshotWriterReaderTest.assertSnapshot(Arrays.asList(records), snapshot); } // Check that listener was notified of the new snapshot try (SnapshotReader snapshot = context.listener.takeSnapshot().get()) { assertEquals(snapshotId, snapshot.snapshotId()); - SnapshotWriterTest.assertSnapshot(Arrays.asList(records), snapshot); + SnapshotWriterReaderTest.assertSnapshot(Arrays.asList(records), snapshot); } } @@ -999,13 +999,13 @@ public void testFetchSnapshotResponsePartialData() throws Exception { // Check that the snapshot was written to the log try (RawSnapshotReader snapshot = context.log.readSnapshot(snapshotId).get()) { assertEquals(memorySnapshot.buffer().remaining(), snapshot.sizeInBytes()); - SnapshotWriterTest.assertSnapshot(Arrays.asList(records), snapshot); + SnapshotWriterReaderTest.assertSnapshot(Arrays.asList(records), snapshot); } // Check that listener was notified of the new snapshot try (SnapshotReader snapshot = context.listener.takeSnapshot().get()) { assertEquals(snapshotId, snapshot.snapshotId()); - SnapshotWriterTest.assertSnapshot(Arrays.asList(records), snapshot); + SnapshotWriterReaderTest.assertSnapshot(Arrays.asList(records), snapshot); } } diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java index c6fd167ea3d96..8beea19124b25 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -85,8 +85,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue; public final class RaftClientTestContext { - private static final StringSerde STRING_SERDE = new StringSerde(); - + public final RecordSerde serde = Builder.SERDE; final TopicPartition metadataPartition = Builder.METADATA_PARTITION; final int electionBackoffMaxMs = Builder.ELECTION_BACKOFF_MAX_MS; final int electionFetchMaxWaitMs = Builder.FETCH_MAX_WAIT_MS; @@ -113,6 +112,7 @@ public final class RaftClientTestContext { public static final class Builder { static final int DEFAULT_ELECTION_TIMEOUT_MS = 10000; + private static final RecordSerde SERDE = new StringSerde(); private static final TopicPartition METADATA_PARTITION = new TopicPartition("metadata", 0); private static final int ELECTION_BACKOFF_MAX_MS = 100; private static final int FETCH_MAX_WAIT_MS = 0; @@ -208,7 +208,7 @@ public RaftClientTestContext build() throws IOException { ELECTION_BACKOFF_MAX_MS, FETCH_TIMEOUT_MS, appendLingerMs); KafkaRaftClient client = new KafkaRaftClient<>( - STRING_SERDE, + SERDE, channel, messageQueue, log, @@ -301,7 +301,7 @@ static MemoryRecords buildBatch( ByteBuffer buffer = ByteBuffer.allocate(512); BatchBuilder builder = new BatchBuilder<>( buffer, - STRING_SERDE, + Builder.SERDE, CompressionType.NONE, baseOffset, timestamp, diff --git a/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterTest.java b/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterReaderTest.java similarity index 87% rename from raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterTest.java rename to raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterReaderTest.java index 8588196c99457..04a2f003bc875 100644 --- a/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterTest.java +++ b/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterReaderTest.java @@ -20,19 +20,19 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Optional; import java.util.Random; import java.util.Set; -import org.apache.kafka.common.record.BufferSupplier.GrowableBufferSupplier; +import org.apache.kafka.common.record.BufferSupplier; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.raft.RaftClientTestContext; import org.junit.jupiter.api.Test; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; -final public class SnapshotWriterTest { +final public class SnapshotWriterReaderTest { private final int localId = 0; private final Set voters = Collections.singleton(localId); @@ -49,7 +49,7 @@ public void testWritingSnapshot() throws IOException { snapshot.freeze(); } - try (RawSnapshotReader reader = context.log.readSnapshot(id).get()) { + try (SnapshotReader reader = readSnapshot(context, id)) { assertSnapshot(expected, reader); } } @@ -66,7 +66,7 @@ public void testAbortedSnapshot() throws IOException { }); } - assertFalse(context.log.readSnapshot(id).isPresent()); + assertEquals(Optional.empty(), context.log.readSnapshot(id)); } @Test @@ -100,13 +100,24 @@ private List> buildRecords(int recordsPerBatch, int batches) { return result; } + private SnapshotReader readSnapshot( + RaftClientTestContext context, + OffsetAndEpoch snapshotId + ) { + return new SnapshotReader<>( + context.log.readSnapshot(snapshotId).get(), + context.serde, + BufferSupplier.create() + ); + } + public static void assertSnapshot(List> batches, RawSnapshotReader reader) { List expected = new ArrayList<>(); batches.forEach(expected::addAll); List actual = new ArrayList<>(expected.size()); reader.forEach(batch -> { - batch.streamingIterator(new GrowableBufferSupplier()).forEachRemaining(record -> { + batch.streamingIterator(BufferSupplier.create()).forEachRemaining(record -> { actual.add(Utils.utf8(record.value())); }); }); From d8b4e13872b3ed9adbc5a991aca161dc67041645 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Thu, 18 Feb 2021 08:22:07 -0700 Subject: [PATCH 05/19] Use functional style when constructing SnapshotReader --- .../java/org/apache/kafka/raft/KafkaRaftClient.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index e13247e5bf8d0..e365a9a49f6b7 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -336,20 +336,20 @@ private void updateListenersProgress(List listenerContexts, lon } private Optional> oldestSnapshot() { - if (log.oldestSnapshotId().isPresent()) { + return log.oldestSnapshotId().flatMap(oldestSnapshotId -> { try { return log - .readSnapshot(log.oldestSnapshotId().get()) + .readSnapshot(oldestSnapshotId) .map(reader -> new SnapshotReader<>(reader, serde, BufferSupplier.create())); } catch (IOException e) { logger.error( - String.format("Unable to read snapshot: %s", log.oldestSnapshotId().get()), + String.format("Unable to read snapshot: %s", oldestSnapshotId), e ); - } - } - return Optional.empty(); + return Optional.empty(); + } + }); } private void maybeFireHandleCommit(long baseOffset, int epoch, List records) { From 6bbed7fa9ca862b3db601c8b8d1c21d8a382b14c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Thu, 11 Mar 2021 11:47:50 -0800 Subject: [PATCH 06/19] Fix test for snapshot releading --- checkstyle/import-control.xml | 1 + .../apache/kafka/raft/KafkaRaftClient.java | 1 + .../org/apache/kafka/raft/LeaderState.java | 34 ++++++++++-- .../raft/KafkaRaftClientSnapshotTest.java | 31 ++++++----- .../kafka/raft/KafkaRaftClientTest.java | 52 +++++++++++++----- .../java/org/apache/kafka/raft/MockLog.java | 22 ++++---- .../org/apache/kafka/raft/MockLogTest.java | 53 ++++++++++++++----- .../kafka/shell/MetadataNodeManager.java | 21 ++++++++ 8 files changed, 164 insertions(+), 51 deletions(-) diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 96336f2f797e6..d275754dbdfb5 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -283,6 +283,7 @@ + diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 0b0b041652c8e..2ff508ae5c106 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -2452,6 +2452,7 @@ void fireHandleResign(int epoch) { public synchronized void onClose(BatchReader reader) { OptionalLong lastOffset = reader.lastOffset(); + if (lastOffset.isPresent()) { nextOffset = lastOffset.getAsLong() + 1; } diff --git a/raft/src/main/java/org/apache/kafka/raft/LeaderState.java b/raft/src/main/java/org/apache/kafka/raft/LeaderState.java index 0f0f728f7c04f..acd3c3fb0c4fe 100644 --- a/raft/src/main/java/org/apache/kafka/raft/LeaderState.java +++ b/raft/src/main/java/org/apache/kafka/raft/LeaderState.java @@ -275,6 +275,16 @@ else if (!that.endOffset.isPresent()) else return Long.compare(that.endOffset.get().offset, this.endOffset.get().offset); } + + @Override + public String toString() { + return String.format( + "ReplicaState(nodeId=%s, endOffset=%s, lastFetchTimestamp=%s)", + nodeId, + endOffset, + lastFetchTimestamp + ); + } } private static class VoterState extends ReplicaState { @@ -285,15 +295,29 @@ public VoterState(int nodeId, super(nodeId); this.hasAcknowledgedLeader = hasAcknowledgedLeader; } + + @Override + public String toString() { + return String.format( + "VoterState(nodeId=%s, endOffset=%s, lastFetchTimestamp=%s, hasAcknowledgedLeader=%s)", + nodeId, + endOffset, + lastFetchTimestamp, + hasAcknowledgedLeader + ); + } } @Override public String toString() { - return "Leader(" + - "localId=" + localId + - ", epoch=" + epoch + - ", epochStartOffset=" + epochStartOffset + - ')'; + return String.format( + "Leader(localId=%s, epoch=%s, epochStartOffset=%s, highWatermark=%s, voterReplicaStates=%s)", + localId, + epoch, + epochStartOffset, + highWatermark, + voterReplicaStates + ); } @Override diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java index d48b4512e47df..ba0746653de09 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java @@ -42,6 +42,7 @@ import java.util.List; import java.util.Optional; import java.util.OptionalInt; +import java.util.OptionalLong; import java.util.Set; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -85,7 +86,7 @@ public void testFollowerListenerNotified() throws Exception { int leaderId = localId + 1; Set voters = Utils.mkSet(localId, leaderId); int epoch = 2; - OffsetAndEpoch snapshotId = new OffsetAndEpoch(3L, 1); + OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 1); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .appendToLog(snapshotId.epoch, Arrays.asList("a", "b", "c")) @@ -122,7 +123,7 @@ public void testSecondListenerNotified() throws Exception { int leaderId = localId + 1; Set voters = Utils.mkSet(localId, leaderId); int epoch = 2; - OffsetAndEpoch snapshotId = new OffsetAndEpoch(3L, 1); + OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 1); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .appendToLog(snapshotId.epoch, Arrays.asList("a", "b", "c")) @@ -174,16 +175,15 @@ public void testListenerRenotified() throws Exception { context.becomeLeader(); int epoch = context.currentEpoch(); - // Stop the the listener from reading commit batches + // Stop the listener from reading commit batches context.listener.updateReadCommit(false); // Advance the highWatermark - context.deliverRequest( - context.fetchRequest(epoch, otherNodeId, snapshotId.offset, snapshotId.epoch, 0) - ); + long localLogEndOffset = context.log.endOffset().offset; + context.deliverRequest(context.fetchRequest(epoch, otherNodeId, localLogEndOffset, epoch, 0)); context.pollUntilResponse(); context.assertSentFetchPartitionResponse(Errors.NONE, epoch, OptionalInt.of(localId)); - assertEquals(snapshotId.offset, context.client.highWatermark().getAsLong()); + assertEquals(localLogEndOffset, context.client.highWatermark().getAsLong()); // Check that listener was notified of the new snapshot try (SnapshotReader snapshot = context.listener.takeSnapshot().get()) { @@ -192,10 +192,14 @@ public void testListenerRenotified() throws Exception { } // Generate a new snapshot - OffsetAndEpoch secondSnapshot = new OffsetAndEpoch(6, snapshotId.epoch); - try (SnapshotWriter snapshot = context.client.createSnapshot(snapshotId)) { + OffsetAndEpoch secondSnapshot = new OffsetAndEpoch(localLogEndOffset, epoch); + try (SnapshotWriter snapshot = context.client.createSnapshot(secondSnapshot)) { snapshot.freeze(); } + context.client.poll(); + + // Resume the listener from reading commit batches + context.listener.updateReadCommit(true); context.client.poll(); // Check that listener was notified of the second snapshot @@ -265,6 +269,7 @@ public void testFetchRequestWithLargerLastFetchedEpoch() throws Exception { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .appendToLog(oldestSnapshotId.epoch, Arrays.asList("a", "b", "c")) + .appendToLog(oldestSnapshotId.epoch, Arrays.asList("d", "e", "f")) .withAppendLingerMs(1) .build(); @@ -285,7 +290,7 @@ public void testFetchRequestWithLargerLastFetchedEpoch() throws Exception { } context.client.poll(); - context.client.scheduleAppend(epoch, Arrays.asList("d", "e", "f")); + context.client.scheduleAppend(epoch, Arrays.asList("g", "h", "i")); context.time.sleep(context.appendLingerMs()); context.client.poll(); @@ -352,7 +357,8 @@ public void testFetchRequestAtLogStartOffsetWithValidEpoch() throws Exception { 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("d", "e", "f")) + .appendToLog(oldestSnapshotId.epoch + 2, Arrays.asList("g", "h", "i")) .withAppendLingerMs(1) .build(); @@ -393,7 +399,8 @@ public void testFetchRequestAtLogStartOffsetWithInvalidEpoch() throws Exception 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("d", "e", "f")) + .appendToLog(oldestSnapshotId.epoch + 2, Arrays.asList("g", "h", "i")) .withAppendLingerMs(1) .build(); diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java index 1c716f42fc082..402ec2a488dcf 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -39,6 +39,7 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -1774,11 +1775,29 @@ public void testLeaderAppendSingleMemberQuorum() throws Exception { // Now try reading it int otherNodeId = 1; - context.deliverRequest(context.fetchRequest(1, otherNodeId, 0L, 0, 500)); - context.pollUntilResponse(); + List batches = new ArrayList<>(2); + boolean appended = true; + + // Continue to fetch until the leader returns an empty response + while (appended) { + long fetchOffset = 0; + int lastFetchedEpoch = 0; + if (!batches.isEmpty()) { + MutableRecordBatch lastBatch = batches.get(batches.size() - 1); + fetchOffset = lastBatch.lastOffset() + 1; + lastFetchedEpoch = lastBatch.partitionLeaderEpoch(); + } + + context.deliverRequest(context.fetchRequest(1, otherNodeId, fetchOffset, lastFetchedEpoch, 0)); + context.pollUntilResponse(); + + MemoryRecords fetchedRecords = context.assertSentFetchPartitionResponse(Errors.NONE, 1, OptionalInt.of(localId)); + List fetchedBatch = Utils.toList(fetchedRecords.batchIterator()); + batches.addAll(fetchedBatch); + + appended = !fetchedBatch.isEmpty(); + } - MemoryRecords fetchedRecords = context.assertSentFetchPartitionResponse(Errors.NONE, 1, OptionalInt.of(localId)); - List batches = Utils.toList(fetchedRecords.batchIterator()); assertEquals(2, batches.size()); MutableRecordBatch leaderChangeBatch = batches.get(0); @@ -2020,18 +2039,24 @@ public void testHandleClaimCallbackFiresAfterHighWatermarkReachesEpochStartOffse // watermark advances and we can start sending committed data to the // listener. context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 9L, 2, 500)); + context.client.poll(); assertEquals(OptionalInt.empty(), context.listener.currentClaimedEpoch()); - assertEquals(3, context.listener.numCommittedBatches()); + assertEquals(1, context.listener.numCommittedBatches()); assertEquals(batch1, context.listener.commitWithBaseOffset(0L)); + + context.client.poll(); + assertEquals(OptionalInt.empty(), context.listener.currentClaimedEpoch()); + assertEquals(2, context.listener.numCommittedBatches()); assertEquals(batch2, context.listener.commitWithBaseOffset(3L)); - assertEquals(batch3, context.listener.commitWithBaseOffset(6L)); - assertEquals(OptionalLong.of(8L), context.listener.lastCommitOffset()); // Now that the listener has caught up to the start of the leader epoch, // we expect the `handleClaim` callback. context.client.poll(); assertEquals(OptionalInt.of(epoch), context.listener.currentClaimedEpoch()); + assertEquals(3, context.listener.numCommittedBatches()); + assertEquals(batch3, context.listener.commitWithBaseOffset(6L)); + assertEquals(OptionalLong.of(8L), context.listener.lastCommitOffset()); } @Test @@ -2057,20 +2082,21 @@ public void testLateRegisteredListenerCatchesUp() throws Exception { // Let the initial listener catch up context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 9L, 2, 500)); - context.client.poll(); + context.pollUntil(() -> OptionalInt.of(epoch).equals(context.listener.currentClaimedEpoch())); assertEquals(OptionalLong.of(9L), context.client.highWatermark()); - context.client.poll(); + assertEquals(OptionalLong.of(8L), context.listener.lastCommitOffset()); assertEquals(OptionalInt.of(epoch), context.listener.currentClaimedEpoch()); + // Ensure that the `handleClaim` callback was not fired early + assertEquals(9L, context.listener.claimedEpochStartOffset(epoch)); // Register a second listener and allow it to catch up to the high watermark RaftClientTestContext.MockListener secondListener = new RaftClientTestContext.MockListener(); context.client.register(secondListener); - context.client.poll(); + context.pollUntil(() -> OptionalInt.of(epoch).equals(secondListener.currentClaimedEpoch())); assertEquals(OptionalLong.of(8L), secondListener.lastCommitOffset()); assertEquals(OptionalInt.of(epoch), context.listener.currentClaimedEpoch()); - // Ensure that the `handleClaim` callback was not fired early - assertEquals(9L, context.listener.claimedEpochStartOffset(epoch)); + assertEquals(9L, secondListener.claimedEpochStartOffset(epoch)); } @Test @@ -2161,6 +2187,7 @@ public void testHandleCommitCallbackFiresInVotedState() throws Exception { context.assertVotedCandidate(candidateEpoch, otherNodeId); // Note the offset is 8 because the record at offset 9 is a control record + context.pollUntil(() -> secondListener.lastCommitOffset().equals(OptionalLong.of(8L))); assertEquals(OptionalLong.of(8L), secondListener.lastCommitOffset()); assertEquals(OptionalInt.empty(), secondListener.currentClaimedEpoch()); } @@ -2208,6 +2235,7 @@ public void testHandleCommitCallbackFiresInCandidateState() throws Exception { context.assertVotedCandidate(candidateEpoch, localId); // Note the offset is 8 because the record at offset 9 is a control record + context.pollUntil(() -> secondListener.lastCommitOffset().equals(OptionalLong.of(8L))); assertEquals(OptionalLong.of(8L), secondListener.lastCommitOffset()); assertEquals(OptionalInt.empty(), secondListener.currentClaimedEpoch()); } diff --git a/raft/src/test/java/org/apache/kafka/raft/MockLog.java b/raft/src/test/java/org/apache/kafka/raft/MockLog.java index a448e8baa5011..96e2735bfcc55 100644 --- a/raft/src/test/java/org/apache/kafka/raft/MockLog.java +++ b/raft/src/test/java/org/apache/kafka/raft/MockLog.java @@ -358,33 +358,30 @@ public LogFetchInfo read(long startOffset, Isolation isolation) { } ByteBuffer buffer = ByteBuffer.allocate(512); - LogEntry firstEntry = null; + LogOffsetMetadata batchStartOffset = null; for (LogBatch batch : batches) { // Note that start offset is inclusive while max offset is exclusive. We only return // complete batches, so batches which end at an offset larger than the max offset are // filtered, which is effectively the same as having the consumer drop an incomplete // batch returned in a fetch response. - if (batch.lastOffset() >= startOffset) { - if (batch.lastOffset() < maxOffset) { - buffer = batch.writeTo(buffer); - } + if (batch.lastOffset() >= startOffset && batch.lastOffset() < maxOffset && !batch.entries.isEmpty()) { + buffer = batch.writeTo(buffer); + batchStartOffset = batch.entries.get(0).logOffsetMetadata(); - if (firstEntry == null && !batch.entries.isEmpty()) { - firstEntry = batch.entries.get(0); - } + break; } } buffer.flip(); Records records = MemoryRecords.readableRecords(buffer); - if (firstEntry == null) { + if (batchStartOffset == null) { throw new RuntimeException("Expected to find at least one entry starting from offset " + startOffset + " but found none"); } - return new LogFetchInfo(records, firstEntry.logOffsetMetadata()); + return new LogFetchInfo(records, batchStartOffset); } @Override @@ -590,6 +587,11 @@ ByteBuffer writeTo(ByteBuffer buffer) { builder.close(); return builder.buffer(); } + + @Override + public String toString() { + return String.format("LogBatch(entries=%s, epoch=%s, isControlBatch=%s)", entries, epoch, isControlBatch); + } } private static class EpochStartOffset { diff --git a/raft/src/test/java/org/apache/kafka/raft/MockLogTest.java b/raft/src/test/java/org/apache/kafka/raft/MockLogTest.java index 094dcf6de208c..aaae775d5b356 100644 --- a/raft/src/test/java/org/apache/kafka/raft/MockLogTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/MockLogTest.java @@ -91,18 +91,25 @@ public void testAppendAsLeaderHelper() { records = log.read(0, Isolation.UNCOMMITTED).records; batches = Utils.toList(records.batches().iterator()); - assertEquals(2, batches.size()); + assertEquals(1, batches.size()); fetchedRecords = Utils.toList(records.records().iterator()); - assertEquals(3, fetchedRecords.size()); + assertEquals(1, fetchedRecords.size()); assertEquals(recordOne, new SimpleRecord(fetchedRecords.get(0))); assertEquals(0, fetchedRecords.get(0).offset()); - assertEquals(recordTwo, new SimpleRecord(fetchedRecords.get(1))); - assertEquals(1, fetchedRecords.get(1).offset()); + records = log.read(fetchedRecords.size(), Isolation.UNCOMMITTED).records; + batches = Utils.toList(records.batches().iterator()); + assertEquals(1, batches.size()); + + fetchedRecords = Utils.toList(records.records().iterator()); + assertEquals(2, fetchedRecords.size()); + + assertEquals(recordTwo, new SimpleRecord(fetchedRecords.get(0))); + assertEquals(1, fetchedRecords.get(0).offset()); - assertEquals(recordThree, new SimpleRecord(fetchedRecords.get(2))); - assertEquals(2, fetchedRecords.get(2).offset()); + assertEquals(recordThree, new SimpleRecord(fetchedRecords.get(1))); + assertEquals(2, fetchedRecords.get(1).offset()); } @Test @@ -595,15 +602,32 @@ public void testDoesntTruncateFully() throws IOException { } private Optional readOffsets(long startOffset, Isolation isolation) { - Records records = log.read(startOffset, isolation).records; + // The current MockLog implementation reads at most one batch + long firstReadOffset = -1L; long lastReadOffset = -1L; - for (Record record : records.records()) { - if (firstReadOffset < 0) - firstReadOffset = record.offset(); - if (record.offset() > lastReadOffset) - lastReadOffset = record.offset(); + + long currentStart = startOffset; + boolean foundRecord = true; + while (foundRecord) { + foundRecord = false; + + Records records = log.read(currentStart, isolation).records; + for (Record record : records.records()) { + foundRecord = true; + + if (firstReadOffset < 0L) { + firstReadOffset = record.offset(); + } + + if (record.offset() > lastReadOffset) { + lastReadOffset = record.offset(); + } + } + + currentStart = lastReadOffset + 1; } + if (firstReadOffset < 0) { return Optional.empty(); } else { @@ -633,6 +657,11 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(startOffset, endOffset); } + + @Override + public String toString() { + return String.format("OffsetRange(startOffset=%s, endOffset=%s)", startOffset, endOffset); + } } private void appendAsLeader(Collection records, int epoch) { diff --git a/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java b/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java index 739e0278d5cda..4a3984c094985 100644 --- a/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java +++ b/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java @@ -43,11 +43,13 @@ import org.apache.kafka.queue.KafkaEventQueue; import org.apache.kafka.raft.BatchReader; import org.apache.kafka.raft.RaftClient; +import org.apache.kafka.snapshot.SnapshotReader; import org.apache.kafka.shell.MetadataNode.DirectoryNode; import org.apache.kafka.shell.MetadataNode.FileNode; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.function.Consumer; @@ -105,6 +107,25 @@ public void handleCommits(long lastOffset, List messages) { }, null); } + @Override + public void handleSnapshot(SnapshotReader reader) { + // TODO: Create Jira: Need to cover the case where handle snapshot invalidates previous commits + // Need to handle that reader.snapshotId() means that every record up to that offset is committed + try { + for (List batch : reader) { + for (ApiMessageAndVersion messageAndVersion : batch) { + handleMessage(messageAndVersion.message()); + } + } + } finally { + try { + reader.close(); + } catch (IOException e) { + log.error("Unable to close snapshot {}", reader.snapshotId(), e); + } + } + } + @Override public void handleNewLeader(MetaLogLeader leader) { appendEvent("handleNewLeader", () -> { From c72053bbedb176ed32df4ee1e0c3eb9f601f7881 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Thu, 11 Mar 2021 12:36:00 -0800 Subject: [PATCH 07/19] Change oldest to earliest --- .../java/org/apache/kafka/raft/KafkaRaftClient.java | 11 +++-------- .../apache/kafka/raft/metadata/MetaLogRaftShim.java | 3 +-- 2 files changed, 4 insertions(+), 10 deletions(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index b960da9549d99..7c744c61b27ae 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -138,11 +138,6 @@ * than the leader's log start offset. This API is similar to the Fetch API since the snapshot is stored * as FileRecords, but we use {@link UnalignedRecords} in FetchSnapshotResponse because the records * are not necessarily offset-aligned. - * - * 5) {@link FetchSnapshotRequestData}: Sent by the followers to the epoch leader when fetching a snapshot. - * Followers need to fetch snapshots when the follower's log end offset is less than the leader's log - * start offset. The follower discover this case when the leader replies with a fetch response that - * contains a snapshot id. */ public class KafkaRaftClient implements RaftClient { private static final int RETRY_BACKOFF_BASE_MS = 100; @@ -2123,7 +2118,7 @@ private long pollUnattachedAsObserver(UnattachedState state, long currentTimeMs) } private long pollCurrentState(long currentTimeMs) throws IOException { - maybeUpdateOldestSnapshotId(); + maybeUpdateEarliestSnapshotId(); if (quorum.isLeader()) { return pollLeader(currentTimeMs); @@ -2187,7 +2182,7 @@ private boolean maybeCompleteShutdown(long currentTimeMs) { return false; } - private void maybeUpdateOldestSnapshotId() { + private void maybeUpdateEarliestSnapshotId() { log.latestSnapshotId().ifPresent(snapshotId -> { quorum.highWatermark().ifPresent(highWatermark -> { if (highWatermark.offset >= snapshotId.offset) { @@ -2392,7 +2387,7 @@ public synchronized OptionalLong nextExpectedOffset() { /** * This API is used when the Listener needs to be notified of a new snapshot. This happens - * when the context's next offset is less than then log start offset. + * when the context's next offset is less than the log start offset. */ public void fireHandleSnapshot(SnapshotReader reader) { synchronized (this) { diff --git a/raft/src/main/java/org/apache/kafka/raft/metadata/MetaLogRaftShim.java b/raft/src/main/java/org/apache/kafka/raft/metadata/MetaLogRaftShim.java index f35a121978727..37483c3c4112e 100644 --- a/raft/src/main/java/org/apache/kafka/raft/metadata/MetaLogRaftShim.java +++ b/raft/src/main/java/org/apache/kafka/raft/metadata/MetaLogRaftShim.java @@ -128,11 +128,10 @@ public void handleCommit(BatchReader reader) { @Override public void handleSnapshot(SnapshotReader reader) { - // TODO: Implement this + // TODO: Create Jira: Handle loading commit in ListenerShim try { reader.close(); } catch (IOException e) { - // TODO: handle exception } } From 259ea30f0815d0609e1c6560e31876ca1ac8cc1c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Fri, 19 Mar 2021 10:02:42 -0700 Subject: [PATCH 08/19] Add Serde based Records iterator --- .../org/apache/kafka/raft/BatchReader.java | 17 +- .../apache/kafka/raft/KafkaRaftClient.java | 12 +- .../raft/internals/RecordsBatchReader.java | 174 ++++---------- .../raft/internals/SerdeRecordsIterator.java | 216 ++++++++++++++++++ .../raft/internals/MemoryBatchReaderTest.java | 19 +- .../internals/RecordsBatchReaderTest.java | 16 +- 6 files changed, 296 insertions(+), 158 deletions(-) create mode 100644 raft/src/main/java/org/apache/kafka/raft/internals/SerdeRecordsIterator.java diff --git a/raft/src/main/java/org/apache/kafka/raft/BatchReader.java b/raft/src/main/java/org/apache/kafka/raft/BatchReader.java index e5f9e38612a0d..3c8fc300086e8 100644 --- a/raft/src/main/java/org/apache/kafka/raft/BatchReader.java +++ b/raft/src/main/java/org/apache/kafka/raft/BatchReader.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.raft; +import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Objects; @@ -63,16 +64,18 @@ public interface BatchReader extends Iterator>, AutoClos class Batch { private final long baseOffset; private final int epoch; + private final long lastOffset; private final List records; - public Batch(long baseOffset, int epoch, List records) { + private Batch(long baseOffset, int epoch, long lastOffset, List records) { this.baseOffset = baseOffset; this.epoch = epoch; + this.lastOffset = lastOffset; this.records = records; } public long lastOffset() { - return baseOffset + records.size() - 1; + return lastOffset; } public long baseOffset() { @@ -92,6 +95,7 @@ public String toString() { return "Batch(" + "baseOffset=" + baseOffset + ", epoch=" + epoch + + ", lastOffset=" + lastOffset + ", records=" + records + ')'; } @@ -110,6 +114,13 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(baseOffset, epoch, records); } - } + public static Batch empty(long baseOffset, int epoch, long lastOffset) { + return new Batch<>(baseOffset, epoch, lastOffset, Collections.emptyList()); + } + + public static Batch of(long baseOffset, int epoch, List records) { + return new Batch<>(baseOffset, epoch, baseOffset + records.size(), records); + } + } } diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 7c744c61b27ae..e7585fc8aa649 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -69,6 +69,7 @@ import org.apache.kafka.raft.internals.KafkaRaftMetrics; import org.apache.kafka.raft.internals.MemoryBatchReader; import org.apache.kafka.raft.internals.RecordsBatchReader; +import org.apache.kafka.raft.internals.SerdeRecordsIterator; import org.apache.kafka.raft.internals.ThresholdPurgatory; import org.apache.kafka.snapshot.RawSnapshotReader; import org.apache.kafka.snapshot.RawSnapshotWriter; @@ -2405,9 +2406,12 @@ public void fireHandleSnapshot(SnapshotReader reader) { * data in memory, we let the state machine read the records from disk. */ public void fireHandleCommit(long baseOffset, Records records) { - BufferSupplier bufferSupplier = BufferSupplier.create(); - RecordsBatchReader reader = new RecordsBatchReader<>(baseOffset, records, - serde, bufferSupplier, this); + // TODO: The max batch size is not techinically correct that the accumulator may create a bigger batch + RecordsBatchReader reader = new RecordsBatchReader<>( + baseOffset, + new SerdeRecordsIterator<>(records, serde, BufferSupplier.create(), MAX_BATCH_SIZE_BYTES), + this + ); fireHandleCommit(reader); } @@ -2419,7 +2423,7 @@ public void fireHandleCommit(long baseOffset, Records records) { * followers. */ public void fireHandleCommit(long baseOffset, int epoch, List records) { - BatchReader.Batch batch = new BatchReader.Batch<>(baseOffset, epoch, records); + BatchReader.Batch batch = BatchReader.Batch.of(baseOffset, epoch, records); MemoryBatchReader reader = new MemoryBatchReader<>(Collections.singletonList(batch), this); fireHandleCommit(reader); } diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java index 0817138e3ff0a..6b16d22812155 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java @@ -16,136 +16,55 @@ */ package org.apache.kafka.raft.internals; -import org.apache.kafka.common.protocol.DataInputStreamReadable; -import org.apache.kafka.common.protocol.Readable; -import org.apache.kafka.common.utils.BufferSupplier; -import org.apache.kafka.common.record.DefaultRecordBatch; -import org.apache.kafka.common.record.FileRecords; -import org.apache.kafka.common.record.MemoryRecords; -import org.apache.kafka.common.record.MutableRecordBatch; -import org.apache.kafka.common.record.Records; import org.apache.kafka.raft.BatchReader; -import org.apache.kafka.raft.RecordSerde; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; import java.util.NoSuchElementException; +import java.util.Optional; import java.util.OptionalLong; public class RecordsBatchReader implements BatchReader { private final long baseOffset; - private final Records records; - private final RecordSerde serde; - private final BufferSupplier bufferSupplier; + private final SerdeRecordsIterator iterator; private final CloseListener> closeListener; - private Iterator batchIterator; private long lastReturnedOffset; - private Batch nextBatch; + + private Optional> nextBatch = Optional.empty(); private boolean isClosed = false; - private ByteBuffer allocatedBuffer = null; public RecordsBatchReader( long baseOffset, - Records records, - RecordSerde serde, - BufferSupplier bufferSupplier, + SerdeRecordsIterator iterator, CloseListener> closeListener ) { this.baseOffset = baseOffset; - this.records = records; - this.serde = serde; - this.bufferSupplier = bufferSupplier; + this.iterator = iterator; this.closeListener = closeListener; this.lastReturnedOffset = baseOffset; } - private void materializeIterator() throws IOException { - if (records instanceof MemoryRecords) { - batchIterator = ((MemoryRecords) records).batchIterator(); - } else if (records instanceof FileRecords) { - this.allocatedBuffer = bufferSupplier.get(records.sizeInBytes()); - ((FileRecords) records).readInto(allocatedBuffer, 0); - MemoryRecords memRecords = MemoryRecords.readableRecords(allocatedBuffer); - batchIterator = memRecords.batchIterator(); - } else { - throw new IllegalStateException("Unexpected Records type " + records.getClass()); - } - } - - private void findNextDataBatch() { - if (batchIterator == null) { - try { - materializeIterator(); - } catch (IOException e) { - throw new RuntimeException("Failed to read records into memory", e); - } - } - - while (batchIterator.hasNext()) { - MutableRecordBatch nextBatch = batchIterator.next(); - if (!(nextBatch instanceof DefaultRecordBatch)) { - throw new IllegalStateException(); - } - - DefaultRecordBatch batch = (DefaultRecordBatch) nextBatch; - if (!batch.isControlBatch()) { - this.nextBatch = readBatch(batch); - return; - } else { - this.lastReturnedOffset = batch.lastOffset(); - } - } - } - - private Batch readBatch(DefaultRecordBatch batch) { - Integer numRecords = batch.countOrNull(); - if (numRecords == null) { - throw new IllegalStateException(); - } - - List records = new ArrayList<>(numRecords); - try (DataInputStreamReadable input = new DataInputStreamReadable( - batch.recordInputStream(bufferSupplier))) { - for (int i = 0; i < numRecords; i++) { - T record = readRecord(input); - records.add(record); - } - return new Batch<>( - batch.baseOffset(), - batch.partitionLeaderEpoch(), - records - ); - } - } - @Override public boolean hasNext() { - if (nextBatch != null) { - return true; - } else { - findNextDataBatch(); - return nextBatch != null; + checkIfClosed(); + + if (!nextBatch.isPresent()) { + nextBatch = nextBatch(); } + + return nextBatch.isPresent(); } @Override public Batch next() { - if (nextBatch != null) { - Batch res = nextBatch; - nextBatch = null; - lastReturnedOffset = res.lastOffset(); - return res; - } else { - findNextDataBatch(); - if (nextBatch == null) { - throw new NoSuchElementException(); - } - return next(); + if (hasNext()) { + throw new NoSuchElementException("Records batch reader doesn't have any more elements"); } + + Batch batch = nextBatch.get(); + nextBatch = Optional.empty(); + + lastReturnedOffset = batch.lastOffset(); + return batch; } @Override @@ -153,6 +72,7 @@ public long baseOffset() { return baseOffset; } + // TODO: rename this property it is misleading. It is not techinically the last offset of the batch reader public OptionalLong lastOffset() { if (isClosed) { return OptionalLong.of(lastReturnedOffset); @@ -163,48 +83,32 @@ public OptionalLong lastOffset() { @Override public void close() { - isClosed = true; + if (!isClosed) { + isClosed = true; - if (allocatedBuffer != null) { - bufferSupplier.release(allocatedBuffer); + // TODO: this closes the iterator even though it was passed through the constructor + iterator.close(); + closeListener.onClose(this); } - - closeListener.onClose(this); } - public T readRecord(Readable input) { - // Read size of body in bytes - input.readVarint(); - - // Read unused attributes - input.readByte(); - - long timestampDelta = input.readVarlong(); - if (timestampDelta != 0) { - throw new IllegalArgumentException(); - } - - // Read offset delta - input.readVarint(); - - int keySize = input.readVarint(); - if (keySize != -1) { - throw new IllegalArgumentException("Unexpected key size " + keySize); - } - - int valueSize = input.readVarint(); - if (valueSize < 0) { - throw new IllegalArgumentException(); + private void checkIfClosed() { + if (isClosed) { + throw new IllegalStateException("Records batch reader was closed"); } + } - T record = serde.read(input, valueSize); + private Optional> nextBatch() { + while (iterator.hasNext()) { + Batch batch = iterator.next(); - int numHeaders = input.readVarint(); - if (numHeaders != 0) { - throw new IllegalArgumentException(); + if (batch.records().isEmpty()) { + lastReturnedOffset = batch.lastOffset(); + } else { + return Optional.of(batch); + } } - return record; + return Optional.empty(); } - } diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/SerdeRecordsIterator.java b/raft/src/main/java/org/apache/kafka/raft/internals/SerdeRecordsIterator.java new file mode 100644 index 0000000000000..5de95a1001827 --- /dev/null +++ b/raft/src/main/java/org/apache/kafka/raft/internals/SerdeRecordsIterator.java @@ -0,0 +1,216 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.raft.internals; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.Optional; +import org.apache.kafka.common.protocol.DataInputStreamReadable; +import org.apache.kafka.common.protocol.Readable; +import org.apache.kafka.common.record.DefaultRecordBatch; +import org.apache.kafka.common.record.FileRecords; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.MutableRecordBatch; +import org.apache.kafka.common.record.Records; +import org.apache.kafka.common.utils.BufferSupplier; +import org.apache.kafka.raft.BatchReader.Batch; +import org.apache.kafka.raft.RecordSerde; + +public final class SerdeRecordsIterator implements Iterator>, AutoCloseable { + private final Records records; + private final RecordSerde serde; + private final BufferSupplier bufferSupplier; + private final int maxBatchSize; + + private Optional> nextBatches = Optional.empty(); + private Optional> nextBatch = Optional.empty(); + // Buffer used to as the backing store for nextBatches if needed + private Optional allocatedBuffer = Optional.empty(); + // Number of bytes from records that read + private int bytesRead = 0; + private boolean isClosed = false; + + public SerdeRecordsIterator( + Records records, + RecordSerde serde, + BufferSupplier bufferSupplier, + int maxBatchSize + ) { + this.records = records; + this.serde = serde; + this.bufferSupplier = bufferSupplier; + this.maxBatchSize = maxBatchSize; + } + + @Override + public boolean hasNext() { + checkIfClosed(); + + if (!nextBatch.isPresent()) { + nextBatch = nextBatch(); + } + + return nextBatch.isPresent(); + } + + @Override + public Batch next() { + if (hasNext()) { + throw new NoSuchElementException("Batch iterator doesn't have any more elements"); + } + + Batch batch = nextBatch.get(); + nextBatch = Optional.empty(); + + return batch; + } + + @Override + public void close() { + isClosed = true; + allocatedBuffer.ifPresent(bufferSupplier::release); + allocatedBuffer = Optional.empty(); + } + + private void checkIfClosed() { + if (isClosed) { + throw new IllegalStateException("Serde record batch itererator was closed"); + } + } + + private Optional> nextBatches() { + int recordSize = records.sizeInBytes(); + if (bytesRead < recordSize) { + final MemoryRecords memoryRecords; + if (records instanceof MemoryRecords) { + bytesRead = recordSize; + memoryRecords = (MemoryRecords) records; + } else if (records instanceof FileRecords) { + final ByteBuffer buffer; + if (allocatedBuffer.isPresent()) { + buffer = allocatedBuffer.get(); + buffer.compact(); + } else { + buffer = bufferSupplier.get(Math.min(maxBatchSize, records.sizeInBytes())); + allocatedBuffer = Optional.of(buffer); + } + + int start = buffer.position(); + try { + ((FileRecords) records).readInto(buffer, bytesRead); + } catch (IOException e) { + throw new RuntimeException("Failed to read records into memory", e); + } + + bytesRead += buffer.position() - start; + memoryRecords = MemoryRecords.readableRecords(buffer); + } else { + throw new IllegalStateException(String.format("Unexpected Records type %s", records.getClass())); + } + + return Optional.of(memoryRecords.batchIterator()); + } else { + return Optional.empty(); + } + } + + private Optional> nextBatch() { + if (!nextBatches.isPresent()) { + nextBatches = nextBatches(); + } + + while (nextBatches.isPresent()) { + while (nextBatches.get().hasNext()) { + MutableRecordBatch nextBatch = nextBatches.get().next(); + if (!(nextBatch instanceof DefaultRecordBatch)) { + throw new IllegalStateException( + String.format("DefaultRecordBatch expected by record type was %s", nextBatch.getClass()) + ); + } + + return Optional.of(readBatch((DefaultRecordBatch) nextBatch)); + } + + nextBatches(); + } + + return Optional.empty(); + } + + private Batch readBatch(DefaultRecordBatch batch) { + final Batch result; + if (batch.isControlBatch()) { + result = Batch.empty(batch.baseOffset(), batch.partitionLeaderEpoch(), batch.lastOffset()); + } else { + Integer numRecords = batch.countOrNull(); + if (numRecords == null) { + throw new IllegalStateException("Expected a record count for the records batch"); + } + + List records = new ArrayList<>(numRecords); + try (DataInputStreamReadable input = new DataInputStreamReadable(batch.recordInputStream(bufferSupplier))) { + for (int i = 0; i < numRecords; i++) { + T record = readRecord(input); + records.add(record); + } + } + + result = Batch.of(batch.baseOffset(), batch.partitionLeaderEpoch(), records); + } + + return result; + } + + private T readRecord(Readable input) { + // Read size of body in bytes + input.readVarint(); + + // Read unused attributes + input.readByte(); + + long timestampDelta = input.readVarlong(); + if (timestampDelta != 0) { + throw new IllegalArgumentException(); + } + + // Read offset delta + input.readVarint(); + + int keySize = input.readVarint(); + if (keySize != -1) { + throw new IllegalArgumentException("Unexpected key size " + keySize); + } + + int valueSize = input.readVarint(); + if (valueSize < 0) { + throw new IllegalArgumentException(); + } + + T record = serde.read(input, valueSize); + + int numHeaders = input.readVarint(); + if (numHeaders != 0) { + throw new IllegalArgumentException(); + } + + return record; + } +} diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/MemoryBatchReaderTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/MemoryBatchReaderTest.java index 90cb52714198b..255b759cce306 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/MemoryBatchReaderTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/MemoryBatchReaderTest.java @@ -31,17 +31,22 @@ class MemoryBatchReaderTest { @Test public void testIteration() { - BatchReader.Batch batch1 = new BatchReader.Batch<>(0L, 1, - Arrays.asList("a", "b", "c")); - BatchReader.Batch batch2 = new BatchReader.Batch<>(3L, 2, - Arrays.asList("d", "e")); - BatchReader.Batch batch3 = new BatchReader.Batch<>(5L, 2, - Arrays.asList("f", "g", "h", "i")); + BatchReader.Batch batch1 = BatchReader.Batch.of( + 0L, 1, Arrays.asList("a", "b", "c") + ); + BatchReader.Batch batch2 = BatchReader.Batch.of( + 3L, 2, Arrays.asList("d", "e") + ); + BatchReader.Batch batch3 = BatchReader.Batch.of( + 5L, 2, Arrays.asList("f", "g", "h", "i") + ); @SuppressWarnings("unchecked") CloseListener> listener = Mockito.mock(CloseListener.class); MemoryBatchReader reader = new MemoryBatchReader<>( - Arrays.asList(batch1, batch2, batch3), listener); + Arrays.asList(batch1, batch2, batch3), + listener + ); assertEquals(0L, reader.baseOffset()); assertEquals(OptionalLong.of(8L), reader.lastOffset()); diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsBatchReaderTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsBatchReaderTest.java index 78ffd51befaf5..a180c76375c62 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsBatchReaderTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsBatchReaderTest.java @@ -53,9 +53,9 @@ public void testReadFromMemoryRecords(CompressionType compressionType) { long baseOffset = 57; List> batches = asList( - new BatchReader.Batch<>(baseOffset, 1, asList("a", "b", "c")), - new BatchReader.Batch<>(baseOffset + 3, 2, asList("d", "e")), - new BatchReader.Batch<>(baseOffset + 5, 2, asList("f")) + BatchReader.Batch.of(baseOffset, 1, asList("a", "b", "c")), + BatchReader.Batch.of(baseOffset + 3, 2, asList("d", "e")), + BatchReader.Batch.of(baseOffset + 5, 2, asList("f")) ); MemoryRecords memRecords = buildRecords(compressionType, batches); @@ -68,9 +68,9 @@ public void testReadFromFileRecords(CompressionType compressionType) throws Exce long baseOffset = 57; List> batches = asList( - new BatchReader.Batch<>(baseOffset, 1, asList("a", "b", "c")), - new BatchReader.Batch<>(baseOffset + 3, 2, asList("d", "e")), - new BatchReader.Batch<>(baseOffset + 5, 2, asList("f")) + BatchReader.Batch.of(baseOffset, 1, asList("a", "b", "c")), + BatchReader.Batch.of(baseOffset + 3, 2, asList("d", "e")), + BatchReader.Batch.of(baseOffset + 5, 2, asList("f")) ); MemoryRecords memRecords = buildRecords(compressionType, batches); @@ -136,9 +136,7 @@ private void testBatchReader( RecordsBatchReader reader = new RecordsBatchReader<>( baseOffset, - records, - serde, - bufferSupplier, + new SerdeRecordsIterator<>(records, serde, bufferSupplier, Integer.MAX_VALUE), closeListener ); From 5b9805bf2c03dc21816e0146174d12a42981b561 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Tue, 23 Mar 2021 20:29:38 -0700 Subject: [PATCH 09/19] Add fix the buffer allocation pattern --- .../org/apache/kafka/raft/BatchReader.java | 9 +- .../apache/kafka/raft/KafkaRaftClient.java | 31 ++-- .../apache/kafka/raft/ReplicatedCounter.java | 5 +- .../raft/internals/RecordsBatchReader.java | 25 ++- .../raft/internals/SerdeRecordsIterator.java | 16 +- .../kafka/snapshot/FileRawSnapshotReader.java | 13 +- .../kafka/snapshot/RawSnapshotReader.java | 22 ++- .../apache/kafka/snapshot/SnapshotReader.java | 140 ++++---------- .../raft/KafkaRaftClientSnapshotTest.java | 4 +- .../java/org/apache/kafka/raft/MockLog.java | 13 +- .../internals/RecordsBatchReaderTest.java | 56 +----- .../internals/SerdeRecordsIteratorTest.java | 171 ++++++++++++++++++ .../kafka/snapshot/FileRawSnapshotTest.java | 62 ++++--- .../snapshot/SnapshotWriterReaderTest.java | 35 ++-- .../kafka/shell/MetadataNodeManager.java | 3 +- 15 files changed, 356 insertions(+), 249 deletions(-) create mode 100644 raft/src/test/java/org/apache/kafka/raft/internals/SerdeRecordsIteratorTest.java diff --git a/raft/src/main/java/org/apache/kafka/raft/BatchReader.java b/raft/src/main/java/org/apache/kafka/raft/BatchReader.java index 3c8fc300086e8..cf01c039c08a6 100644 --- a/raft/src/main/java/org/apache/kafka/raft/BatchReader.java +++ b/raft/src/main/java/org/apache/kafka/raft/BatchReader.java @@ -61,7 +61,7 @@ public interface BatchReader extends Iterator>, AutoClos @Override void close(); - class Batch { + final class Batch implements Iterable { private final long baseOffset; private final int epoch; private final long lastOffset; @@ -90,6 +90,11 @@ public int epoch() { return epoch; } + @Override + public Iterator iterator() { + return records.iterator(); + } + @Override public String toString() { return "Batch(" + @@ -120,7 +125,7 @@ public static Batch empty(long baseOffset, int epoch, long lastOffset) { } public static Batch of(long baseOffset, int epoch, List records) { - return new Batch<>(baseOffset, epoch, baseOffset + records.size(), records); + return new Batch<>(baseOffset, epoch, baseOffset + records.size() - 1, records); } } } diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index e7585fc8aa649..7194425f8afe5 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -69,7 +69,6 @@ import org.apache.kafka.raft.internals.KafkaRaftMetrics; import org.apache.kafka.raft.internals.MemoryBatchReader; import org.apache.kafka.raft.internals.RecordsBatchReader; -import org.apache.kafka.raft.internals.SerdeRecordsIterator; import org.apache.kafka.raft.internals.ThresholdPurgatory; import org.apache.kafka.snapshot.RawSnapshotReader; import org.apache.kafka.snapshot.RawSnapshotWriter; @@ -340,9 +339,10 @@ private void updateListenersProgress(List listenerContexts, lon private Optional> earliestSnapshot() { return log.earliestSnapshotId().flatMap(earliestSnapshoId -> { try { + // TODO: The buffer size is not technically correct. The BatchAccumulator allows for bigger buffers return log .readSnapshot(earliestSnapshoId) - .map(reader -> new SnapshotReader<>(reader, serde, BufferSupplier.create())); + .map(reader -> SnapshotReader.of(reader, serde, BufferSupplier.create(), MAX_BATCH_SIZE_BYTES)); } catch (IOException e) { logger.error( String.format("Unable to read snapshot: %s", earliestSnapshoId), @@ -1280,7 +1280,8 @@ private FetchSnapshotResponseData handleFetchSnapshotRequest( } try (RawSnapshotReader snapshot = snapshotOpt.get()) { - if (partitionSnapshot.position() < 0 || partitionSnapshot.position() >= snapshot.sizeInBytes()) { + long snapshotSize = snapshot.sizeInBytes(); + if (partitionSnapshot.position() < 0 || partitionSnapshot.position() >= snapshotSize) { return FetchSnapshotResponse.singleton( log.topicPartition(), responsePartitionSnapshot -> addQuorumLeader(responsePartitionSnapshot) @@ -1290,18 +1291,18 @@ private FetchSnapshotResponseData handleFetchSnapshotRequest( int maxSnapshotSize; try { - maxSnapshotSize = Math.toIntExact(snapshot.sizeInBytes()); + maxSnapshotSize = Math.toIntExact(snapshotSize); } catch (ArithmeticException e) { maxSnapshotSize = Integer.MAX_VALUE; } if (partitionSnapshot.position() > Integer.MAX_VALUE) { + // TODO: This should return an error response instead of throwing an exception throw new IllegalStateException(String.format("Trying to fetch a snapshot with position: %d lager than Int.MaxValue", partitionSnapshot.position())); } - UnalignedRecords records = snapshot.read(partitionSnapshot.position(), Math.min(data.maxBytes(), maxSnapshotSize)); - - long snapshotSize = snapshot.sizeInBytes(); + // TODO: I think this slice of records is closed when the snapshot is close in the try (...) above. + UnalignedRecords records = snapshot.slice(partitionSnapshot.position(), Math.min(data.maxBytes(), maxSnapshotSize)); return FetchSnapshotResponse.singleton( log.topicPartition(), @@ -2406,13 +2407,17 @@ public void fireHandleSnapshot(SnapshotReader reader) { * data in memory, we let the state machine read the records from disk. */ public void fireHandleCommit(long baseOffset, Records records) { - // TODO: The max batch size is not techinically correct that the accumulator may create a bigger batch - RecordsBatchReader reader = new RecordsBatchReader<>( - baseOffset, - new SerdeRecordsIterator<>(records, serde, BufferSupplier.create(), MAX_BATCH_SIZE_BYTES), - this + // TODO: The max batch size is not techinically correct. The accumulator may create a bigger batch + fireHandleCommit( + RecordsBatchReader.of( + baseOffset, + records, + serde, + BufferSupplier.create(), + MAX_BATCH_SIZE_BYTES, + this + ) ); - fireHandleCommit(reader); } /** diff --git a/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java b/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java index 85324a3f3ae0d..4631cf74a93b9 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java +++ b/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java @@ -18,11 +18,11 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.raft.BatchReader.Batch; import org.apache.kafka.snapshot.SnapshotReader; import org.slf4j.Logger; import java.io.IOException; -import java.util.List; import java.util.Optional; import static java.util.Collections.singletonList; @@ -101,7 +101,8 @@ public synchronized void handleSnapshot(SnapshotReader reader) { try { try (SnapshotReader snapshot = reader) { log.debug("Loading snapshot {}", snapshot.snapshotId()); - for (List batch : snapshot) { + while (snapshot.hasNext()) { + Batch batch = snapshot.next(); for (Integer value : batch) { log.debug("Setting value: {}", value); this.committed = value; diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java index 6b16d22812155..4a953a10d44b7 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java @@ -16,13 +16,16 @@ */ package org.apache.kafka.raft.internals; +import org.apache.kafka.common.record.Records; +import org.apache.kafka.common.utils.BufferSupplier; import org.apache.kafka.raft.BatchReader; +import org.apache.kafka.raft.RecordSerde; import java.util.NoSuchElementException; import java.util.Optional; import java.util.OptionalLong; -public class RecordsBatchReader implements BatchReader { +public final class RecordsBatchReader implements BatchReader { private final long baseOffset; private final SerdeRecordsIterator iterator; private final CloseListener> closeListener; @@ -32,7 +35,7 @@ public class RecordsBatchReader implements BatchReader { private Optional> nextBatch = Optional.empty(); private boolean isClosed = false; - public RecordsBatchReader( + private RecordsBatchReader( long baseOffset, SerdeRecordsIterator iterator, CloseListener> closeListener @@ -56,7 +59,7 @@ public boolean hasNext() { @Override public Batch next() { - if (hasNext()) { + if (!hasNext()) { throw new NoSuchElementException("Records batch reader doesn't have any more elements"); } @@ -86,12 +89,26 @@ public void close() { if (!isClosed) { isClosed = true; - // TODO: this closes the iterator even though it was passed through the constructor iterator.close(); closeListener.onClose(this); } } + public static RecordsBatchReader of( + long baseOffset, + Records records, + RecordSerde serde, + BufferSupplier bufferSupplier, + int maxBatchSize, + CloseListener> closeListener + ) { + return new RecordsBatchReader<>( + baseOffset, + new SerdeRecordsIterator<>(records, serde, bufferSupplier, maxBatchSize), + closeListener + ); + } + private void checkIfClosed() { if (isClosed) { throw new IllegalStateException("Records batch reader was closed"); diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/SerdeRecordsIterator.java b/raft/src/main/java/org/apache/kafka/raft/internals/SerdeRecordsIterator.java index 5de95a1001827..bf1c08a872147 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/SerdeRecordsIterator.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/SerdeRecordsIterator.java @@ -73,7 +73,7 @@ public boolean hasNext() { @Override public Batch next() { - if (hasNext()) { + if (!hasNext()) { throw new NoSuchElementException("Batch iterator doesn't have any more elements"); } @@ -108,6 +108,7 @@ private Optional> nextBatches() { if (allocatedBuffer.isPresent()) { buffer = allocatedBuffer.get(); buffer.compact(); + // TODO: catch the case where compact didn't do anything } else { buffer = bufferSupplier.get(Math.min(maxBatchSize, records.sizeInBytes())); allocatedBuffer = Optional.of(buffer); @@ -120,8 +121,8 @@ private Optional> nextBatches() { throw new RuntimeException("Failed to read records into memory", e); } - bytesRead += buffer.position() - start; - memoryRecords = MemoryRecords.readableRecords(buffer); + bytesRead += buffer.limit() - start; + memoryRecords = MemoryRecords.readableRecords(buffer.slice()); } else { throw new IllegalStateException(String.format("Unexpected Records type %s", records.getClass())); } @@ -138,18 +139,21 @@ private Optional> nextBatch() { } while (nextBatches.isPresent()) { - while (nextBatches.get().hasNext()) { + if (nextBatches.get().hasNext()) { MutableRecordBatch nextBatch = nextBatches.get().next(); + + // Update the buffer position to reflect the read batch + allocatedBuffer.ifPresent(buffer -> buffer.position(buffer.position() + nextBatch.sizeInBytes())); + if (!(nextBatch instanceof DefaultRecordBatch)) { throw new IllegalStateException( String.format("DefaultRecordBatch expected by record type was %s", nextBatch.getClass()) ); } - return Optional.of(readBatch((DefaultRecordBatch) nextBatch)); } - nextBatches(); + nextBatches = nextBatches(); } return Optional.empty(); diff --git a/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotReader.java b/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotReader.java index d0218c79cc427..820230ee64636 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotReader.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotReader.java @@ -17,14 +17,12 @@ package org.apache.kafka.snapshot; import org.apache.kafka.common.record.FileRecords; -import org.apache.kafka.common.record.RecordBatch; +import org.apache.kafka.common.record.Records; import org.apache.kafka.common.record.UnalignedRecords; -import org.apache.kafka.common.utils.Utils; import org.apache.kafka.raft.OffsetAndEpoch; import java.io.IOException; import java.nio.file.Path; -import java.util.Iterator; public final class FileRawSnapshotReader implements RawSnapshotReader { private final FileRecords fileRecords; @@ -46,12 +44,13 @@ public long sizeInBytes() { } @Override - public Iterator iterator() { - return Utils.covariantCast(fileRecords.batchIterator()); + public UnalignedRecords slice(long position, int size) { + return fileRecords.sliceUnaligned(Math.toIntExact(position), size); } - public UnalignedRecords read(long position, int size) { - return fileRecords.sliceUnaligned(Math.toIntExact(position), size); + @Override + public Records records() { + return fileRecords; } @Override diff --git a/raft/src/main/java/org/apache/kafka/snapshot/RawSnapshotReader.java b/raft/src/main/java/org/apache/kafka/snapshot/RawSnapshotReader.java index 6d1ff28950b33..bbd42e1aacf35 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/RawSnapshotReader.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/RawSnapshotReader.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.snapshot; -import org.apache.kafka.common.record.RecordBatch; +import org.apache.kafka.common.record.Records; import org.apache.kafka.common.record.UnalignedRecords; import org.apache.kafka.raft.OffsetAndEpoch; @@ -26,7 +26,7 @@ /** * Interface for reading snapshots as a sequence of records. */ -public interface RawSnapshotReader extends Closeable, Iterable { +public interface RawSnapshotReader extends Closeable { /** * Returns the end offset and epoch for the snapshot. */ @@ -40,14 +40,16 @@ public interface RawSnapshotReader extends Closeable, Iterable { long sizeInBytes() throws IOException; /** - * Reads bytes from position into the given buffer. + * Creates a slize of unaligned records from the position up to a size. * - * It is not guarantee that the given buffer will be filled. - * - * @param size size to read from snapshot file - * @param position the starting position in the snapshot to read - * @return the region read from snapshot - * @throws IOException for any IO error while reading the snapshot + * @param position the starting position of the slice in the snapshot + * @param size the maximum size of the slice + * @return an unaligned slice of records in the snapshot + */ + UnalignedRecords slice(long position, int size); + + /** + * TODO: */ - UnalignedRecords read(long position, int size) throws IOException; + Records records(); } diff --git a/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java index a6de8f634ae40..7ba394849f123 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java @@ -19,49 +19,47 @@ import java.io.Closeable; import java.io.IOException; -import java.util.ArrayList; import java.util.Iterator; -import java.util.List; -import java.util.NoSuchElementException; -import org.apache.kafka.common.protocol.ByteBufferAccessor; -import org.apache.kafka.common.record.Record; -import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.utils.BufferSupplier; +import org.apache.kafka.raft.BatchReader.Batch; import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.raft.RecordSerde; +import org.apache.kafka.raft.internals.SerdeRecordsIterator; -public final class SnapshotReader implements Closeable, Iterable> { - private final RawSnapshotReader snapshot; - private final RecordSerde serde; - private final BufferSupplier bufferSupplier; +/** + * A type for reading an immutable snapshot. + * + * A snapshot reader can be used to scan through all of the objects T in a snapshot. It + * is assumed that the content of the snapshot represents all of the objects T for the topic + * partition from offset 0 up to but not including the end offset in the snapshot id. + */ +public final class SnapshotReader implements Closeable, Iterator> { + private final OffsetAndEpoch snapshotId; + private final SerdeRecordsIterator iterator; - /** - * A type for reading an immutable snapshot. - * - * A snapshot reader can be used to scan through all of the objects T in a snapshot. It - * is assumed that the content of the snapshot represents all of the objects T for the topic - * partition from offset 0 up to but not including the end offset in the snapshot id. - */ - public SnapshotReader( - RawSnapshotReader snapshot, - RecordSerde serde, - BufferSupplier bufferSupplier + private SnapshotReader( + OffsetAndEpoch snapshotId, + SerdeRecordsIterator iterator ) { - this.snapshot = snapshot; - this.serde = serde; - this.bufferSupplier = bufferSupplier; + this.snapshotId = snapshotId; + this.iterator = iterator; } /** * Returns the end offset and epoch for the snapshot. */ public OffsetAndEpoch snapshotId() { - return snapshot.snapshotId(); + return snapshotId; } @Override - public Iterator> iterator() { - return new SnapshotReaderIterator(snapshot.iterator()); + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public Batch next() { + return iterator.next(); } /** @@ -70,84 +68,18 @@ public Iterator> iterator() { * @throws IOException for any IO error during close */ public void close() throws IOException { - snapshot.close(); + iterator.close(); } - - final class SnapshotReaderIterator implements Iterator> { - private final Iterator iterator; - private List nextBatch; - - SnapshotReaderIterator(Iterator iterator) { - this.iterator = iterator; - } - - @Override - public boolean hasNext() { - if (nextBatch == null) { - nextBatch = findNext(); - } - - return nextBatch != null; - } - - @Override - public List next() { - if (nextBatch == null) { - nextBatch = findNext(); - } - - if (nextBatch == null) { - throw new NoSuchElementException( - String.format("Snapshot (%s) doesn't have any more elements", snapshotId()) - ); - } else { - List result = nextBatch; - nextBatch = null; - return result; - } - } - - private List findNext() { - RecordBatch batch = null; - while (iterator.hasNext()) { - RecordBatch current = iterator.next(); - if (!current.isControlBatch()) { - batch = current; - break; - } - } - - if (batch == null) { - return null; - } - - if (batch.countOrNull() == null) { - throw new IllegalStateException( - String.format("Expected a record count the the batch (%s)", batch) - ); - } - List data = new ArrayList<>(batch.countOrNull()); - - Iterator records = batch.streamingIterator(bufferSupplier); - while (records.hasNext()) { - Record record = records.next(); - - if (record.hasValue()) { - data.add( - serde.read(new ByteBufferAccessor(record.value()), record.value().remaining()) - ); - } else { - throw new IllegalStateException( - String.format( - "Expected all records in the snapshot (%s) to have a value", - snapshotId() - ) - ); - } - } - - return data; - } + public static SnapshotReader of( + RawSnapshotReader snapshot, + RecordSerde serde, + BufferSupplier bufferSupplier, + int maxBatchSize + ) { + return new SnapshotReader<>( + snapshot.snapshotId(), + new SerdeRecordsIterator<>(snapshot.records(), serde, bufferSupplier, maxBatchSize) + ); } } diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java index daf7dc6a70bbf..59b8914b775b5 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java @@ -573,7 +573,7 @@ public void testFetchSnapshotRequestAsLeader() throws Exception { assertEquals(0, response.position()); assertEquals(snapshot.sizeInBytes(), response.unalignedRecords().sizeInBytes()); - UnalignedMemoryRecords memoryRecords = (UnalignedMemoryRecords) snapshot.read(0, Math.toIntExact(snapshot.sizeInBytes())); + UnalignedMemoryRecords memoryRecords = (UnalignedMemoryRecords) snapshot.slice(0, Math.toIntExact(snapshot.sizeInBytes())); assertEquals(memoryRecords.buffer(), ((UnalignedMemoryRecords) response.unalignedRecords()).buffer()); } @@ -617,7 +617,7 @@ public void testPartialFetchSnapshotRequestAsLeader() throws Exception { assertEquals(0, response.position()); assertEquals(snapshot.sizeInBytes() / 2, response.unalignedRecords().sizeInBytes()); - UnalignedMemoryRecords memoryRecords = (UnalignedMemoryRecords) snapshot.read(0, Math.toIntExact(snapshot.sizeInBytes())); + UnalignedMemoryRecords memoryRecords = (UnalignedMemoryRecords) snapshot.slice(0, Math.toIntExact(snapshot.sizeInBytes())); ByteBuffer snapshotBuffer = memoryRecords.buffer(); ByteBuffer responseBuffer = ByteBuffer.allocate(Math.toIntExact(snapshot.sizeInBytes())); diff --git a/raft/src/test/java/org/apache/kafka/raft/MockLog.java b/raft/src/test/java/org/apache/kafka/raft/MockLog.java index 19773f8893f43..c88b3ff2209be 100644 --- a/raft/src/test/java/org/apache/kafka/raft/MockLog.java +++ b/raft/src/test/java/org/apache/kafka/raft/MockLog.java @@ -36,7 +36,6 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collections; -import java.util.Iterator; import java.util.List; import java.util.NavigableMap; import java.util.NoSuchElementException; @@ -687,18 +686,18 @@ public long sizeInBytes() { } @Override - public Iterator iterator() { - return Utils.covariantCast(data.batchIterator()); - } - - @Override - public UnalignedRecords read(long position, int size) { + public UnalignedRecords slice(long position, int size) { ByteBuffer buffer = data.buffer(); buffer.position(Math.toIntExact(position)); buffer.limit(Math.min(buffer.limit(), Math.toIntExact(position + size))); return new UnalignedMemoryRecords(buffer.slice()); } + @Override + public Records records() { + return data; + } + @Override public void close() {} } diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsBatchReaderTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsBatchReaderTest.java index a180c76375c62..8c94d2d7b7c12 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsBatchReaderTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsBatchReaderTest.java @@ -21,7 +21,6 @@ import org.apache.kafka.common.record.FileRecords; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.Records; -import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.raft.BatchReader; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; @@ -34,7 +33,6 @@ import java.util.NoSuchElementException; import java.util.Set; -import static java.util.Arrays.asList; import static org.apache.kafka.test.TestUtils.tempFile; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -44,7 +42,6 @@ class RecordsBatchReaderTest { private static final int MAX_BATCH_BYTES = 128; - private final MockTime time = new MockTime(); private final StringSerde serde = new StringSerde(); @ParameterizedTest @@ -52,13 +49,9 @@ class RecordsBatchReaderTest { public void testReadFromMemoryRecords(CompressionType compressionType) { long baseOffset = 57; - List> batches = asList( - BatchReader.Batch.of(baseOffset, 1, asList("a", "b", "c")), - BatchReader.Batch.of(baseOffset + 3, 2, asList("d", "e")), - BatchReader.Batch.of(baseOffset + 5, 2, asList("f")) - ); + List> batches = SerdeRecordsIteratorTest.createBatches(baseOffset); + MemoryRecords memRecords = SerdeRecordsIteratorTest.buildRecords(compressionType, batches); - MemoryRecords memRecords = buildRecords(compressionType, batches); testBatchReader(baseOffset, memRecords, batches); } @@ -67,13 +60,8 @@ public void testReadFromMemoryRecords(CompressionType compressionType) { public void testReadFromFileRecords(CompressionType compressionType) throws Exception { long baseOffset = 57; - List> batches = asList( - BatchReader.Batch.of(baseOffset, 1, asList("a", "b", "c")), - BatchReader.Batch.of(baseOffset + 3, 2, asList("d", "e")), - BatchReader.Batch.of(baseOffset + 5, 2, asList("f")) - ); - - MemoryRecords memRecords = buildRecords(compressionType, batches); + List> batches = SerdeRecordsIteratorTest.createBatches(baseOffset); + MemoryRecords memRecords = SerdeRecordsIteratorTest.buildRecords(compressionType, batches); FileRecords fileRecords = FileRecords.open(tempFile()); fileRecords.append(memRecords); @@ -81,35 +69,6 @@ public void testReadFromFileRecords(CompressionType compressionType) throws Exce testBatchReader(baseOffset, fileRecords, batches); } - private MemoryRecords buildRecords( - CompressionType compressionType, - List> batches - ) { - ByteBuffer buffer = ByteBuffer.allocate(1024); - - for (BatchReader.Batch batch : batches) { - BatchBuilder builder = new BatchBuilder<>( - buffer, - serde, - compressionType, - batch.baseOffset(), - time.milliseconds(), - false, - batch.epoch(), - MAX_BATCH_BYTES - ); - - for (String record : batch.records()) { - builder.appendRecord(record, null); - } - - builder.build(); - } - - buffer.flip(); - return MemoryRecords.readableRecords(buffer); - } - private void testBatchReader( long baseOffset, Records records, @@ -134,9 +93,12 @@ private void testBatchReader( @SuppressWarnings("unchecked") CloseListener> closeListener = Mockito.mock(CloseListener.class); - RecordsBatchReader reader = new RecordsBatchReader<>( + RecordsBatchReader reader = RecordsBatchReader.of( baseOffset, - new SerdeRecordsIterator<>(records, serde, bufferSupplier, Integer.MAX_VALUE), + records, + serde, + bufferSupplier, + MAX_BATCH_BYTES, closeListener ); diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/SerdeRecordsIteratorTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/SerdeRecordsIteratorTest.java new file mode 100644 index 0000000000000..7b5daeb3f467f --- /dev/null +++ b/raft/src/test/java/org/apache/kafka/raft/internals/SerdeRecordsIteratorTest.java @@ -0,0 +1,171 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.raft.internals; + + + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.IdentityHashMap; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.stream.Stream; +import org.apache.kafka.common.record.CompressionType; +import org.apache.kafka.common.record.FileRecords; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.Records; +import org.apache.kafka.common.utils.BufferSupplier; +import org.apache.kafka.raft.BatchReader; +import org.apache.kafka.raft.RecordSerde; +import org.apache.kafka.test.TestUtils; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.EnumSource; +import org.junit.jupiter.params.provider.MethodSource; +import org.mockito.Mockito; +import static java.util.Arrays.asList; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public final class SerdeRecordsIteratorTest { + private static final int MAX_BATCH_BYTES = 128; + private static final RecordSerde STRING_SERDE = new StringSerde(); + + private static Stream emptyRecords() throws IOException { + return Stream.of( + FileRecords.open(TestUtils.tempFile()), + MemoryRecords.EMPTY + ).map(Arguments::of); + } + + @ParameterizedTest + @MethodSource("emptyRecords") + void testEmptyRecords(Records records) throws IOException { + testIterator(Collections.emptyList(), records); + } + + @ParameterizedTest + @EnumSource(CompressionType.class) + public void testMemoryRecords(CompressionType compressionType) { + List> batches = createBatches(57); + + MemoryRecords memRecords = buildRecords(compressionType, batches); + testIterator(batches, memRecords); + } + + @ParameterizedTest + @EnumSource(CompressionType.class) + public void testFileRecords(CompressionType compressionType) throws IOException { + List> batches = createBatches(57); + + MemoryRecords memRecords = buildRecords(compressionType, batches); + FileRecords fileRecords = FileRecords.open(TestUtils.tempFile()); + fileRecords.append(memRecords); + + testIterator(batches, fileRecords); + } + + // TODO: test what happends if max batch size is less that a record batch + + private void testIterator( + List> expectedBatches, + Records records + ) { + Set allocatedBuffers = Collections.newSetFromMap(new IdentityHashMap<>()); + + SerdeRecordsIterator iterator = createIterator( + records, + mockBufferSupplier(allocatedBuffers), + MAX_BATCH_BYTES + ); + + for (BatchReader.Batch batch : expectedBatches) { + assertTrue(iterator.hasNext()); + assertEquals(batch, iterator.next()); + } + + assertFalse(iterator.hasNext()); + assertThrows(NoSuchElementException.class, iterator::next); + + iterator.close(); + assertEquals(Collections.emptySet(), allocatedBuffers); + } + + static SerdeRecordsIterator createIterator(Records records, BufferSupplier bufferSupplier, int maxBatchSize) { + return new SerdeRecordsIterator<>(records, STRING_SERDE, bufferSupplier, maxBatchSize); + } + + static BufferSupplier mockBufferSupplier(Set buffers) { + BufferSupplier bufferSupplier = Mockito.mock(BufferSupplier.class); + + Mockito.when(bufferSupplier.get(Mockito.anyInt())).thenAnswer(invocation -> { + int size = invocation.getArgument(0); + ByteBuffer buffer = ByteBuffer.allocate(size); + buffers.add(buffer); + return buffer; + }); + + Mockito.doAnswer(invocation -> { + ByteBuffer released = invocation.getArgument(0); + buffers.remove(released); + return null; + }).when(bufferSupplier).release(Mockito.any(ByteBuffer.class)); + + return bufferSupplier; + } + + public static List> createBatches(long baseOffset) { + return asList( + BatchReader.Batch.of(baseOffset, 1, asList("a", "b", "c")), + BatchReader.Batch.of(baseOffset + 3, 2, asList("d", "e")), + BatchReader.Batch.of(baseOffset + 5, 2, asList("f")) + ); + } + + public static MemoryRecords buildRecords( + CompressionType compressionType, + List> batches + ) { + ByteBuffer buffer = ByteBuffer.allocate(1024); + + for (BatchReader.Batch batch : batches) { + BatchBuilder builder = new BatchBuilder<>( + buffer, + STRING_SERDE, + compressionType, + batch.baseOffset(), + 12345L, + false, + batch.epoch(), + MAX_BATCH_BYTES + ); + + for (String record : batch.records()) { + builder.appendRecord(record, null); + } + + builder.build(); + } + + buffer.flip(); + return MemoryRecords.readableRecords(buffer); + } +} diff --git a/raft/src/test/java/org/apache/kafka/snapshot/FileRawSnapshotTest.java b/raft/src/test/java/org/apache/kafka/snapshot/FileRawSnapshotTest.java index dc4f6359d6cd1..ef38d465c5440 100644 --- a/raft/src/test/java/org/apache/kafka/snapshot/FileRawSnapshotTest.java +++ b/raft/src/test/java/org/apache/kafka/snapshot/FileRawSnapshotTest.java @@ -62,14 +62,14 @@ public void tearDown() throws IOException { public void testWritingSnapshot() throws IOException { OffsetAndEpoch offsetAndEpoch = new OffsetAndEpoch(10L, 3); int bufferSize = 256; - int batches = 10; + int numberOfBatches = 10; int expectedSize = 0; try (FileRawSnapshotWriter snapshot = createSnapshotWriter(tempDir, offsetAndEpoch)) { assertEquals(0, snapshot.sizeInBytes()); UnalignedMemoryRecords records = buildRecords(ByteBuffer.wrap(randomBytes(bufferSize))); - for (int i = 0; i < batches; i++) { + for (int i = 0; i < numberOfBatches; i++) { snapshot.append(records); expectedSize += records.sizeInBytes(); } @@ -88,13 +88,13 @@ public void testWritingSnapshot() throws IOException { public void testWriteReadSnapshot() throws IOException { OffsetAndEpoch offsetAndEpoch = new OffsetAndEpoch(10L, 3); int bufferSize = 256; - int batches = 10; + int numberOfBatches = 10; ByteBuffer expectedBuffer = ByteBuffer.wrap(randomBytes(bufferSize)); try (FileRawSnapshotWriter snapshot = createSnapshotWriter(tempDir, offsetAndEpoch)) { UnalignedMemoryRecords records = buildRecords(expectedBuffer); - for (int i = 0; i < batches; i++) { + for (int i = 0; i < numberOfBatches; i++) { snapshot.append(records); } @@ -104,7 +104,10 @@ public void testWriteReadSnapshot() throws IOException { try (FileRawSnapshotReader snapshot = FileRawSnapshotReader.open(tempDir, offsetAndEpoch)) { int countBatches = 0; int countRecords = 0; - for (RecordBatch batch : snapshot) { + + Iterator batches = Utils.covariantCast(snapshot.records().batchIterator()); + while (batches.hasNext()) { + RecordBatch batch = batches.next(); countBatches += 1; Iterator records = batch.streamingIterator(new GrowableBufferSupplier()); @@ -120,8 +123,8 @@ public void testWriteReadSnapshot() throws IOException { } } - assertEquals(batches, countBatches); - assertEquals(batches, countRecords); + assertEquals(numberOfBatches, countBatches); + assertEquals(numberOfBatches, countRecords); } } @@ -151,8 +154,8 @@ public void testPartialWriteReadSnapshot() throws IOException { int totalSize = Math.toIntExact(snapshot.sizeInBytes()); assertEquals(expectedBuffer.remaining(), totalSize); - UnalignedFileRecords record1 = (UnalignedFileRecords) snapshot.read(0, totalSize / 2); - UnalignedFileRecords record2 = (UnalignedFileRecords) snapshot.read(totalSize / 2, totalSize - totalSize / 2); + UnalignedFileRecords record1 = (UnalignedFileRecords) snapshot.slice(0, totalSize / 2); + UnalignedFileRecords record2 = (UnalignedFileRecords) snapshot.slice(totalSize / 2, totalSize - totalSize / 2); assertEquals(buffer1, TestUtils.toBuffer(record1)); assertEquals(buffer2, TestUtils.toBuffer(record2)); @@ -170,10 +173,10 @@ public void testBatchWriteReadSnapshot() throws IOException { OffsetAndEpoch offsetAndEpoch = new OffsetAndEpoch(10L, 3); int bufferSize = 256; int batchSize = 3; - int batches = 10; + int numberOfBatches = 10; try (FileRawSnapshotWriter snapshot = createSnapshotWriter(tempDir, offsetAndEpoch)) { - for (int i = 0; i < batches; i++) { + for (int i = 0; i < numberOfBatches; i++) { ByteBuffer[] buffers = IntStream .range(0, batchSize) .mapToObj(ignore -> ByteBuffer.wrap(randomBytes(bufferSize))).toArray(ByteBuffer[]::new); @@ -187,7 +190,10 @@ public void testBatchWriteReadSnapshot() throws IOException { try (FileRawSnapshotReader snapshot = FileRawSnapshotReader.open(tempDir, offsetAndEpoch)) { int countBatches = 0; int countRecords = 0; - for (RecordBatch batch : snapshot) { + + Iterator batches = Utils.covariantCast(snapshot.records().batchIterator()); + while (batches.hasNext()) { + RecordBatch batch = batches.next(); countBatches += 1; Iterator records = batch.streamingIterator(new GrowableBufferSupplier()); @@ -202,8 +208,8 @@ public void testBatchWriteReadSnapshot() throws IOException { } } - assertEquals(batches, countBatches); - assertEquals(batches * batchSize, countRecords); + assertEquals(numberOfBatches, countBatches); + assertEquals(numberOfBatches * batchSize, countRecords); } } @@ -212,11 +218,11 @@ public void testBufferWriteReadSnapshot() throws IOException { OffsetAndEpoch offsetAndEpoch = new OffsetAndEpoch(10L, 3); int bufferSize = 256; int batchSize = 3; - int batches = 10; + int numberOfBatches = 10; int expectedSize = 0; try (FileRawSnapshotWriter snapshot = createSnapshotWriter(tempDir, offsetAndEpoch)) { - for (int i = 0; i < batches; i++) { + for (int i = 0; i < numberOfBatches; i++) { ByteBuffer[] buffers = IntStream .range(0, batchSize) .mapToObj(ignore -> ByteBuffer.wrap(randomBytes(bufferSize))).toArray(ByteBuffer[]::new); @@ -239,7 +245,9 @@ public void testBufferWriteReadSnapshot() throws IOException { int countBatches = 0; int countRecords = 0; - for (RecordBatch batch : snapshot) { + Iterator batches = Utils.covariantCast(snapshot.records().batchIterator()); + while (batches.hasNext()) { + RecordBatch batch = batches.next(); countBatches += 1; Iterator records = batch.streamingIterator(new GrowableBufferSupplier()); @@ -254,8 +262,8 @@ public void testBufferWriteReadSnapshot() throws IOException { } } - assertEquals(batches, countBatches); - assertEquals(batches * batchSize, countRecords); + assertEquals(numberOfBatches, countBatches); + assertEquals(numberOfBatches * batchSize, countRecords); } } @@ -263,11 +271,11 @@ public void testBufferWriteReadSnapshot() throws IOException { public void testAbortedSnapshot() throws IOException { OffsetAndEpoch offsetAndEpoch = new OffsetAndEpoch(20L, 2); int bufferSize = 256; - int batches = 10; + int numberOfBatches = 10; try (FileRawSnapshotWriter snapshot = createSnapshotWriter(tempDir, offsetAndEpoch)) { UnalignedMemoryRecords records = buildRecords(ByteBuffer.wrap(randomBytes(bufferSize))); - for (int i = 0; i < batches; i++) { + for (int i = 0; i < numberOfBatches; i++) { snapshot.append(records); } } @@ -281,11 +289,11 @@ public void testAbortedSnapshot() throws IOException { public void testAppendToFrozenSnapshot() throws IOException { OffsetAndEpoch offsetAndEpoch = new OffsetAndEpoch(10L, 3); int bufferSize = 256; - int batches = 10; + int numberOfBatches = 10; try (FileRawSnapshotWriter snapshot = createSnapshotWriter(tempDir, offsetAndEpoch)) { UnalignedMemoryRecords records = buildRecords(ByteBuffer.wrap(randomBytes(bufferSize))); - for (int i = 0; i < batches; i++) { + for (int i = 0; i < numberOfBatches; i++) { snapshot.append(records); } @@ -296,18 +304,18 @@ public void testAppendToFrozenSnapshot() throws IOException { // File should exist and the size should be greater than the sum of all the buffers assertTrue(Files.exists(Snapshots.snapshotPath(tempDir, offsetAndEpoch))); - assertTrue(Files.size(Snapshots.snapshotPath(tempDir, offsetAndEpoch)) > bufferSize * batches); + assertTrue(Files.size(Snapshots.snapshotPath(tempDir, offsetAndEpoch)) > bufferSize * numberOfBatches); } @Test public void testCreateSnapshotWithSameId() throws IOException { OffsetAndEpoch offsetAndEpoch = new OffsetAndEpoch(20L, 2); int bufferSize = 256; - int batches = 1; + int numberOfBatches = 1; try (FileRawSnapshotWriter snapshot = createSnapshotWriter(tempDir, offsetAndEpoch)) { UnalignedMemoryRecords records = buildRecords(ByteBuffer.wrap(randomBytes(bufferSize))); - for (int i = 0; i < batches; i++) { + for (int i = 0; i < numberOfBatches; i++) { snapshot.append(records); } @@ -317,7 +325,7 @@ public void testCreateSnapshotWithSameId() throws IOException { // Create another snapshot with the same id try (FileRawSnapshotWriter snapshot = createSnapshotWriter(tempDir, offsetAndEpoch)) { UnalignedMemoryRecords records = buildRecords(ByteBuffer.wrap(randomBytes(bufferSize))); - for (int i = 0; i < batches; i++) { + for (int i = 0; i < numberOfBatches; i++) { snapshot.append(records); } diff --git a/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterReaderTest.java b/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterReaderTest.java index 561f5eadacaf2..a8a7d2e046e07 100644 --- a/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterReaderTest.java +++ b/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterReaderTest.java @@ -24,9 +24,10 @@ import java.util.Random; import java.util.Set; import org.apache.kafka.common.utils.BufferSupplier; -import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.raft.BatchReader.Batch; import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.raft.RaftClientTestContext; +import org.apache.kafka.raft.internals.StringSerde; import org.junit.jupiter.api.Test; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -49,7 +50,7 @@ public void testWritingSnapshot() throws IOException { snapshot.freeze(); } - try (SnapshotReader reader = readSnapshot(context, id)) { + try (SnapshotReader reader = readSnapshot(context, id, Integer.MAX_VALUE)) { assertSnapshot(expected, reader); } } @@ -102,27 +103,22 @@ private List> buildRecords(int recordsPerBatch, int batches) { private SnapshotReader readSnapshot( RaftClientTestContext context, - OffsetAndEpoch snapshotId + OffsetAndEpoch snapshotId, + int maxBatchSize ) { - return new SnapshotReader<>( + return SnapshotReader.of( context.log.readSnapshot(snapshotId).get(), context.serde, - BufferSupplier.create() + BufferSupplier.create(), + maxBatchSize ); } public static void assertSnapshot(List> batches, RawSnapshotReader reader) { - List expected = new ArrayList<>(); - batches.forEach(expected::addAll); - - List actual = new ArrayList<>(expected.size()); - reader.forEach(batch -> { - batch.streamingIterator(BufferSupplier.create()).forEachRemaining(record -> { - actual.add(Utils.utf8(record.value())); - }); - }); - - assertEquals(expected, actual); + assertSnapshot( + batches, + SnapshotReader.of(reader, new StringSerde(), BufferSupplier.create(), Integer.MAX_VALUE) + ); } public static void assertSnapshot(List> batches, SnapshotReader reader) { @@ -130,7 +126,12 @@ public static void assertSnapshot(List> batches, SnapshotReader actual = new ArrayList<>(expected.size()); - reader.forEach(actual::addAll); + while (reader.hasNext()) { + Batch batch = reader.next(); + for (String value : batch) { + actual.add(value); + } + } assertEquals(expected, actual); } diff --git a/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java b/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java index 4a3984c094985..0764606db4ddf 100644 --- a/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java +++ b/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java @@ -112,7 +112,8 @@ public void handleSnapshot(SnapshotReader reader) { // TODO: Create Jira: Need to cover the case where handle snapshot invalidates previous commits // Need to handle that reader.snapshotId() means that every record up to that offset is committed try { - for (List batch : reader) { + while (reader.hasNext()) { + BatchReader.Batch batch = reader.next(); for (ApiMessageAndVersion messageAndVersion : batch) { handleMessage(messageAndVersion.message()); } From 1cbf5c2774d355ed9e75faaca1e3f4242ff50a39 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Wed, 24 Mar 2021 12:38:42 -0700 Subject: [PATCH 10/19] Handle the case where the max batch size is smaller than the batch --- checkstyle/import-control.xml | 2 -- .../apache/kafka/raft/KafkaRaftClient.java | 32 +++++++++---------- .../apache/kafka/raft/ReplicatedCounter.java | 23 ++++++------- .../org/apache/kafka/raft/ReplicatedLog.java | 3 +- .../raft/internals/SerdeRecordsIterator.java | 13 +++++++- .../kafka/raft/metadata/MetaLogRaftShim.java | 6 +--- .../kafka/snapshot/FileRawSnapshotWriter.java | 32 +++++++++---------- .../apache/kafka/snapshot/SnapshotReader.java | 8 ++--- .../internals/SerdeRecordsIteratorTest.java | 14 +++++++- .../kafka/shell/MetadataNodeManager.java | 7 +--- 10 files changed, 73 insertions(+), 67 deletions(-) diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 28a4423497e84..110e393f6b204 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -282,7 +282,6 @@ - @@ -410,7 +409,6 @@ - diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 15c502ef7b063..04fb1ddb0211b 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -312,7 +312,7 @@ private void updateListenersProgress(List listenerContexts, lon for (ListenerContext listenerContext : listenerContexts) { listenerContext.nextExpectedOffset().ifPresent(nextExpectedOffset -> { if (nextExpectedOffset < log.startOffset() && nextExpectedOffset < highWatermark) { - SnapshotReader snapshot = earliestSnapshot().orElseThrow(() -> { + SnapshotReader snapshot = latestSnapshot().orElseThrow(() -> { return new IllegalStateException( String.format( "Snapshot expected when next offset is %s, log start offset is %s and high-watermark is %s", @@ -336,18 +336,14 @@ private void updateListenersProgress(List listenerContexts, lon } } - private Optional> earliestSnapshot() { - return log.earliestSnapshotId().flatMap(earliestSnapshoId -> { + private Optional> latestSnapshot() { + return log.latestSnapshotId().flatMap(snapshoId -> { try { - // TODO: The buffer size is not technically correct. The BatchAccumulator allows for bigger buffers return log - .readSnapshot(earliestSnapshoId) + .readSnapshot(snapshoId) .map(reader -> SnapshotReader.of(reader, serde, BufferSupplier.create(), MAX_BATCH_SIZE_BYTES)); } catch (IOException e) { - logger.error( - String.format("Unable to read snapshot: %s", earliestSnapshoId), - e - ); + logger.error("Unable to read snapshot: {}", snapshoId, e); return Optional.empty(); } @@ -1328,6 +1324,17 @@ private FetchSnapshotResponseData handleFetchSnapshotRequest( ); } + if (partitionSnapshot.position() > Integer.MAX_VALUE) { + throw new IllegalStateException( + String.format( + "Trying to fetch a snapshot with size (%s) and a position (%s) larger than %s", + snapshotSize, + partitionSnapshot.position(), + Integer.MAX_VALUE + ) + ); + } + int maxSnapshotSize; try { maxSnapshotSize = Math.toIntExact(snapshotSize); @@ -1335,12 +1342,6 @@ private FetchSnapshotResponseData handleFetchSnapshotRequest( maxSnapshotSize = Integer.MAX_VALUE; } - if (partitionSnapshot.position() > Integer.MAX_VALUE) { - // TODO: This should return an error response instead of throwing an exception - throw new IllegalStateException(String.format("Trying to fetch a snapshot with position: %d lager than Int.MaxValue", partitionSnapshot.position())); - } - - // TODO: I think this slice of records is closed when the snapshot is close in the try (...) above. UnalignedRecords records = snapshot.slice(partitionSnapshot.position(), Math.min(data.maxBytes(), maxSnapshotSize)); return FetchSnapshotResponse.singleton( @@ -2450,7 +2451,6 @@ public void fireHandleSnapshot(SnapshotReader reader) { * data in memory, we let the state machine read the records from disk. */ public void fireHandleCommit(long baseOffset, Records records) { - // TODO: The max batch size is not techinically correct. The accumulator may create a bigger batch fireHandleCommit( RecordsBatchReader.of( baseOffset, diff --git a/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java b/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java index 4631cf74a93b9..32346e5a13f19 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java +++ b/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java @@ -22,7 +22,6 @@ import org.apache.kafka.snapshot.SnapshotReader; import org.slf4j.Logger; -import java.io.IOException; import java.util.Optional; import static java.util.Collections.singletonList; @@ -99,20 +98,18 @@ public synchronized void handleCommit(BatchReader reader) { @Override public synchronized void handleSnapshot(SnapshotReader reader) { try { - try (SnapshotReader snapshot = reader) { - log.debug("Loading snapshot {}", snapshot.snapshotId()); - while (snapshot.hasNext()) { - Batch batch = snapshot.next(); - for (Integer value : batch) { - log.debug("Setting value: {}", value); - this.committed = value; - this.uncommitted = value; - } + log.debug("Loading snapshot {}", reader.snapshotId()); + while (reader.hasNext()) { + Batch batch = reader.next(); + for (Integer value : batch) { + log.debug("Setting value: {}", value); + this.committed = value; + this.uncommitted = value; } - log.debug("Finished loading snapshot. Set value: {}", this.committed); } - } catch (IOException e) { - log.error(String.format("Unable to read snapshot %s", reader.snapshotId()), e); + log.debug("Finished loading snapshot. Set value: {}", this.committed); + } finally { + reader.close(); } } diff --git a/raft/src/main/java/org/apache/kafka/raft/ReplicatedLog.java b/raft/src/main/java/org/apache/kafka/raft/ReplicatedLog.java index d385e0b948f5e..bec779c13abad 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ReplicatedLog.java +++ b/raft/src/main/java/org/apache/kafka/raft/ReplicatedLog.java @@ -225,7 +225,8 @@ default long truncateToEndOffset(OffsetAndEpoch endOffset) { /** * Create a writable snapshot for the given snapshot id. * - * See {@link RawSnapshotWriter} for details on how to use this object. + * See {@link RawSnapshotWriter} for details on how to use this object. The caller of + * this method is responsible for invoking {@link RawSnapshotWriter#close()}. * * @param snapshotId the end offset and epoch that identifies the snapshot * @return a writable snapshot diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/SerdeRecordsIterator.java b/raft/src/main/java/org/apache/kafka/raft/internals/SerdeRecordsIterator.java index bf1c08a872147..5277fee760f43 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/SerdeRecordsIterator.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/SerdeRecordsIterator.java @@ -108,7 +108,18 @@ private Optional> nextBatches() { if (allocatedBuffer.isPresent()) { buffer = allocatedBuffer.get(); buffer.compact(); - // TODO: catch the case where compact didn't do anything + + if (!buffer.hasRemaining()) { + // The buffer is not big enough to read an entire batch + throw new IllegalStateException( + String.format( + "Unable to read batch from file records buffer %s with maximum batch %s and record size %s", + buffer, + maxBatchSize, + records.sizeInBytes() + ) + ); + } } else { buffer = bufferSupplier.get(Math.min(maxBatchSize, records.sizeInBytes())); allocatedBuffer = Optional.of(buffer); diff --git a/raft/src/main/java/org/apache/kafka/raft/metadata/MetaLogRaftShim.java b/raft/src/main/java/org/apache/kafka/raft/metadata/MetaLogRaftShim.java index 37483c3c4112e..aa5a7b06d066c 100644 --- a/raft/src/main/java/org/apache/kafka/raft/metadata/MetaLogRaftShim.java +++ b/raft/src/main/java/org/apache/kafka/raft/metadata/MetaLogRaftShim.java @@ -26,7 +26,6 @@ import org.apache.kafka.raft.RaftClient; import org.apache.kafka.snapshot.SnapshotReader; -import java.io.IOException; import java.util.List; import java.util.stream.Collectors; @@ -129,10 +128,7 @@ public void handleCommit(BatchReader reader) { @Override public void handleSnapshot(SnapshotReader reader) { // TODO: Create Jira: Handle loading commit in ListenerShim - try { - reader.close(); - } catch (IOException e) { - } + reader.close(); } @Override diff --git a/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotWriter.java b/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotWriter.java index 342639487e448..5f28beea647ca 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotWriter.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotWriter.java @@ -60,21 +60,13 @@ public long sizeInBytes() throws IOException { @Override public void append(UnalignedMemoryRecords records) throws IOException { - if (frozen) { - throw new IllegalStateException( - String.format("Append is not supported. Snapshot is already frozen: id = %s; temp path = %s", snapshotId, tempSnapshotPath) - ); - } + checkIfFrozen("Append"); Utils.writeFully(channel, records.buffer()); } @Override public void append(MemoryRecords records) throws IOException { - if (frozen) { - throw new IllegalStateException( - String.format("Append is not supported. Snapshot is already frozen: id = %s; temp path = %s", snapshotId, tempSnapshotPath) - ); - } + checkIfFrozen("Append"); Utils.writeFully(channel, records.buffer()); } @@ -85,16 +77,11 @@ public boolean isFrozen() { @Override public void freeze() throws IOException { - if (frozen) { - throw new IllegalStateException( - String.format("Freeze is not supported. Snapshot is already frozen: id = %s; temp path = %s", snapshotId, tempSnapshotPath) - ); - } + checkIfFrozen("Freeze"); channel.close(); frozen = true; - // Set readonly and ignore the result if (!tempSnapshotPath.toFile().setReadOnly()) { throw new IOException(String.format("Unable to set file (%s) as read-only", tempSnapshotPath)); } @@ -125,6 +112,19 @@ public String toString() { ); } + void checkIfFrozen(String operation) { + if (frozen) { + throw new IllegalStateException( + String.format( + "%s is not supported. Snapshot is already frozen: id = %s; temp path = %s", + operation, + snapshotId, + tempSnapshotPath + ) + ); + } + } + /** * Create a snapshot writer for topic partition log dir and snapshot id. * diff --git a/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java index 7ba394849f123..ac17026f5e74b 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java @@ -17,8 +17,6 @@ package org.apache.kafka.snapshot; -import java.io.Closeable; -import java.io.IOException; import java.util.Iterator; import org.apache.kafka.common.utils.BufferSupplier; import org.apache.kafka.raft.BatchReader.Batch; @@ -33,7 +31,7 @@ * is assumed that the content of the snapshot represents all of the objects T for the topic * partition from offset 0 up to but not including the end offset in the snapshot id. */ -public final class SnapshotReader implements Closeable, Iterator> { +public final class SnapshotReader implements AutoCloseable, Iterator> { private final OffsetAndEpoch snapshotId; private final SerdeRecordsIterator iterator; @@ -64,10 +62,8 @@ public Batch next() { /** * Closes the snapshot reader. - * - * @throws IOException for any IO error during close */ - public void close() throws IOException { + public void close() { iterator.close(); } diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/SerdeRecordsIteratorTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/SerdeRecordsIteratorTest.java index 7b5daeb3f467f..65da8a6ffd5a0 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/SerdeRecordsIteratorTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/SerdeRecordsIteratorTest.java @@ -34,6 +34,7 @@ import org.apache.kafka.raft.BatchReader; import org.apache.kafka.raft.RecordSerde; import org.apache.kafka.test.TestUtils; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.EnumSource; @@ -83,7 +84,18 @@ public void testFileRecords(CompressionType compressionType) throws IOException testIterator(batches, fileRecords); } - // TODO: test what happends if max batch size is less that a record batch + @Test + public void testMaxBatchTooSmall() throws IOException { + List> batches = createBatches(57); + + MemoryRecords memRecords = buildRecords(CompressionType.NONE, batches); + FileRecords fileRecords = FileRecords.open(TestUtils.tempFile()); + fileRecords.append(memRecords); + + SerdeRecordsIterator iterator = createIterator(fileRecords, BufferSupplier.create(), 10); + assertThrows(IllegalStateException.class, iterator::hasNext); + assertThrows(IllegalStateException.class, iterator::next); + } private void testIterator( List> expectedBatches, diff --git a/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java b/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java index 0764606db4ddf..cf70d3b460909 100644 --- a/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java +++ b/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java @@ -49,7 +49,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.function.Consumer; @@ -119,11 +118,7 @@ public void handleSnapshot(SnapshotReader reader) { } } } finally { - try { - reader.close(); - } catch (IOException e) { - log.error("Unable to close snapshot {}", reader.snapshotId(), e); - } + reader.close(); } } From a859ab976fc3219a65c661ab43b090325d532c55 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Wed, 24 Mar 2021 13:22:05 -0700 Subject: [PATCH 11/19] Add more documentation to RawSnapshotReader --- .../apache/kafka/raft/internals/RecordsBatchReader.java | 1 - .../org/apache/kafka/raft/metadata/MetaLogRaftShim.java | 1 - .../java/org/apache/kafka/snapshot/RawSnapshotReader.java | 7 ++++--- 3 files changed, 4 insertions(+), 5 deletions(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java index 4a953a10d44b7..fb8e15e0c132f 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java @@ -75,7 +75,6 @@ public long baseOffset() { return baseOffset; } - // TODO: rename this property it is misleading. It is not techinically the last offset of the batch reader public OptionalLong lastOffset() { if (isClosed) { return OptionalLong.of(lastReturnedOffset); diff --git a/raft/src/main/java/org/apache/kafka/raft/metadata/MetaLogRaftShim.java b/raft/src/main/java/org/apache/kafka/raft/metadata/MetaLogRaftShim.java index aa5a7b06d066c..22b7ca067baa2 100644 --- a/raft/src/main/java/org/apache/kafka/raft/metadata/MetaLogRaftShim.java +++ b/raft/src/main/java/org/apache/kafka/raft/metadata/MetaLogRaftShim.java @@ -127,7 +127,6 @@ public void handleCommit(BatchReader reader) { @Override public void handleSnapshot(SnapshotReader reader) { - // TODO: Create Jira: Handle loading commit in ListenerShim reader.close(); } diff --git a/raft/src/main/java/org/apache/kafka/snapshot/RawSnapshotReader.java b/raft/src/main/java/org/apache/kafka/snapshot/RawSnapshotReader.java index bbd42e1aacf35..4f4523cbcf8fd 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/RawSnapshotReader.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/RawSnapshotReader.java @@ -21,7 +21,6 @@ import org.apache.kafka.raft.OffsetAndEpoch; import java.io.Closeable; -import java.io.IOException; /** * Interface for reading snapshots as a sequence of records. @@ -37,7 +36,7 @@ public interface RawSnapshotReader extends Closeable { * * @throws IOException for any IO error while reading the size */ - long sizeInBytes() throws IOException; + long sizeInBytes(); /** * Creates a slize of unaligned records from the position up to a size. @@ -49,7 +48,9 @@ public interface RawSnapshotReader extends Closeable { UnalignedRecords slice(long position, int size); /** - * TODO: + * Returns all of the records backing this snapshot reader. + * + * @return all of the records for this snapshot */ Records records(); } From 033e332639a5cf1f65031cfa128982ba6b5df04f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Wed, 24 Mar 2021 15:48:08 -0700 Subject: [PATCH 12/19] Generate snapshot in ReplicatedCounter and fix simulation --- .../apache/kafka/raft/KafkaRaftClient.java | 60 ++++++++------- .../org/apache/kafka/raft/RaftClient.java | 13 +--- .../apache/kafka/raft/ReplicatedCounter.java | 61 ++++++++++----- .../org/apache/kafka/raft/ReplicatedLog.java | 8 +- .../kafka/snapshot/FileRawSnapshotWriter.java | 77 ++++++++++++------- .../kafka/snapshot/RawSnapshotWriter.java | 23 ++---- .../apache/kafka/snapshot/SnapshotWriter.java | 17 ++-- .../kafka/raft/RaftEventSimulationTest.java | 56 +++++++++++--- 8 files changed, 184 insertions(+), 131 deletions(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 04fb1ddb0211b..0cb9486ec2955 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -338,15 +338,9 @@ private void updateListenersProgress(List listenerContexts, lon private Optional> latestSnapshot() { return log.latestSnapshotId().flatMap(snapshoId -> { - try { - return log - .readSnapshot(snapshoId) - .map(reader -> SnapshotReader.of(reader, serde, BufferSupplier.create(), MAX_BATCH_SIZE_BYTES)); - } catch (IOException e) { - logger.error("Unable to read snapshot: {}", snapshoId, e); - - return Optional.empty(); - } + return log + .readSnapshot(snapshoId) + .map(reader -> SnapshotReader.of(reader, serde, BufferSupplier.create(), MAX_BATCH_SIZE_BYTES)); }); } @@ -379,24 +373,29 @@ private void fireHandleResign(int epoch) { } @Override - public void initialize() throws IOException { - quorum.initialize(new OffsetAndEpoch(log.endOffset().offset, log.lastFetchedEpoch())); + public void initialize() { + try { + quorum.initialize(new OffsetAndEpoch(log.endOffset().offset, log.lastFetchedEpoch())); - long currentTimeMs = time.milliseconds(); - if (quorum.isLeader()) { - throw new IllegalStateException("Voter cannot initialize as a Leader"); - } else if (quorum.isCandidate()) { - onBecomeCandidate(currentTimeMs); - } else if (quorum.isFollower()) { - onBecomeFollower(currentTimeMs); - } + long currentTimeMs = time.milliseconds(); + if (quorum.isLeader()) { + throw new IllegalStateException("Voter cannot initialize as a Leader"); + } else if (quorum.isCandidate()) { + onBecomeCandidate(currentTimeMs); + } else if (quorum.isFollower()) { + onBecomeFollower(currentTimeMs); + } - // When there is only a single voter, become candidate immediately - if (quorum.isVoter() - && quorum.remoteVoters().isEmpty() - && !quorum.isLeader() - && !quorum.isCandidate()) { - transitionToCandidate(currentTimeMs); + // When there is only a single voter, become candidate immediately + if (quorum.isVoter() + && quorum.remoteVoters().isEmpty() + && !quorum.isLeader() + && !quorum.isCandidate()) { + + transitionToCandidate(currentTimeMs); + } + } catch (IOException e) { + throw new RuntimeException(e); } } @@ -1447,10 +1446,15 @@ private boolean handleFetchSnapshotResponse( ); } - if (!(partitionSnapshot.unalignedRecords() instanceof MemoryRecords)) { + final UnalignedMemoryRecords records; + if (partitionSnapshot.unalignedRecords() instanceof MemoryRecords) { + records = new UnalignedMemoryRecords(((MemoryRecords) partitionSnapshot.unalignedRecords()).buffer()); + } else if (partitionSnapshot.unalignedRecords() instanceof UnalignedMemoryRecords) { + records = (UnalignedMemoryRecords) partitionSnapshot.unalignedRecords(); + } else { throw new IllegalStateException(String.format("Received unexpected fetch snapshot response: %s", partitionSnapshot)); } - snapshot.append(new UnalignedMemoryRecords(((MemoryRecords) partitionSnapshot.unalignedRecords()).buffer())); + snapshot.append(records); if (snapshot.sizeInBytes() == partitionSnapshot.size()) { // Finished fetching the snapshot. @@ -2324,7 +2328,7 @@ public CompletableFuture shutdown(int timeoutMs) { } @Override - public SnapshotWriter createSnapshot(OffsetAndEpoch snapshotId) throws IOException { + public SnapshotWriter createSnapshot(OffsetAndEpoch snapshotId) { return new SnapshotWriter<>( log.createSnapshot(snapshotId), MAX_BATCH_SIZE_BYTES, diff --git a/raft/src/main/java/org/apache/kafka/raft/RaftClient.java b/raft/src/main/java/org/apache/kafka/raft/RaftClient.java index e2c706ff22223..7293f796eb772 100644 --- a/raft/src/main/java/org/apache/kafka/raft/RaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/RaftClient.java @@ -19,12 +19,10 @@ import org.apache.kafka.snapshot.SnapshotReader; import org.apache.kafka.snapshot.SnapshotWriter; -import java.io.Closeable; -import java.io.IOException; import java.util.List; import java.util.concurrent.CompletableFuture; -public interface RaftClient extends Closeable { +public interface RaftClient extends AutoCloseable { interface Listener { /** @@ -79,12 +77,9 @@ default void handleResign(int epoch) {} } /** - * Initialize the client. - * This should only be called once on startup. - * - * @throws IOException For any IO errors during initialization + * Initialize the client. This should only be called once on startup. */ - void initialize() throws IOException; + void initialize(); /** * Register a listener to get commit/leader notifications. @@ -170,5 +165,5 @@ default void handleResign(int epoch) {} * @param snapshotId the end offset and epoch that identifies the snapshot * @return a writable snapshot */ - SnapshotWriter createSnapshot(OffsetAndEpoch snapshotId) throws IOException; + SnapshotWriter createSnapshot(OffsetAndEpoch snapshotId); } diff --git a/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java b/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java index 32346e5a13f19..f958420aeacb0 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java +++ b/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java @@ -20,9 +20,10 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.raft.BatchReader.Batch; import org.apache.kafka.snapshot.SnapshotReader; +import org.apache.kafka.snapshot.SnapshotWriter; import org.slf4j.Logger; -import java.util.Optional; +import java.util.OptionalInt; import static java.util.Collections.singletonList; @@ -31,9 +32,12 @@ public class ReplicatedCounter implements RaftClient.Listener { private final Logger log; private final RaftClient client; - private int committed; - private int uncommitted; - private Optional claimedEpoch; + private int committed = 0; + private int uncommitted = 0; + private OptionalInt claimedEpoch = OptionalInt.empty(); + private long lastSnapshotEndOffset = 0; + private long nextReadOffset = 0; + private int readEpoch = 0; public ReplicatedCounter( int nodeId, @@ -42,11 +46,7 @@ public ReplicatedCounter( ) { this.nodeId = nodeId; this.client = client; - this.log = logContext.logger(ReplicatedCounter.class); - - this.committed = 0; - this.uncommitted = 0; - this.claimedEpoch = Optional.empty(); + log = logContext.logger(ReplicatedCounter.class); } public synchronized boolean isWritable() { @@ -58,7 +58,7 @@ public synchronized void increment() { throw new KafkaException("Counter is not currently writable"); } - int epoch = claimedEpoch.get(); + int epoch = claimedEpoch.getAsInt(); uncommitted += 1; Long offset = client.scheduleAppend(epoch, singletonList(uncommitted)); if (offset != null) { @@ -70,26 +70,37 @@ public synchronized void increment() { @Override public synchronized void handleCommit(BatchReader reader) { try { - int initialValue = this.committed; + int initialValue = committed; while (reader.hasNext()) { BatchReader.Batch batch = reader.next(); log.debug("Handle commit of batch with records {} at base offset {}", batch.records(), batch.baseOffset()); for (Integer value : batch.records()) { - if (value != this.committed + 1) { + if (value != committed + 1) { throw new AssertionError( String.format( "Expected next committed value to be %s, but instead found %s on node %s", - this.committed + 1, + committed + 1, value, nodeId ) ); } - this.committed = value; + committed = value; } + + nextReadOffset = batch.lastOffset() + 1; + readEpoch = batch.epoch(); } log.debug("Counter incremented from {} to {}", initialValue, committed); + + if (lastSnapshotEndOffset + 10 < nextReadOffset) { + log.debug("Generating new snapshot at {} since next commit offset is {}", lastSnapshotEndOffset, nextReadOffset); + try (SnapshotWriter snapshot = client.createSnapshot(new OffsetAndEpoch(nextReadOffset, readEpoch))) { + snapshot.append(singletonList(committed)); + snapshot.freeze(); + } + } } finally { reader.close(); } @@ -101,13 +112,23 @@ public synchronized void handleSnapshot(SnapshotReader reader) { log.debug("Loading snapshot {}", reader.snapshotId()); while (reader.hasNext()) { Batch batch = reader.next(); + if (batch.records().size() != 1) { + throw new AssertionError( + String.format( + "Expected the snapshot at %s to only contain one record %s", + reader.snapshotId(), + batch.records() + ) + ); + } + for (Integer value : batch) { log.debug("Setting value: {}", value); - this.committed = value; - this.uncommitted = value; + committed = value; + uncommitted = value; } } - log.debug("Finished loading snapshot. Set value: {}", this.committed); + log.debug("Finished loading snapshot. Set value: {}", committed); } finally { reader.close(); } @@ -117,14 +138,14 @@ public synchronized void handleSnapshot(SnapshotReader reader) { public synchronized void handleClaim(int epoch) { log.debug("Counter uncommitted value initialized to {} after claiming leadership in epoch {}", committed, epoch); - this.uncommitted = committed; - this.claimedEpoch = Optional.of(epoch); + uncommitted = committed; + claimedEpoch = OptionalInt.of(epoch); } @Override public synchronized void handleResign(int epoch) { log.debug("Counter uncommitted value reset after resigning leadership"); this.uncommitted = -1; - this.claimedEpoch = Optional.empty(); + this.claimedEpoch = OptionalInt.empty(); } } diff --git a/raft/src/main/java/org/apache/kafka/raft/ReplicatedLog.java b/raft/src/main/java/org/apache/kafka/raft/ReplicatedLog.java index bec779c13abad..7350459ba1619 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ReplicatedLog.java +++ b/raft/src/main/java/org/apache/kafka/raft/ReplicatedLog.java @@ -21,11 +21,9 @@ import org.apache.kafka.snapshot.RawSnapshotReader; import org.apache.kafka.snapshot.RawSnapshotWriter; -import java.io.Closeable; -import java.io.IOException; import java.util.Optional; -public interface ReplicatedLog extends Closeable { +public interface ReplicatedLog extends AutoCloseable { /** * Write a set of records to the local leader log. These messages will either @@ -231,7 +229,7 @@ default long truncateToEndOffset(OffsetAndEpoch endOffset) { * @param snapshotId the end offset and epoch that identifies the snapshot * @return a writable snapshot */ - RawSnapshotWriter createSnapshot(OffsetAndEpoch snapshotId) throws IOException; + RawSnapshotWriter createSnapshot(OffsetAndEpoch snapshotId); /** * Opens a readable snapshot for the given snapshot id. @@ -244,7 +242,7 @@ default long truncateToEndOffset(OffsetAndEpoch endOffset) { * @return an Optional with a readable snapshot, if the snapshot exists, otherwise * returns an empty Optional */ - Optional readSnapshot(OffsetAndEpoch snapshotId) throws IOException; + Optional readSnapshot(OffsetAndEpoch snapshotId); /** * Returns the latest snapshot id if one exists. diff --git a/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotWriter.java b/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotWriter.java index 5f28beea647ca..e259aefaba962 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotWriter.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotWriter.java @@ -54,20 +54,32 @@ public OffsetAndEpoch snapshotId() { } @Override - public long sizeInBytes() throws IOException { - return channel.size(); + public long sizeInBytes() { + try { + return channel.size(); + } catch (IOException e) { + throw new RuntimeException(e); + } } @Override - public void append(UnalignedMemoryRecords records) throws IOException { - checkIfFrozen("Append"); - Utils.writeFully(channel, records.buffer()); + public void append(UnalignedMemoryRecords records) { + try { + checkIfFrozen("Append"); + Utils.writeFully(channel, records.buffer()); + } catch (IOException e) { + throw new RuntimeException(e); + } } @Override - public void append(MemoryRecords records) throws IOException { - checkIfFrozen("Append"); - Utils.writeFully(channel, records.buffer()); + public void append(MemoryRecords records) { + try { + checkIfFrozen("Append"); + Utils.writeFully(channel, records.buffer()); + } catch (IOException e) { + throw new RuntimeException(e); + } } @Override @@ -76,29 +88,34 @@ public boolean isFrozen() { } @Override - public void freeze() throws IOException { - checkIfFrozen("Freeze"); + public void freeze() { + try { + checkIfFrozen("Freeze"); - channel.close(); - frozen = true; + channel.close(); + frozen = true; - if (!tempSnapshotPath.toFile().setReadOnly()) { - throw new IOException(String.format("Unable to set file (%s) as read-only", tempSnapshotPath)); - } + if (!tempSnapshotPath.toFile().setReadOnly()) { + throw new IllegalStateException(String.format("Unable to set file (%s) as read-only", tempSnapshotPath)); + } - Path destination = Snapshots.moveRename(tempSnapshotPath, snapshotId); - Utils.atomicMoveWithFallback(tempSnapshotPath, destination); + Path destination = Snapshots.moveRename(tempSnapshotPath, snapshotId); + Utils.atomicMoveWithFallback(tempSnapshotPath, destination); - replicatedLog.ifPresent(log -> log.onSnapshotFrozen(snapshotId)); + replicatedLog.ifPresent(log -> log.onSnapshotFrozen(snapshotId)); + } catch (IOException e) { + throw new RuntimeException(e); + } } @Override - public void close() throws IOException { + public void close() { try { channel.close(); - } finally { // This is a noop if freeze was called before calling close Files.deleteIfExists(tempSnapshotPath); + } catch (IOException e) { + throw new RuntimeException(e); } } @@ -136,14 +153,18 @@ public static FileRawSnapshotWriter create( Path logDir, OffsetAndEpoch snapshotId, Optional replicatedLog - ) throws IOException { - Path path = Snapshots.createTempFile(logDir, snapshotId); + ) { + try { + Path path = Snapshots.createTempFile(logDir, snapshotId); - return new FileRawSnapshotWriter( - path, - FileChannel.open(path, Utils.mkSet(StandardOpenOption.WRITE, StandardOpenOption.APPEND)), - snapshotId, - replicatedLog - ); + return new FileRawSnapshotWriter( + path, + FileChannel.open(path, Utils.mkSet(StandardOpenOption.WRITE, StandardOpenOption.APPEND)), + snapshotId, + replicatedLog + ); + } catch (IOException e) { + throw new RuntimeException(e); + } } } diff --git a/raft/src/main/java/org/apache/kafka/snapshot/RawSnapshotWriter.java b/raft/src/main/java/org/apache/kafka/snapshot/RawSnapshotWriter.java index f8ec58f17fcd2..07d8271e953f6 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/RawSnapshotWriter.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/RawSnapshotWriter.java @@ -21,13 +21,10 @@ import org.apache.kafka.common.record.UnalignedMemoryRecords; import org.apache.kafka.raft.OffsetAndEpoch; -import java.io.Closeable; -import java.io.IOException; - /** * Interface for writing snapshot as a sequence of records. */ -public interface RawSnapshotWriter extends Closeable { +public interface RawSnapshotWriter extends AutoCloseable { /** * Returns the end offset and epoch for the snapshot. */ @@ -35,10 +32,8 @@ public interface RawSnapshotWriter extends Closeable { /** * Returns the number of bytes for the snapshot. - * - * @throws IOException for any IO error while reading the size */ - long sizeInBytes() throws IOException; + long sizeInBytes(); /** * Fully appends the memory record set to the snapshot. @@ -47,9 +42,8 @@ public interface RawSnapshotWriter extends Closeable { * snapshot. * * @param records the region to append - * @throws IOException for any IO error during append */ - void append(MemoryRecords records) throws IOException; + void append(MemoryRecords records); /** * Fully appends the memory record set to the snapshot, the difference with {@link RawSnapshotWriter#append(MemoryRecords)} @@ -59,9 +53,8 @@ public interface RawSnapshotWriter extends Closeable { * snapshot. * * @param records the region to append - * @throws IOException for any IO error during append */ - void append(UnalignedMemoryRecords records) throws IOException; + void append(UnalignedMemoryRecords records); /** * Returns true if the snapshot has been frozen, otherwise false is returned. @@ -72,17 +65,13 @@ public interface RawSnapshotWriter extends Closeable { /** * Freezes the snapshot and marking it as immutable. - * - * @throws IOException for any IO error during freezing */ - void freeze() throws IOException; + void freeze(); /** * Closes the snapshot writer. * * If close is called without first calling freeze the snapshot is aborted. - * - * @throws IOException for any IO error during close */ - void close() throws IOException; + void close(); } diff --git a/raft/src/main/java/org/apache/kafka/snapshot/SnapshotWriter.java b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotWriter.java index 8d94a2449cf2c..e9b3c64e0362f 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/SnapshotWriter.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotWriter.java @@ -25,8 +25,6 @@ import org.apache.kafka.raft.internals.BatchAccumulator; import org.apache.kafka.raft.internals.BatchAccumulator.CompletedBatch; -import java.io.Closeable; -import java.io.IOException; import java.util.List; /** @@ -42,7 +40,7 @@ * * @see org.apache.kafka.raft.RaftClient#createSnapshot(OffsetAndEpoch) */ -final public class SnapshotWriter implements Closeable { +final public class SnapshotWriter implements AutoCloseable { final private RawSnapshotWriter snapshot; final private BatchAccumulator accumulator; final private Time time; @@ -102,10 +100,9 @@ public boolean isFrozen() { * The list of record passed are guaranteed to get written together. * * @param records the list of records to append to the snapshot - * @throws IOException for any IO error while appending * @throws IllegalStateException if append is called when isFrozen is true */ - public void append(List records) throws IOException { + public void append(List records) { if (snapshot.isFrozen()) { String message = String.format( "Append not supported. Snapshot is already frozen: id = '%s'.", @@ -124,10 +121,8 @@ public void append(List records) throws IOException { /** * Freezes the snapshot by flushing all pending writes and marking it as immutable. - * - * @throws IOException for any IO error during freezing */ - public void freeze() throws IOException { + public void freeze() { appendBatches(accumulator.drain()); snapshot.freeze(); accumulator.close(); @@ -137,15 +132,13 @@ public void freeze() throws IOException { * Closes the snapshot writer. * * If close is called without first calling freeze the snapshot is aborted. - * - * @throws IOException for any IO error during close */ - public void close() throws IOException { + public void close() { snapshot.close(); accumulator.close(); } - private void appendBatches(List> batches) throws IOException { + private void appendBatches(List> batches) { try { for (CompletedBatch batch : batches) { snapshot.append(batch.data); diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java index 223c2694c767d..eae5b70e1e040 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java @@ -27,6 +27,7 @@ import org.apache.kafka.common.protocol.Readable; import org.apache.kafka.common.protocol.Writable; import org.apache.kafka.common.protocol.types.Type; +import org.apache.kafka.common.utils.BufferSupplier; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; @@ -34,9 +35,9 @@ import org.apache.kafka.raft.MockLog.LogBatch; import org.apache.kafka.raft.MockLog.LogEntry; import org.apache.kafka.raft.internals.BatchMemoryPool; +import org.apache.kafka.snapshot.SnapshotReader; import org.junit.jupiter.api.Tag; -import java.io.IOException; import java.net.InetSocketAddress; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -53,11 +54,13 @@ import java.util.Random; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; import java.util.function.Supplier; import java.util.stream.Collectors; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; @@ -719,7 +722,8 @@ void start(int nodeId) { persistentState.store, logContext, time, - random + random, + serde ); node.initialize(); running.put(nodeId, node); @@ -737,6 +741,7 @@ private static class RaftNode { final ReplicatedCounter counter; final Time time; final Random random; + final RecordSerde intSerde; private RaftNode( int nodeId, @@ -747,7 +752,8 @@ private RaftNode( MockQuorumStateStore store, LogContext logContext, Time time, - Random random + Random random, + RecordSerde intSerde ) { this.nodeId = nodeId; this.client = client; @@ -759,15 +765,12 @@ private RaftNode( this.time = time; this.random = random; this.counter = new ReplicatedCounter(nodeId, client, logContext); + this.intSerde = intSerde; } void initialize() { - try { - client.register(this.counter); - client.initialize(); - } catch (IOException e) { - throw new RuntimeException(e); - } + client.register(this.counter); + client.initialize(); } void poll() { @@ -988,14 +991,43 @@ private int parseSequenceNumber(ByteBuffer value) { return (int) Type.INT32.read(value); } - private void assertCommittedData(int nodeId, KafkaRaftClient manager, MockLog log) { + private void assertCommittedData(RaftNode node) { + final int nodeId = node.nodeId; + final KafkaRaftClient manager = node.client; + final MockLog log = node.log; + OptionalLong highWatermark = manager.highWatermark(); if (!highWatermark.isPresent()) { // We cannot do validation if the current high watermark is unknown return; } - for (LogBatch batch : log.readBatches(0L, highWatermark)) { + AtomicLong startOffset = new AtomicLong(0); + log.earliestSnapshotId().ifPresent(snapshoId -> { + assertTrue(snapshoId.offset <= highWatermark.getAsLong()); + startOffset.set(snapshoId.offset); + + try (SnapshotReader snapshot = + SnapshotReader.of(log.readSnapshot(snapshoId).get(), node.intSerde, BufferSupplier.create(), Integer.MAX_VALUE)) { + // Expect only one batch with only one record + assertTrue(snapshot.hasNext()); + BatchReader.Batch batch = snapshot.next(); + assertFalse(snapshot.hasNext()); + assertEquals(1, batch.records().size()); + + long offset = snapshoId.offset - 1; + int sequence = batch.records().get(0); + committedSequenceNumbers.putIfAbsent(offset, sequence); + + assertEquals( + committedSequenceNumbers.get(offset), + sequence, + String.format("Committed sequence at offset %s changed on node %s", offset, nodeId) + ); + } + }); + + for (LogBatch batch : log.readBatches(startOffset.get(), highWatermark)) { if (batch.isControlBatch) { continue; } @@ -1017,7 +1049,7 @@ private void assertCommittedData(int nodeId, KafkaRaftClient manager, M @Override public void validate() { - cluster.forAllRunning(node -> assertCommittedData(node.nodeId, node.client, node.log)); + cluster.forAllRunning(this::assertCommittedData); } } From 2dd49397a1848e0b9b9c11aa72d5455c5fccfc3e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Wed, 24 Mar 2021 15:49:53 -0700 Subject: [PATCH 13/19] Remove TODO since Jira was created --- .../main/java/org/apache/kafka/shell/MetadataNodeManager.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java b/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java index cf70d3b460909..5a3b91361ac82 100644 --- a/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java +++ b/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java @@ -108,8 +108,6 @@ public void handleCommits(long lastOffset, List messages) { @Override public void handleSnapshot(SnapshotReader reader) { - // TODO: Create Jira: Need to cover the case where handle snapshot invalidates previous commits - // Need to handle that reader.snapshotId() means that every record up to that offset is committed try { while (reader.hasNext()) { BatchReader.Batch batch = reader.next(); From 8adbd4c3f8f1d008ad42cd59738722ea4128077a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Fri, 9 Apr 2021 14:30:43 -0700 Subject: [PATCH 14/19] Use Colletions.emptyIterator instead of Optional.empty --- .../apache/kafka/raft/KafkaRaftClient.java | 16 ++--- .../apache/kafka/raft/ReplicatedCounter.java | 22 ++++--- .../raft/internals/SerdeRecordsIterator.java | 61 +++++++++---------- 3 files changed, 52 insertions(+), 47 deletions(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 0cb9486ec2955..7452b1f2b130e 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -337,11 +337,13 @@ private void updateListenersProgress(List listenerContexts, lon } private Optional> latestSnapshot() { - return log.latestSnapshotId().flatMap(snapshoId -> { - return log - .readSnapshot(snapshoId) - .map(reader -> SnapshotReader.of(reader, serde, BufferSupplier.create(), MAX_BATCH_SIZE_BYTES)); - }); + return log.latestSnapshotId().flatMap(snapshotId -> + log + .readSnapshot(snapshotId) + .map(reader -> + SnapshotReader.of(reader, serde, BufferSupplier.create(), MAX_BATCH_SIZE_BYTES) + ) + ); } private void maybeFireHandleCommit(long baseOffset, int epoch, List records) { @@ -2168,7 +2170,7 @@ private long pollUnattachedAsObserver(UnattachedState state, long currentTimeMs) } private long pollCurrentState(long currentTimeMs) throws IOException { - maybeUpdateEarliestSnapshotId(); + maybeDeleteBeforeSnapshot(); if (quorum.isLeader()) { return pollLeader(currentTimeMs); @@ -2232,7 +2234,7 @@ private boolean maybeCompleteShutdown(long currentTimeMs) { return false; } - private void maybeUpdateEarliestSnapshotId() { + private void maybeDeleteBeforeSnapshot() { log.latestSnapshotId().ifPresent(snapshotId -> { quorum.highWatermark().ifPresent(highWatermark -> { if (highWatermark.offset >= snapshotId.offset) { diff --git a/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java b/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java index f958420aeacb0..b884b59e1f6b2 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java +++ b/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java @@ -31,6 +31,7 @@ public class ReplicatedCounter implements RaftClient.Listener { private final int nodeId; private final Logger log; private final RaftClient client; + private final int snapshotDelayInRecords = 10; private int committed = 0; private int uncommitted = 0; @@ -70,31 +71,34 @@ public synchronized void increment() { @Override public synchronized void handleCommit(BatchReader reader) { try { - int initialValue = committed; + int initialCommitted = committed; while (reader.hasNext()) { BatchReader.Batch batch = reader.next(); - log.debug("Handle commit of batch with records {} at base offset {}", - batch.records(), batch.baseOffset()); - for (Integer value : batch.records()) { - if (value != committed + 1) { + log.debug( + "Handle commit of batch with records {} at base offset {}", + batch.records(), + batch.baseOffset() + ); + for (Integer nextCommitted: batch.records()) { + if (nextCommitted != committed + 1) { throw new AssertionError( String.format( "Expected next committed value to be %s, but instead found %s on node %s", committed + 1, - value, + nextCommitted, nodeId ) ); } - committed = value; + committed = nextCommitted; } nextReadOffset = batch.lastOffset() + 1; readEpoch = batch.epoch(); } - log.debug("Counter incremented from {} to {}", initialValue, committed); + log.debug("Counter incremented from {} to {}", initialCommitted, committed); - if (lastSnapshotEndOffset + 10 < nextReadOffset) { + if (lastSnapshotEndOffset + snapshotDelayInRecords < nextReadOffset) { log.debug("Generating new snapshot at {} since next commit offset is {}", lastSnapshotEndOffset, nextReadOffset); try (SnapshotWriter snapshot = client.createSnapshot(new OffsetAndEpoch(nextReadOffset, readEpoch))) { snapshot.append(singletonList(committed)); diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/SerdeRecordsIterator.java b/raft/src/main/java/org/apache/kafka/raft/internals/SerdeRecordsIterator.java index 5277fee760f43..07a2caba7906c 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/SerdeRecordsIterator.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/SerdeRecordsIterator.java @@ -19,6 +19,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; @@ -40,11 +41,11 @@ public final class SerdeRecordsIterator implements Iterator>, AutoCl private final BufferSupplier bufferSupplier; private final int maxBatchSize; - private Optional> nextBatches = Optional.empty(); + private Iterator nextBatches = Collections.emptyIterator(); private Optional> nextBatch = Optional.empty(); // Buffer used to as the backing store for nextBatches if needed private Optional allocatedBuffer = Optional.empty(); - // Number of bytes from records that read + // Number of bytes from records read up to now private int bytesRead = 0; private boolean isClosed = false; @@ -96,7 +97,7 @@ private void checkIfClosed() { } } - private Optional> nextBatches() { + private Iterator nextBatches() { int recordSize = records.sizeInBytes(); if (bytesRead < recordSize) { final MemoryRecords memoryRecords; @@ -108,18 +109,6 @@ private Optional> nextBatches() { if (allocatedBuffer.isPresent()) { buffer = allocatedBuffer.get(); buffer.compact(); - - if (!buffer.hasRemaining()) { - // The buffer is not big enough to read an entire batch - throw new IllegalStateException( - String.format( - "Unable to read batch from file records buffer %s with maximum batch %s and record size %s", - buffer, - maxBatchSize, - records.sizeInBytes() - ) - ); - } } else { buffer = bufferSupplier.get(Math.min(maxBatchSize, records.sizeInBytes())); allocatedBuffer = Optional.of(buffer); @@ -138,33 +127,43 @@ private Optional> nextBatches() { throw new IllegalStateException(String.format("Unexpected Records type %s", records.getClass())); } - return Optional.of(memoryRecords.batchIterator()); - } else { - return Optional.empty(); + Iterator batches = memoryRecords.batchIterator(); + if (!batches.hasNext()) { + // The buffer is not big enough to read an entire batch + throw new IllegalStateException( + String.format( + "Unable to read batch from records buffer %s with maximum batch %s and record size %s", + allocatedBuffer, + maxBatchSize, + records.sizeInBytes() + ) + ); + } + + return batches; } + + return Collections.emptyIterator(); } private Optional> nextBatch() { - if (!nextBatches.isPresent()) { + if (!nextBatches.hasNext()) { nextBatches = nextBatches(); } - while (nextBatches.isPresent()) { - if (nextBatches.get().hasNext()) { - MutableRecordBatch nextBatch = nextBatches.get().next(); + if (nextBatches.hasNext()) { + MutableRecordBatch nextBatch = nextBatches.next(); - // Update the buffer position to reflect the read batch - allocatedBuffer.ifPresent(buffer -> buffer.position(buffer.position() + nextBatch.sizeInBytes())); + // Update the buffer position to reflect the read batch + allocatedBuffer.ifPresent(buffer -> buffer.position(buffer.position() + nextBatch.sizeInBytes())); - if (!(nextBatch instanceof DefaultRecordBatch)) { - throw new IllegalStateException( - String.format("DefaultRecordBatch expected by record type was %s", nextBatch.getClass()) - ); - } - return Optional.of(readBatch((DefaultRecordBatch) nextBatch)); + if (!(nextBatch instanceof DefaultRecordBatch)) { + throw new IllegalStateException( + String.format("DefaultRecordBatch expected by record type was %s", nextBatch.getClass()) + ); } - nextBatches = nextBatches(); + return Optional.of(readBatch((DefaultRecordBatch) nextBatch)); } return Optional.empty(); From 029fea5610a159e907556055af428e1f5a03997f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Fri, 9 Apr 2021 21:25:41 -0700 Subject: [PATCH 15/19] Add for reading batches greater than the suggest size --- .../apache/kafka/raft/KafkaRaftClient.java | 1 - .../apache/kafka/raft/ReplicatedCounter.java | 6 +- .../raft/internals/RecordsBatchReader.java | 10 +-- ...ordsIterator.java => RecordsIterator.java} | 88 ++++++++++--------- .../kafka/snapshot/RawSnapshotReader.java | 2 - .../apache/kafka/snapshot/SnapshotReader.java | 8 +- .../java/org/apache/kafka/raft/MockLog.java | 16 ++-- .../kafka/raft/RaftEventSimulationTest.java | 11 +-- .../internals/RecordsBatchReaderTest.java | 8 +- ...atorTest.java => RecordsIteratorTest.java} | 86 ++++++++++-------- 10 files changed, 129 insertions(+), 107 deletions(-) rename raft/src/main/java/org/apache/kafka/raft/internals/{SerdeRecordsIterator.java => RecordsIterator.java} (77%) rename raft/src/test/java/org/apache/kafka/raft/internals/{SerdeRecordsIteratorTest.java => RecordsIteratorTest.java} (73%) diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 3f957df3dadc4..be324d35dfdeb 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -391,7 +391,6 @@ public void initialize() { // When there is only a single voter, become candidate immediately if (quorum.isVoter() && quorum.remoteVoters().isEmpty() - && !quorum.isLeader() && !quorum.isCandidate()) { transitionToCandidate(currentTimeMs); diff --git a/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java b/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java index b884b59e1f6b2..6e7a30f2f8d7e 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java +++ b/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java @@ -37,8 +37,6 @@ public class ReplicatedCounter implements RaftClient.Listener { private int uncommitted = 0; private OptionalInt claimedEpoch = OptionalInt.empty(); private long lastSnapshotEndOffset = 0; - private long nextReadOffset = 0; - private int readEpoch = 0; public ReplicatedCounter( int nodeId, @@ -72,6 +70,9 @@ public synchronized void increment() { public synchronized void handleCommit(BatchReader reader) { try { int initialCommitted = committed; + long nextReadOffset = 0; + int readEpoch = 0; + while (reader.hasNext()) { BatchReader.Batch batch = reader.next(); log.debug( @@ -103,6 +104,7 @@ public synchronized void handleCommit(BatchReader reader) { try (SnapshotWriter snapshot = client.createSnapshot(new OffsetAndEpoch(nextReadOffset, readEpoch))) { snapshot.append(singletonList(committed)); snapshot.freeze(); + lastSnapshotEndOffset = nextReadOffset; } } } finally { diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java index fb8e15e0c132f..2da14b50fbe11 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java @@ -27,7 +27,7 @@ public final class RecordsBatchReader implements BatchReader { private final long baseOffset; - private final SerdeRecordsIterator iterator; + private final RecordsIterator iterator; private final CloseListener> closeListener; private long lastReturnedOffset; @@ -37,7 +37,7 @@ public final class RecordsBatchReader implements BatchReader { private RecordsBatchReader( long baseOffset, - SerdeRecordsIterator iterator, + RecordsIterator iterator, CloseListener> closeListener ) { this.baseOffset = baseOffset; @@ -48,7 +48,7 @@ private RecordsBatchReader( @Override public boolean hasNext() { - checkIfClosed(); + ensureOpen(); if (!nextBatch.isPresent()) { nextBatch = nextBatch(); @@ -103,12 +103,12 @@ public static RecordsBatchReader of( ) { return new RecordsBatchReader<>( baseOffset, - new SerdeRecordsIterator<>(records, serde, bufferSupplier, maxBatchSize), + new RecordsIterator<>(records, serde, bufferSupplier, maxBatchSize), closeListener ); } - private void checkIfClosed() { + private void ensureOpen() { if (isClosed) { throw new IllegalStateException("Records batch reader was closed"); } diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/SerdeRecordsIterator.java b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java similarity index 77% rename from raft/src/main/java/org/apache/kafka/raft/internals/SerdeRecordsIterator.java rename to raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java index 07a2caba7906c..2172c0b92b20f 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/SerdeRecordsIterator.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java @@ -35,35 +35,35 @@ import org.apache.kafka.raft.BatchReader.Batch; import org.apache.kafka.raft.RecordSerde; -public final class SerdeRecordsIterator implements Iterator>, AutoCloseable { +public final class RecordsIterator implements Iterator>, AutoCloseable { private final Records records; private final RecordSerde serde; private final BufferSupplier bufferSupplier; - private final int maxBatchSize; + private final int batchSize; private Iterator nextBatches = Collections.emptyIterator(); private Optional> nextBatch = Optional.empty(); - // Buffer used to as the backing store for nextBatches if needed + // Buffer used as the backing store for nextBatches if needed private Optional allocatedBuffer = Optional.empty(); // Number of bytes from records read up to now private int bytesRead = 0; private boolean isClosed = false; - public SerdeRecordsIterator( + public RecordsIterator( Records records, RecordSerde serde, BufferSupplier bufferSupplier, - int maxBatchSize + int batchSize ) { this.records = records; this.serde = serde; this.bufferSupplier = bufferSupplier; - this.maxBatchSize = maxBatchSize; + this.batchSize = Math.max(batchSize, Records.HEADER_SIZE_UP_TO_MAGIC); } @Override public boolean hasNext() { - checkIfClosed(); + ensureOpen(); if (!nextBatch.isPresent()) { nextBatch = nextBatch(); @@ -91,12 +91,50 @@ public void close() { allocatedBuffer = Optional.empty(); } - private void checkIfClosed() { + private void ensureOpen() { if (isClosed) { throw new IllegalStateException("Serde record batch itererator was closed"); } } + private MemoryRecords readFileRecords(FileRecords fileRecords, ByteBuffer buffer) { + int start = buffer.position(); + try { + fileRecords.readInto(buffer, bytesRead); + } catch (IOException e) { + throw new RuntimeException("Failed to read records into memory", e); + } + + bytesRead += buffer.limit() - start; + return MemoryRecords.readableRecords(buffer.slice()); + } + + private MemoryRecords createMemoryRecords(FileRecords fileRecords) { + final ByteBuffer buffer; + if (allocatedBuffer.isPresent()) { + buffer = allocatedBuffer.get(); + buffer.compact(); + } else { + buffer = bufferSupplier.get(Math.min(batchSize, records.sizeInBytes())); + allocatedBuffer = Optional.of(buffer); + } + + MemoryRecords memoryRecords = readFileRecords(fileRecords, buffer); + + if (memoryRecords.firstBatchSize() < buffer.remaining()) { + return memoryRecords; + } else { + // Not enough bytes read; create a bigger buffer + ByteBuffer newBuffer = bufferSupplier.get(memoryRecords.firstBatchSize()); + allocatedBuffer = Optional.of(newBuffer); + + newBuffer.put(buffer); + bufferSupplier.release(buffer); + + return readFileRecords(fileRecords, newBuffer); + } + } + private Iterator nextBatches() { int recordSize = records.sizeInBytes(); if (bytesRead < recordSize) { @@ -105,42 +143,12 @@ private Iterator nextBatches() { bytesRead = recordSize; memoryRecords = (MemoryRecords) records; } else if (records instanceof FileRecords) { - final ByteBuffer buffer; - if (allocatedBuffer.isPresent()) { - buffer = allocatedBuffer.get(); - buffer.compact(); - } else { - buffer = bufferSupplier.get(Math.min(maxBatchSize, records.sizeInBytes())); - allocatedBuffer = Optional.of(buffer); - } - - int start = buffer.position(); - try { - ((FileRecords) records).readInto(buffer, bytesRead); - } catch (IOException e) { - throw new RuntimeException("Failed to read records into memory", e); - } - - bytesRead += buffer.limit() - start; - memoryRecords = MemoryRecords.readableRecords(buffer.slice()); + memoryRecords = createMemoryRecords((FileRecords) records); } else { throw new IllegalStateException(String.format("Unexpected Records type %s", records.getClass())); } - Iterator batches = memoryRecords.batchIterator(); - if (!batches.hasNext()) { - // The buffer is not big enough to read an entire batch - throw new IllegalStateException( - String.format( - "Unable to read batch from records buffer %s with maximum batch %s and record size %s", - allocatedBuffer, - maxBatchSize, - records.sizeInBytes() - ) - ); - } - - return batches; + return memoryRecords.batchIterator(); } return Collections.emptyIterator(); diff --git a/raft/src/main/java/org/apache/kafka/snapshot/RawSnapshotReader.java b/raft/src/main/java/org/apache/kafka/snapshot/RawSnapshotReader.java index 4f4523cbcf8fd..506728d10ae41 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/RawSnapshotReader.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/RawSnapshotReader.java @@ -33,8 +33,6 @@ public interface RawSnapshotReader extends Closeable { /** * Returns the number of bytes for the snapshot. - * - * @throws IOException for any IO error while reading the size */ long sizeInBytes(); diff --git a/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java index ac17026f5e74b..3940fb99c6351 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java @@ -22,7 +22,7 @@ import org.apache.kafka.raft.BatchReader.Batch; import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.raft.RecordSerde; -import org.apache.kafka.raft.internals.SerdeRecordsIterator; +import org.apache.kafka.raft.internals.RecordsIterator; /** * A type for reading an immutable snapshot. @@ -33,11 +33,11 @@ */ public final class SnapshotReader implements AutoCloseable, Iterator> { private final OffsetAndEpoch snapshotId; - private final SerdeRecordsIterator iterator; + private final RecordsIterator iterator; private SnapshotReader( OffsetAndEpoch snapshotId, - SerdeRecordsIterator iterator + RecordsIterator iterator ) { this.snapshotId = snapshotId; this.iterator = iterator; @@ -75,7 +75,7 @@ public static SnapshotReader of( ) { return new SnapshotReader<>( snapshot.snapshotId(), - new SerdeRecordsIterator<>(snapshot.records(), serde, bufferSupplier, maxBatchSize) + new RecordsIterator<>(snapshot.records(), serde, bufferSupplier, maxBatchSize) ); } } diff --git a/raft/src/test/java/org/apache/kafka/raft/MockLog.java b/raft/src/test/java/org/apache/kafka/raft/MockLog.java index aa61faaf836c0..4c45979f4a499 100644 --- a/raft/src/test/java/org/apache/kafka/raft/MockLog.java +++ b/raft/src/test/java/org/apache/kafka/raft/MockLog.java @@ -423,14 +423,20 @@ public void onSnapshotFrozen(OffsetAndEpoch snapshotId) {} @Override public boolean deleteBeforeSnapshot(OffsetAndEpoch snapshotId) { - if (logStartOffset() > snapshotId.offset || - highWatermark.offset < snapshotId.offset) { - + if (logStartOffset() > snapshotId.offset) { + throw new OffsetOutOfRangeException( + String.format( + "New log start (%s) is less than the curent log start offset (%s)", + snapshotId, + logStartOffset() + ) + ); + } + if (highWatermark.offset < snapshotId.offset) { throw new OffsetOutOfRangeException( String.format( - "New log start (%s) is less than start offset (%s) or is greater than the high watermark (%s)", + "New log start (%s) is greater than the high watermark (%s)", snapshotId, - logStartOffset(), highWatermark.offset ) ); diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java index c2679b1e1d647..5b85f6e0b3e85 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java @@ -1001,19 +1001,20 @@ private void assertCommittedData(RaftNode node) { } AtomicLong startOffset = new AtomicLong(0); - log.earliestSnapshotId().ifPresent(snapshoId -> { - assertTrue(snapshoId.offset <= highWatermark.getAsLong()); - startOffset.set(snapshoId.offset); + log.earliestSnapshotId().ifPresent(snapshotId -> { + assertTrue(snapshotId.offset <= highWatermark.getAsLong()); + startOffset.set(snapshotId.offset); try (SnapshotReader snapshot = - SnapshotReader.of(log.readSnapshot(snapshoId).get(), node.intSerde, BufferSupplier.create(), Integer.MAX_VALUE)) { + SnapshotReader.of(log.readSnapshot(snapshotId).get(), node.intSerde, BufferSupplier.create(), Integer.MAX_VALUE)) { // Expect only one batch with only one record assertTrue(snapshot.hasNext()); BatchReader.Batch batch = snapshot.next(); assertFalse(snapshot.hasNext()); assertEquals(1, batch.records().size()); - long offset = snapshoId.offset - 1; + // The snapshotId offset is an "end offset" + long offset = snapshotId.offset - 1; int sequence = batch.records().get(0); committedSequenceNumbers.putIfAbsent(offset, sequence); diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsBatchReaderTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsBatchReaderTest.java index 8c94d2d7b7c12..bc332d145993d 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsBatchReaderTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsBatchReaderTest.java @@ -49,8 +49,8 @@ class RecordsBatchReaderTest { public void testReadFromMemoryRecords(CompressionType compressionType) { long baseOffset = 57; - List> batches = SerdeRecordsIteratorTest.createBatches(baseOffset); - MemoryRecords memRecords = SerdeRecordsIteratorTest.buildRecords(compressionType, batches); + List> batches = RecordsIteratorTest.createBatches(baseOffset); + MemoryRecords memRecords = RecordsIteratorTest.buildRecords(compressionType, batches); testBatchReader(baseOffset, memRecords, batches); } @@ -60,8 +60,8 @@ public void testReadFromMemoryRecords(CompressionType compressionType) { public void testReadFromFileRecords(CompressionType compressionType) throws Exception { long baseOffset = 57; - List> batches = SerdeRecordsIteratorTest.createBatches(baseOffset); - MemoryRecords memRecords = SerdeRecordsIteratorTest.buildRecords(compressionType, batches); + List> batches = RecordsIteratorTest.createBatches(baseOffset); + MemoryRecords memRecords = RecordsIteratorTest.buildRecords(compressionType, batches); FileRecords fileRecords = FileRecords.open(tempFile()); fileRecords.append(memRecords); diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/SerdeRecordsIteratorTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java similarity index 73% rename from raft/src/test/java/org/apache/kafka/raft/internals/SerdeRecordsIteratorTest.java rename to raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java index 65da8a6ffd5a0..7d732f2e87bf3 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/SerdeRecordsIteratorTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java @@ -20,12 +20,18 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.IdentityHashMap; import java.util.List; import java.util.NoSuchElementException; +import java.util.Random; import java.util.Set; +import java.util.stream.Collectors; import java.util.stream.Stream; +import net.jqwik.api.ForAll; +import net.jqwik.api.Property; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.FileRecords; import org.apache.kafka.common.record.MemoryRecords; @@ -34,20 +40,16 @@ import org.apache.kafka.raft.BatchReader; import org.apache.kafka.raft.RecordSerde; import org.apache.kafka.test.TestUtils; -import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; -import org.junit.jupiter.params.provider.EnumSource; import org.junit.jupiter.params.provider.MethodSource; import org.mockito.Mockito; -import static java.util.Arrays.asList; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; -public final class SerdeRecordsIteratorTest { - private static final int MAX_BATCH_BYTES = 128; +public final class RecordsIteratorTest { private static final RecordSerde STRING_SERDE = new StringSerde(); private static Stream emptyRecords() throws IOException { @@ -63,19 +65,23 @@ void testEmptyRecords(Records records) throws IOException { testIterator(Collections.emptyList(), records); } - @ParameterizedTest - @EnumSource(CompressionType.class) - public void testMemoryRecords(CompressionType compressionType) { - List> batches = createBatches(57); + @Property + public void testMemoryRecords( + @ForAll CompressionType compressionType, + @ForAll long seed + ) { + List> batches = createBatches(seed); MemoryRecords memRecords = buildRecords(compressionType, batches); testIterator(batches, memRecords); } - @ParameterizedTest - @EnumSource(CompressionType.class) - public void testFileRecords(CompressionType compressionType) throws IOException { - List> batches = createBatches(57); + @Property + public void testFileRecords( + @ForAll CompressionType compressionType, + @ForAll long seed + ) throws IOException { + List> batches = createBatches(seed); MemoryRecords memRecords = buildRecords(compressionType, batches); FileRecords fileRecords = FileRecords.open(TestUtils.tempFile()); @@ -84,29 +90,15 @@ public void testFileRecords(CompressionType compressionType) throws IOException testIterator(batches, fileRecords); } - @Test - public void testMaxBatchTooSmall() throws IOException { - List> batches = createBatches(57); - - MemoryRecords memRecords = buildRecords(CompressionType.NONE, batches); - FileRecords fileRecords = FileRecords.open(TestUtils.tempFile()); - fileRecords.append(memRecords); - - SerdeRecordsIterator iterator = createIterator(fileRecords, BufferSupplier.create(), 10); - assertThrows(IllegalStateException.class, iterator::hasNext); - assertThrows(IllegalStateException.class, iterator::next); - } - private void testIterator( List> expectedBatches, Records records ) { Set allocatedBuffers = Collections.newSetFromMap(new IdentityHashMap<>()); - SerdeRecordsIterator iterator = createIterator( + RecordsIterator iterator = createIterator( records, - mockBufferSupplier(allocatedBuffers), - MAX_BATCH_BYTES + mockBufferSupplier(allocatedBuffers) ); for (BatchReader.Batch batch : expectedBatches) { @@ -121,8 +113,8 @@ private void testIterator( assertEquals(Collections.emptySet(), allocatedBuffers); } - static SerdeRecordsIterator createIterator(Records records, BufferSupplier bufferSupplier, int maxBatchSize) { - return new SerdeRecordsIterator<>(records, STRING_SERDE, bufferSupplier, maxBatchSize); + static RecordsIterator createIterator(Records records, BufferSupplier bufferSupplier) { + return new RecordsIterator<>(records, STRING_SERDE, bufferSupplier, Records.HEADER_SIZE_UP_TO_MAGIC); } static BufferSupplier mockBufferSupplier(Set buffers) { @@ -144,19 +136,35 @@ static BufferSupplier mockBufferSupplier(Set buffers) { return bufferSupplier; } - public static List> createBatches(long baseOffset) { - return asList( - BatchReader.Batch.of(baseOffset, 1, asList("a", "b", "c")), - BatchReader.Batch.of(baseOffset + 3, 2, asList("d", "e")), - BatchReader.Batch.of(baseOffset + 5, 2, asList("f")) - ); + public static List> createBatches(long seed) { + Random random = new Random(seed); + long baseOffset = random.nextInt(100); + int epoch = random.nextInt(3) + 1; + + int numberOfBatches = random.nextInt(100) + 1; + List> batches = new ArrayList<>(numberOfBatches); + for (int i = 0; i < numberOfBatches; i++) { + int numberOfRecords = random.nextInt(100) + 1; + List records = random + .ints(numberOfRecords, 0, 10) + .mapToObj(String::valueOf) + .collect(Collectors.toList()); + + batches.add(BatchReader.Batch.of(baseOffset, epoch, records)); + baseOffset += records.size(); + if (i % 5 == 0) { + epoch += random.nextInt(3); + } + } + + return batches; } public static MemoryRecords buildRecords( CompressionType compressionType, List> batches ) { - ByteBuffer buffer = ByteBuffer.allocate(1024); + ByteBuffer buffer = ByteBuffer.allocate(102400); for (BatchReader.Batch batch : batches) { BatchBuilder builder = new BatchBuilder<>( @@ -167,7 +175,7 @@ public static MemoryRecords buildRecords( 12345L, false, batch.epoch(), - MAX_BATCH_BYTES + 1024 ); for (String record : batch.records()) { From d55165ea5e05e530e29dbb8c9bf77502fc33bec9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Tue, 27 Apr 2021 14:56:58 -0700 Subject: [PATCH 16/19] Fix imports --- .../org/apache/kafka/raft/internals/RecordsIteratorTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java index 7d732f2e87bf3..40c6aad25da21 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java @@ -21,7 +21,6 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.IdentityHashMap; import java.util.List; From 5a3aafdd55a232ea5ec9edd9843fa7408496d14a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Thu, 29 Apr 2021 13:10:19 -0700 Subject: [PATCH 17/19] Add snapshot at log start validation for the simulation --- .../scala/kafka/tools/TestRaftServer.scala | 5 +- .../java/org/apache/kafka/raft/Batch.java | 133 ++++++++++++++++++ .../org/apache/kafka/raft/BatchReader.java | 73 +--------- .../apache/kafka/raft/KafkaRaftClient.java | 5 +- .../apache/kafka/raft/ReplicatedCounter.java | 3 +- .../raft/internals/MemoryBatchReader.java | 1 + .../raft/internals/RecordsBatchReader.java | 1 + .../kafka/raft/internals/RecordsIterator.java | 5 +- .../kafka/raft/metadata/MetaLogRaftShim.java | 3 +- .../apache/kafka/snapshot/SnapshotReader.java | 2 +- .../raft/KafkaRaftClientSnapshotTest.java | 22 +-- .../kafka/raft/KafkaRaftClientTest.java | 30 ++-- .../java/org/apache/kafka/raft/MockLog.java | 18 ++- .../org/apache/kafka/raft/MockLogTest.java | 110 ++++++--------- .../kafka/raft/RaftClientTestContext.java | 10 +- .../kafka/raft/RaftEventSimulationTest.java | 51 ++++++- .../raft/internals/MemoryBatchReaderTest.java | 7 +- .../internals/RecordsBatchReaderTest.java | 9 +- .../raft/internals/RecordsIteratorTest.java | 22 ++- .../snapshot/SnapshotWriterReaderTest.java | 2 +- .../kafka/shell/MetadataNodeManager.java | 5 +- 21 files changed, 306 insertions(+), 211 deletions(-) create mode 100644 raft/src/main/java/org/apache/kafka/raft/Batch.java diff --git a/core/src/main/scala/kafka/tools/TestRaftServer.scala b/core/src/main/scala/kafka/tools/TestRaftServer.scala index 822b54fd18aa2..3d432159ce4c9 100644 --- a/core/src/main/scala/kafka/tools/TestRaftServer.scala +++ b/core/src/main/scala/kafka/tools/TestRaftServer.scala @@ -35,8 +35,7 @@ import org.apache.kafka.common.security.scram.internals.ScramMechanism import org.apache.kafka.common.security.token.delegation.internals.DelegationTokenCache import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{TopicPartition, Uuid, protocol} -import org.apache.kafka.raft.BatchReader.Batch -import org.apache.kafka.raft.{BatchReader, RaftClient, RaftConfig, RecordSerde} +import org.apache.kafka.raft.{Batch, BatchReader, RaftClient, RaftConfig, RecordSerde} import org.apache.kafka.snapshot.SnapshotReader import scala.jdk.CollectionConverters._ @@ -233,7 +232,7 @@ class TestRaftServer( } case HandleSnapshot(reader) => - // Ignore snapshots; only interested on records appended by this leader + // Ignore snapshots; only interested in records appended by this leader reader.close() case Shutdown => // Ignore shutdown command diff --git a/raft/src/main/java/org/apache/kafka/raft/Batch.java b/raft/src/main/java/org/apache/kafka/raft/Batch.java new file mode 100644 index 0000000000000..daa1e05c07a12 --- /dev/null +++ b/raft/src/main/java/org/apache/kafka/raft/Batch.java @@ -0,0 +1,133 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.raft; + +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; + +/** + * A batch of records. + * + * This type contains a list of records `T` along with the information associated with those records. + */ +public final class Batch implements Iterable { + private final long baseOffset; + private final int epoch; + private final long lastOffset; + private final List records; + + private Batch(long baseOffset, int epoch, long lastOffset, List records) { + this.baseOffset = baseOffset; + this.epoch = epoch; + this.lastOffset = lastOffset; + this.records = records; + } + + /** + * The offset of the last record in the batch. + */ + public long lastOffset() { + return lastOffset; + } + + /** + * The offset of the first record in the batch. + */ + public long baseOffset() { + return baseOffset; + } + + /** + * The list of records in the batch. + */ + public List records() { + return records; + } + + /** + * The epoch of the leader that appended the record batch. + */ + public int epoch() { + return epoch; + } + + @Override + public Iterator iterator() { + return records.iterator(); + } + + @Override + public String toString() { + return "Batch(" + + "baseOffset=" + baseOffset + + ", epoch=" + epoch + + ", lastOffset=" + lastOffset + + ", records=" + records + + ')'; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Batch batch = (Batch) o; + return baseOffset == batch.baseOffset && + epoch == batch.epoch && + Objects.equals(records, batch.records); + } + + @Override + public int hashCode() { + return Objects.hash(baseOffset, epoch, records); + } + + /** + * Create a batch without any records. + * + * Internally this is used to propagate offset information for control batches which do not decode to the type T. + * + * @param baseOffset offset of the batch + * @param epoch epoch of the leader that created this batch + * @param lastOffset offset of the last record of this batch + */ + public static Batch empty(long baseOffset, int epoch, long lastOffset) { + return new Batch<>(baseOffset, epoch, lastOffset, Collections.emptyList()); + } + + /** + * Create a batch with the given base offset, epoch and records. + * + * @param baseOffset offset of the first record in the batch + * @param epoch epoch of the leader that created this batch + * @param records the list of records in this batch + */ + public static Batch of(long baseOffset, int epoch, List records) { + if (records.isEmpty()) { + throw new IllegalArgumentException( + String.format( + "Batch must contain at least one record; baseOffset = %s; epoch = %s", + baseOffset, + epoch + ) + ); + } + + return new Batch<>(baseOffset, epoch, baseOffset + records.size() - 1, records); + } +} diff --git a/raft/src/main/java/org/apache/kafka/raft/BatchReader.java b/raft/src/main/java/org/apache/kafka/raft/BatchReader.java index cf01c039c08a6..6469af2095f57 100644 --- a/raft/src/main/java/org/apache/kafka/raft/BatchReader.java +++ b/raft/src/main/java/org/apache/kafka/raft/BatchReader.java @@ -16,10 +16,7 @@ */ package org.apache.kafka.raft; -import java.util.Collections; import java.util.Iterator; -import java.util.List; -import java.util.Objects; import java.util.OptionalLong; /** @@ -33,7 +30,7 @@ * * @param record type (see {@link org.apache.kafka.raft.RecordSerde}) */ -public interface BatchReader extends Iterator>, AutoCloseable { +public interface BatchReader extends Iterator>, AutoCloseable { /** * Get the base offset of the readable batches. Note that this value is a constant @@ -60,72 +57,4 @@ public interface BatchReader extends Iterator>, AutoClos */ @Override void close(); - - final class Batch implements Iterable { - private final long baseOffset; - private final int epoch; - private final long lastOffset; - private final List records; - - private Batch(long baseOffset, int epoch, long lastOffset, List records) { - this.baseOffset = baseOffset; - this.epoch = epoch; - this.lastOffset = lastOffset; - this.records = records; - } - - public long lastOffset() { - return lastOffset; - } - - public long baseOffset() { - return baseOffset; - } - - public List records() { - return records; - } - - public int epoch() { - return epoch; - } - - @Override - public Iterator iterator() { - return records.iterator(); - } - - @Override - public String toString() { - return "Batch(" + - "baseOffset=" + baseOffset + - ", epoch=" + epoch + - ", lastOffset=" + lastOffset + - ", records=" + records + - ')'; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - Batch batch = (Batch) o; - return baseOffset == batch.baseOffset && - epoch == batch.epoch && - Objects.equals(records, batch.records); - } - - @Override - public int hashCode() { - return Objects.hash(baseOffset, epoch, records); - } - - public static Batch empty(long baseOffset, int epoch, long lastOffset) { - return new Batch<>(baseOffset, epoch, lastOffset, Collections.emptyList()); - } - - public static Batch of(long baseOffset, int epoch, List records) { - return new Batch<>(baseOffset, epoch, baseOffset + records.size() - 1, records); - } - } } diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index be324d35dfdeb..e010122e5af38 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -315,7 +315,8 @@ private void updateListenersProgress(List listenerContexts, lon SnapshotReader snapshot = latestSnapshot().orElseThrow(() -> { return new IllegalStateException( String.format( - "Snapshot expected when next offset is %s, log start offset is %s and high-watermark is %s", + "Snapshot expected since next offset of %s is %s, log start offset is %s and high-watermark is %s", + listenerContext.listener.getClass().getTypeName(), nextExpectedOffset, log.startOffset(), highWatermark @@ -2446,7 +2447,7 @@ public void fireHandleCommit(long baseOffset, Records records) { * followers. */ public void fireHandleCommit(long baseOffset, int epoch, List records) { - BatchReader.Batch batch = BatchReader.Batch.of(baseOffset, epoch, records); + Batch batch = Batch.of(baseOffset, epoch, records); MemoryBatchReader reader = new MemoryBatchReader<>(Collections.singletonList(batch), this); fireHandleCommit(reader); } diff --git a/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java b/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java index 6e7a30f2f8d7e..634ce37392f17 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java +++ b/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java @@ -18,7 +18,6 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.utils.LogContext; -import org.apache.kafka.raft.BatchReader.Batch; import org.apache.kafka.snapshot.SnapshotReader; import org.apache.kafka.snapshot.SnapshotWriter; import org.slf4j.Logger; @@ -74,7 +73,7 @@ public synchronized void handleCommit(BatchReader reader) { int readEpoch = 0; while (reader.hasNext()) { - BatchReader.Batch batch = reader.next(); + Batch batch = reader.next(); log.debug( "Handle commit of batch with records {} at base offset {}", batch.records(), diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/MemoryBatchReader.java b/raft/src/main/java/org/apache/kafka/raft/internals/MemoryBatchReader.java index 2ff2b8cdda7b6..07ad1bbcb4533 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/MemoryBatchReader.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/MemoryBatchReader.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.raft.internals; +import org.apache.kafka.raft.Batch; import org.apache.kafka.raft.BatchReader; import java.util.Iterator; diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java index 2da14b50fbe11..a55815d0b5e5e 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.record.Records; import org.apache.kafka.common.utils.BufferSupplier; +import org.apache.kafka.raft.Batch; import org.apache.kafka.raft.BatchReader; import org.apache.kafka.raft.RecordSerde; diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java index 2172c0b92b20f..46155b57ee4f6 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java @@ -32,7 +32,7 @@ import org.apache.kafka.common.record.MutableRecordBatch; import org.apache.kafka.common.record.Records; import org.apache.kafka.common.utils.BufferSupplier; -import org.apache.kafka.raft.BatchReader.Batch; +import org.apache.kafka.raft.Batch; import org.apache.kafka.raft.RecordSerde; public final class RecordsIterator implements Iterator>, AutoCloseable { @@ -121,7 +121,8 @@ private MemoryRecords createMemoryRecords(FileRecords fileRecords) { MemoryRecords memoryRecords = readFileRecords(fileRecords, buffer); - if (memoryRecords.firstBatchSize() < buffer.remaining()) { + // firstBatchSize() is always non-null because the minimum buffer is HEADER_SIZE_UP_TO_MAGIC. + if (memoryRecords.firstBatchSize() <= buffer.remaining()) { return memoryRecords; } else { // Not enough bytes read; create a bigger buffer diff --git a/raft/src/main/java/org/apache/kafka/raft/metadata/MetaLogRaftShim.java b/raft/src/main/java/org/apache/kafka/raft/metadata/MetaLogRaftShim.java index 22b7ca067baa2..0bcf2c67055b1 100644 --- a/raft/src/main/java/org/apache/kafka/raft/metadata/MetaLogRaftShim.java +++ b/raft/src/main/java/org/apache/kafka/raft/metadata/MetaLogRaftShim.java @@ -21,6 +21,7 @@ import org.apache.kafka.metalog.MetaLogLeader; import org.apache.kafka.metalog.MetaLogListener; import org.apache.kafka.metalog.MetaLogManager; +import org.apache.kafka.raft.Batch; import org.apache.kafka.raft.BatchReader; import org.apache.kafka.raft.LeaderAndEpoch; import org.apache.kafka.raft.RaftClient; @@ -114,7 +115,7 @@ public void handleCommit(BatchReader reader) { // not a leader. We want to move this IO to the state machine so that // it does not block Raft replication while (reader.hasNext()) { - BatchReader.Batch batch = reader.next(); + Batch batch = reader.next(); List records = batch.records().stream() .map(ApiMessageAndVersion::message) .collect(Collectors.toList()); diff --git a/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java index 3940fb99c6351..af00cdb2286bf 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java @@ -19,7 +19,7 @@ import java.util.Iterator; import org.apache.kafka.common.utils.BufferSupplier; -import org.apache.kafka.raft.BatchReader.Batch; +import org.apache.kafka.raft.Batch; import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.raft.RecordSerde; import org.apache.kafka.raft.internals.RecordsIterator; diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java index 40b99e92e174c..6fd5147421165 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java @@ -58,7 +58,7 @@ public void testLeaderListernerNotified() throws Exception { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .appendToLog(snapshotId.epoch, Arrays.asList("a", "b", "c")) .appendToLog(snapshotId.epoch, Arrays.asList("d", "e", "f")) - .withSnapshot(snapshotId) + .withEmptySnapshot(snapshotId) .deleteBeforeSnapshot(snapshotId) .build(); @@ -73,7 +73,7 @@ public void testLeaderListernerNotified() throws Exception { assertEquals(localLogEndOffset, context.client.highWatermark().getAsLong()); // Check that listener was notified of the new snapshot - try (SnapshotReader snapshot = context.listener.takeSnapshot().get()) { + try (SnapshotReader snapshot = context.listener.drainHandledSnapshot().get()) { assertEquals(snapshotId, snapshot.snapshotId()); SnapshotWriterReaderTest.assertSnapshot(Arrays.asList(), snapshot); } @@ -90,7 +90,7 @@ public void testFollowerListenerNotified() throws Exception { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .appendToLog(snapshotId.epoch, Arrays.asList("a", "b", "c")) .appendToLog(snapshotId.epoch, Arrays.asList("d", "e", "f")) - .withSnapshot(snapshotId) + .withEmptySnapshot(snapshotId) .deleteBeforeSnapshot(snapshotId) .withElectedLeader(epoch, leaderId) .build(); @@ -110,7 +110,7 @@ public void testFollowerListenerNotified() throws Exception { context.assertSentFetchRequest(epoch, localLogEndOffset, snapshotId.epoch); // Check that listener was notified of the new snapshot - try (SnapshotReader snapshot = context.listener.takeSnapshot().get()) { + try (SnapshotReader snapshot = context.listener.drainHandledSnapshot().get()) { assertEquals(snapshotId, snapshot.snapshotId()); SnapshotWriterReaderTest.assertSnapshot(Arrays.asList(), snapshot); } @@ -127,7 +127,7 @@ public void testSecondListenerNotified() throws Exception { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .appendToLog(snapshotId.epoch, Arrays.asList("a", "b", "c")) .appendToLog(snapshotId.epoch, Arrays.asList("d", "e", "f")) - .withSnapshot(snapshotId) + .withEmptySnapshot(snapshotId) .withElectedLeader(epoch, leaderId) .build(); @@ -150,7 +150,7 @@ public void testSecondListenerNotified() throws Exception { context.client.poll(); // Check that the second listener was notified of the new snapshot - try (SnapshotReader snapshot = secondListener.takeSnapshot().get()) { + try (SnapshotReader snapshot = secondListener.drainHandledSnapshot().get()) { assertEquals(snapshotId, snapshot.snapshotId()); SnapshotWriterReaderTest.assertSnapshot(Arrays.asList(), snapshot); } @@ -167,7 +167,7 @@ public void testListenerRenotified() throws Exception { .appendToLog(snapshotId.epoch, Arrays.asList("a", "b", "c")) .appendToLog(snapshotId.epoch, Arrays.asList("d", "e", "f")) .appendToLog(snapshotId.epoch, Arrays.asList("g", "h", "i")) - .withSnapshot(snapshotId) + .withEmptySnapshot(snapshotId) .deleteBeforeSnapshot(snapshotId) .build(); @@ -185,7 +185,7 @@ public void testListenerRenotified() throws Exception { assertEquals(localLogEndOffset, context.client.highWatermark().getAsLong()); // Check that listener was notified of the new snapshot - try (SnapshotReader snapshot = context.listener.takeSnapshot().get()) { + try (SnapshotReader snapshot = context.listener.drainHandledSnapshot().get()) { assertEquals(snapshotId, snapshot.snapshotId()); SnapshotWriterReaderTest.assertSnapshot(Arrays.asList(), snapshot); } @@ -202,7 +202,7 @@ public void testListenerRenotified() throws Exception { context.client.poll(); // Check that listener was notified of the second snapshot - try (SnapshotReader snapshot = context.listener.takeSnapshot().get()) { + try (SnapshotReader snapshot = context.listener.drainHandledSnapshot().get()) { assertEquals(secondSnapshot, snapshot.snapshotId()); SnapshotWriterReaderTest.assertSnapshot(Arrays.asList(), snapshot); } @@ -915,7 +915,7 @@ public void testFetchResponseWithSnapshotId() throws Exception { } // Check that listener was notified of the new snapshot - try (SnapshotReader snapshot = context.listener.takeSnapshot().get()) { + try (SnapshotReader snapshot = context.listener.drainHandledSnapshot().get()) { assertEquals(snapshotId, snapshot.snapshotId()); SnapshotWriterReaderTest.assertSnapshot(Arrays.asList(records), snapshot); } @@ -1019,7 +1019,7 @@ public void testFetchSnapshotResponsePartialData() throws Exception { } // Check that listener was notified of the new snapshot - try (SnapshotReader snapshot = context.listener.takeSnapshot().get()) { + try (SnapshotReader snapshot = context.listener.drainHandledSnapshot().get()) { assertEquals(snapshotId, snapshot.snapshotId()); SnapshotWriterReaderTest.assertSnapshot(Arrays.asList(records), snapshot); } diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java index 908544a4cba61..6322b0ac734f7 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -2240,6 +2240,7 @@ public void testHandleClaimCallbackFiresAfterHighWatermarkReachesEpochStartOffse List batch2 = Arrays.asList("4", "5", "6"); List batch3 = Arrays.asList("7", "8", "9"); + List> expectedBatches = Arrays.asList(batch1, batch2, batch3); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .appendToLog(1, batch1) .appendToLog(1, batch2) @@ -2271,23 +2272,24 @@ public void testHandleClaimCallbackFiresAfterHighWatermarkReachesEpochStartOffse // watermark advances and we can start sending committed data to the // listener. Note that the `LeaderChange` control record is filtered. context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 10L, epoch, 500)); - context.client.poll(); - assertEquals(OptionalInt.empty(), context.listener.currentClaimedEpoch()); - assertEquals(1, context.listener.numCommittedBatches()); - assertEquals(batch1, context.listener.commitWithBaseOffset(0L)); + context.pollUntil(() -> { + int committedBatches = context.listener.numCommittedBatches(); + long baseOffset = 0; + for (int index = 0; index < committedBatches; index++) { + List expectedBatch = expectedBatches.get(index); + assertEquals(expectedBatch, context.listener.commitWithBaseOffset(baseOffset)); + baseOffset += expectedBatch.size(); + } - context.client.poll(); - assertEquals(OptionalInt.empty(), context.listener.currentClaimedEpoch()); - assertEquals(2, context.listener.numCommittedBatches()); - assertEquals(batch2, context.listener.commitWithBaseOffset(3L)); + return context.listener.currentClaimedEpoch().isPresent(); + }); - // Now that the listener has caught up to the start of the leader epoch, - // we expect the `handleClaim` callback. - context.client.poll(); assertEquals(OptionalInt.of(epoch), context.listener.currentClaimedEpoch()); - assertEquals(3, context.listener.numCommittedBatches()); - assertEquals(batch3, context.listener.commitWithBaseOffset(6L)); - assertEquals(OptionalLong.of(8L), context.listener.lastCommitOffset()); + // Note that last committed offset is inclusive, hence we subtract 1. + assertEquals( + OptionalLong.of(expectedBatches.stream().mapToInt(List::size).sum() - 1), + context.listener.lastCommitOffset() + ); } @Test diff --git a/raft/src/test/java/org/apache/kafka/raft/MockLog.java b/raft/src/test/java/org/apache/kafka/raft/MockLog.java index 4c45979f4a499..bf03a06ee8f4f 100644 --- a/raft/src/test/java/org/apache/kafka/raft/MockLog.java +++ b/raft/src/test/java/org/apache/kafka/raft/MockLog.java @@ -362,6 +362,7 @@ public LogFetchInfo read(long startOffset, Isolation isolation) { } ByteBuffer buffer = ByteBuffer.allocate(512); + int batchCount = 0; LogOffsetMetadata batchStartOffset = null; for (LogBatch batch : batches) { @@ -371,9 +372,17 @@ public LogFetchInfo read(long startOffset, Isolation isolation) { // batch returned in a fetch response. if (batch.lastOffset() >= startOffset && batch.lastOffset() < maxOffset && !batch.entries.isEmpty()) { buffer = batch.writeTo(buffer); - batchStartOffset = batch.entries.get(0).logOffsetMetadata(); - break; + if (batchStartOffset == null) { + batchStartOffset = batch.entries.get(0).logOffsetMetadata(); + } + + // Read on the mock log should return at most 2 batches. This is a simple solution + // for testing interesting partial read scenarios. + batchCount += 1; + if (batchCount >= 2) { + break; + } } } @@ -446,11 +455,6 @@ public boolean deleteBeforeSnapshot(OffsetAndEpoch snapshotId) { if (snapshots.containsKey(snapshotId)) { snapshots.headMap(snapshotId, false).clear(); - // Update the high watermark if it is less than the new log start offset - if (snapshotId.offset > highWatermark.offset) { - updateHighWatermark(new LogOffsetMetadata(snapshotId.offset)); - } - batches.removeIf(entry -> entry.lastOffset() < snapshotId.offset); AtomicReference> last = new AtomicReference<>(Optional.empty()); diff --git a/raft/src/test/java/org/apache/kafka/raft/MockLogTest.java b/raft/src/test/java/org/apache/kafka/raft/MockLogTest.java index c633e44834e12..492410b75a2dd 100644 --- a/raft/src/test/java/org/apache/kafka/raft/MockLogTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/MockLogTest.java @@ -75,56 +75,6 @@ public void testTopicId() { assertEquals(topicId, log.topicId()); } - @Test - public void testAppendAsLeaderHelper() { - int epoch = 2; - SimpleRecord recordOne = new SimpleRecord("one".getBytes()); - appendAsLeader(Collections.singleton(recordOne), epoch); - assertEquals(epoch, log.lastFetchedEpoch()); - assertEquals(0L, log.startOffset()); - assertEquals(1L, log.endOffset().offset); - - Records records = log.read(0, Isolation.UNCOMMITTED).records; - List batches = Utils.toList(records.batches().iterator()); - - RecordBatch batch = batches.get(0); - assertEquals(0, batch.baseOffset()); - assertEquals(0, batch.lastOffset()); - - List fetchedRecords = Utils.toList(batch.iterator()); - assertEquals(1, fetchedRecords.size()); - assertEquals(recordOne, new SimpleRecord(fetchedRecords.get(0))); - assertEquals(0, fetchedRecords.get(0).offset()); - - SimpleRecord recordTwo = new SimpleRecord("two".getBytes()); - SimpleRecord recordThree = new SimpleRecord("three".getBytes()); - appendAsLeader(Arrays.asList(recordTwo, recordThree), epoch); - assertEquals(0L, log.startOffset()); - assertEquals(3L, log.endOffset().offset); - - records = log.read(0, Isolation.UNCOMMITTED).records; - batches = Utils.toList(records.batches().iterator()); - assertEquals(1, batches.size()); - - fetchedRecords = Utils.toList(records.records().iterator()); - assertEquals(1, fetchedRecords.size()); - assertEquals(recordOne, new SimpleRecord(fetchedRecords.get(0))); - assertEquals(0, fetchedRecords.get(0).offset()); - - records = log.read(fetchedRecords.size(), Isolation.UNCOMMITTED).records; - batches = Utils.toList(records.batches().iterator()); - assertEquals(1, batches.size()); - - fetchedRecords = Utils.toList(records.records().iterator()); - assertEquals(2, fetchedRecords.size()); - - assertEquals(recordTwo, new SimpleRecord(fetchedRecords.get(0))); - assertEquals(1, fetchedRecords.get(0).offset()); - - assertEquals(recordThree, new SimpleRecord(fetchedRecords.get(1))); - assertEquals(2, fetchedRecords.get(1).offset()); - } - @Test public void testTruncateTo() { int epoch = 2; @@ -181,28 +131,26 @@ public void testAssignEpochStartOffset() { @Test public void testAppendAsLeader() { - SimpleRecord recordFoo = new SimpleRecord("foo".getBytes()); - final int currentEpoch = 3; - final long initialOffset = log.endOffset().offset; + int epoch = 2; + SimpleRecord recordOne = new SimpleRecord("one".getBytes()); + List expectedRecords = new ArrayList<>(); - log.appendAsLeader( - MemoryRecords.withRecords(initialOffset, CompressionType.NONE, recordFoo), - currentEpoch - ); + expectedRecords.add(recordOne); + appendAsLeader(Collections.singleton(recordOne), epoch); - assertEquals(0, log.startOffset()); - assertEquals(1, log.endOffset().offset); - assertEquals(currentEpoch, log.lastFetchedEpoch()); + assertEquals(new OffsetAndEpoch(expectedRecords.size(), epoch), log.endOffsetForEpoch(epoch)); + assertEquals(epoch, log.lastFetchedEpoch()); + validateReadRecords(expectedRecords, log); - Records records = log.read(0, Isolation.UNCOMMITTED).records; - List extractRecords = new ArrayList<>(); - for (Record record : records.records()) { - extractRecords.add(record.value()); - } + SimpleRecord recordTwo = new SimpleRecord("two".getBytes()); + SimpleRecord recordThree = new SimpleRecord("three".getBytes()); + expectedRecords.add(recordTwo); + expectedRecords.add(recordThree); + appendAsLeader(Arrays.asList(recordTwo, recordThree), epoch); - assertEquals(1, extractRecords.size()); - assertEquals(recordFoo.value(), extractRecords.get(0)); - assertEquals(new OffsetAndEpoch(1, currentEpoch), log.endOffsetForEpoch(currentEpoch)); + assertEquals(new OffsetAndEpoch(expectedRecords.size(), epoch), log.endOffsetForEpoch(epoch)); + assertEquals(epoch, log.lastFetchedEpoch()); + validateReadRecords(expectedRecords, log); } @Test @@ -890,4 +838,30 @@ private void appendBatch(int numRecords, int epoch) { appendAsLeader(records, epoch); } + + private static void validateReadRecords(List expectedRecords, MockLog log) { + assertEquals(0L, log.startOffset()); + assertEquals(expectedRecords.size(), log.endOffset().offset); + + int currentOffset = 0; + while (currentOffset < log.endOffset().offset) { + Records records = log.read(currentOffset, Isolation.UNCOMMITTED).records; + List batches = Utils.toList(records.batches().iterator()); + + assertTrue(batches.size() > 0); + for (RecordBatch batch : batches) { + assertTrue(batch.countOrNull() > 0); + assertEquals(currentOffset, batch.baseOffset()); + assertEquals(currentOffset + batch.countOrNull() - 1, batch.lastOffset()); + + for (Record record : batch) { + assertEquals(currentOffset, record.offset()); + assertEquals(expectedRecords.get(currentOffset), new SimpleRecord(record)); + currentOffset += 1; + } + + assertEquals(currentOffset - 1, batch.lastOffset()); + } + } + } } diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java index a3a0c820aaba5..a70e5b638225b 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -187,7 +187,7 @@ Builder appendToLog(int epoch, List records) { return this; } - Builder withSnapshot(OffsetAndEpoch snapshotId) throws IOException { + Builder withEmptySnapshot(OffsetAndEpoch snapshotId) throws IOException { try (RawSnapshotWriter snapshot = log.createSnapshot(snapshotId)) { snapshot.freeze(); } @@ -1052,7 +1052,7 @@ FetchResponseData divergingFetchResponse( } static class MockListener implements RaftClient.Listener { - private final List> commits = new ArrayList<>(); + private final List> commits = new ArrayList<>(); private final List> savedBatches = new ArrayList<>(); private final Map claimedEpochStartOffsets = new HashMap<>(); private OptionalInt currentClaimedEpoch = OptionalInt.empty(); @@ -1067,7 +1067,7 @@ Long claimedEpochStartOffset(int epoch) { return claimedEpochStartOffsets.get(epoch); } - BatchReader.Batch lastCommit() { + Batch lastCommit() { if (commits.isEmpty()) { return null; } else { @@ -1103,7 +1103,7 @@ List commitWithLastOffset(long lastOffset) { .orElse(null); } - Optional> takeSnapshot() { + Optional> drainHandledSnapshot() { Optional> temp = snapshot; snapshot = Optional.empty(); return temp; @@ -1126,7 +1126,7 @@ void readBatch(BatchReader reader) { while (reader.hasNext()) { long nextOffset = lastCommitOffset().isPresent() ? lastCommitOffset().getAsLong() + 1 : 0L; - BatchReader.Batch batch = reader.next(); + Batch batch = reader.next(); // We expect monotonic offsets, but not necessarily sequential // offsets since control records will be filtered. assertTrue(batch.baseOffset() >= nextOffset, diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java index 5b85f6e0b3e85..b6ec2056822e7 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java @@ -350,6 +350,7 @@ private EventScheduler schedulerWithDefaultInvariants(Cluster cluster) { scheduler.addInvariant(new MonotonicEpoch(cluster)); scheduler.addInvariant(new MajorityReachedHighWatermark(cluster)); scheduler.addInvariant(new SingleLeader(cluster)); + scheduler.addInvariant(new SnapshotAtLogStart(cluster)); scheduler.addValidation(new ConsistentCommittedData(cluster)); return scheduler; } @@ -966,6 +967,54 @@ public void verify() { } } + private static class SnapshotAtLogStart implements Invariant { + final Cluster cluster; + + private SnapshotAtLogStart(Cluster cluster) { + this.cluster = cluster; + } + + @Override + public void verify() { + for (Map.Entry nodeEntry : cluster.nodes.entrySet()) { + int nodeId = nodeEntry.getKey(); + ReplicatedLog log = nodeEntry.getValue().log; + log.earliestSnapshotId().ifPresent(earliestSnapshotId -> { + assertTrue( + log.startOffset() <= earliestSnapshotId.offset, + String.format( + "invalid log start offset (%s) and snapshotId offset (%s): nodeId = %s", + log.startOffset(), + earliestSnapshotId.offset, + nodeId + ) + ); + assertEquals( + log.validateOffsetAndEpoch( + earliestSnapshotId.offset, + earliestSnapshotId.epoch + ).kind(), + ValidOffsetAndEpoch.Kind.VALID, + String.format("invalid offset and epoch for ealiest snapshot: nodeId = %s", nodeId) + ); + + if (log.startOffset() > 0) { + assertEquals( + log.startOffset(), + earliestSnapshotId.offset, + String.format("mising snapshot at log start offset: nodeId = %s", nodeId) + ); + assertEquals( + ValidOffsetAndEpoch.valid(earliestSnapshotId), + log.validateOffsetAndEpoch(earliestSnapshotId.offset, earliestSnapshotId.epoch), + String.format("invalid leader epoch cache: nodeId = %s", nodeId) + ); + } + }); + } + } + } + /** * Validating the committed data is expensive, so we do this as a {@link Validation}. We depend * on the following external invariants: @@ -1009,7 +1058,7 @@ private void assertCommittedData(RaftNode node) { SnapshotReader.of(log.readSnapshot(snapshotId).get(), node.intSerde, BufferSupplier.create(), Integer.MAX_VALUE)) { // Expect only one batch with only one record assertTrue(snapshot.hasNext()); - BatchReader.Batch batch = snapshot.next(); + Batch batch = snapshot.next(); assertFalse(snapshot.hasNext()); assertEquals(1, batch.records().size()); diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/MemoryBatchReaderTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/MemoryBatchReaderTest.java index 255b759cce306..25b843129a9a2 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/MemoryBatchReaderTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/MemoryBatchReaderTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.raft.internals; +import org.apache.kafka.raft.Batch; import org.apache.kafka.raft.BatchReader; import org.junit.jupiter.api.Test; import org.mockito.Mockito; @@ -31,13 +32,13 @@ class MemoryBatchReaderTest { @Test public void testIteration() { - BatchReader.Batch batch1 = BatchReader.Batch.of( + Batch batch1 = Batch.of( 0L, 1, Arrays.asList("a", "b", "c") ); - BatchReader.Batch batch2 = BatchReader.Batch.of( + Batch batch2 = Batch.of( 3L, 2, Arrays.asList("d", "e") ); - BatchReader.Batch batch3 = BatchReader.Batch.of( + Batch batch3 = Batch.of( 5L, 2, Arrays.asList("f", "g", "h", "i") ); diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsBatchReaderTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsBatchReaderTest.java index bc332d145993d..e340738965e14 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsBatchReaderTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsBatchReaderTest.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.record.FileRecords; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.Records; +import org.apache.kafka.raft.Batch; import org.apache.kafka.raft.BatchReader; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; @@ -49,7 +50,7 @@ class RecordsBatchReaderTest { public void testReadFromMemoryRecords(CompressionType compressionType) { long baseOffset = 57; - List> batches = RecordsIteratorTest.createBatches(baseOffset); + List> batches = RecordsIteratorTest.createBatches(baseOffset); MemoryRecords memRecords = RecordsIteratorTest.buildRecords(compressionType, batches); testBatchReader(baseOffset, memRecords, batches); @@ -60,7 +61,7 @@ public void testReadFromMemoryRecords(CompressionType compressionType) { public void testReadFromFileRecords(CompressionType compressionType) throws Exception { long baseOffset = 57; - List> batches = RecordsIteratorTest.createBatches(baseOffset); + List> batches = RecordsIteratorTest.createBatches(baseOffset); MemoryRecords memRecords = RecordsIteratorTest.buildRecords(compressionType, batches); FileRecords fileRecords = FileRecords.open(tempFile()); @@ -72,7 +73,7 @@ public void testReadFromFileRecords(CompressionType compressionType) throws Exce private void testBatchReader( long baseOffset, Records records, - List> expectedBatches + List> expectedBatches ) { BufferSupplier bufferSupplier = Mockito.mock(BufferSupplier.class); Set allocatedBuffers = Collections.newSetFromMap(new IdentityHashMap<>()); @@ -102,7 +103,7 @@ private void testBatchReader( closeListener ); - for (BatchReader.Batch batch : expectedBatches) { + for (Batch batch : expectedBatches) { assertTrue(reader.hasNext()); assertEquals(batch, reader.next()); } diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java index 40c6aad25da21..e450b5266440c 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java @@ -16,8 +16,6 @@ */ package org.apache.kafka.raft.internals; - - import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -36,7 +34,7 @@ import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.Records; import org.apache.kafka.common.utils.BufferSupplier; -import org.apache.kafka.raft.BatchReader; +import org.apache.kafka.raft.Batch; import org.apache.kafka.raft.RecordSerde; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.params.ParameterizedTest; @@ -69,7 +67,7 @@ public void testMemoryRecords( @ForAll CompressionType compressionType, @ForAll long seed ) { - List> batches = createBatches(seed); + List> batches = createBatches(seed); MemoryRecords memRecords = buildRecords(compressionType, batches); testIterator(batches, memRecords); @@ -80,7 +78,7 @@ public void testFileRecords( @ForAll CompressionType compressionType, @ForAll long seed ) throws IOException { - List> batches = createBatches(seed); + List> batches = createBatches(seed); MemoryRecords memRecords = buildRecords(compressionType, batches); FileRecords fileRecords = FileRecords.open(TestUtils.tempFile()); @@ -90,7 +88,7 @@ public void testFileRecords( } private void testIterator( - List> expectedBatches, + List> expectedBatches, Records records ) { Set allocatedBuffers = Collections.newSetFromMap(new IdentityHashMap<>()); @@ -100,7 +98,7 @@ private void testIterator( mockBufferSupplier(allocatedBuffers) ); - for (BatchReader.Batch batch : expectedBatches) { + for (Batch batch : expectedBatches) { assertTrue(iterator.hasNext()); assertEquals(batch, iterator.next()); } @@ -135,13 +133,13 @@ static BufferSupplier mockBufferSupplier(Set buffers) { return bufferSupplier; } - public static List> createBatches(long seed) { + public static List> createBatches(long seed) { Random random = new Random(seed); long baseOffset = random.nextInt(100); int epoch = random.nextInt(3) + 1; int numberOfBatches = random.nextInt(100) + 1; - List> batches = new ArrayList<>(numberOfBatches); + List> batches = new ArrayList<>(numberOfBatches); for (int i = 0; i < numberOfBatches; i++) { int numberOfRecords = random.nextInt(100) + 1; List records = random @@ -149,7 +147,7 @@ public static List> createBatches(long seed) { .mapToObj(String::valueOf) .collect(Collectors.toList()); - batches.add(BatchReader.Batch.of(baseOffset, epoch, records)); + batches.add(Batch.of(baseOffset, epoch, records)); baseOffset += records.size(); if (i % 5 == 0) { epoch += random.nextInt(3); @@ -161,11 +159,11 @@ public static List> createBatches(long seed) { public static MemoryRecords buildRecords( CompressionType compressionType, - List> batches + List> batches ) { ByteBuffer buffer = ByteBuffer.allocate(102400); - for (BatchReader.Batch batch : batches) { + for (Batch batch : batches) { BatchBuilder builder = new BatchBuilder<>( buffer, STRING_SERDE, diff --git a/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterReaderTest.java b/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterReaderTest.java index a8a7d2e046e07..be862101d73ba 100644 --- a/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterReaderTest.java +++ b/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterReaderTest.java @@ -24,7 +24,7 @@ import java.util.Random; import java.util.Set; import org.apache.kafka.common.utils.BufferSupplier; -import org.apache.kafka.raft.BatchReader.Batch; +import org.apache.kafka.raft.Batch; import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.raft.RaftClientTestContext; import org.apache.kafka.raft.internals.StringSerde; diff --git a/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java b/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java index 5a3b91361ac82..dde0e40defb2c 100644 --- a/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java +++ b/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java @@ -41,6 +41,7 @@ import org.apache.kafka.metalog.MetaLogListener; import org.apache.kafka.queue.EventQueue; import org.apache.kafka.queue.KafkaEventQueue; +import org.apache.kafka.raft.Batch; import org.apache.kafka.raft.BatchReader; import org.apache.kafka.raft.RaftClient; import org.apache.kafka.snapshot.SnapshotReader; @@ -84,7 +85,7 @@ public void handleCommit(BatchReader reader) { try { // TODO: handle lastOffset while (reader.hasNext()) { - BatchReader.Batch batch = reader.next(); + Batch batch = reader.next(); for (ApiMessageAndVersion messageAndVersion : batch.records()) { handleMessage(messageAndVersion.message()); } @@ -110,7 +111,7 @@ public void handleCommits(long lastOffset, List messages) { public void handleSnapshot(SnapshotReader reader) { try { while (reader.hasNext()) { - BatchReader.Batch batch = reader.next(); + Batch batch = reader.next(); for (ApiMessageAndVersion messageAndVersion : batch) { handleMessage(messageAndVersion.message()); } From 7b8a320fff8c22727c2ac309b7e0487e7edf2635 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Fri, 30 Apr 2021 13:52:29 -0700 Subject: [PATCH 18/19] Improve snapshot verity performance --- .../kafka/raft/RaftEventSimulationTest.java | 29 +++++++++---------- 1 file changed, 14 insertions(+), 15 deletions(-) diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java index b6ec2056822e7..cc86ff6cba9a1 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java @@ -980,38 +980,37 @@ public void verify() { int nodeId = nodeEntry.getKey(); ReplicatedLog log = nodeEntry.getValue().log; log.earliestSnapshotId().ifPresent(earliestSnapshotId -> { + long logStartOffset = log.startOffset(); + ValidOffsetAndEpoch validateOffsetAndEpoch = log.validateOffsetAndEpoch( + earliestSnapshotId.offset, + earliestSnapshotId.epoch + ); + assertTrue( - log.startOffset() <= earliestSnapshotId.offset, + logStartOffset <= earliestSnapshotId.offset, String.format( "invalid log start offset (%s) and snapshotId offset (%s): nodeId = %s", - log.startOffset(), + logStartOffset, earliestSnapshotId.offset, nodeId ) ); assertEquals( - log.validateOffsetAndEpoch( - earliestSnapshotId.offset, - earliestSnapshotId.epoch - ).kind(), - ValidOffsetAndEpoch.Kind.VALID, - String.format("invalid offset and epoch for ealiest snapshot: nodeId = %s", nodeId) + ValidOffsetAndEpoch.valid(earliestSnapshotId), + validateOffsetAndEpoch, + String.format("invalid leader epoch cache: nodeId = %s", nodeId) ); - if (log.startOffset() > 0) { + if (logStartOffset > 0) { assertEquals( - log.startOffset(), + logStartOffset, earliestSnapshotId.offset, String.format("mising snapshot at log start offset: nodeId = %s", nodeId) ); - assertEquals( - ValidOffsetAndEpoch.valid(earliestSnapshotId), - log.validateOffsetAndEpoch(earliestSnapshotId.offset, earliestSnapshotId.epoch), - String.format("invalid leader epoch cache: nodeId = %s", nodeId) - ); } }); } + } } From fbbf95304c34a52855ad56d51561e822d64ecc54 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Fri, 30 Apr 2021 14:36:33 -0700 Subject: [PATCH 19/19] Use supplier for assert message --- .../org/apache/kafka/raft/RaftEventSimulationTest.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java index cc86ff6cba9a1..32e701a3854e6 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java @@ -988,7 +988,7 @@ public void verify() { assertTrue( logStartOffset <= earliestSnapshotId.offset, - String.format( + () -> String.format( "invalid log start offset (%s) and snapshotId offset (%s): nodeId = %s", logStartOffset, earliestSnapshotId.offset, @@ -998,19 +998,18 @@ public void verify() { assertEquals( ValidOffsetAndEpoch.valid(earliestSnapshotId), validateOffsetAndEpoch, - String.format("invalid leader epoch cache: nodeId = %s", nodeId) + () -> String.format("invalid leader epoch cache: nodeId = %s", nodeId) ); if (logStartOffset > 0) { assertEquals( logStartOffset, earliestSnapshotId.offset, - String.format("mising snapshot at log start offset: nodeId = %s", nodeId) + () -> String.format("mising snapshot at log start offset: nodeId = %s", nodeId) ); } }); } - } }