hachikuji commented on a change in pull request #10085: URL: https://github.com/apache/kafka/pull/10085#discussion_r607232953
########## File path: raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java ########## @@ -355,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() Review comment: Since we're in here already, this check is not needed since we already ruled it out above. ########## File path: raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java ########## @@ -68,20 +70,65 @@ public synchronized void increment() { @Override public synchronized void handleCommit(BatchReader<Integer> reader) { try { - int initialValue = this.committed; + int initialValue = committed; while (reader.hasNext()) { BatchReader.Batch<Integer> 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) { - throw new AssertionError("Expected next committed value to be " + - (this.committed + 1) + ", but instead found " + value + " on node " + nodeId); + if (value != committed + 1) { + throw new AssertionError( + String.format( + "Expected next committed value to be %s, but instead found %s on node %s", + 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) { Review comment: Looks like we are trying to do snapshots every 10 records. We could probably get rid of `lastSnapshotEndOffset` and use `committed % 10` or something like that. It may also be useful to be able to control the frequency of snapshots with a parameter. ########## File path: raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java ########## @@ -326,6 +336,14 @@ private void updateListenersProgress(List<ListenerContext> listenerContexts, lon } } + private Optional<SnapshotReader<T>> latestSnapshot() { + return log.latestSnapshotId().flatMap(snapshoId -> { Review comment: nit: missing t in snapshotId ########## File path: raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java ########## @@ -2154,8 +2182,14 @@ private boolean maybeCompleteShutdown(long currentTimeMs) { return false; } - private void maybeUpdateOldestSnapshotId() { - log.latestSnapshotId().ifPresent(log::deleteBeforeSnapshot); + private void maybeUpdateEarliestSnapshotId() { Review comment: Hmm, I guess I still see this the other way around. Why would the raft client care about updating the log start offset if not to delete old snapshots? What would that even mean outside the context of snapshot deletion? ########## File path: raft/src/main/java/org/apache/kafka/raft/internals/SerdeRecordsIterator.java ########## @@ -0,0 +1,231 @@ +/* + * 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<T> implements Iterator<Batch<T>>, AutoCloseable { Review comment: nit: it's really more of a _batch_ iterator. Could we cal this `RecordsBatchIterator` to go along with `RecordBatchReader`? ########## File path: raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java ########## @@ -163,48 +85,46 @@ public OptionalLong lastOffset() { @Override public void close() { - isClosed = true; + if (!isClosed) { + isClosed = true; - if (allocatedBuffer != null) { - bufferSupplier.release(allocatedBuffer); + 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); - } + public static <T> RecordsBatchReader<T> of( + long baseOffset, + Records records, + RecordSerde<T> serde, + BufferSupplier bufferSupplier, + int maxBatchSize, + CloseListener<BatchReader<T>> closeListener + ) { + return new RecordsBatchReader<>( + baseOffset, + new SerdeRecordsIterator<>(records, serde, bufferSupplier, maxBatchSize), + closeListener + ); + } - int valueSize = input.readVarint(); - if (valueSize < 0) { - throw new IllegalArgumentException(); + private void checkIfClosed() { Review comment: nit: maybe `ensureNotClosed` or `ensureOpen`? ########## File path: raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java ########## @@ -68,20 +70,65 @@ public synchronized void increment() { @Override public synchronized void handleCommit(BatchReader<Integer> reader) { try { - int initialValue = this.committed; + int initialValue = committed; while (reader.hasNext()) { BatchReader.Batch<Integer> 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) { - throw new AssertionError("Expected next committed value to be " + - (this.committed + 1) + ", but instead found " + value + " on node " + nodeId); + if (value != committed + 1) { + throw new AssertionError( + String.format( + "Expected next committed value to be %s, but instead found %s on node %s", + committed + 1, + value, + nodeId + ) + ); } - this.committed = value; + committed = value; } + + nextReadOffset = batch.lastOffset() + 1; Review comment: As far as I can tell, we should be able to convert these two fields to local variables. ########## File path: raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java ########## @@ -988,14 +991,43 @@ private int parseSequenceNumber(ByteBuffer value) { return (int) Type.INT32.read(value); } - private void assertCommittedData(int nodeId, KafkaRaftClient<Integer> manager, MockLog log) { + private void assertCommittedData(RaftNode node) { + final int nodeId = node.nodeId; + final KafkaRaftClient<Integer> 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 -> { Review comment: nit: typo `snapshoId` ########## File path: raft/src/main/java/org/apache/kafka/raft/internals/SerdeRecordsIterator.java ########## @@ -0,0 +1,231 @@ +/* + * 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<T> implements Iterator<Batch<T>>, AutoCloseable { + private final Records records; + private final RecordSerde<T> serde; + private final BufferSupplier bufferSupplier; + private final int maxBatchSize; + + private Optional<Iterator<MutableRecordBatch>> nextBatches = Optional.empty(); + private Optional<Batch<T>> nextBatch = Optional.empty(); + // Buffer used to as the backing store for nextBatches if needed + private Optional<ByteBuffer> allocatedBuffer = Optional.empty(); + // Number of bytes from records that read + private int bytesRead = 0; + private boolean isClosed = false; + + public SerdeRecordsIterator( + Records records, + RecordSerde<T> 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<T> next() { + if (!hasNext()) { + throw new NoSuchElementException("Batch iterator doesn't have any more elements"); + } + + Batch<T> 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<Iterator<MutableRecordBatch>> 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(); + + 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); + } + + 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; Review comment: Maybe I am missing it, but how does this ensure proper alignment? The batch at the end of the buffer could be incomplete, no? ########## File path: raft/src/test/java/org/apache/kafka/raft/MockLog.java ########## @@ -418,48 +414,49 @@ public RawSnapshotWriter createSnapshot(OffsetAndEpoch snapshotId) { public void onSnapshotFrozen(OffsetAndEpoch snapshotId) {} @Override - public boolean deleteBeforeSnapshot(OffsetAndEpoch logStartSnapshotId) { - if (logStartOffset() > logStartSnapshotId.offset || - highWatermark.offset < logStartSnapshotId.offset) { + public boolean deleteBeforeSnapshot(OffsetAndEpoch snapshotId) { + if (logStartOffset() > snapshotId.offset || + 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)", Review comment: nit: maybe we can just have two separate cases so that we can give a clearer message -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org