From dd8447964ef1c606280903468a178800d289ea1c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Tue, 30 Jan 2024 16:28:06 -0800 Subject: [PATCH 01/34] KAFKA-16207; Add KRaftVersionRecord and VotersRecord --- .../common/record/ControlRecordType.java | 10 +- .../common/record/ControlRecordUtils.java | 70 +++++++--- .../kafka/common/record/MemoryRecords.java | 64 +++++++++- .../common/record/MemoryRecordsBuilder.java | 23 +++- .../common/message/KRaftVersionRecord.json | 27 ++++ .../common/message/VotersRecord.json | 49 +++++++ .../common/record/ControlRecordTypeTest.java | 54 ++++++++ .../common/record/ControlRecordUtilsTest.java | 10 ++ .../scala/kafka/tools/DumpLogSegments.scala | 11 +- .../org/apache/kafka/raft/ControlRecord.java | 16 +++ .../raft/internals/BatchAccumulator.java | 8 +- .../kafka/raft/internals/RecordsIterator.java | 12 +- .../kafka/snapshot/RecordsSnapshotReader.java | 8 +- .../apache/kafka/raft/ControlRecordTest.java | 6 +- .../raft/internals/RecordsIteratorTest.java | 120 +++++++++++------- 15 files changed, 406 insertions(+), 82 deletions(-) create mode 100644 clients/src/main/resources/common/message/KRaftVersionRecord.json create mode 100644 clients/src/main/resources/common/message/VotersRecord.json diff --git a/clients/src/main/java/org/apache/kafka/common/record/ControlRecordType.java b/clients/src/main/java/org/apache/kafka/common/record/ControlRecordType.java index 39268044db34..9e52777bfb40 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/ControlRecordType.java +++ b/clients/src/main/java/org/apache/kafka/common/record/ControlRecordType.java @@ -44,11 +44,15 @@ public enum ControlRecordType { ABORT((short) 0), COMMIT((short) 1), - // Raft quorum related control messages. + // KRaft quorum related control messages LEADER_CHANGE((short) 2), SNAPSHOT_HEADER((short) 3), SNAPSHOT_FOOTER((short) 4), + // KRaft membership changes messages + KRAFT_VERSION((short) 5), + VOTERS((short) 6), + // UNKNOWN is used to indicate a control type which the client is not aware of and should be ignored UNKNOWN((short) -1); @@ -108,6 +112,10 @@ public static ControlRecordType fromTypeId(short typeId) { return SNAPSHOT_HEADER; case 4: return SNAPSHOT_FOOTER; + case 5: + return KRAFT_VERSION; + case 6: + return VOTERS; default: return UNKNOWN; diff --git a/clients/src/main/java/org/apache/kafka/common/record/ControlRecordUtils.java b/clients/src/main/java/org/apache/kafka/common/record/ControlRecordUtils.java index 3b1fd21f7875..3f8d4a56943b 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/ControlRecordUtils.java +++ b/clients/src/main/java/org/apache/kafka/common/record/ControlRecordUtils.java @@ -16,9 +16,11 @@ */ package org.apache.kafka.common.record; +import org.apache.kafka.common.message.KRaftVersionRecord; import org.apache.kafka.common.message.LeaderChangeMessage; -import org.apache.kafka.common.message.SnapshotHeaderRecord; import org.apache.kafka.common.message.SnapshotFooterRecord; +import org.apache.kafka.common.message.SnapshotHeaderRecord; +import org.apache.kafka.common.message.VotersRecord; import org.apache.kafka.common.protocol.ByteBufferAccessor; import java.nio.ByteBuffer; @@ -27,49 +29,77 @@ * Utility class for easy interaction with control records. */ public class ControlRecordUtils { + public static final short KRAFT_VERSION_CURRENT_VERSION = 0; public static final short LEADER_CHANGE_CURRENT_VERSION = 0; - public static final short SNAPSHOT_HEADER_CURRENT_VERSION = 0; public static final short SNAPSHOT_FOOTER_CURRENT_VERSION = 0; + public static final short SNAPSHOT_HEADER_CURRENT_VERSION = 0; + public static final short VOTERS_CURRENT_VERSION = 0; public static LeaderChangeMessage deserializeLeaderChangeMessage(Record record) { ControlRecordType recordType = ControlRecordType.parse(record.key()); - if (recordType != ControlRecordType.LEADER_CHANGE) { - throw new IllegalArgumentException( - "Expected LEADER_CHANGE control record type(2), but found " + recordType.toString()); - } + validateControlRecordType(ControlRecordType.LEADER_CHANGE, recordType); + return deserializeLeaderChangeMessage(record.value()); } public static LeaderChangeMessage deserializeLeaderChangeMessage(ByteBuffer data) { - ByteBufferAccessor byteBufferAccessor = new ByteBufferAccessor(data.slice()); - return new LeaderChangeMessage(byteBufferAccessor, LEADER_CHANGE_CURRENT_VERSION); + return new LeaderChangeMessage(new ByteBufferAccessor(data.slice()), LEADER_CHANGE_CURRENT_VERSION); } public static SnapshotHeaderRecord deserializeSnapshotHeaderRecord(Record record) { ControlRecordType recordType = ControlRecordType.parse(record.key()); - if (recordType != ControlRecordType.SNAPSHOT_HEADER) { - throw new IllegalArgumentException( - "Expected SNAPSHOT_HEADER control record type(3), but found " + recordType.toString()); - } + validateControlRecordType(ControlRecordType.SNAPSHOT_HEADER, recordType); + return deserializeSnapshotHeaderRecord(record.value()); } public static SnapshotHeaderRecord deserializeSnapshotHeaderRecord(ByteBuffer data) { - ByteBufferAccessor byteBufferAccessor = new ByteBufferAccessor(data.slice()); - return new SnapshotHeaderRecord(byteBufferAccessor, SNAPSHOT_HEADER_CURRENT_VERSION); + return new SnapshotHeaderRecord(new ByteBufferAccessor(data.slice()), SNAPSHOT_HEADER_CURRENT_VERSION); } public static SnapshotFooterRecord deserializeSnapshotFooterRecord(Record record) { ControlRecordType recordType = ControlRecordType.parse(record.key()); - if (recordType != ControlRecordType.SNAPSHOT_FOOTER) { - throw new IllegalArgumentException( - "Expected SNAPSHOT_FOOTER control record type(4), but found " + recordType.toString()); - } + validateControlRecordType(ControlRecordType.SNAPSHOT_FOOTER, recordType); + return deserializeSnapshotFooterRecord(record.value()); } public static SnapshotFooterRecord deserializeSnapshotFooterRecord(ByteBuffer data) { - ByteBufferAccessor byteBufferAccessor = new ByteBufferAccessor(data.slice()); - return new SnapshotFooterRecord(byteBufferAccessor, SNAPSHOT_FOOTER_CURRENT_VERSION); + return new SnapshotFooterRecord(new ByteBufferAccessor(data.slice()), SNAPSHOT_FOOTER_CURRENT_VERSION); + } + + public static KRaftVersionRecord deserializeKRaftVersionRecord(Record record) { + ControlRecordType recordType = ControlRecordType.parse(record.key()); + validateControlRecordType(ControlRecordType.KRAFT_VERSION, recordType); + + return deserializeKRaftVersionRecord(record.value()); + } + + public static KRaftVersionRecord deserializeKRaftVersionRecord(ByteBuffer data) { + return new KRaftVersionRecord(new ByteBufferAccessor(data.slice()), KRAFT_VERSION_CURRENT_VERSION); + } + + public static VotersRecord deserializeVotersRecord(Record record) { + ControlRecordType recordType = ControlRecordType.parse(record.key()); + validateControlRecordType(ControlRecordType.VOTERS, recordType); + + return deserializeVotersRecord(record.value()); + } + + public static VotersRecord deserializeVotersRecord(ByteBuffer data) { + return new VotersRecord(new ByteBufferAccessor(data.slice()), VOTERS_CURRENT_VERSION); + } + + private static void validateControlRecordType(ControlRecordType expected, ControlRecordType actual) { + if (actual != expected) { + throw new IllegalArgumentException( + String.format( + "Expected %s control record type(%d), but found %s", + expected, + expected.type(), + actual + ) + ); + } } } diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index 3aa233c34e94..987fa8b1d109 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -18,9 +18,11 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.CorruptRecordException; +import org.apache.kafka.common.message.KRaftVersionRecord; import org.apache.kafka.common.message.LeaderChangeMessage; -import org.apache.kafka.common.message.SnapshotHeaderRecord; import org.apache.kafka.common.message.SnapshotFooterRecord; +import org.apache.kafka.common.message.SnapshotHeaderRecord; +import org.apache.kafka.common.message.VotersRecord; import org.apache.kafka.common.network.TransferableChannel; import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetention; import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetentionResult; @@ -154,7 +156,7 @@ public FilterResult filterTo(TopicPartition partition, RecordFilter filter, Byte /** * Note: This method is also used to convert the first timestamp of the batch (which is usually the timestamp of the first record) - * to the delete horizon of the tombstones or txn markers which are present in the batch. + * to the delete horizon of the tombstones or txn markers which are present in the batch. */ private static FilterResult filterTo(TopicPartition partition, Iterable batches, RecordFilter filter, ByteBuffer destinationBuffer, int maxRecordBatchSize, @@ -807,4 +809,62 @@ private static void writeSnapshotFooterRecord( builder.appendSnapshotFooterMessage(timestamp, snapshotFooterRecord); } } + + public static MemoryRecords withKRaftVersionRecord( + long initialOffset, + long timestamp, + int leaderEpoch, + ByteBuffer buffer, + KRaftVersionRecord kraftVersionRecord + ) { + writeKRaftVersionRecord(buffer, initialOffset, timestamp, leaderEpoch, kraftVersionRecord); + buffer.flip(); + return MemoryRecords.readableRecords(buffer); + } + + private static void writeKRaftVersionRecord( + ByteBuffer buffer, + long initialOffset, + long timestamp, + int leaderEpoch, + KRaftVersionRecord kraftVersionRecord + ) { + try (MemoryRecordsBuilder builder = new MemoryRecordsBuilder( + buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, + TimestampType.CREATE_TIME, initialOffset, timestamp, + RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, + false, true, leaderEpoch, buffer.capacity()) + ) { + builder.appendKRaftVersionMessage(timestamp, kraftVersionRecord); + } + } + + public static MemoryRecords withVotersRecord( + long initialOffset, + long timestamp, + int leaderEpoch, + ByteBuffer buffer, + VotersRecord votersRecord + ) { + writeVotersRecord(buffer, initialOffset, timestamp, leaderEpoch, votersRecord); + buffer.flip(); + return MemoryRecords.readableRecords(buffer); + } + + private static void writeVotersRecord( + ByteBuffer buffer, + long initialOffset, + long timestamp, + int leaderEpoch, + VotersRecord votersRecord + ) { + try (MemoryRecordsBuilder builder = new MemoryRecordsBuilder( + buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, + TimestampType.CREATE_TIME, initialOffset, timestamp, + RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, + false, true, leaderEpoch, buffer.capacity()) + ) { + builder.appendVotersMessage(timestamp, votersRecord); + } + } } diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java index 143a515bcd00..88d2ebca3c86 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java @@ -18,9 +18,11 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.header.Header; +import org.apache.kafka.common.message.KRaftVersionRecord; import org.apache.kafka.common.message.LeaderChangeMessage; -import org.apache.kafka.common.message.SnapshotHeaderRecord; import org.apache.kafka.common.message.SnapshotFooterRecord; +import org.apache.kafka.common.message.SnapshotHeaderRecord; +import org.apache.kafka.common.message.VotersRecord; import org.apache.kafka.common.protocol.MessageUtil; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.utils.ByteBufferOutputStream; @@ -584,11 +586,12 @@ public void append(SimpleRecord record) { /** * Append a control record at the next sequential offset. + * * @param timestamp The record timestamp * @param type The control record type (cannot be UNKNOWN) * @param value The control record value */ - private void appendControlRecord(long timestamp, ControlRecordType type, ByteBuffer value) { + public void appendControlRecord(long timestamp, ControlRecordType type, ByteBuffer value) { Struct keyStruct = type.recordKey(); ByteBuffer key = ByteBuffer.allocate(keyStruct.sizeOf()); keyStruct.writeTo(key); @@ -632,6 +635,22 @@ public void appendSnapshotFooterMessage(long timestamp, SnapshotFooterRecord sna ); } + public void appendKRaftVersionMessage(long timestamp, KRaftVersionRecord kraftVersionRecord) { + appendControlRecord( + timestamp, + ControlRecordType.KRAFT_VERSION, + MessageUtil.toByteBuffer(kraftVersionRecord, ControlRecordUtils.KRAFT_VERSION_CURRENT_VERSION) + ); + } + + public void appendVotersMessage(long timestamp, VotersRecord votersRecord) { + appendControlRecord( + timestamp, + ControlRecordType.VOTERS, + MessageUtil.toByteBuffer(votersRecord, ControlRecordUtils.VOTERS_CURRENT_VERSION) + ); + } + /** * Add a legacy record without doing offset/magic validation (this should only be used in testing). * @param offset The offset of the record diff --git a/clients/src/main/resources/common/message/KRaftVersionRecord.json b/clients/src/main/resources/common/message/KRaftVersionRecord.json new file mode 100644 index 000000000000..8610f75fe5d7 --- /dev/null +++ b/clients/src/main/resources/common/message/KRaftVersionRecord.json @@ -0,0 +1,27 @@ +// 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. + +{ + "type": "data", + "name": "KRaftVersionRecord", + "validVersions": "0", + "flexibleVersions": "0+", + "fields": [ + { "name": "Version", "type": "int16", "versions": "0+", + "about": "The version of the kraft version record" }, + { "name": "KRaftVersion", "type": "int16", "versions": "0+", + "about": "The kraft protocol version" } + ] +} diff --git a/clients/src/main/resources/common/message/VotersRecord.json b/clients/src/main/resources/common/message/VotersRecord.json new file mode 100644 index 000000000000..239b25759d92 --- /dev/null +++ b/clients/src/main/resources/common/message/VotersRecord.json @@ -0,0 +1,49 @@ +// 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. + +{ + "type": "data", + "name": "VotersRecord", + "validVersions": "0", + "flexibleVersions": "0+", + "fields": [ + { "name": "Version", "type": "int16", "versions": "0+", + "about": "The version of the voters record" }, + { "name": "Voters", "type": "[]Voter", "versions": "0+", "fields": [ + { "name": "VoterId", "type": "int32", "versions": "0+", "entityType": "brokerId", + "about": "The replica id of the voter in the topic partition" }, + { "name": "VoterUuid", "type": "uuid", "versions": "0+", + "about": "The directory id of the voter in the topic partition" }, + { "name": "EndPoints", "type": "[]Endpoint", "versions": "0+", + "about": "The endpoint that can be used to communicate with the voter", "fields": [ + { "name": "Name", "type": "string", "versions": "0+", "mapKey": true, + "about": "The name of the endpoint" }, + { "name": "Host", "type": "string", "versions": "0+", + "about": "The hostname" }, + { "name": "Port", "type": "uint16", "versions": "0+", + "about": "The port" }, + { "name": "SecurityProtocol", "type": "int16", "versions": "0+", + "about": "The security protocol" } + ]}, + { "name": "KRaftVersionFeature", "type": "KRaftVersionFeature", "versions": "0+", + "about": "The range of versions of the protocol that the replica supports", "fields": [ + { "name": "MinSupportedVersion", "type": "int16", "versions": "0+", + "about": "The minimum supported KRaft protocol version" }, + { "name": "MaxSupportedVersion", "type": "int16", "versions": "0+", + "about": "The maximum supported KRaft protocol version" } + ]} + ]} + ] +} diff --git a/clients/src/test/java/org/apache/kafka/common/record/ControlRecordTypeTest.java b/clients/src/test/java/org/apache/kafka/common/record/ControlRecordTypeTest.java index fcf6ffd3ab81..fc0be11b127a 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/ControlRecordTypeTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/ControlRecordTypeTest.java @@ -45,4 +45,58 @@ public void testParseUnknownVersion() { assertEquals(ControlRecordType.ABORT, type); } + @Test + public void testLeaderChange() { + ByteBuffer buffer = ByteBuffer.allocate(32); + buffer.putShort(ControlRecordType.CURRENT_CONTROL_RECORD_KEY_VERSION); + buffer.putShort((short) 2); + buffer.flip(); + + ControlRecordType type = ControlRecordType.parse(buffer); + assertEquals(ControlRecordType.LEADER_CHANGE, type); + } + + @Test + public void testSnapshotHeader() { + ByteBuffer buffer = ByteBuffer.allocate(32); + buffer.putShort(ControlRecordType.CURRENT_CONTROL_RECORD_KEY_VERSION); + buffer.putShort((short) 3); + buffer.flip(); + + ControlRecordType type = ControlRecordType.parse(buffer); + assertEquals(ControlRecordType.SNAPSHOT_HEADER, type); + } + + @Test + public void testSnapshotFooter() { + ByteBuffer buffer = ByteBuffer.allocate(32); + buffer.putShort(ControlRecordType.CURRENT_CONTROL_RECORD_KEY_VERSION); + buffer.putShort((short) 4); + buffer.flip(); + + ControlRecordType type = ControlRecordType.parse(buffer); + assertEquals(ControlRecordType.SNAPSHOT_FOOTER, type); + } + + @Test + public void testKRaftVersion() { + ByteBuffer buffer = ByteBuffer.allocate(32); + buffer.putShort(ControlRecordType.CURRENT_CONTROL_RECORD_KEY_VERSION); + buffer.putShort((short) 5); + buffer.flip(); + + ControlRecordType type = ControlRecordType.parse(buffer); + assertEquals(ControlRecordType.KRAFT_VERSION, type); + } + + @Test + public void testVoters() { + ByteBuffer buffer = ByteBuffer.allocate(32); + buffer.putShort(ControlRecordType.CURRENT_CONTROL_RECORD_KEY_VERSION); + buffer.putShort((short) 6); + buffer.flip(); + + ControlRecordType type = ControlRecordType.parse(buffer); + assertEquals(ControlRecordType.VOTERS, type); + } } diff --git a/clients/src/test/java/org/apache/kafka/common/record/ControlRecordUtilsTest.java b/clients/src/test/java/org/apache/kafka/common/record/ControlRecordUtilsTest.java index fc95f531512a..643c69b687a9 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/ControlRecordUtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/ControlRecordUtilsTest.java @@ -19,9 +19,11 @@ import java.nio.ByteBuffer; import java.util.Collections; import org.apache.kafka.common.message.LeaderChangeMessage.Voter; +import org.apache.kafka.common.message.KRaftVersionRecord; import org.apache.kafka.common.message.LeaderChangeMessage; import org.apache.kafka.common.message.SnapshotFooterRecord; import org.apache.kafka.common.message.SnapshotHeaderRecord; +import org.apache.kafka.common.message.VotersRecord; import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.ObjectSerializationCache; import org.junit.jupiter.api.Test; @@ -46,6 +48,14 @@ public void testCurrentVersions() { SnapshotFooterRecord.HIGHEST_SUPPORTED_VERSION, ControlRecordUtils.SNAPSHOT_FOOTER_CURRENT_VERSION ); + assertEquals( + KRaftVersionRecord.HIGHEST_SUPPORTED_VERSION, + ControlRecordUtils.KRAFT_VERSION_CURRENT_VERSION + ); + assertEquals( + VotersRecord.HIGHEST_SUPPORTED_VERSION, + ControlRecordUtils.VOTERS_CURRENT_VERSION + ); } @Test diff --git a/core/src/main/scala/kafka/tools/DumpLogSegments.scala b/core/src/main/scala/kafka/tools/DumpLogSegments.scala index 8a37e6592207..55ced144cc19 100755 --- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala +++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala @@ -25,7 +25,10 @@ import kafka.log._ import kafka.serializer.Decoder import kafka.utils._ import kafka.utils.Implicits._ -import org.apache.kafka.common.message.{SnapshotFooterRecordJsonConverter, SnapshotHeaderRecordJsonConverter} +import org.apache.kafka.common.message.KRaftVersionRecordJsonConverter +import org.apache.kafka.common.message.SnapshotFooterRecordJsonConverter +import org.apache.kafka.common.message.SnapshotHeaderRecordJsonConverter +import org.apache.kafka.common.message.VotersRecordJsonConverter import org.apache.kafka.common.metadata.{MetadataJsonConverters, MetadataRecordType} import org.apache.kafka.common.protocol.ByteBufferAccessor import org.apache.kafka.common.record._ @@ -307,6 +310,12 @@ object DumpLogSegments { case ControlRecordType.SNAPSHOT_FOOTER => val footer = ControlRecordUtils.deserializeSnapshotFooterRecord(record) print(s" SnapshotFooter ${SnapshotFooterRecordJsonConverter.write(footer, footer.version())}") + case ControlRecordType.KRAFT_VERSION => + val kraftVersion = ControlRecordUtils.deserializeKRaftVersionRecord(record) + print(s" KRaftVersion ${KRaftVersionRecordJsonConverter.write(kraftVersion,, kraftVersion.version())}") + case ControlRecordType.VOTERS=> + val voters = ControlRecordUtils.deserializeVotersRecord(record) + print(s" Voets ${VotersRecordJsonConverter.write(voters,, voters.version())}") case controlType => print(s" controlType: $controlType($controlTypeId)") } diff --git a/raft/src/main/java/org/apache/kafka/raft/ControlRecord.java b/raft/src/main/java/org/apache/kafka/raft/ControlRecord.java index bf6857707205..f0b999cc0f78 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ControlRecord.java +++ b/raft/src/main/java/org/apache/kafka/raft/ControlRecord.java @@ -17,9 +17,11 @@ package org.apache.kafka.raft; import java.util.Objects; +import org.apache.kafka.common.message.KRaftVersionRecord; import org.apache.kafka.common.message.LeaderChangeMessage; import org.apache.kafka.common.message.SnapshotFooterRecord; import org.apache.kafka.common.message.SnapshotHeaderRecord; +import org.apache.kafka.common.message.VotersRecord; import org.apache.kafka.common.protocol.ApiMessage; import org.apache.kafka.common.record.ControlRecordType; @@ -54,6 +56,16 @@ public ControlRecord(ControlRecordType recordType, ApiMessage message) { throwIllegalArgument(recordType, message); } break; + case KRAFT_VERSION: + if (!(message instanceof KRaftVersionRecord)) { + throwIllegalArgument(recordType, message); + } + break; + case VOTERS: + if (!(message instanceof VotersRecord)) { + throwIllegalArgument(recordType, message); + } + break; default: throw new IllegalArgumentException(String.format("Unknown control record type %s", recordType)); } @@ -74,6 +86,10 @@ public short version() { return ((SnapshotHeaderRecord) message).version(); case SNAPSHOT_FOOTER: return ((SnapshotFooterRecord) message).version(); + case KRAFT_VERSION: + return ((KRaftVersionRecord) message).version(); + case VOTERS: + return ((VotersRecord) message).version(); default: throw new IllegalStateException(String.format("Unknown control record type %s", recordType)); } diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java b/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java index aa66d4f9951a..febe0969afbb 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java @@ -215,7 +215,7 @@ private void completeCurrentBatch() { * batch that will be appended. The memory records returned must contain one * control batch and that control batch have one record. */ - private void appendControlMessage(Function valueCreator) { + private void appendControlMessages(Function valueCreator) { appendLock.lock(); try { ByteBuffer buffer = memoryPool.tryAllocate(256); @@ -256,7 +256,7 @@ public void appendLeaderChangeMessage( LeaderChangeMessage leaderChangeMessage, long currentTimestamp ) { - appendControlMessage(buffer -> MemoryRecords.withLeaderChangeMessage( + appendControlMessages(buffer -> MemoryRecords.withLeaderChangeMessage( this.nextOffset, currentTimestamp, this.epoch, @@ -277,7 +277,7 @@ public void appendSnapshotHeaderRecord( SnapshotHeaderRecord snapshotHeaderRecord, long currentTimestamp ) { - appendControlMessage(buffer -> MemoryRecords.withSnapshotHeaderRecord( + appendControlMessages(buffer -> MemoryRecords.withSnapshotHeaderRecord( this.nextOffset, currentTimestamp, this.epoch, @@ -297,7 +297,7 @@ public void appendSnapshotFooterRecord( SnapshotFooterRecord snapshotFooterRecord, long currentTimestamp ) { - appendControlMessage(buffer -> MemoryRecords.withSnapshotFooterRecord( + appendControlMessages(buffer -> MemoryRecords.withSnapshotFooterRecord( this.nextOffset, currentTimestamp, this.epoch, 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 6cfbe7349e52..3dc53cdf2294 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 @@ -214,7 +214,11 @@ private Batch readBatch(DefaultRecordBatch batch) { if (batch.isControlBatch()) { List records = new ArrayList<>(numRecords); for (int i = 0; i < numRecords; i++) { - ControlRecord record = readRecord(input, batch.sizeInBytes(), RecordsIterator::decodeControlRecord); + ControlRecord record = readRecord( + input, + batch.sizeInBytes(), + RecordsIterator::decodeControlRecord + ); records.add(record); } result = Batch.control( @@ -366,6 +370,12 @@ private static ControlRecord decodeControlRecord(Optional key, Optio case SNAPSHOT_FOOTER: message = ControlRecordUtils.deserializeSnapshotFooterRecord(value.get()); break; + case KRAFT_VERSION: + message = ControlRecordUtils.deserializeKRaftVersionRecord(value.get()); + break; + case VOTERS: + message = ControlRecordUtils.deserializeVotersRecord(value.get()); + break; default: throw new IllegalArgumentException(String.format("Unknown control record type %s", type)); } diff --git a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotReader.java b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotReader.java index 11aed79ae08b..c9c42a1da9ac 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotReader.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotReader.java @@ -123,6 +123,8 @@ public static RecordsSnapshotReader of( * Returns the next non-control Batch */ private Optional> nextBatch() { + // TODO: Should this return control records? It may need to return control record if I decide to use + // this type in the internal KRaft listener. Another option is to use the RecordsIterator directly. while (iterator.hasNext()) { Batch batch = iterator.next(); @@ -130,8 +132,10 @@ private Optional> nextBatch() { // This must be the first batch which is expected to be a control batch with one record for // the snapshot header. if (batch.controlRecords().isEmpty()) { - throw new IllegalStateException("First batch is not a control batch with at least one record"); - } else if (!ControlRecordType.SNAPSHOT_HEADER.equals(batch.controlRecords().get(0).type())) { + throw new IllegalStateException( + "First batch is not a control batch with at least one record" + ); + } else if (ControlRecordType.SNAPSHOT_HEADER != batch.controlRecords().get(0).type()) { throw new IllegalStateException( String.format( "First control record is not a snapshot header (%s)", diff --git a/raft/src/test/java/org/apache/kafka/raft/ControlRecordTest.java b/raft/src/test/java/org/apache/kafka/raft/ControlRecordTest.java index 5ed500995e27..de03aa8785b9 100644 --- a/raft/src/test/java/org/apache/kafka/raft/ControlRecordTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/ControlRecordTest.java @@ -16,9 +16,11 @@ */ package org.apache.kafka.raft; +import org.apache.kafka.common.message.KRaftVersionRecord; import org.apache.kafka.common.message.LeaderChangeMessage; import org.apache.kafka.common.message.SnapshotFooterRecord; import org.apache.kafka.common.message.SnapshotHeaderRecord; +import org.apache.kafka.common.message.VotersRecord; import org.apache.kafka.common.protocol.ApiMessage; import org.apache.kafka.common.record.ControlRecordType; import org.junit.jupiter.api.Test; @@ -33,6 +35,8 @@ void testCtr() { new ControlRecord(ControlRecordType.LEADER_CHANGE, new LeaderChangeMessage()); new ControlRecord(ControlRecordType.SNAPSHOT_HEADER, new SnapshotHeaderRecord()); new ControlRecord(ControlRecordType.SNAPSHOT_FOOTER, new SnapshotFooterRecord()); + new ControlRecord(ControlRecordType.KRAFT_VERSION, new KRaftVersionRecord()); + new ControlRecord(ControlRecordType.VOTERS, new VotersRecord()); // Invalid constructions assertThrows( @@ -54,6 +58,6 @@ void testControlRecordTypeValues() { // If this test fails then it means that ControlRecordType was changed. Please review the // implementation for ControlRecord to see if it needs to be updated based on the changes // to ControlRecordType. - assertEquals(6, ControlRecordType.values().length); + assertEquals(8, ControlRecordType.values().length); } } 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 9433dbe1a969..1f1d6470e5b4 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 @@ -34,16 +34,23 @@ import net.jqwik.api.Property; import org.apache.kafka.common.errors.CorruptRecordException; import org.apache.kafka.common.memory.MemoryPool; +import org.apache.kafka.common.message.KRaftVersionRecord; import org.apache.kafka.common.message.LeaderChangeMessage; import org.apache.kafka.common.message.SnapshotFooterRecord; import org.apache.kafka.common.message.SnapshotHeaderRecord; +import org.apache.kafka.common.message.VotersRecord; import org.apache.kafka.common.protocol.ApiMessage; +import org.apache.kafka.common.protocol.MessageUtil; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.ControlRecordType; +import org.apache.kafka.common.record.ControlRecordUtils; 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.MemoryRecordsBuilder; +import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.record.Records; +import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.utils.BufferSupplier; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.raft.Batch; @@ -193,23 +200,13 @@ public void testControlRecordIteration() { } @ParameterizedTest - @EnumSource(value = ControlRecordType.class, names = {"LEADER_CHANGE", "SNAPSHOT_HEADER", "SNAPSHOT_FOOTER"}) + @EnumSource( + value = ControlRecordType.class, + names = {"LEADER_CHANGE", "SNAPSHOT_HEADER", "SNAPSHOT_FOOTER", "KRAFT_VERSION", "VOTERS"} + ) void testWithAllSupportedControlRecords(ControlRecordType type) { MemoryRecords records = buildControlRecords(type); - final ApiMessage expectedMessage; - switch (type) { - case LEADER_CHANGE: - expectedMessage = new LeaderChangeMessage(); - break; - case SNAPSHOT_HEADER: - expectedMessage = new SnapshotHeaderRecord(); - break; - case SNAPSHOT_FOOTER: - expectedMessage = new SnapshotFooterRecord(); - break; - default: - throw new RuntimeException("Should not happen. Poorly configured test"); - } + ApiMessage expectedMessage = defaultControlRecord(type); try (RecordsIterator iterator = createIterator(records, BufferSupplier.NO_CACHING, true)) { assertTrue(iterator.hasNext()); @@ -226,7 +223,7 @@ void testControlRecordTypeValues() { // If this test fails then it means that ControlRecordType was changed. Please review the // implementation for RecordsIterator to see if it needs to be updated based on the changes // to ControlRecordType. - assertEquals(6, ControlRecordType.values().length); + assertEquals(8, ControlRecordType.values().length); } private void testIterator( @@ -308,40 +305,33 @@ public static List> createBatches(long seed) { } public static MemoryRecords buildControlRecords(ControlRecordType type) { - final MemoryRecords records; - switch (type) { - case LEADER_CHANGE: - records = MemoryRecords.withLeaderChangeMessage( - 0, - 0, - 1, - ByteBuffer.allocate(128), - new LeaderChangeMessage() - ); - break; - case SNAPSHOT_HEADER: - records = MemoryRecords.withSnapshotHeaderRecord( - 0, - 0, - 1, - ByteBuffer.allocate(128), - new SnapshotHeaderRecord() - ); - break; - case SNAPSHOT_FOOTER: - records = MemoryRecords.withSnapshotFooterRecord( - 0, - 0, - 1, - ByteBuffer.allocate(128), - new SnapshotFooterRecord() - ); - break; - default: - throw new RuntimeException(String.format("Control record type %s is not supported", type)); + ByteBuffer buffer = ByteBuffer.allocate(128); + + try (MemoryRecordsBuilder builder = new MemoryRecordsBuilder( + buffer, + RecordBatch.CURRENT_MAGIC_VALUE, + CompressionType.NONE, + TimestampType.CREATE_TIME, + 0, // initialOffset + 0, // timestamp + RecordBatch.NO_PRODUCER_ID, + RecordBatch.NO_PRODUCER_EPOCH, + RecordBatch.NO_SEQUENCE, + false, + true, + 1, // leaderEpoch + buffer.capacity() + ) + ) { + builder.appendControlRecord( + 0, + type, + MessageUtil.toByteBuffer(defaultControlRecord(type), defaultControlRecordVersion(type)) + ); } - return records; + buffer.flip(); + return MemoryRecords.readableRecords(buffer); } public static MemoryRecords buildRecords( @@ -415,4 +405,38 @@ static TestBatch from(Batch batch) { return new TestBatch<>(batch.baseOffset(), batch.epoch(), batch.appendTimestamp(), batch.records()); } } + + private static ApiMessage defaultControlRecord(ControlRecordType type) { + switch (type) { + case LEADER_CHANGE: + return new LeaderChangeMessage(); + case SNAPSHOT_HEADER: + return new SnapshotHeaderRecord(); + case SNAPSHOT_FOOTER: + return new SnapshotFooterRecord(); + case KRAFT_VERSION: + return new KRaftVersionRecord(); + case VOTERS: + return new VotersRecord(); + default: + throw new RuntimeException("Should not happen. Poorly configured test"); + } + } + + private static short defaultControlRecordVersion(ControlRecordType type) { + switch (type) { + case LEADER_CHANGE: + return ControlRecordUtils.LEADER_CHANGE_CURRENT_VERSION; + case SNAPSHOT_HEADER: + return ControlRecordUtils.SNAPSHOT_HEADER_CURRENT_VERSION; + case SNAPSHOT_FOOTER: + return ControlRecordUtils.SNAPSHOT_FOOTER_CURRENT_VERSION; + case KRAFT_VERSION: + return ControlRecordUtils.KRAFT_VERSION_CURRENT_VERSION; + case VOTERS: + return ControlRecordUtils.VOTERS_CURRENT_VERSION; + default: + throw new RuntimeException("Should not happen. Poorly configured test"); + } + } } From 87a43c6a98e911944758b0c925b46be17b17657b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Wed, 14 Feb 2024 14:21:19 -0800 Subject: [PATCH 02/34] Some observation while reviewing the KIP and implementation --- .../src/main/java/org/apache/kafka/raft/KafkaRaftClient.java | 5 +++++ 1 file changed, 5 insertions(+) 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 e9fca1d6b2b5..84be74ad95f8 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -1584,6 +1584,11 @@ private void maybeTransition( // The request or response indicates the leader of the current epoch, // which is currently unknown transitionToFollower(epoch, leaderId.getAsInt(), currentTimeMs); + } else { + // TODO: implement this to prospective. Talk about reconfiguration without pre-vote + // Maybe if the prospective doesn't exist then transition to candidate. + // TODO: take a look at the simulation as to why resign with check quorum shows liveness + // } } From c9de8a171fcc7642da39c08fca97e9021d79518b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Mon, 25 Mar 2024 13:10:23 -0700 Subject: [PATCH 03/34] Fix compilation errors --- core/src/main/scala/kafka/tools/DumpLogSegments.scala | 4 ++-- .../src/main/java/org/apache/kafka/raft/KafkaRaftClient.java | 5 ----- 2 files changed, 2 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/kafka/tools/DumpLogSegments.scala b/core/src/main/scala/kafka/tools/DumpLogSegments.scala index 55ced144cc19..45705ee0f05b 100755 --- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala +++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala @@ -312,10 +312,10 @@ object DumpLogSegments { print(s" SnapshotFooter ${SnapshotFooterRecordJsonConverter.write(footer, footer.version())}") case ControlRecordType.KRAFT_VERSION => val kraftVersion = ControlRecordUtils.deserializeKRaftVersionRecord(record) - print(s" KRaftVersion ${KRaftVersionRecordJsonConverter.write(kraftVersion,, kraftVersion.version())}") + print(s" KRaftVersion ${KRaftVersionRecordJsonConverter.write(kraftVersion, kraftVersion.version())}") case ControlRecordType.VOTERS=> val voters = ControlRecordUtils.deserializeVotersRecord(record) - print(s" Voets ${VotersRecordJsonConverter.write(voters,, voters.version())}") + print(s" Voets ${VotersRecordJsonConverter.write(voters, voters.version())}") case controlType => print(s" controlType: $controlType($controlTypeId)") } 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 84be74ad95f8..e9fca1d6b2b5 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -1584,11 +1584,6 @@ private void maybeTransition( // The request or response indicates the leader of the current epoch, // which is currently unknown transitionToFollower(epoch, leaderId.getAsInt(), currentTimeMs); - } else { - // TODO: implement this to prospective. Talk about reconfiguration without pre-vote - // Maybe if the prospective doesn't exist then transition to candidate. - // TODO: take a look at the simulation as to why resign with check quorum shows liveness - // } } From 54b164eac9ad242d9160b319c71e5fc00da1ae57 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Mon, 25 Mar 2024 13:42:06 -0700 Subject: [PATCH 04/34] KAFKA-16207; Match KIP schema --- clients/src/main/resources/common/message/VotersRecord.json | 4 +--- core/src/main/scala/kafka/tools/DumpLogSegments.scala | 2 +- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/clients/src/main/resources/common/message/VotersRecord.json b/clients/src/main/resources/common/message/VotersRecord.json index 239b25759d92..12e8dcec1a1c 100644 --- a/clients/src/main/resources/common/message/VotersRecord.json +++ b/clients/src/main/resources/common/message/VotersRecord.json @@ -33,9 +33,7 @@ { "name": "Host", "type": "string", "versions": "0+", "about": "The hostname" }, { "name": "Port", "type": "uint16", "versions": "0+", - "about": "The port" }, - { "name": "SecurityProtocol", "type": "int16", "versions": "0+", - "about": "The security protocol" } + "about": "The port" } ]}, { "name": "KRaftVersionFeature", "type": "KRaftVersionFeature", "versions": "0+", "about": "The range of versions of the protocol that the replica supports", "fields": [ diff --git a/core/src/main/scala/kafka/tools/DumpLogSegments.scala b/core/src/main/scala/kafka/tools/DumpLogSegments.scala index 45705ee0f05b..37a5cd8098be 100755 --- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala +++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala @@ -315,7 +315,7 @@ object DumpLogSegments { print(s" KRaftVersion ${KRaftVersionRecordJsonConverter.write(kraftVersion, kraftVersion.version())}") case ControlRecordType.VOTERS=> val voters = ControlRecordUtils.deserializeVotersRecord(record) - print(s" Voets ${VotersRecordJsonConverter.write(voters, voters.version())}") + print(s" Votes ${VotersRecordJsonConverter.write(voters, voters.version())}") case controlType => print(s" controlType: $controlType($controlTypeId)") } From 04fcf4cd069f8c09b4089a769a168997c12472af Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Wed, 27 Mar 2024 12:04:20 -0700 Subject: [PATCH 05/34] KAFKA-16207; Implement VoterSet and VoterSetHistory --- .../common/message/VotersRecord.json | 2 +- .../org/apache/kafka/raft/RaftConfig.java | 2 +- .../apache/kafka/raft/internals/VoterSet.java | 207 ++++++++++++++++++ .../kafka/raft/internals/VoterSetHistory.java | 97 ++++++++ .../raft/internals/VoterSetHistoryTest.java | 73 ++++++ 5 files changed, 379 insertions(+), 2 deletions(-) create mode 100644 raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java create mode 100644 raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java create mode 100644 raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java diff --git a/clients/src/main/resources/common/message/VotersRecord.json b/clients/src/main/resources/common/message/VotersRecord.json index 12e8dcec1a1c..2f040424a5c8 100644 --- a/clients/src/main/resources/common/message/VotersRecord.json +++ b/clients/src/main/resources/common/message/VotersRecord.json @@ -26,7 +26,7 @@ "about": "The replica id of the voter in the topic partition" }, { "name": "VoterUuid", "type": "uuid", "versions": "0+", "about": "The directory id of the voter in the topic partition" }, - { "name": "EndPoints", "type": "[]Endpoint", "versions": "0+", + { "name": "Endpoints", "type": "[]Endpoint", "versions": "0+", "about": "The endpoint that can be used to communicate with the voter", "fields": [ { "name": "Name", "type": "string", "versions": "0+", "mapKey": true, "about": "The name of the endpoint" }, diff --git a/raft/src/main/java/org/apache/kafka/raft/RaftConfig.java b/raft/src/main/java/org/apache/kafka/raft/RaftConfig.java index 5c5865f35812..c10898000594 100644 --- a/raft/src/main/java/org/apache/kafka/raft/RaftConfig.java +++ b/raft/src/main/java/org/apache/kafka/raft/RaftConfig.java @@ -251,7 +251,7 @@ public static List voterConnectionsToNodes(Map connection.getValue() instanceof InetAddressSpec) .map(connection -> { InetAddressSpec spec = (InetAddressSpec) connection.getValue(); - return new Node(connection.getKey(), spec.address.getHostName(), spec.address.getPort()); + return new Node(connection.getKey(), spec.address.getHostString(), spec.address.getPort()); }) .collect(Collectors.toList()); } diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java new file mode 100644 index 000000000000..515dd48385a4 --- /dev/null +++ b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java @@ -0,0 +1,207 @@ +/* + * 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.net.InetSocketAddress; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.HashSet; +import java.util.stream.Collectors; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.message.VotersRecord; +import org.apache.kafka.common.utils.Utils; + +// TODO: write unittest for VoterSet +// TODO: Write documentation +final public class VoterSet { + private final Map voters; + + VoterSet(Map voters) { + this.voters = voters; + } + + Optional voterAddress(int voter, String listener) { + return Optional.ofNullable(voters.get(voter)) + .flatMap(voterNode -> voterNode.address(listener)); + } + + VotersRecord toVotersRecord(short version) { + return new VotersRecord() + .setVersion(version) + .setVoters( + voters + .values() + .stream() + .map(voter -> { + Iterator endpoints = voter + .listeners() + .entrySet() + .stream() + .map(entry -> + new VotersRecord.Endpoint() + .setName(entry.getKey()) + .setHost(entry.getValue().getHostString()) + .setPort(entry.getValue().getPort()) + ) + .iterator(); + + return new VotersRecord.Voter() + .setVoterId(voter.id()) + .setVoterUuid(voter.uuid().orElse(Uuid.ZERO_UUID)) + .setEndpoints(new VotersRecord.EndpointCollection(endpoints)) + .setKRaftVersionFeature(voter.feature()); + }) + .collect(Collectors.toList()) + ); + } + + boolean hasOverlappingMajority(VoterSet that) { + if (Utils.diff(HashSet::new, voters.keySet(), that.voters.keySet()).size() > 2) return false; + if (Utils.diff(HashSet::new, that.voters.keySet(), voters.keySet()).size() > 2) return false; + + return true; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + VoterSet that = (VoterSet) o; + + return voters.equals(that.voters); + } + + @Override + public int hashCode() { + return Objects.hashCode(voters); + } + + @Override + public String toString() { + return String.format("VoterSet(voters=%s)", voters); + } + + final static class VoterNode { + private final int id; + private final Optional uuid; + private final Map listeners; + // TODO: is there a better type for this? + private final VotersRecord.KRaftVersionFeature feature; + + VoterNode( + int id, + Optional uuid, + Map listeners, + VotersRecord.KRaftVersionFeature feature + ) { + this.id = id; + this.uuid = uuid; + this.listeners = listeners; + this.feature = feature; + } + + int id() { + return id; + } + + Optional uuid() { + return uuid; + } + + Map listeners() { + return listeners; + } + + VotersRecord.KRaftVersionFeature feature() { + return feature; + } + + + Optional address(String listener) { + return Optional.ofNullable(listeners.get(listener)); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + VoterNode that = (VoterNode) o; + + if (id != that.id) return false; + if (!Objects.equals(uuid, that.uuid)) return false; + if (!Objects.equals(feature, that.feature)) return false; + if (!Objects.equals(listeners, that.listeners)) return false; + + return true; + } + + @Override + public int hashCode() { + return Objects.hash(id, uuid, listeners, feature); + } + + @Override + public String toString() { + return String.format("VoterNode(id=%d, uuid=%s, listeners=%s, feature=%s)", id, uuid, listeners, feature); + } + } + + public static VoterSet fromVotersRecord(VotersRecord voters) { + Map voterNodes = new HashMap<>(voters.voters().size()); + for (VotersRecord.Voter voter: voters.voters()) { + final Optional uuid; + if (!voter.voterUuid().equals(Uuid.ZERO_UUID)) { + uuid = Optional.of(voter.voterUuid()); + } else { + uuid = Optional.empty(); + } + + Map listeners = new HashMap<>(voter.endpoints().size()); + for (VotersRecord.Endpoint endpoint : voter.endpoints()) { + listeners.put(endpoint.name(), InetSocketAddress.createUnresolved(endpoint.host(), endpoint.port())); + } + + voterNodes.put(voter.voterId(), new VoterNode(voter.voterId(), uuid, listeners, voter.kRaftVersionFeature())); + } + + return new VoterSet(voterNodes); + } + + public static VoterSet fromAddressSpecs(String listener, Map voters) { + Map voterNodes = voters + .entrySet() + .stream() + .collect( + Collectors.toMap( + Map.Entry::getKey, + entry -> new VoterNode( + entry.getKey(), + Optional.empty(), + Collections.singletonMap(listener, entry.getValue()), + new VotersRecord.KRaftVersionFeature() + ) + ) + ); + return new VoterSet(voterNodes); + } +} diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java new file mode 100644 index 000000000000..0b44cfb08dc2 --- /dev/null +++ b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java @@ -0,0 +1,97 @@ +/* + * 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.util.Map.Entry; +import java.util.NavigableMap; +import java.util.Optional; +import java.util.TreeMap; +import java.util.Map; + +// TODO: write unittest for VoterSetHistory +// TODO: Write documentation +final public class VoterSetHistory { + Optional staticVoterSet; + TreeMap votersHistory = new TreeMap<>(); + + public VoterSetHistory(Optional staticVoterSet) { + this.staticVoterSet = staticVoterSet; + } + + public void nextVoterSet(long offset, VoterSet voters) { + Optional> lastEntry = Optional.ofNullable(votersHistory.lastEntry()); + Optional currentOffset = lastEntry.map(Map.Entry::getKey); + if (currentOffset.isPresent() && offset <= currentOffset.get()) { + throw new IllegalArgumentException( + String.format("Next offset %d must be greater than the last offset %d", offset, currentOffset.get()) + ); + } + + if (lastEntry.isPresent() && lastEntry.get().getKey() >= 0) { + // If the last voter set comes from the replicated log then the majorities must overlap. This ignores + // the static voter set and the bootstrapped voter set since they come from the configuration and the KRaft + // leader never guaranteed that they are the same across all replicas. + VoterSet lastVoterSet = lastEntry.get().getValue(); + if (!lastVoterSet.hasOverlappingMajority(voters)) { + throw new IllegalArgumentException( + String.format( + "Last voter set %s doesn't have an overlapping majority with the new voter set %s", + lastVoterSet, + voters + ) + ); + } + } + + votersHistory.compute( + offset, + (key, value) -> { + if (value != null) { + throw new IllegalArgumentException( + String.format("Rejected %s sincea voter set already exist at %d: %s", voters, offset, value) + ); + } + + return voters; + } + ); + } + + // TODO: document that this doesn't include the static configuration + public Optional voterSetAt(long offset) { + return Optional.ofNullable(votersHistory.floorEntry(offset)).map(Entry::getValue); + } + + public VoterSet latestVoterSet() { + return Optional + .ofNullable(votersHistory.lastEntry()) + .map(Entry::getValue) + .orElseGet(() -> staticVoterSet.orElseThrow(() -> new IllegalStateException("No voter set found"))); + } + + void truncateTo(long endOffset) { + votersHistory.tailMap(endOffset, false).clear(); + } + + void trimPrefixTo(long startOffset) { + NavigableMap lesserVoters = votersHistory.headMap(startOffset, true); + while (lesserVoters.size() > 1) { + // Poll and ignore the entry to remove the first entry + lesserVoters.pollFirstEntry(); + } + } +} diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java new file mode 100644 index 000000000000..8cd82a954abb --- /dev/null +++ b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java @@ -0,0 +1,73 @@ +/* + * 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.net.InetSocketAddress; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.message.VotersRecord; +import org.junit.jupiter.api.Test; +import static org.junit.jupiter.api.Assertions.assertEquals; + +final public class VoterSetHistoryTest { + @Test + void testStaicVoterSet() { + VoterSet staticVoterSet = createVoterSet(Arrays.asList(1, 2, 3)); + VoterSetHistory votersHistory = new VoterSetHistory(Optional.of(staticVoterSet)); + + validateStaticVoterSet(staticVoterSet, votersHistory); + + // Should be a no-op + votersHistory.truncateTo(100); + validateStaticVoterSet(staticVoterSet, votersHistory); + + // Should be a no-op + votersHistory.trimPrefixTo(100); + validateStaticVoterSet(staticVoterSet, votersHistory); + } + + private void validateStaticVoterSet(VoterSet expected, VoterSetHistory votersHistory) { + assertEquals(Optional.empty(), votersHistory.voterSetAt(0)); + assertEquals(Optional.empty(), votersHistory.voterSetAt(100)); + assertEquals(expected, votersHistory.latestVoterSet()); + } + + private VoterSet createVoterSet(List replicas) { + return new VoterSet( + replicas + .stream() + .collect( + Collectors.toMap( + replica -> replica, + replica -> new VoterSet.VoterNode( + replica, + Optional.of(Uuid.randomUuid()), + Collections.singletonMap( + "LISTNER", + InetSocketAddress.createUnresolved(String.format("replica-%d", replica), 1234) + ), + new VotersRecord.KRaftVersionFeature() + ) + ) + ) + ); + } +} From 702e739c671a8e93102ed736b2bafbeb612ccaf6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Thu, 28 Mar 2024 11:27:20 -0700 Subject: [PATCH 06/34] KAFKA-16207; Add impl for internal log listener --- .../apache/kafka/raft/KafkaRaftClient.java | 9 +- .../kafka/raft/internals/CloseListener.java | 3 +- .../raft/internals/InternalLogListener.java | 139 ++++++++++++++++++ .../raft/internals/RecordsBatchReader.java | 1 + .../kafka/raft/internals/VoterSetHistory.java | 16 +- 5 files changed, 158 insertions(+), 10 deletions(-) create mode 100644 raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java 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 e9fca1d6b2b5..f12c9c830a8a 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -1200,6 +1200,8 @@ private void appendAsFollower( log.flush(false); } + // TODO: Update the internal listener + OffsetAndEpoch endOffset = endOffset(); kafkaRaftMetrics.updateFetchedRecords(info.lastOffset - info.firstOffset + 1); kafkaRaftMetrics.updateLogEnd(endOffset); @@ -1210,6 +1212,9 @@ private LogAppendInfo appendAsLeader( Records records ) { LogAppendInfo info = log.appendAsLeader(records, quorum.epoch()); + + // TODO: Notify the internal listener + OffsetAndEpoch endOffset = endOffset(); kafkaRaftMetrics.updateAppendRecords(info.lastOffset - info.firstOffset + 1); kafkaRaftMetrics.updateLogEnd(endOffset); @@ -1926,12 +1931,12 @@ private void appendBatch( future.whenComplete((commitTimeMs, exception) -> { if (exception != null) { - logger.debug("Failed to commit {} records at {}", batch.numRecords, offsetAndEpoch, exception); + logger.debug("Failed to commit {} records up to last offset {}", batch.numRecords, offsetAndEpoch, exception); } else { long elapsedTime = Math.max(0, commitTimeMs - appendTimeMs); double elapsedTimePerRecord = (double) elapsedTime / batch.numRecords; kafkaRaftMetrics.updateCommitLatency(elapsedTimePerRecord, appendTimeMs); - logger.debug("Completed commit of {} records at {}", batch.numRecords, offsetAndEpoch); + logger.debug("Completed commit of {} records up to last offset {}", batch.numRecords, offsetAndEpoch); batch.records.ifPresent(records -> { maybeFireHandleCommit(batch.baseOffset, epoch, batch.appendTimestamp(), batch.sizeInBytes(), records); }); diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/CloseListener.java b/raft/src/main/java/org/apache/kafka/raft/internals/CloseListener.java index e54ff96ce41a..2f15d40e3f12 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/CloseListener.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/CloseListener.java @@ -16,8 +16,7 @@ */ package org.apache.kafka.raft.internals; +@FunctionalInterface public interface CloseListener { - void onClose(T closeable); - } diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java b/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java new file mode 100644 index 000000000000..b1781cbd341f --- /dev/null +++ b/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java @@ -0,0 +1,139 @@ +/* + * 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 org.apache.kafka.common.message.KRaftVersionRecord; +import org.apache.kafka.common.message.VotersRecord; +import org.apache.kafka.common.utils.BufferSupplier; +import org.apache.kafka.raft.Batch; +import org.apache.kafka.raft.ControlRecord; +import org.apache.kafka.raft.Isolation; +import org.apache.kafka.raft.LogFetchInfo; +import org.apache.kafka.raft.ReplicatedLog; +import org.apache.kafka.server.common.serialization.RecordSerde; +import java.util.OptionalLong; +import java.util.Optional; +import org.apache.kafka.snapshot.RawSnapshotReader; +import org.apache.kafka.snapshot.SnapshotReader; +import org.apache.kafka.snapshot.RecordsSnapshotReader; +import static org.apache.kafka.raft.KafkaRaftClient.MAX_BATCH_SIZE_BYTES; + +// TODO: File an issue to remove the RecordSerde. The internal listener should just skip data record batches +// TODO: Document this class and methods +// TODO: Add unnitest for it +final class InternalLogListener { + private final ReplicatedLog log; + private final VoterSetHistory voterSetHistory; + private final RecordSerde serde; + private final BufferSupplier bufferSupplier; + + // TODO: We need to keep the kraft version history so that the right value is included in the snapshot + private short kraftVersion = 0; + private long nextOffset = 0; + + InternalLogListener( + Optional staticVoterSet, + ReplicatedLog log, + RecordSerde serde, + BufferSupplier bufferSupplier + ) { + this.log = log; + this.voterSetHistory = new VoterSetHistory(staticVoterSet); + this.serde = serde; + this.bufferSupplier = bufferSupplier; + } + + void updateListerner() { + maybeLoadSnapshot(); + maybeLoadLog(); + } + + short kraftVersion() { + return kraftVersion; + } + + private void maybeLoadLog() { + while (log.endOffset().offset > nextOffset) { + LogFetchInfo info = log.read(nextOffset, Isolation.UNCOMMITTED); + try (RecordsIterator iterator = new RecordsIterator<>( + info.records, + serde, + bufferSupplier, + MAX_BATCH_SIZE_BYTES, + true // Validate batch CRC + ) + ) { + while (iterator.hasNext()) { + Batch batch = iterator.next(); + handleBatch(batch, OptionalLong.empty()); + nextOffset = batch.lastOffset() + 1; + } + } + } + } + + private void maybeLoadSnapshot() { + Optional rawSnapshot = log.latestSnapshot(); + if (rawSnapshot.isPresent() && (nextOffset == 0 || nextOffset > log.startOffset())) { + // Clear the current state + kraftVersion = 0; + voterSetHistory.clear(); + + // Load the snapshot since the listener is at the start of the log or the log doesn't have the next entry. + try (SnapshotReader reader = RecordsSnapshotReader.of( + rawSnapshot.get(), + serde, + bufferSupplier, + MAX_BATCH_SIZE_BYTES, + true // Validate batch CRC + ) + ) { + OptionalLong currentOffset = OptionalLong.of(reader.lastContainedLogOffset()); + while (reader.hasNext()) { + Batch batch = reader.next(); + handleBatch(batch, currentOffset); + } + + nextOffset = reader.lastContainedLogOffset() + 1; + } + } + } + + private void handleBatch(Batch batch, OptionalLong overrideOffset) { + int index = 0; + for (ControlRecord record : batch.controlRecords()) { + long currentOffset = overrideOffset.orElse(batch.baseOffset() + index); + switch (record.type()) { + case VOTERS: + voterSetHistory.nextVoterSet( + currentOffset, + VoterSet.fromVotersRecord((VotersRecord) record.message()) + ); + break; + + case KRAFT_VERSION: + kraftVersion = ((KRaftVersionRecord) record.message()).kRaftVersion(); + break; + + default: + // Skip the rest of the control records + break; + } + ++index; + } + } +} 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 84e2d9fe3d0e..a9f4e106aa1d 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 @@ -76,6 +76,7 @@ public long baseOffset() { return baseOffset; } + @Override public OptionalLong lastOffset() { if (isClosed) { return OptionalLong.of(lastReturnedOffset); diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java index 0b44cfb08dc2..eac7c4462cb2 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java @@ -25,14 +25,14 @@ // TODO: write unittest for VoterSetHistory // TODO: Write documentation final public class VoterSetHistory { - Optional staticVoterSet; - TreeMap votersHistory = new TreeMap<>(); + private final Optional staticVoterSet; + private final TreeMap votersHistory = new TreeMap<>(); - public VoterSetHistory(Optional staticVoterSet) { + VoterSetHistory(Optional staticVoterSet) { this.staticVoterSet = staticVoterSet; } - public void nextVoterSet(long offset, VoterSet voters) { + void nextVoterSet(long offset, VoterSet voters) { Optional> lastEntry = Optional.ofNullable(votersHistory.lastEntry()); Optional currentOffset = lastEntry.map(Map.Entry::getKey); if (currentOffset.isPresent() && offset <= currentOffset.get()) { @@ -72,11 +72,11 @@ public void nextVoterSet(long offset, VoterSet voters) { } // TODO: document that this doesn't include the static configuration - public Optional voterSetAt(long offset) { + Optional voterSetAt(long offset) { return Optional.ofNullable(votersHistory.floorEntry(offset)).map(Entry::getValue); } - public VoterSet latestVoterSet() { + VoterSet latestVoterSet() { return Optional .ofNullable(votersHistory.lastEntry()) .map(Entry::getValue) @@ -94,4 +94,8 @@ void trimPrefixTo(long startOffset) { lesserVoters.pollFirstEntry(); } } + + void clear() { + votersHistory.clear(); + } } From fae8b8466811ee9e3c7d93bce7d50fd7e2870ca6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Thu, 28 Mar 2024 16:46:58 -0700 Subject: [PATCH 07/34] KAFKA-16207; Add historical kraft version --- .../apache/kafka/raft/internals/History.java | 54 +++++++++++++ .../raft/internals/InternalLogListener.java | 19 ++--- .../kafka/raft/internals/TreeMapHistory.java | 80 +++++++++++++++++++ .../kafka/raft/internals/VoterSetHistory.java | 74 +++++++---------- .../raft/internals/VoterSetHistoryTest.java | 6 +- 5 files changed, 172 insertions(+), 61 deletions(-) create mode 100644 raft/src/main/java/org/apache/kafka/raft/internals/History.java create mode 100644 raft/src/main/java/org/apache/kafka/raft/internals/TreeMapHistory.java diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/History.java b/raft/src/main/java/org/apache/kafka/raft/internals/History.java new file mode 100644 index 000000000000..d2b9cb4654ba --- /dev/null +++ b/raft/src/main/java/org/apache/kafka/raft/internals/History.java @@ -0,0 +1,54 @@ +/* + * 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.util.Optional; + +// TODO: document this type +public interface History { + public void addAt(long offset, T newValue); + + public Optional valueAt(long offset); + + public Optional> lastEntry(); + + public void truncateTo(long endOffset); + + public void trimPrefixTo(long startOffset); + + public void clear(); + + final public static class Entry { + private final long offset; + private final T value; + + public Entry(long offset, T value) { + this.offset = offset; + this.value = value; + } + + public long offset() { + return offset; + } + + public T value() { + return value; + } + + // TODO: override equals, hashCode and toString + } +} diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java b/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java index b1781cbd341f..9f5290f18d16 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java @@ -37,12 +37,12 @@ // TODO: Add unnitest for it final class InternalLogListener { private final ReplicatedLog log; - private final VoterSetHistory voterSetHistory; private final RecordSerde serde; private final BufferSupplier bufferSupplier; - // TODO: We need to keep the kraft version history so that the right value is included in the snapshot - private short kraftVersion = 0; + private final VoterSetHistory voterSetHistory; + private final History kraftVersion = new TreeMapHistory<>(); + private long nextOffset = 0; InternalLogListener( @@ -62,10 +62,6 @@ void updateListerner() { maybeLoadLog(); } - short kraftVersion() { - return kraftVersion; - } - private void maybeLoadLog() { while (log.endOffset().offset > nextOffset) { LogFetchInfo info = log.read(nextOffset, Isolation.UNCOMMITTED); @@ -90,7 +86,7 @@ private void maybeLoadSnapshot() { Optional rawSnapshot = log.latestSnapshot(); if (rawSnapshot.isPresent() && (nextOffset == 0 || nextOffset > log.startOffset())) { // Clear the current state - kraftVersion = 0; + kraftVersion.clear(); voterSetHistory.clear(); // Load the snapshot since the listener is at the start of the log or the log doesn't have the next entry. @@ -119,14 +115,11 @@ private void handleBatch(Batch batch, OptionalLong overrideOffset) { long currentOffset = overrideOffset.orElse(batch.baseOffset() + index); switch (record.type()) { case VOTERS: - voterSetHistory.nextVoterSet( - currentOffset, - VoterSet.fromVotersRecord((VotersRecord) record.message()) - ); + voterSetHistory.addAt(currentOffset, VoterSet.fromVotersRecord((VotersRecord) record.message())); break; case KRAFT_VERSION: - kraftVersion = ((KRaftVersionRecord) record.message()).kRaftVersion(); + kraftVersion.addAt(currentOffset, ((KRaftVersionRecord) record.message()).kRaftVersion()); break; default: diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/TreeMapHistory.java b/raft/src/main/java/org/apache/kafka/raft/internals/TreeMapHistory.java new file mode 100644 index 000000000000..c4c41eaaf2d2 --- /dev/null +++ b/raft/src/main/java/org/apache/kafka/raft/internals/TreeMapHistory.java @@ -0,0 +1,80 @@ +/* + * 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.util.NavigableMap; +import java.util.Optional; +import java.util.TreeMap; +import java.util.Map; + +// TODO: document this type +// TODO: add unittest +final public class TreeMapHistory implements History { + private final NavigableMap history = new TreeMap<>(); + + @Override + public void addAt(long offset, T newValue) { + Map.Entry lastEntry = history.lastEntry(); + if (lastEntry != null && offset <= lastEntry.getKey()) { + throw new IllegalArgumentException( + String.format("Next offset %d must be greater than the last offset %d", offset, lastEntry.getKey()) + ); + } + + history.compute( + offset, + (key, oldValue) -> { + if (oldValue != null) { + throw new IllegalArgumentException( + String.format("Rejected %s since a value already exist at %d: %s", newValue, offset, oldValue) + ); + } + + return newValue; + } + ); + } + + @Override + public Optional valueAt(long offset) { + return Optional.ofNullable(history.floorEntry(offset)).map(Map.Entry::getValue); + } + + @Override + public Optional> lastEntry() { + return Optional.ofNullable(history.lastEntry()).map(entry -> new Entry<>(entry.getKey(), entry.getValue())); + } + + @Override + public void truncateTo(long endOffset) { + history.tailMap(endOffset, false).clear(); + } + + @Override + public void trimPrefixTo(long startOffset) { + NavigableMap lesserValues = history.headMap(startOffset, true); + while (lesserValues.size() > 1) { + // Poll and ignore the entry to remove the first entry + lesserValues.pollFirstEntry(); + } + } + + @Override + public void clear() { + history.clear(); + } +} diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java index eac7c4462cb2..7fa98036f3b3 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java @@ -16,36 +16,26 @@ */ package org.apache.kafka.raft.internals; -import java.util.Map.Entry; -import java.util.NavigableMap; import java.util.Optional; -import java.util.TreeMap; -import java.util.Map; // TODO: write unittest for VoterSetHistory // TODO: Write documentation -final public class VoterSetHistory { +final public class VoterSetHistory implements History { private final Optional staticVoterSet; - private final TreeMap votersHistory = new TreeMap<>(); + private final History votersHistory = new TreeMapHistory<>(); VoterSetHistory(Optional staticVoterSet) { this.staticVoterSet = staticVoterSet; } - void nextVoterSet(long offset, VoterSet voters) { - Optional> lastEntry = Optional.ofNullable(votersHistory.lastEntry()); - Optional currentOffset = lastEntry.map(Map.Entry::getKey); - if (currentOffset.isPresent() && offset <= currentOffset.get()) { - throw new IllegalArgumentException( - String.format("Next offset %d must be greater than the last offset %d", offset, currentOffset.get()) - ); - } - - if (lastEntry.isPresent() && lastEntry.get().getKey() >= 0) { + @Override + public void addAt(long offset, VoterSet voters) { + Optional> lastEntry = votersHistory.lastEntry(); + if (lastEntry.isPresent() && lastEntry.get().offset() >= 0) { // If the last voter set comes from the replicated log then the majorities must overlap. This ignores // the static voter set and the bootstrapped voter set since they come from the configuration and the KRaft // leader never guaranteed that they are the same across all replicas. - VoterSet lastVoterSet = lastEntry.get().getValue(); + VoterSet lastVoterSet = lastEntry.get().value(); if (!lastVoterSet.hasOverlappingMajority(voters)) { throw new IllegalArgumentException( String.format( @@ -57,45 +47,39 @@ void nextVoterSet(long offset, VoterSet voters) { } } - votersHistory.compute( - offset, - (key, value) -> { - if (value != null) { - throw new IllegalArgumentException( - String.format("Rejected %s sincea voter set already exist at %d: %s", voters, offset, value) - ); - } - - return voters; - } - ); + votersHistory.addAt(offset, voters); } // TODO: document that this doesn't include the static configuration - Optional voterSetAt(long offset) { - return Optional.ofNullable(votersHistory.floorEntry(offset)).map(Entry::getValue); + @Override + public Optional valueAt(long offset) { + return votersHistory.valueAt(offset); } - VoterSet latestVoterSet() { - return Optional - .ofNullable(votersHistory.lastEntry()) - .map(Entry::getValue) - .orElseGet(() -> staticVoterSet.orElseThrow(() -> new IllegalStateException("No voter set found"))); + @Override + public Optional> lastEntry() { + Optional> result = votersHistory.lastEntry(); + if (result.isPresent()) return result; + + return staticVoterSet.map(value -> new History.Entry<>(-1, value)); } - void truncateTo(long endOffset) { - votersHistory.tailMap(endOffset, false).clear(); + public VoterSet lastValue() { + return lastEntry().orElseThrow(() -> new IllegalStateException("No voter set found")).value(); } - void trimPrefixTo(long startOffset) { - NavigableMap lesserVoters = votersHistory.headMap(startOffset, true); - while (lesserVoters.size() > 1) { - // Poll and ignore the entry to remove the first entry - lesserVoters.pollFirstEntry(); - } + @Override + public void truncateTo(long endOffset) { + votersHistory.truncateTo(endOffset); + } + + @Override + public void trimPrefixTo(long startOffset) { + votersHistory.trimPrefixTo(startOffset); } - void clear() { + @Override + public void clear() { votersHistory.clear(); } } diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java index 8cd82a954abb..db4e8d262545 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java @@ -45,9 +45,9 @@ void testStaicVoterSet() { } private void validateStaticVoterSet(VoterSet expected, VoterSetHistory votersHistory) { - assertEquals(Optional.empty(), votersHistory.voterSetAt(0)); - assertEquals(Optional.empty(), votersHistory.voterSetAt(100)); - assertEquals(expected, votersHistory.latestVoterSet()); + assertEquals(Optional.empty(), votersHistory.valueAt(0)); + assertEquals(Optional.empty(), votersHistory.valueAt(100)); + assertEquals(expected, votersHistory.lastValue()); } private VoterSet createVoterSet(List replicas) { From 25d9dfdee931de305a685c8ab9f5da271c8c03e4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Wed, 3 Apr 2024 18:24:44 -0700 Subject: [PATCH 08/34] KAFKA-16207; Import refactor to delay voter address evaluation --- .../main/scala/kafka/raft/RaftManager.scala | 16 +-- .../scala/kafka/server/KafkaRaftServer.scala | 1 + .../kafka/raft/KafkaNetworkChannel.java | 5 +- .../apache/kafka/raft/KafkaRaftClient.java | 124 ++++++++++++------ .../org/apache/kafka/raft/NetworkChannel.java | 4 +- .../org/apache/kafka/raft/RaftClient.java | 5 - .../org/apache/kafka/raft/RaftConfig.java | 19 +-- .../raft/internals/InternalLogListener.java | 4 +- .../kafka/raft/KafkaNetworkChannelTest.java | 21 +-- .../apache/kafka/raft/MockNetworkChannel.java | 3 +- .../kafka/raft/RaftClientTestContext.java | 31 +++-- .../kafka/raft/RaftEventSimulationTest.java | 38 ++++-- 12 files changed, 150 insertions(+), 121 deletions(-) diff --git a/core/src/main/scala/kafka/raft/RaftManager.scala b/core/src/main/scala/kafka/raft/RaftManager.scala index a9e64fb967be..cd1d7cb1d74e 100644 --- a/core/src/main/scala/kafka/raft/RaftManager.scala +++ b/core/src/main/scala/kafka/raft/RaftManager.scala @@ -137,20 +137,7 @@ class KafkaRaftManager[T]( private val clientDriver = new KafkaRaftClientDriver[T](client, threadNamePrefix, fatalFaultHandler, logContext) def startup(): Unit = { - // Update the voter endpoints (if valid) with what's in RaftConfig - val voterAddresses: util.Map[Integer, AddressSpec] = controllerQuorumVotersFuture.get() - for (voterAddressEntry <- voterAddresses.entrySet.asScala) { - voterAddressEntry.getValue match { - case spec: InetAddressSpec => - netChannel.updateEndpoint(voterAddressEntry.getKey, spec) - case _: UnknownAddressSpec => - info(s"Skipping channel update for destination ID: ${voterAddressEntry.getKey} " + - s"because of non-routable endpoint: ${NON_ROUTABLE_ADDRESS.toString}") - case invalid: AddressSpec => - warn(s"Unexpected address spec (type: ${invalid.getClass}) for channel update for " + - s"destination ID: ${voterAddressEntry.getKey}") - } - } + client.initialize(controllerQuorumVotersFuture.get()) netChannel.start() clientDriver.start() } @@ -196,7 +183,6 @@ class KafkaRaftManager[T]( nodeId, raftConfig ) - client.initialize() client } diff --git a/core/src/main/scala/kafka/server/KafkaRaftServer.scala b/core/src/main/scala/kafka/server/KafkaRaftServer.scala index 4ae5a8b02379..4dd32a6ee95a 100644 --- a/core/src/main/scala/kafka/server/KafkaRaftServer.scala +++ b/core/src/main/scala/kafka/server/KafkaRaftServer.scala @@ -65,6 +65,7 @@ class KafkaRaftServer( metaPropsEnsemble.clusterId().get() ) + // TODO: Map the value to InetSocketAddress and throw a configuration error if not possible private val controllerQuorumVotersFuture = CompletableFuture.completedFuture( RaftConfig.parseVoterConnections(config.quorumVoters)) diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaNetworkChannel.java b/raft/src/main/java/org/apache/kafka/raft/KafkaNetworkChannel.java index bc77c1638b9e..5ec91752cbeb 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaNetworkChannel.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaNetworkChannel.java @@ -39,6 +39,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; @@ -148,8 +149,8 @@ private ApiMessage errorResponse(ApiMessage request, Errors error) { } @Override - public void updateEndpoint(int id, RaftConfig.InetAddressSpec spec) { - Node node = new Node(id, spec.address.getHostString(), spec.address.getPort()); + public void updateEndpoint(int id, InetSocketAddress address) { + Node node = new Node(id, address.getHostString(), address.getPort()); endpoints.put(id, node); } 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 f12c9c830a8a..e6cdfed84ffb 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -65,10 +65,12 @@ import org.apache.kafka.raft.internals.BlockingMessageQueue; import org.apache.kafka.raft.internals.CloseListener; import org.apache.kafka.raft.internals.FuturePurgatory; +import org.apache.kafka.raft.internals.InternalLogListener; 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.ThresholdPurgatory; +import org.apache.kafka.raft.internals.VoterSet; import org.apache.kafka.server.common.serialization.RecordSerde; import org.apache.kafka.snapshot.RawSnapshotReader; import org.apache.kafka.snapshot.RawSnapshotWriter; @@ -78,6 +80,7 @@ import org.apache.kafka.snapshot.SnapshotReader; import org.slf4j.Logger; +import java.net.InetSocketAddress; import java.util.Collections; import java.util.IdentityHashMap; import java.util.Iterator; @@ -139,13 +142,14 @@ * as FileRecords, but we use {@link UnalignedRecords} in FetchSnapshotResponse because the records * are not necessarily offset-aligned. */ -public class KafkaRaftClient implements RaftClient { +final public class KafkaRaftClient implements RaftClient { private static final int RETRY_BACKOFF_BASE_MS = 100; public static final int MAX_FETCH_WAIT_MS = 500; public static final int MAX_BATCH_SIZE_BYTES = 8 * 1024 * 1024; public static final int MAX_FETCH_SIZE_BYTES = MAX_BATCH_SIZE_BYTES; private final AtomicReference shutdown = new AtomicReference<>(); + private final LogContext logContext; private final Logger logger; private final Time time; private final int fetchMaxWaitMs; @@ -159,14 +163,17 @@ public class KafkaRaftClient implements RaftClient { private final MemoryPool memoryPool; private final RaftMessageQueue messageQueue; private final RaftConfig raftConfig; - private final KafkaRaftMetrics kafkaRaftMetrics; - private final QuorumState quorum; - private final RequestManager requestManager; private final RaftMetadataLogCleanerManager snapshotCleaner; private final Map, ListenerContext> listenerContexts = new IdentityHashMap<>(); private final ConcurrentLinkedQueue> pendingRegistrations = new ConcurrentLinkedQueue<>(); + // Components that needs to be initialized because they depend on the voter set + private volatile InternalLogListener internalListener; + private volatile KafkaRaftMetrics kafkaRaftMetrics; + private volatile QuorumState quorum; + private volatile RequestManager requestManager; + /** * Create a new instance. * @@ -177,27 +184,21 @@ public KafkaRaftClient( RecordSerde serde, NetworkChannel channel, ReplicatedLog log, - QuorumStateStore quorumStateStore, Time time, - Metrics metrics, ExpirationService expirationService, LogContext logContext, String clusterId, - OptionalInt nodeId, RaftConfig raftConfig ) { this(serde, channel, new BlockingMessageQueue(), log, - quorumStateStore, new BatchMemoryPool(5, MAX_BATCH_SIZE_BYTES), time, - metrics, expirationService, MAX_FETCH_WAIT_MS, clusterId, - nodeId, logContext, new Random(), raftConfig); @@ -208,18 +209,16 @@ public KafkaRaftClient( NetworkChannel channel, RaftMessageQueue messageQueue, ReplicatedLog log, - QuorumStateStore quorumStateStore, MemoryPool memoryPool, Time time, - Metrics metrics, ExpirationService expirationService, int fetchMaxWaitMs, String clusterId, - OptionalInt nodeId, LogContext logContext, Random random, RaftConfig raftConfig ) { + this.logContext = logContext; this.serde = serde; this.channel = channel; this.messageQueue = messageQueue; @@ -234,28 +233,6 @@ public KafkaRaftClient( this.random = random; this.raftConfig = raftConfig; this.snapshotCleaner = new RaftMetadataLogCleanerManager(logger, time, 60000, log::maybeClean); - Set quorumVoterIds = raftConfig.quorumVoterIds(); - this.requestManager = new RequestManager(quorumVoterIds, raftConfig.retryBackoffMs(), - raftConfig.requestTimeoutMs(), random); - this.quorum = new QuorumState( - nodeId, - quorumVoterIds, - raftConfig.electionTimeoutMs(), - raftConfig.fetchTimeoutMs(), - quorumStateStore, - time, - logContext, - random); - this.kafkaRaftMetrics = new KafkaRaftMetrics(metrics, "raft", quorum); - // All Raft voters are statically configured and known at startup - // so there are no unknown voter connections. Report this metric as 0. - kafkaRaftMetrics.updateNumUnknownVoterConnections(0); - - // Update the voter endpoints with what's in RaftConfig - Map voterAddresses = raftConfig.quorumVoterConnections(); - voterAddresses.entrySet().stream() - .filter(e -> e.getValue() instanceof RaftConfig.InetAddressSpec) - .forEach(e -> this.channel.updateEndpoint(e.getKey(), (RaftConfig.InetAddressSpec) e.getValue())); } private void updateFollowerHighWatermark( @@ -370,8 +347,50 @@ private void maybeFireLeaderChange() { } } - @Override - public void initialize() { + // TODO: need a way to verify that initialize has been called. + + public void initialize( + OptionalInt nodeId, + Map voterAddresses, + QuorumStateStore quorumStateStore, + Metrics metrics + ) { + // TODO: create the internal log listener + internalListener = new InternalLogListener( + Optional.of(VoterSet.fromAddressSpecs("TODO", voterAddresses)), + log, + serde, + BufferSupplier.create() + ); + + // TODO: Fix this to use internal listener + requestManager = new RequestManager( + voterAddresses.keySet(), + raftConfig.retryBackoffMs(), + raftConfig.requestTimeoutMs(), + random + ); + + // TODO: Fix this to use internal listener + quorum = new QuorumState( + nodeId, + voterAddresses.keySet(), + raftConfig.electionTimeoutMs(), + raftConfig.fetchTimeoutMs(), + quorumStateStore, + time, + logContext, + random); + + kafkaRaftMetrics = new KafkaRaftMetrics(metrics, "raft", quorum); + // All Raft voters are statically configured and known at startup + // so there are no unknown voter connections. Report this metric as 0. + kafkaRaftMetrics.updateNumUnknownVoterConnections(0); + + // Update the voter endpoints with what's in RaftConfig + // TODO: Create issue to fix this. Eventually, this needs to be dynamic based on the internal listener + voterAddresses.entrySet().forEach(entry -> channel.updateEndpoint(entry.getKey(), entry.getValue())); + quorum.initialize(new OffsetAndEpoch(log.endOffset().offset, log.lastFetchedEpoch())); long currentTimeMs = time.milliseconds(); @@ -407,7 +426,11 @@ public void unregister(Listener listener) { @Override public LeaderAndEpoch leaderAndEpoch() { - return quorum.leaderAndEpoch(); + if (!isInitialized()) { + return LeaderAndEpoch.UNKNOWN; + } else { + return quorum.leaderAndEpoch(); + } } @Override @@ -2300,6 +2323,10 @@ public void handle(RaftRequest.Inbound request) { * requests and send any needed outbound requests. */ public void poll() { + if (!isInitialized()) { + new IllegalStateException("KafkaRaftClient must be initialized before polling"); + } + long startPollTimeMs = time.milliseconds(); if (maybeCompleteShutdown(startPollTimeMs)) { return; @@ -2335,6 +2362,10 @@ public long scheduleAtomicAppend(int epoch, OptionalLong requiredBaseOffset, Lis } private long append(int epoch, List records, OptionalLong requiredBaseOffset, boolean isAtomic) { + if (!isInitialized()) { + new NotLeaderException("Append failed because the replication is not the current leader"); + } + LeaderState leaderState = quorum.maybeLeaderState().orElseThrow( () -> new NotLeaderException("Append failed because the replication is not the current leader") ); @@ -2449,18 +2480,25 @@ public void close() { } } - QuorumState quorum() { - return quorum; - } - + @Override public OptionalLong highWatermark() { - if (quorum.highWatermark().isPresent()) { + if (isInitialized() && quorum.highWatermark().isPresent()) { return OptionalLong.of(quorum.highWatermark().get().offset); } else { return OptionalLong.empty(); } } + // Visible only for test + QuorumState quorum() { + // because this is only called by test is it okay to return null + return quorum; + } + + private boolean isInitialized() { + return internalListener != null && quorum != null && requestManager != null && kafkaRaftMetrics != null; + } + private class GracefulShutdown { final Timer finishTimer; final CompletableFuture completeFuture; diff --git a/raft/src/main/java/org/apache/kafka/raft/NetworkChannel.java b/raft/src/main/java/org/apache/kafka/raft/NetworkChannel.java index e527adf6f9b7..6c715b378877 100644 --- a/raft/src/main/java/org/apache/kafka/raft/NetworkChannel.java +++ b/raft/src/main/java/org/apache/kafka/raft/NetworkChannel.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.raft; +import java.net.InetSocketAddress; + /** * A simple network interface with few assumptions. We do not assume ordering * of requests or even that every outbound request will receive a response. @@ -37,7 +39,7 @@ public interface NetworkChannel extends AutoCloseable { /** * Update connection information for the given id. */ - void updateEndpoint(int id, RaftConfig.InetAddressSpec address); + void updateEndpoint(int id, InetSocketAddress address); default void close() throws InterruptedException {} 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 485d8101c7a3..13006b7624ae 100644 --- a/raft/src/main/java/org/apache/kafka/raft/RaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/RaftClient.java @@ -87,11 +87,6 @@ default void handleLeaderChange(LeaderAndEpoch leader) {} default void beginShutdown() {} } - /** - * Initialize the client. This should only be called once on startup. - */ - void initialize(); - /** * Register a listener to get commit, snapshot and leader notifications. * diff --git a/raft/src/main/java/org/apache/kafka/raft/RaftConfig.java b/raft/src/main/java/org/apache/kafka/raft/RaftConfig.java index c10898000594..0ab12de40c96 100644 --- a/raft/src/main/java/org/apache/kafka/raft/RaftConfig.java +++ b/raft/src/main/java/org/apache/kafka/raft/RaftConfig.java @@ -29,11 +29,10 @@ import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Set; import java.util.stream.Collectors; /** - * RaftConfig encapsulates configuration specific to the Raft quorum voter nodes. + * RaftConfig encapsulates configuration specific to the cluster metadata KRaft voter nodes. * Specifically, this class parses the voter node endpoints into an AddressSpec * for use with the KafkaRaftClient/KafkaNetworkChannel. * @@ -98,7 +97,6 @@ public class RaftConfig { private final int electionBackoffMaxMs; private final int fetchTimeoutMs; private final int appendLingerMs; - private final Map voterConnections; public interface AddressSpec { } @@ -139,17 +137,17 @@ private UnknownAddressSpec() { } public RaftConfig(AbstractConfig abstractConfig) { - this(parseVoterConnections(abstractConfig.getList(QUORUM_VOTERS_CONFIG)), + this( abstractConfig.getInt(QUORUM_REQUEST_TIMEOUT_MS_CONFIG), abstractConfig.getInt(QUORUM_RETRY_BACKOFF_MS_CONFIG), abstractConfig.getInt(QUORUM_ELECTION_TIMEOUT_MS_CONFIG), abstractConfig.getInt(QUORUM_ELECTION_BACKOFF_MAX_MS_CONFIG), abstractConfig.getInt(QUORUM_FETCH_TIMEOUT_MS_CONFIG), - abstractConfig.getInt(QUORUM_LINGER_MS_CONFIG)); + abstractConfig.getInt(QUORUM_LINGER_MS_CONFIG) + ); } public RaftConfig( - Map voterConnections, int requestTimeoutMs, int retryBackoffMs, int electionTimeoutMs, @@ -157,7 +155,6 @@ public RaftConfig( int fetchTimeoutMs, int appendLingerMs ) { - this.voterConnections = voterConnections; this.requestTimeoutMs = requestTimeoutMs; this.retryBackoffMs = retryBackoffMs; this.electionTimeoutMs = electionTimeoutMs; @@ -190,14 +187,6 @@ public int appendLingerMs() { return appendLingerMs; } - public Set quorumVoterIds() { - return quorumVoterConnections().keySet(); - } - - public Map quorumVoterConnections() { - return voterConnections; - } - private static Integer parseVoterId(String idString) { try { return Integer.parseInt(idString); diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java b/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java index 9f5290f18d16..28b58883541b 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java @@ -35,7 +35,7 @@ // TODO: File an issue to remove the RecordSerde. The internal listener should just skip data record batches // TODO: Document this class and methods // TODO: Add unnitest for it -final class InternalLogListener { +final public class InternalLogListener { private final ReplicatedLog log; private final RecordSerde serde; private final BufferSupplier bufferSupplier; @@ -45,7 +45,7 @@ final class InternalLogListener { private long nextOffset = 0; - InternalLogListener( + public InternalLogListener( Optional staticVoterSet, ReplicatedLog log, RecordSerde serde, diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaNetworkChannelTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaNetworkChannelTest.java index 3a1d097fc7a7..f7ff08b7075b 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaNetworkChannelTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaNetworkChannelTest.java @@ -108,8 +108,7 @@ public void testSendToUnknownDestination() throws ExecutionException, Interrupte public void testSendToBlackedOutDestination() throws ExecutionException, InterruptedException { int destinationId = 2; Node destinationNode = new Node(destinationId, "127.0.0.1", 9092); - channel.updateEndpoint(destinationId, new RaftConfig.InetAddressSpec( - new InetSocketAddress(destinationNode.host(), destinationNode.port()))); + channel.updateEndpoint(destinationId, new InetSocketAddress(destinationNode.host(), destinationNode.port())); client.backoff(destinationNode, 500); assertBrokerNotAvailable(destinationId); } @@ -118,8 +117,7 @@ public void testSendToBlackedOutDestination() throws ExecutionException, Interru public void testWakeupClientOnSend() throws InterruptedException, ExecutionException { int destinationId = 2; Node destinationNode = new Node(destinationId, "127.0.0.1", 9092); - channel.updateEndpoint(destinationId, new RaftConfig.InetAddressSpec( - new InetSocketAddress(destinationNode.host(), destinationNode.port()))); + channel.updateEndpoint(destinationId, new InetSocketAddress(destinationNode.host(), destinationNode.port())); client.enableBlockingUntilWakeup(1); @@ -145,8 +143,7 @@ public void testWakeupClientOnSend() throws InterruptedException, ExecutionExcep public void testSendAndDisconnect() throws ExecutionException, InterruptedException { int destinationId = 2; Node destinationNode = new Node(destinationId, "127.0.0.1", 9092); - channel.updateEndpoint(destinationId, new RaftConfig.InetAddressSpec( - new InetSocketAddress(destinationNode.host(), destinationNode.port()))); + channel.updateEndpoint(destinationId, new InetSocketAddress(destinationNode.host(), destinationNode.port())); for (ApiKeys apiKey : RAFT_APIS) { AbstractResponse response = buildResponse(buildTestErrorResponse(apiKey, Errors.INVALID_REQUEST)); @@ -159,8 +156,7 @@ public void testSendAndDisconnect() throws ExecutionException, InterruptedExcept public void testSendAndFailAuthentication() throws ExecutionException, InterruptedException { int destinationId = 2; Node destinationNode = new Node(destinationId, "127.0.0.1", 9092); - channel.updateEndpoint(destinationId, new RaftConfig.InetAddressSpec( - new InetSocketAddress(destinationNode.host(), destinationNode.port()))); + channel.updateEndpoint(destinationId, new InetSocketAddress(destinationNode.host(), destinationNode.port())); for (ApiKeys apiKey : RAFT_APIS) { client.createPendingAuthenticationError(destinationNode, 100); @@ -181,8 +177,7 @@ private void assertBrokerNotAvailable(int destinationId) throws ExecutionExcepti public void testSendAndReceiveOutboundRequest() throws ExecutionException, InterruptedException { int destinationId = 2; Node destinationNode = new Node(destinationId, "127.0.0.1", 9092); - channel.updateEndpoint(destinationId, new RaftConfig.InetAddressSpec( - new InetSocketAddress(destinationNode.host(), destinationNode.port()))); + channel.updateEndpoint(destinationId, new InetSocketAddress(destinationNode.host(), destinationNode.port())); for (ApiKeys apiKey : RAFT_APIS) { Errors expectedError = Errors.INVALID_REQUEST; @@ -197,8 +192,7 @@ public void testSendAndReceiveOutboundRequest() throws ExecutionException, Inter public void testUnsupportedVersionError() throws ExecutionException, InterruptedException { int destinationId = 2; Node destinationNode = new Node(destinationId, "127.0.0.1", 9092); - channel.updateEndpoint(destinationId, new RaftConfig.InetAddressSpec( - new InetSocketAddress(destinationNode.host(), destinationNode.port()))); + channel.updateEndpoint(destinationId, new InetSocketAddress(destinationNode.host(), destinationNode.port())); for (ApiKeys apiKey : RAFT_APIS) { client.prepareUnsupportedVersionResponse(request -> request.apiKey() == apiKey); @@ -211,8 +205,7 @@ public void testUnsupportedVersionError() throws ExecutionException, Interrupted public void testFetchRequestDowngrade(short version) { int destinationId = 2; Node destinationNode = new Node(destinationId, "127.0.0.1", 9092); - channel.updateEndpoint(destinationId, new RaftConfig.InetAddressSpec( - new InetSocketAddress(destinationNode.host(), destinationNode.port()))); + channel.updateEndpoint(destinationId, new InetSocketAddress(destinationNode.host(), destinationNode.port())); sendTestRequest(ApiKeys.FETCH, destinationId); channel.pollOnce(); diff --git a/raft/src/test/java/org/apache/kafka/raft/MockNetworkChannel.java b/raft/src/test/java/org/apache/kafka/raft/MockNetworkChannel.java index 2a9793170f9b..379290240e01 100644 --- a/raft/src/test/java/org/apache/kafka/raft/MockNetworkChannel.java +++ b/raft/src/test/java/org/apache/kafka/raft/MockNetworkChannel.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.protocol.ApiKeys; +import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.HashMap; import java.util.Iterator; @@ -57,7 +58,7 @@ public void send(RaftRequest.Outbound request) { } @Override - public void updateEndpoint(int id, RaftConfig.InetAddressSpec address) { + public void updateEndpoint(int id, InetSocketAddress address) { // empty } 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 68241ae70b19..839e25fe5c21 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -76,6 +76,7 @@ import java.util.OptionalLong; import java.util.Set; import java.util.function.Consumer; +import java.util.function.Function; import java.util.stream.Collectors; import static org.apache.kafka.raft.LeaderState.CHECK_QUORUM_TIMEOUT_FACTOR; @@ -234,31 +235,41 @@ public RaftClientTestContext build() throws IOException { Metrics metrics = new Metrics(time); MockNetworkChannel channel = new MockNetworkChannel(voters); MockListener listener = new MockListener(localId); - Map voterAddressMap = voters.stream() - .collect(Collectors.toMap(id -> id, RaftClientTestContext::mockAddress)); - RaftConfig raftConfig = new RaftConfig(voterAddressMap, requestTimeoutMs, RETRY_BACKOFF_MS, electionTimeoutMs, - ELECTION_BACKOFF_MAX_MS, FETCH_TIMEOUT_MS, appendLingerMs); + Map voterAddressMap = voters + .stream() + .collect(Collectors.toMap(Function.identity(), RaftClientTestContext::mockAddress)); + + RaftConfig raftConfig = new RaftConfig( + requestTimeoutMs, + RETRY_BACKOFF_MS, + electionTimeoutMs, + ELECTION_BACKOFF_MAX_MS, + FETCH_TIMEOUT_MS, + appendLingerMs + ); KafkaRaftClient client = new KafkaRaftClient<>( SERDE, channel, messageQueue, log, - quorumStateStore, memoryPool, time, - metrics, new MockExpirationService(time), FETCH_MAX_WAIT_MS, clusterId.toString(), - localId, logContext, random, raftConfig ); client.register(listener); - client.initialize(); + client.initialize( + localId, + voterAddressMap, + quorumStateStore, + metrics + ); RaftClientTestContext context = new RaftClientTestContext( clusterId, @@ -814,8 +825,8 @@ private List collectBeginEpochRequests(int epoch) { return requests; } - private static RaftConfig.AddressSpec mockAddress(int id) { - return new RaftConfig.InetAddressSpec(new InetSocketAddress("localhost", 9990 + id)); + private static InetSocketAddress mockAddress(int id) { + return new InetSocketAddress("localhost", 9990 + id); } EndQuorumEpochResponseData endEpochResponse( 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 a359e24778bb..3be305bb36f9 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java @@ -59,6 +59,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; +import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -708,8 +709,8 @@ void killAndDeletePersistentState(int nodeId) { nodes.put(nodeId, new PersistentState(nodeId)); } - private static RaftConfig.AddressSpec nodeAddress(int id) { - return new RaftConfig.InetAddressSpec(new InetSocketAddress("localhost", 9990 + id)); + private static InetSocketAddress nodeAddress(int id) { + return new InetSocketAddress("localhost", 9990 + id); } void start(int nodeId) { @@ -717,10 +718,18 @@ void start(int nodeId) { PersistentState persistentState = nodes.get(nodeId); MockNetworkChannel channel = new MockNetworkChannel(correlationIdCounter, voters); MockMessageQueue messageQueue = new MockMessageQueue(); - Map voterAddressMap = voters.stream() - .collect(Collectors.toMap(id -> id, Cluster::nodeAddress)); - RaftConfig raftConfig = new RaftConfig(voterAddressMap, REQUEST_TIMEOUT_MS, RETRY_BACKOFF_MS, ELECTION_TIMEOUT_MS, - ELECTION_JITTER_MS, FETCH_TIMEOUT_MS, LINGER_MS); + Map voterAddressMap = voters + .stream() + .collect(Collectors.toMap(Function.identity(), Cluster::nodeAddress)); + + RaftConfig raftConfig = new RaftConfig( + REQUEST_TIMEOUT_MS, + RETRY_BACKOFF_MS, + ELECTION_TIMEOUT_MS, + ELECTION_JITTER_MS, + FETCH_TIMEOUT_MS, + LINGER_MS + ); Metrics metrics = new Metrics(time); persistentState.log.reopen(); @@ -733,14 +742,11 @@ void start(int nodeId) { channel, messageQueue, persistentState.log, - persistentState.store, memoryPool, time, - metrics, new MockExpirationService(time), FETCH_MAX_WAIT_MS, clusterId.toString(), - OptionalInt.of(nodeId), logContext, random, raftConfig @@ -757,7 +763,8 @@ void start(int nodeId) { random, serde ); - node.initialize(); + node.initialize(voterAddressMap, metrics); + running.put(nodeId, node); } } @@ -800,9 +807,14 @@ private RaftNode( this.intSerde = intSerde; } - void initialize() { - client.register(this.counter); - client.initialize(); + void initialize(Map voterAddresses, Metrics metrics) { + client.register(counter); + client.initialize( + OptionalInt.of(nodeId), + voterAddresses, + store, + metrics + ); } void poll() { From 9a6b4c201ea2fafdb74cbd9303082d30a8d48444 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Fri, 5 Apr 2024 13:14:03 -0700 Subject: [PATCH 09/34] KAFKA-16207; Fix core and metadata after raft changes --- .../main/scala/kafka/raft/RaftManager.scala | 19 ++++---- .../scala/kafka/server/KafkaRaftServer.scala | 6 +-- .../main/scala/kafka/server/KafkaServer.scala | 13 +++--- .../scala/kafka/server/SharedServer.scala | 11 +++-- .../scala/kafka/tools/TestRaftServer.scala | 2 +- .../kafka/testkit/KafkaClusterTestKit.java | 21 ++++++--- .../kafka/server/QuorumTestHarness.scala | 16 ++++--- .../unit/kafka/raft/RaftManagerTest.scala | 2 +- .../unit/kafka/server/KafkaConfigTest.scala | 21 ++++----- .../image/publisher/SnapshotEmitterTest.java | 5 -- .../apache/kafka/metalog/LocalLogManager.java | 22 +-------- .../kafka/metalog/LocalLogManagerTestEnv.java | 3 -- .../org/apache/kafka/raft/RaftConfig.java | 46 +++++++++++++------ 13 files changed, 92 insertions(+), 95 deletions(-) diff --git a/core/src/main/scala/kafka/raft/RaftManager.scala b/core/src/main/scala/kafka/raft/RaftManager.scala index cd1d7cb1d74e..d03c0d6981e8 100644 --- a/core/src/main/scala/kafka/raft/RaftManager.scala +++ b/core/src/main/scala/kafka/raft/RaftManager.scala @@ -17,11 +17,12 @@ package kafka.raft import java.io.File +import java.net.InetSocketAddress import java.nio.file.Files import java.nio.file.Paths -import java.util import java.util.OptionalInt import java.util.concurrent.CompletableFuture +import java.util.{Map => JMap} import kafka.log.LogManager import kafka.log.UnifiedLog import kafka.server.KafkaConfig @@ -39,7 +40,6 @@ import org.apache.kafka.common.requests.RequestHeader import org.apache.kafka.common.security.JaasContext import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.{LogContext, Time} -import org.apache.kafka.raft.RaftConfig.{AddressSpec, InetAddressSpec, NON_ROUTABLE_ADDRESS, UnknownAddressSpec} import org.apache.kafka.raft.{FileBasedStateStore, KafkaNetworkChannel, KafkaRaftClient, KafkaRaftClientDriver, LeaderAndEpoch, RaftClient, RaftConfig, ReplicatedLog} import org.apache.kafka.server.ProcessRole import org.apache.kafka.server.common.serialization.RecordSerde @@ -98,7 +98,7 @@ class KafkaRaftManager[T]( time: Time, metrics: Metrics, threadNamePrefixOpt: Option[String], - val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, AddressSpec]], + val controllerQuorumVotersFuture: CompletableFuture[JMap[Integer, InetSocketAddress]], fatalFaultHandler: FaultHandler ) extends RaftManager[T] with Logging { @@ -137,7 +137,12 @@ class KafkaRaftManager[T]( private val clientDriver = new KafkaRaftClientDriver[T](client, threadNamePrefix, fatalFaultHandler, logContext) def startup(): Unit = { - client.initialize(controllerQuorumVotersFuture.get()) + client.initialize( + OptionalInt.of(config.nodeId), + controllerQuorumVotersFuture.get(), + new FileBasedStateStore(new File(dataDir, "quorum-state")), + metrics + ) netChannel.start() clientDriver.start() } @@ -167,20 +172,14 @@ class KafkaRaftManager[T]( } private def buildRaftClient(): KafkaRaftClient[T] = { - val quorumStateStore = new FileBasedStateStore(new File(dataDir, "quorum-state")) - val nodeId = OptionalInt.of(config.nodeId) - val client = new KafkaRaftClient( recordSerde, netChannel, replicatedLog, - quorumStateStore, time, - metrics, expirationService, logContext, clusterId, - nodeId, raftConfig ) client diff --git a/core/src/main/scala/kafka/server/KafkaRaftServer.scala b/core/src/main/scala/kafka/server/KafkaRaftServer.scala index 4dd32a6ee95a..4faf137a76d8 100644 --- a/core/src/main/scala/kafka/server/KafkaRaftServer.scala +++ b/core/src/main/scala/kafka/server/KafkaRaftServer.scala @@ -65,16 +65,12 @@ class KafkaRaftServer( metaPropsEnsemble.clusterId().get() ) - // TODO: Map the value to InetSocketAddress and throw a configuration error if not possible - private val controllerQuorumVotersFuture = CompletableFuture.completedFuture( - RaftConfig.parseVoterConnections(config.quorumVoters)) - private val sharedServer = new SharedServer( config, metaPropsEnsemble, time, metrics, - controllerQuorumVotersFuture, + CompletableFuture.completedFuture(RaftConfig.parseVoterConnections(config.quorumVoters, true)), new StandardFaultHandlerFactory(), ) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 233a6b8eb989..e9103cda0359 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -322,7 +322,8 @@ class KafkaServer( if (config.migrationEnabled) { kraftControllerNodes = RaftConfig.voterConnectionsToNodes( - RaftConfig.parseVoterConnections(config.quorumVoters)).asScala + RaftConfig.parseVoterConnections(config.quorumVoters, true) + ).asScala } else { kraftControllerNodes = Seq.empty } @@ -330,7 +331,7 @@ class KafkaServer( config.brokerId, config.interBrokerProtocolVersion, brokerFeatures, - kraftControllerNodes, + kraftControllerNodes, // TODO: This needs to use the raft manager config.migrationEnabled) val controllerNodeProvider = new MetadataCacheControllerNodeProvider(metadataCache, config) @@ -421,8 +422,7 @@ class KafkaServer( logManager.directoryIdsSet) // If the ZK broker is in migration mode, start up a RaftManager to learn about the new KRaft controller - val controllerQuorumVotersFuture = CompletableFuture.completedFuture( - RaftConfig.parseVoterConnections(config.quorumVoters)) + val quorumVoters = RaftConfig.parseVoterConnections(config.quorumVoters, true) raftManager = new KafkaRaftManager[ApiMessageAndVersion]( metaPropsEnsemble.clusterId().get(), config, @@ -432,10 +432,11 @@ class KafkaServer( time, metrics, threadNamePrefix, - controllerQuorumVotersFuture, + CompletableFuture.completedFuture(quorumVoters), fatalFaultHandler = new LoggingFaultHandler("raftManager", () => shutdown()) ) - val controllerNodes = RaftConfig.voterConnectionsToNodes(controllerQuorumVotersFuture.get()).asScala + val controllerNodes = RaftConfig.voterConnectionsToNodes(quorumVoters).asScala + // TODO: This needs to use the raft manager val quorumControllerNodeProvider = RaftControllerNodeProvider(raftManager, config, controllerNodes) val brokerToQuorumChannelManager = new NodeToControllerChannelManagerImpl( controllerNodeProvider = quorumControllerNodeProvider, diff --git a/core/src/main/scala/kafka/server/SharedServer.scala b/core/src/main/scala/kafka/server/SharedServer.scala index 31b4957e3f54..9c2c72504928 100644 --- a/core/src/main/scala/kafka/server/SharedServer.scala +++ b/core/src/main/scala/kafka/server/SharedServer.scala @@ -31,16 +31,17 @@ import org.apache.kafka.image.publisher.{SnapshotEmitter, SnapshotGenerator} import org.apache.kafka.image.publisher.metrics.SnapshotEmitterMetrics import org.apache.kafka.metadata.MetadataRecordSerde import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble -import org.apache.kafka.raft.RaftConfig.AddressSpec import org.apache.kafka.server.ProcessRole import org.apache.kafka.server.common.ApiMessageAndVersion import org.apache.kafka.server.fault.{FaultHandler, LoggingFaultHandler, ProcessTerminatingFaultHandler} import org.apache.kafka.server.metrics.KafkaYammerMetrics -import java.util +import java.net.InetSocketAddress +import java.util.Arrays import java.util.Optional -import java.util.concurrent.{CompletableFuture, TimeUnit} import java.util.concurrent.atomic.AtomicReference +import java.util.concurrent.{CompletableFuture, TimeUnit} +import java.util.{Map => JMap} /** @@ -92,7 +93,7 @@ class SharedServer( val metaPropsEnsemble: MetaPropertiesEnsemble, val time: Time, private val _metrics: Metrics, - val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, AddressSpec]], + val controllerQuorumVotersFuture: CompletableFuture[JMap[Integer, InetSocketAddress]], val faultHandlerFactory: FaultHandlerFactory ) extends Logging { private val logContext: LogContext = new LogContext(s"[SharedServer id=${sharedServerConfig.nodeId}] ") @@ -303,7 +304,7 @@ class SharedServer( setThreadNamePrefix(s"kafka-${sharedServerConfig.nodeId}-"). build() try { - loader.installPublishers(util.Arrays.asList(snapshotGenerator)).get() + loader.installPublishers(Arrays.asList(snapshotGenerator)).get() } catch { case t: Throwable => { error("Unable to install metadata publishers", t) diff --git a/core/src/main/scala/kafka/tools/TestRaftServer.scala b/core/src/main/scala/kafka/tools/TestRaftServer.scala index 092ec67b2199..eef6248a3aaa 100644 --- a/core/src/main/scala/kafka/tools/TestRaftServer.scala +++ b/core/src/main/scala/kafka/tools/TestRaftServer.scala @@ -91,7 +91,7 @@ class TestRaftServer( time, metrics, Some(threadNamePrefix), - CompletableFuture.completedFuture(RaftConfig.parseVoterConnections(config.quorumVoters)), + CompletableFuture.completedFuture(RaftConfig.parseVoterConnections(config.quorumVoters, true)), new ProcessTerminatingFaultHandler.Builder().build() ) diff --git a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java index 4c6d1f9ef4ad..c2c20184b4ae 100644 --- a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java +++ b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java @@ -86,7 +86,7 @@ public class KafkaClusterTestKit implements AutoCloseable { */ private static class ControllerQuorumVotersFutureManager implements AutoCloseable { private final int expectedControllers; - private final CompletableFuture> future = new CompletableFuture<>(); + private final CompletableFuture> future = new CompletableFuture<>(); private final Map controllerPorts = new TreeMap<>(); ControllerQuorumVotersFutureManager(int expectedControllers) { @@ -96,11 +96,17 @@ private static class ControllerQuorumVotersFutureManager implements AutoCloseabl synchronized void registerPort(int nodeId, int port) { controllerPorts.put(nodeId, port); if (controllerPorts.size() >= expectedControllers) { - future.complete(controllerPorts.entrySet().stream(). - collect(Collectors.toMap( - Map.Entry::getKey, - entry -> new RaftConfig.InetAddressSpec(new InetSocketAddress("localhost", entry.getValue())) - ))); + future.complete( + controllerPorts + .entrySet() + .stream() + .collect( + Collectors.toMap( + Map.Entry::getKey, + entry -> new InetSocketAddress("localhost", entry.getValue()) + ) + ) + ); } } @@ -448,7 +454,8 @@ public void waitForReadyBrokers() throws ExecutionException, InterruptedExceptio public String quorumVotersConfig() throws ExecutionException, InterruptedException { Collection controllerNodes = RaftConfig.voterConnectionsToNodes( - controllerQuorumVotersFutureManager.future.get()); + controllerQuorumVotersFutureManager.future.get() + ); StringBuilder bld = new StringBuilder(); String prefix = ""; for (Node node : controllerNodes) { diff --git a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala index aabf3f934536..67641256c6e2 100755 --- a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala +++ b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala @@ -35,7 +35,6 @@ import org.apache.kafka.metadata.bootstrap.BootstrapMetadata import org.apache.kafka.common.metadata.FeatureLevelRecord import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble.VerificationFlag.{REQUIRE_AT_LEAST_ONE_VALID, REQUIRE_METADATA_LOG_DIR} import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion} -import org.apache.kafka.raft.RaftConfig.{AddressSpec, InetAddressSpec} import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion} import org.apache.kafka.server.fault.{FaultHandler, MockFaultHandler} import org.apache.zookeeper.client.ZKClientConfig @@ -88,7 +87,7 @@ class KRaftQuorumImplementation( val controllerServer: ControllerServer, val faultHandlerFactory: FaultHandlerFactory, val metadataDir: File, - val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, AddressSpec]], + val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, InetSocketAddress]], val clusterId: String, val log: Logging, val faultHandler: FaultHandler @@ -353,7 +352,7 @@ abstract class QuorumTestHarness extends Logging { props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") props.setProperty(KafkaConfig.QuorumVotersProp, s"${nodeId}@localhost:0") val config = new KafkaConfig(props) - val controllerQuorumVotersFuture = new CompletableFuture[util.Map[Integer, AddressSpec]] + val controllerQuorumVotersFuture = new CompletableFuture[util.Map[Integer, InetSocketAddress]] val metaPropertiesEnsemble = new MetaPropertiesEnsemble.Loader(). addMetadataLogDir(metadataDir.getAbsolutePath). load() @@ -378,8 +377,9 @@ abstract class QuorumTestHarness extends Logging { error("Error completing controller socket server future", e) controllerQuorumVotersFuture.completeExceptionally(e) } else { - controllerQuorumVotersFuture.complete(Collections.singletonMap(nodeId, - new InetAddressSpec(new InetSocketAddress("localhost", port)))) + controllerQuorumVotersFuture.complete( + Collections.singletonMap(nodeId, new InetSocketAddress("localhost", port)) + ) } }) controllerServer.startup() @@ -389,13 +389,15 @@ abstract class QuorumTestHarness extends Logging { CoreUtils.swallow(sharedServer.stopForController(), this) throw e } - new KRaftQuorumImplementation(controllerServer, + new KRaftQuorumImplementation( + controllerServer, faultHandlerFactory, metadataDir, controllerQuorumVotersFuture, metaProperties.clusterId.get(), this, - faultHandler) + faultHandler + ) } private def newZooKeeperQuorum(): ZooKeeperQuorumImplementation = { diff --git a/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala b/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala index e716bb452d91..b711df6f0acb 100644 --- a/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala +++ b/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala @@ -88,7 +88,7 @@ class RaftManagerTest { Time.SYSTEM, new Metrics(Time.SYSTEM), Option.empty, - CompletableFuture.completedFuture(RaftConfig.parseVoterConnections(config.quorumVoters)), + CompletableFuture.completedFuture(RaftConfig.parseVoterConnections(config.quorumVoters, true)), mock(classOf[FaultHandler]) ) } diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 391a93ab0000..46be71dd2f9b 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -27,7 +27,6 @@ import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.record.{CompressionType, Records} import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.raft.RaftConfig -import org.apache.kafka.raft.RaftConfig.{AddressSpec, InetAddressSpec, UNKNOWN_ADDRESS_SPEC_INSTANCE} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test @@ -1337,29 +1336,29 @@ class KafkaConfigTest { } @Test - def testValidQuorumVotersConfig(): Unit = { - val expected = new util.HashMap[Integer, AddressSpec]() + def testValidQuorumVotersParsing(): Unit = { + val expected = new util.HashMap[Integer, InetSocketAddress]() assertValidQuorumVoters("", expected) - expected.put(1, new InetAddressSpec(new InetSocketAddress("127.0.0.1", 9092))) + expected.put(1, new InetSocketAddress("127.0.0.1", 9092)) assertValidQuorumVoters("1@127.0.0.1:9092", expected) expected.clear() - expected.put(1, UNKNOWN_ADDRESS_SPEC_INSTANCE) + expected.put(1, new InetSocketAddress("0.0.0.0", 0)) assertValidQuorumVoters("1@0.0.0.0:0", expected) expected.clear() - expected.put(1, new InetAddressSpec(new InetSocketAddress("kafka1", 9092))) - expected.put(2, new InetAddressSpec(new InetSocketAddress("kafka2", 9092))) - expected.put(3, new InetAddressSpec(new InetSocketAddress("kafka3", 9092))) + expected.put(1, new InetSocketAddress("kafka1", 9092)) + expected.put(2, new InetSocketAddress("kafka2", 9092)) + expected.put(3, new InetSocketAddress("kafka3", 9092)) assertValidQuorumVoters("1@kafka1:9092,2@kafka2:9092,3@kafka3:9092", expected) } - private def assertValidQuorumVoters(value: String, expectedVoters: util.Map[Integer, AddressSpec]): Unit = { + private def assertValidQuorumVoters(value: String, expectedVoters: util.Map[Integer, InetSocketAddress]): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect) props.setProperty(KafkaConfig.QuorumVotersProp, value) - val raftConfig = new RaftConfig(KafkaConfig.fromProps(props)) - assertEquals(expectedVoters, raftConfig.quorumVoterConnections()) + val addresses = RaftConfig.parseVoterConnections(KafkaConfig.fromProps(props).quorumVoters, false) + assertEquals(expectedVoters, addresses) } @Test diff --git a/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotEmitterTest.java b/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotEmitterTest.java index 088b6c921c37..d8008ad56f8f 100644 --- a/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotEmitterTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotEmitterTest.java @@ -46,11 +46,6 @@ public class SnapshotEmitterTest { static class MockRaftClient implements RaftClient { TreeMap writers = new TreeMap<>(); - @Override - public void initialize() { - // nothing to do - } - @Override public void register(Listener listener) { // nothing to do diff --git a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java index f2e9d22bb597..d5b9ed16c642 100644 --- a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java +++ b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java @@ -484,11 +484,6 @@ void beginShutdown() { */ private final EventQueue eventQueue; - /** - * Whether this LocalLogManager has been initialized. - */ - private boolean initialized = false; - /** * Whether this LocalLogManager has been shut down. */ @@ -633,7 +628,7 @@ class ShutdownEvent implements EventQueue.Event { @Override public void run() throws Exception { try { - if (initialized && !shutdown) { + if (!shutdown) { log.debug("Node {}: beginning shutdown.", nodeId); resign(leader.epoch()); for (MetaLogListenerData listenerData : listeners.values()) { @@ -679,14 +674,6 @@ public CompletableFuture shutdown(int timeoutMs) { return shutdownFuture; } - @Override - public void initialize() { - eventQueue.append(() -> { - log.debug("initialized local log manager for node " + nodeId); - initialized = true; - }); - } - @Override public void register(RaftClient.Listener listener) { CompletableFuture future = new CompletableFuture<>(); @@ -695,7 +682,7 @@ public void register(RaftClient.Listener listener) { log.info("Node {}: can't register because local log manager has " + "already been shut down.", nodeId); future.complete(null); - } else if (initialized) { + } else { int id = System.identityHashCode(listener); if (listeners.putIfAbsent(listener, new MetaLogListenerData(listener)) != null) { log.error("Node {}: can't register because listener {} already exists", nodeId, id); @@ -705,11 +692,6 @@ public void register(RaftClient.Listener listener) { shared.electLeaderIfNeeded(); scheduleLogCheck(); future.complete(null); - } else { - log.info("Node {}: can't register because local log manager has not " + - "been initialized.", nodeId); - future.completeExceptionally(new RuntimeException( - "LocalLogManager was not initialized.")); } }); try { diff --git a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTestEnv.java b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTestEnv.java index c3a7283ab9ed..5dbf7dcad7b5 100644 --- a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTestEnv.java +++ b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTestEnv.java @@ -129,9 +129,6 @@ private LocalLogManagerTestEnv( shared, String.format("LocalLogManager-%d_", nodeId))); } - for (LocalLogManager logManager : newLogManagers) { - logManager.initialize(); - } } catch (Throwable t) { for (LocalLogManager logManager : newLogManagers) { logManager.close(); diff --git a/raft/src/main/java/org/apache/kafka/raft/RaftConfig.java b/raft/src/main/java/org/apache/kafka/raft/RaftConfig.java index 0ab12de40c96..312dc9e43888 100644 --- a/raft/src/main/java/org/apache/kafka/raft/RaftConfig.java +++ b/raft/src/main/java/org/apache/kafka/raft/RaftConfig.java @@ -51,7 +51,7 @@ public class RaftConfig { private static final String QUORUM_PREFIX = "controller.quorum."; // Non-routable address represents an endpoint that does not resolve to any particular node - public static final InetSocketAddress NON_ROUTABLE_ADDRESS = new InetSocketAddress("0.0.0.0", 0); + public static final String NON_ROUTABLE_HOST = "0.0.0.0"; public static final UnknownAddressSpec UNKNOWN_ADDRESS_SPEC_INSTANCE = new UnknownAddressSpec(); public static final String QUORUM_VOTERS_CONFIG = QUORUM_PREFIX + "voters"; @@ -105,7 +105,7 @@ public static class InetAddressSpec implements AddressSpec { public final InetSocketAddress address; public InetAddressSpec(InetSocketAddress address) { - if (address == null || address.equals(NON_ROUTABLE_ADDRESS)) { + if (address == null || address.getHostString().equals(NON_ROUTABLE_HOST)) { throw new IllegalArgumentException("Invalid address: " + address); } this.address = address; @@ -196,7 +196,25 @@ private static Integer parseVoterId(String idString) { } public static Map parseVoterConnections(List voterEntries) { - Map voterMap = new HashMap<>(); + return parseVoterConnections(voterEntries, false) + .entrySet() + .stream() + .collect( + Collectors.toMap( + Map.Entry::getKey, + entry -> { + if (entry.getValue().getHostString().equals(NON_ROUTABLE_HOST)) { + return UNKNOWN_ADDRESS_SPEC_INSTANCE; + } else { + return new InetAddressSpec(entry.getValue()); + } + } + ) + ); + } + + public static Map parseVoterConnections(List voterEntries, boolean routableOnly) { + Map voterMap = new HashMap<>(); for (String voterMapEntry : voterEntries) { String[] idAndAddress = voterMapEntry.split("@"); if (idAndAddress.length != 2) { @@ -220,10 +238,12 @@ public static Map parseVoterConnections(List voter } InetSocketAddress address = new InetSocketAddress(host, port); - if (address.equals(NON_ROUTABLE_ADDRESS)) { - voterMap.put(voterId, UNKNOWN_ADDRESS_SPEC_INSTANCE); + if (address.getHostString().equals(NON_ROUTABLE_HOST) && routableOnly) { + throw new ConfigException( + String.format("Host string ({}) is not routeable", address.getHostString()) + ); } else { - voterMap.put(voterId, new InetAddressSpec(address)); + voterMap.put(voterId, address); } } @@ -231,17 +251,15 @@ public static Map parseVoterConnections(List voter } public static List quorumVoterStringsToNodes(List voters) { - return voterConnectionsToNodes(parseVoterConnections(voters)); + return voterConnectionsToNodes(parseVoterConnections(voters, true)); } - public static List voterConnectionsToNodes(Map voterConnections) { - return voterConnections.entrySet().stream() + public static List voterConnectionsToNodes(Map voterConnections) { + return voterConnections + .entrySet() + .stream() .filter(Objects::nonNull) - .filter(connection -> connection.getValue() instanceof InetAddressSpec) - .map(connection -> { - InetAddressSpec spec = (InetAddressSpec) connection.getValue(); - return new Node(connection.getKey(), spec.address.getHostString(), spec.address.getPort()); - }) + .map(entry -> new Node(entry.getKey(), entry.getValue().getHostString(), entry.getValue().getPort())) .collect(Collectors.toList()); } From 24661b11c2cb6151f740484f3028145742a88988 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Fri, 5 Apr 2024 13:48:18 -0700 Subject: [PATCH 10/34] KAFKA-16207; More voter parsing clean ups --- .../main/scala/kafka/server/KafkaConfig.scala | 14 ++++----- .../scala/kafka/server/KafkaRaftServer.scala | 2 +- .../main/scala/kafka/server/KafkaServer.scala | 4 +-- .../scala/kafka/tools/TestRaftServer.scala | 2 +- .../unit/kafka/raft/RaftManagerTest.scala | 2 +- .../unit/kafka/server/KafkaConfigTest.scala | 6 +--- .../org/apache/kafka/raft/RaftConfig.java | 29 +++++++------------ 7 files changed, 23 insertions(+), 36 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index af0827d0e42f..f02ac1941f81 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -1993,14 +1993,14 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val advertisedListenerNames = effectiveAdvertisedListeners.map(_.listenerName).toSet // validate KRaft-related configs - val voterAddressSpecsByNodeId = RaftConfig.parseVoterConnections(quorumVoters) + val voterIds = RaftConfig.parseVoterIds(quorumVoters) def validateNonEmptyQuorumVotersForKRaft(): Unit = { - if (voterAddressSpecsByNodeId.isEmpty) { + if (voterIds.isEmpty) { throw new ConfigException(s"If using ${KafkaConfig.ProcessRolesProp}, ${KafkaConfig.QuorumVotersProp} must contain a parseable set of voters.") } } def validateNonEmptyQuorumVotersForMigration(): Unit = { - if (voterAddressSpecsByNodeId.isEmpty) { + if (voterIds.isEmpty) { throw new ConfigException(s"If using ${KafkaConfig.MigrationEnabledProp}, ${KafkaConfig.QuorumVotersProp} must contain a parseable set of voters.") } } @@ -2013,8 +2013,8 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami s"The advertised.listeners config must not contain KRaft controller listeners from ${KafkaConfig.ControllerListenerNamesProp} when ${KafkaConfig.ProcessRolesProp} contains the broker role because Kafka clients that send requests via advertised listeners do not send requests to KRaft controllers -- they only send requests to KRaft brokers.") } def validateControllerQuorumVotersMustContainNodeIdForKRaftController(): Unit = { - require(voterAddressSpecsByNodeId.containsKey(nodeId), - s"If ${KafkaConfig.ProcessRolesProp} contains the 'controller' role, the node id $nodeId must be included in the set of voters ${KafkaConfig.QuorumVotersProp}=${voterAddressSpecsByNodeId.asScala.keySet.toSet}") + require(voterIds.contains(nodeId), + s"If ${KafkaConfig.ProcessRolesProp} contains the 'controller' role, the node id $nodeId must be included in the set of voters ${KafkaConfig.QuorumVotersProp}=${voterIds.asScala.toSet}") } def validateControllerListenerExistsForKRaftController(): Unit = { require(controllerListeners.nonEmpty, @@ -2036,8 +2036,8 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami validateControlPlaneListenerEmptyForKRaft() validateAdvertisedListenersDoesNotContainControllerListenersForKRaftBroker() // nodeId must not appear in controller.quorum.voters - require(!voterAddressSpecsByNodeId.containsKey(nodeId), - s"If ${KafkaConfig.ProcessRolesProp} contains just the 'broker' role, the node id $nodeId must not be included in the set of voters ${KafkaConfig.QuorumVotersProp}=${voterAddressSpecsByNodeId.asScala.keySet.toSet}") + require(!voterIds.contains(nodeId), + s"If ${KafkaConfig.ProcessRolesProp} contains just the 'broker' role, the node id $nodeId must not be included in the set of voters ${KafkaConfig.QuorumVotersProp}=${voterIds.asScala.toSet}") // controller.listener.names must be non-empty... require(controllerListenerNames.nonEmpty, s"${KafkaConfig.ControllerListenerNamesProp} must contain at least one value when running KRaft with just the broker role") diff --git a/core/src/main/scala/kafka/server/KafkaRaftServer.scala b/core/src/main/scala/kafka/server/KafkaRaftServer.scala index 4faf137a76d8..fa61ec71baff 100644 --- a/core/src/main/scala/kafka/server/KafkaRaftServer.scala +++ b/core/src/main/scala/kafka/server/KafkaRaftServer.scala @@ -70,7 +70,7 @@ class KafkaRaftServer( metaPropsEnsemble, time, metrics, - CompletableFuture.completedFuture(RaftConfig.parseVoterConnections(config.quorumVoters, true)), + CompletableFuture.completedFuture(RaftConfig.parseVoterConnections(config.quorumVoters)), new StandardFaultHandlerFactory(), ) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index e9103cda0359..0e545ea9633c 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -322,7 +322,7 @@ class KafkaServer( if (config.migrationEnabled) { kraftControllerNodes = RaftConfig.voterConnectionsToNodes( - RaftConfig.parseVoterConnections(config.quorumVoters, true) + RaftConfig.parseVoterConnections(config.quorumVoters) ).asScala } else { kraftControllerNodes = Seq.empty @@ -422,7 +422,7 @@ class KafkaServer( logManager.directoryIdsSet) // If the ZK broker is in migration mode, start up a RaftManager to learn about the new KRaft controller - val quorumVoters = RaftConfig.parseVoterConnections(config.quorumVoters, true) + val quorumVoters = RaftConfig.parseVoterConnections(config.quorumVoters) raftManager = new KafkaRaftManager[ApiMessageAndVersion]( metaPropsEnsemble.clusterId().get(), config, diff --git a/core/src/main/scala/kafka/tools/TestRaftServer.scala b/core/src/main/scala/kafka/tools/TestRaftServer.scala index eef6248a3aaa..092ec67b2199 100644 --- a/core/src/main/scala/kafka/tools/TestRaftServer.scala +++ b/core/src/main/scala/kafka/tools/TestRaftServer.scala @@ -91,7 +91,7 @@ class TestRaftServer( time, metrics, Some(threadNamePrefix), - CompletableFuture.completedFuture(RaftConfig.parseVoterConnections(config.quorumVoters, true)), + CompletableFuture.completedFuture(RaftConfig.parseVoterConnections(config.quorumVoters)), new ProcessTerminatingFaultHandler.Builder().build() ) diff --git a/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala b/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala index b711df6f0acb..e716bb452d91 100644 --- a/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala +++ b/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala @@ -88,7 +88,7 @@ class RaftManagerTest { Time.SYSTEM, new Metrics(Time.SYSTEM), Option.empty, - CompletableFuture.completedFuture(RaftConfig.parseVoterConnections(config.quorumVoters, true)), + CompletableFuture.completedFuture(RaftConfig.parseVoterConnections(config.quorumVoters)), mock(classOf[FaultHandler]) ) } diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 46be71dd2f9b..0dd4262b34d2 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -1343,10 +1343,6 @@ class KafkaConfigTest { expected.put(1, new InetSocketAddress("127.0.0.1", 9092)) assertValidQuorumVoters("1@127.0.0.1:9092", expected) - expected.clear() - expected.put(1, new InetSocketAddress("0.0.0.0", 0)) - assertValidQuorumVoters("1@0.0.0.0:0", expected) - expected.clear() expected.put(1, new InetSocketAddress("kafka1", 9092)) expected.put(2, new InetSocketAddress("kafka2", 9092)) @@ -1357,7 +1353,7 @@ class KafkaConfigTest { private def assertValidQuorumVoters(value: String, expectedVoters: util.Map[Integer, InetSocketAddress]): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect) props.setProperty(KafkaConfig.QuorumVotersProp, value) - val addresses = RaftConfig.parseVoterConnections(KafkaConfig.fromProps(props).quorumVoters, false) + val addresses = RaftConfig.parseVoterConnections(KafkaConfig.fromProps(props).quorumVoters) assertEquals(expectedVoters, addresses) } diff --git a/raft/src/main/java/org/apache/kafka/raft/RaftConfig.java b/raft/src/main/java/org/apache/kafka/raft/RaftConfig.java index 312dc9e43888..1dd913be0dc7 100644 --- a/raft/src/main/java/org/apache/kafka/raft/RaftConfig.java +++ b/raft/src/main/java/org/apache/kafka/raft/RaftConfig.java @@ -29,6 +29,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Set; import java.util.stream.Collectors; /** @@ -195,25 +196,15 @@ private static Integer parseVoterId(String idString) { } } - public static Map parseVoterConnections(List voterEntries) { - return parseVoterConnections(voterEntries, false) - .entrySet() - .stream() - .collect( - Collectors.toMap( - Map.Entry::getKey, - entry -> { - if (entry.getValue().getHostString().equals(NON_ROUTABLE_HOST)) { - return UNKNOWN_ADDRESS_SPEC_INSTANCE; - } else { - return new InetAddressSpec(entry.getValue()); - } - } - ) - ); + public static Map parseVoterConnections(List voterEntries) { + return parseVoterConnections(voterEntries, true); + } + + public static Set parseVoterIds(List voterEntries) { + return parseVoterConnections(voterEntries, false).keySet(); } - public static Map parseVoterConnections(List voterEntries, boolean routableOnly) { + private static Map parseVoterConnections(List voterEntries, boolean routableOnly) { Map voterMap = new HashMap<>(); for (String voterMapEntry : voterEntries) { String[] idAndAddress = voterMapEntry.split("@"); @@ -251,7 +242,7 @@ public static Map parseVoterConnections(List } public static List quorumVoterStringsToNodes(List voters) { - return voterConnectionsToNodes(parseVoterConnections(voters, true)); + return voterConnectionsToNodes(parseVoterConnections(voters)); } public static List voterConnectionsToNodes(Map voterConnections) { @@ -274,7 +265,7 @@ public void ensureValid(String name, Object value) { List voterStrings = (List) value; // Attempt to parse the connect strings - parseVoterConnections(voterStrings); + parseVoterConnections(voterStrings, false); } @Override From cd3277cf02ec405f896350df85eaeb6361ff61c3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Fri, 5 Apr 2024 15:26:43 -0700 Subject: [PATCH 11/34] KAFKA-16207; User the voter set from the config or log --- .../main/scala/kafka/raft/RaftManager.scala | 6 ++- .../apache/kafka/raft/KafkaRaftClient.java | 23 +++++---- .../org/apache/kafka/raft/RaftConfig.java | 47 +------------------ .../raft/internals/InternalLogListener.java | 7 ++- .../apache/kafka/raft/internals/VoterSet.java | 9 +++- .../kafka/raft/RaftClientTestContext.java | 1 + .../kafka/raft/RaftEventSimulationTest.java | 1 + 7 files changed, 36 insertions(+), 58 deletions(-) diff --git a/core/src/main/scala/kafka/raft/RaftManager.scala b/core/src/main/scala/kafka/raft/RaftManager.scala index d03c0d6981e8..70165ab5d5ab 100644 --- a/core/src/main/scala/kafka/raft/RaftManager.scala +++ b/core/src/main/scala/kafka/raft/RaftManager.scala @@ -140,6 +140,7 @@ class KafkaRaftManager[T]( client.initialize( OptionalInt.of(config.nodeId), controllerQuorumVotersFuture.get(), + config.controllerListenerNames.head, new FileBasedStateStore(new File(dataDir, "quorum-state")), metrics ) @@ -208,7 +209,10 @@ class KafkaRaftManager[T]( private def buildNetworkClient(): NetworkClient = { val controllerListenerName = new ListenerName(config.controllerListenerNames.head) - val controllerSecurityProtocol = config.effectiveListenerSecurityProtocolMap.getOrElse(controllerListenerName, SecurityProtocol.forName(controllerListenerName.value())) + val controllerSecurityProtocol = config.effectiveListenerSecurityProtocolMap.getOrElse( + controllerListenerName, + SecurityProtocol.forName(controllerListenerName.value()) + ) val channelBuilder = ChannelBuilders.clientChannelBuilder( controllerSecurityProtocol, JaasContext.Type.SERVER, 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 3973d76f4034..d7ad7cb87e37 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -352,20 +352,23 @@ private void maybeFireLeaderChange() { public void initialize( OptionalInt nodeId, Map voterAddresses, + String listenerName, QuorumStateStore quorumStateStore, Metrics metrics ) { - // TODO: create the internal log listener internalListener = new InternalLogListener( - Optional.of(VoterSet.fromAddressSpecs("TODO", voterAddresses)), + Optional.of(VoterSet.fromAddressSpecs(listenerName, voterAddresses)), log, serde, BufferSupplier.create() ); + // Read the entire log + logger.info("Reading KRaft snapshot and log as part of the initialization"); + internalListener.updateListener(); // TODO: Fix this to use internal listener requestManager = new RequestManager( - voterAddresses.keySet(), + internalListener.lastVoterSet().voterIds(), raftConfig.retryBackoffMs(), raftConfig.requestTimeoutMs(), random @@ -374,22 +377,26 @@ public void initialize( // TODO: Fix this to use internal listener quorum = new QuorumState( nodeId, - voterAddresses.keySet(), + internalListener.lastVoterSet().voterIds(), raftConfig.electionTimeoutMs(), raftConfig.fetchTimeoutMs(), quorumStateStore, time, logContext, - random); + random + ); kafkaRaftMetrics = new KafkaRaftMetrics(metrics, "raft", quorum); // All Raft voters are statically configured and known at startup // so there are no unknown voter connections. Report this metric as 0. kafkaRaftMetrics.updateNumUnknownVoterConnections(0); - // Update the voter endpoints with what's in RaftConfig - // TODO: Create issue to fix this. Eventually, this needs to be dynamic based on the internal listener - voterAddresses.entrySet().forEach(entry -> channel.updateEndpoint(entry.getKey(), entry.getValue())); + // TODO: Create issue to fix this. RaftRequest.Outbound should contain the node not just the id + // we need this because the controller.quorum.bootstrap.server doesn't contain an id. + VoterSet lastVoterSet = internalListener.lastVoterSet(); + for (Integer voterId : lastVoterSet.voterIds()) { + channel.updateEndpoint(voterId, lastVoterSet.voterAddress(voterId, listenerName).get()); + } quorum.initialize(new OffsetAndEpoch(log.endOffset().offset, log.lastFetchedEpoch())); diff --git a/raft/src/main/java/org/apache/kafka/raft/RaftConfig.java b/raft/src/main/java/org/apache/kafka/raft/RaftConfig.java index 1dd913be0dc7..5a86a4ed0838 100644 --- a/raft/src/main/java/org/apache/kafka/raft/RaftConfig.java +++ b/raft/src/main/java/org/apache/kafka/raft/RaftConfig.java @@ -33,13 +33,7 @@ import java.util.stream.Collectors; /** - * RaftConfig encapsulates configuration specific to the cluster metadata KRaft voter nodes. - * Specifically, this class parses the voter node endpoints into an AddressSpec - * for use with the KafkaRaftClient/KafkaNetworkChannel. - * - * If the voter endpoints are not known at startup, a non-routable address can be provided instead. - * For example: `1@0.0.0.0:0,2@0.0.0.0:0,3@0.0.0.0:0` - * This will assign an {@link UnknownAddressSpec} to the voter entries + * RaftConfig encapsulates configuration specific to the cluster metadata KRaft replicas. * * The default raft timeouts are relatively low compared to some other timeouts such as * request.timeout.ms. This is part of a general design philosophy where we see changing @@ -53,7 +47,6 @@ public class RaftConfig { // Non-routable address represents an endpoint that does not resolve to any particular node public static final String NON_ROUTABLE_HOST = "0.0.0.0"; - public static final UnknownAddressSpec UNKNOWN_ADDRESS_SPEC_INSTANCE = new UnknownAddressSpec(); public static final String QUORUM_VOTERS_CONFIG = QUORUM_PREFIX + "voters"; public static final String QUORUM_VOTERS_DOC = "Map of id/endpoint information for " + @@ -99,44 +92,6 @@ public class RaftConfig { private final int fetchTimeoutMs; private final int appendLingerMs; - public interface AddressSpec { - } - - public static class InetAddressSpec implements AddressSpec { - public final InetSocketAddress address; - - public InetAddressSpec(InetSocketAddress address) { - if (address == null || address.getHostString().equals(NON_ROUTABLE_HOST)) { - throw new IllegalArgumentException("Invalid address: " + address); - } - this.address = address; - } - - @Override - public int hashCode() { - return address.hashCode(); - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - - if (obj == null || getClass() != obj.getClass()) { - return false; - } - - final InetAddressSpec that = (InetAddressSpec) obj; - return that.address.equals(address); - } - } - - public static class UnknownAddressSpec implements AddressSpec { - private UnknownAddressSpec() { - } - } - public RaftConfig(AbstractConfig abstractConfig) { this( abstractConfig.getInt(QUORUM_REQUEST_TIMEOUT_MS_CONFIG), diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java b/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java index 28b58883541b..fcb86d307389 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java @@ -57,11 +57,15 @@ public InternalLogListener( this.bufferSupplier = bufferSupplier; } - void updateListerner() { + public void updateListener() { maybeLoadSnapshot(); maybeLoadLog(); } + public VoterSet lastVoterSet() { + return voterSetHistory.lastValue(); + } + private void maybeLoadLog() { while (log.endOffset().offset > nextOffset) { LogFetchInfo info = log.read(nextOffset, Isolation.UNCOMMITTED); @@ -98,6 +102,7 @@ private void maybeLoadSnapshot() { true // Validate batch CRC ) ) { + // TODO: log a message that we are loading a snapshot OptionalLong currentOffset = OptionalLong.of(reader.lastContainedLogOffset()); while (reader.hasNext()) { Batch batch = reader.next(); diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java index 515dd48385a4..24191c4b129e 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java @@ -19,11 +19,12 @@ import java.net.InetSocketAddress; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.Map; import java.util.Objects; import java.util.Optional; -import java.util.HashSet; +import java.util.Set; import java.util.stream.Collectors; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.message.VotersRecord; @@ -38,11 +39,15 @@ final public class VoterSet { this.voters = voters; } - Optional voterAddress(int voter, String listener) { + public Optional voterAddress(int voter, String listener) { return Optional.ofNullable(voters.get(voter)) .flatMap(voterNode -> voterNode.address(listener)); } + public Set voterIds() { + return voters.keySet(); + } + VotersRecord toVotersRecord(short version) { return new VotersRecord() .setVersion(version) 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 7d011b42ba1e..ae9bca2b4381 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -269,6 +269,7 @@ public RaftClientTestContext build() throws IOException { client.initialize( localId, voterAddressMap, + "CONTROLLER", quorumStateStore, metrics ); 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 3be305bb36f9..85bd055f006a 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java @@ -812,6 +812,7 @@ void initialize(Map voterAddresses, Metrics metrics) client.initialize( OptionalInt.of(nodeId), voterAddresses, + "CONTROLLER", store, metrics ); From f3a0fcd4953d0aa37d510f7687170f652a8421a9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Sat, 6 Apr 2024 19:27:24 -0700 Subject: [PATCH 12/34] KAFKA-16207; Return node id before calling initialize --- .../main/scala/kafka/raft/RaftManager.scala | 2 +- .../apache/kafka/raft/KafkaRaftClient.java | 20 +++++++++++++------ .../kafka/raft/RaftClientTestContext.java | 2 +- .../kafka/raft/RaftEventSimulationTest.java | 2 +- 4 files changed, 17 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/kafka/raft/RaftManager.scala b/core/src/main/scala/kafka/raft/RaftManager.scala index 70165ab5d5ab..cb896ffba27d 100644 --- a/core/src/main/scala/kafka/raft/RaftManager.scala +++ b/core/src/main/scala/kafka/raft/RaftManager.scala @@ -138,7 +138,6 @@ class KafkaRaftManager[T]( def startup(): Unit = { client.initialize( - OptionalInt.of(config.nodeId), controllerQuorumVotersFuture.get(), config.controllerListenerNames.head, new FileBasedStateStore(new File(dataDir, "quorum-state")), @@ -174,6 +173,7 @@ class KafkaRaftManager[T]( private def buildRaftClient(): KafkaRaftClient[T] = { val client = new KafkaRaftClient( + OptionalInt.of(config.nodeId), recordSerde, netChannel, replicatedLog, 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 d7ad7cb87e37..822ae2e6236d 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -148,6 +148,7 @@ final public class KafkaRaftClient implements RaftClient { public static final int MAX_BATCH_SIZE_BYTES = 8 * 1024 * 1024; public static final int MAX_FETCH_SIZE_BYTES = MAX_BATCH_SIZE_BYTES; + private final OptionalInt nodeId; private final AtomicReference shutdown = new AtomicReference<>(); private final LogContext logContext; private final Logger logger; @@ -181,6 +182,7 @@ final public class KafkaRaftClient implements RaftClient { * non-participating observer. */ public KafkaRaftClient( + OptionalInt nodeId, RecordSerde serde, NetworkChannel channel, ReplicatedLog log, @@ -190,7 +192,9 @@ public KafkaRaftClient( String clusterId, RaftConfig raftConfig ) { - this(serde, + this( + nodeId, + serde, channel, new BlockingMessageQueue(), log, @@ -201,10 +205,12 @@ public KafkaRaftClient( clusterId, logContext, new Random(), - raftConfig); + raftConfig + ); } KafkaRaftClient( + OptionalInt nodeId, RecordSerde serde, NetworkChannel channel, RaftMessageQueue messageQueue, @@ -218,6 +224,7 @@ public KafkaRaftClient( Random random, RaftConfig raftConfig ) { + this.nodeId = nodeId; this.logContext = logContext; this.serde = serde; this.channel = channel; @@ -350,7 +357,6 @@ private void maybeFireLeaderChange() { // TODO: need a way to verify that initialize has been called. public void initialize( - OptionalInt nodeId, Map voterAddresses, String listenerName, QuorumStateStore quorumStateStore, @@ -442,7 +448,7 @@ public LeaderAndEpoch leaderAndEpoch() { @Override public OptionalInt nodeId() { - return quorum.localId(); + return nodeId; } private OffsetAndEpoch endOffset() { @@ -1230,7 +1236,7 @@ private void appendAsFollower( log.flush(false); } - // TODO: Update the internal listener + internalListener.updateListener(); OffsetAndEpoch endOffset = endOffset(); kafkaRaftMetrics.updateFetchedRecords(info.lastOffset - info.firstOffset + 1); @@ -1243,7 +1249,7 @@ private LogAppendInfo appendAsLeader( ) { LogAppendInfo info = log.appendAsLeader(records, quorum.epoch()); - // TODO: Notify the internal listener + internalListener.updateListener(); OffsetAndEpoch endOffset = endOffset(); kafkaRaftMetrics.updateAppendRecords(info.lastOffset - info.firstOffset + 1); @@ -1505,6 +1511,8 @@ private boolean handleFetchSnapshotResponse( quorum.leaderIdOrSentinel() ); + internalListener.updateListener(); + updateFollowerHighWatermark(state, OptionalLong.of(log.highWatermark().offset)); } else { throw new IllegalStateException( 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 ae9bca2b4381..a246c4ae0d1b 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -251,6 +251,7 @@ public RaftClientTestContext build() throws IOException { ); KafkaRaftClient client = new KafkaRaftClient<>( + localId, SERDE, channel, messageQueue, @@ -267,7 +268,6 @@ public RaftClientTestContext build() throws IOException { client.register(listener); client.initialize( - localId, voterAddressMap, "CONTROLLER", quorumStateStore, 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 85bd055f006a..1ccd7dca5025 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java @@ -738,6 +738,7 @@ void start(int nodeId) { MemoryPool memoryPool = new BatchMemoryPool(2, KafkaRaftClient.MAX_BATCH_SIZE_BYTES); KafkaRaftClient client = new KafkaRaftClient<>( + OptionalInt.of(nodeId), serde, channel, messageQueue, @@ -810,7 +811,6 @@ private RaftNode( void initialize(Map voterAddresses, Metrics metrics) { client.register(counter); client.initialize( - OptionalInt.of(nodeId), voterAddresses, "CONTROLLER", store, From 08e1831e589c298acc04354536a8a01758f857a2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Sun, 7 Apr 2024 11:53:16 -0700 Subject: [PATCH 13/34] KAFKA-16207; Add a builder for record snaapshot writer --- .../kafka/tools/DumpLogSegmentsTest.scala | 15 +- .../apache/kafka/metalog/LocalLogManager.java | 25 ++- .../apache/kafka/raft/KafkaRaftClient.java | 20 +-- .../raft/internals/InternalLogListener.java | 2 +- .../kafka/snapshot/RecordsSnapshotWriter.java | 148 ++++++++++-------- .../raft/KafkaRaftClientSnapshotTest.java | 15 +- .../raft/internals/RecordsIteratorTest.java | 17 +- 7 files changed, 117 insertions(+), 125 deletions(-) diff --git a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala index a75041b49e16..456289287ee8 100644 --- a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala +++ b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala @@ -28,7 +28,6 @@ import kafka.tools.DumpLogSegments.TimeIndexDumpErrors import kafka.utils.TestUtils import org.apache.kafka.common.Uuid import org.apache.kafka.common.config.TopicConfig -import org.apache.kafka.common.memory.MemoryPool import org.apache.kafka.common.metadata.{PartitionChangeRecord, RegisterBrokerRecord, TopicRecord} import org.apache.kafka.common.protocol.{ByteBufferAccessor, ObjectSerializationCache} import org.apache.kafka.common.record.{CompressionType, ControlRecordType, EndTransactionMarker, MemoryRecords, RecordVersion, SimpleRecord} @@ -323,15 +322,11 @@ class DumpLogSegmentsTest { val lastContainedLogTimestamp = 10000 TestUtils.resource( - RecordsSnapshotWriter.createWithHeader( - () => metadataLog.createNewSnapshot(new OffsetAndEpoch(0, 0)), - 1024, - MemoryPool.NONE, - new MockTime, - lastContainedLogTimestamp, - CompressionType.NONE, - MetadataRecordSerde.INSTANCE, - ).get() + new RecordsSnapshotWriter.Builder() + .setTime(new MockTime) + .setLastContainedLogTimestamp(lastContainedLogTimestamp) + .setRawSnapshotWriter(metadataLog.createNewSnapshot(new OffsetAndEpoch(0, 0)).get) + .build(MetadataRecordSerde.INSTANCE) ) { snapshotWriter => snapshotWriter.append(metadataRecords.asJava) snapshotWriter.freeze() diff --git a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java index d5b9ed16c642..3b82d5765c97 100644 --- a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java +++ b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java @@ -17,9 +17,7 @@ package org.apache.kafka.metalog; -import org.apache.kafka.common.memory.MemoryPool; import org.apache.kafka.common.protocol.ObjectSerializationCache; -import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.utils.BufferSupplier; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; @@ -817,22 +815,19 @@ public Optional> createSnapshot( OffsetAndEpoch snapshotId, long lastContainedLogTimestamp ) { - return RecordsSnapshotWriter.createWithHeader( - () -> createNewSnapshot(snapshotId), - 1024, - MemoryPool.NONE, - new MockTime(), - lastContainedLogTimestamp, - CompressionType.NONE, - new MetadataRecordSerde() + return Optional.of( + new RecordsSnapshotWriter.Builder() + .setLastContainedLogTimestamp(lastContainedLogTimestamp) + .setTime(new MockTime()) + .setRawSnapshotWriter(createNewSnapshot(snapshotId)) + .build(new MetadataRecordSerde()) ); } - private Optional createNewSnapshot(OffsetAndEpoch snapshotId) { - return Optional.of( - new MockRawSnapshotWriter(snapshotId, buffer -> { - shared.addSnapshot(new MockRawSnapshotReader(snapshotId, buffer)); - }) + private RawSnapshotWriter createNewSnapshot(OffsetAndEpoch snapshotId) { + return new MockRawSnapshotWriter( + snapshotId, + buffer -> shared.addSnapshot(new MockRawSnapshotReader(snapshotId, buffer)) ); } 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 822ae2e6236d..7d1ad879c462 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -2468,17 +2468,17 @@ public void resign(int epoch) { @Override public Optional> createSnapshot( OffsetAndEpoch snapshotId, - long lastContainedLogTime + long lastContainedLogTimestamp ) { - return RecordsSnapshotWriter.createWithHeader( - () -> log.createNewSnapshot(snapshotId), - MAX_BATCH_SIZE_BYTES, - memoryPool, - time, - lastContainedLogTime, - CompressionType.NONE, - serde - ); + return log.createNewSnapshot(snapshotId).map(writer -> + new RecordsSnapshotWriter.Builder() + .setLastContainedLogTimestamp(lastContainedLogTimestamp) + .setTime(time) + .setMaxBatchSize(MAX_BATCH_SIZE_BYTES) + .setMemoryPool(memoryPool) + .setRawSnapshotWriter(writer) + .build(serde) + ); } @Override diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java b/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java index fcb86d307389..63ce637a924b 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java @@ -88,7 +88,7 @@ private void maybeLoadLog() { private void maybeLoadSnapshot() { Optional rawSnapshot = log.latestSnapshot(); - if (rawSnapshot.isPresent() && (nextOffset == 0 || nextOffset > log.startOffset())) { + if (rawSnapshot.isPresent() && (nextOffset == 0 || nextOffset < log.startOffset())) { // Clear the current state kraftVersion.clear(); voterSetHistory.clear(); diff --git a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java index 818a346ac0ae..bbfdc4d6001d 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java @@ -18,39 +18,36 @@ package org.apache.kafka.snapshot; import org.apache.kafka.common.memory.MemoryPool; +import org.apache.kafka.common.message.SnapshotFooterRecord; +import org.apache.kafka.common.message.SnapshotHeaderRecord; import org.apache.kafka.common.record.CompressionType; +import org.apache.kafka.common.record.ControlRecordUtils; import org.apache.kafka.common.utils.Time; import org.apache.kafka.raft.OffsetAndEpoch; -import org.apache.kafka.server.common.serialization.RecordSerde; -import org.apache.kafka.raft.internals.BatchAccumulator; import org.apache.kafka.raft.internals.BatchAccumulator.CompletedBatch; -import org.apache.kafka.common.message.SnapshotHeaderRecord; -import org.apache.kafka.common.message.SnapshotFooterRecord; -import org.apache.kafka.common.record.ControlRecordUtils; +import org.apache.kafka.raft.internals.BatchAccumulator; +//import org.apache.kafka.raft.internals.VoterSet; +import org.apache.kafka.server.common.serialization.RecordSerde; import java.util.Optional; import java.util.List; import java.util.OptionalLong; -import java.util.function.Supplier; final public class RecordsSnapshotWriter implements SnapshotWriter { final private RawSnapshotWriter snapshot; final private BatchAccumulator accumulator; final private Time time; - final private long lastContainedLogTimestamp; private RecordsSnapshotWriter( RawSnapshotWriter snapshot, int maxBatchSize, MemoryPool memoryPool, Time time, - long lastContainedLogTimestamp, CompressionType compressionType, RecordSerde serde ) { this.snapshot = snapshot; this.time = time; - this.lastContainedLogTimestamp = lastContainedLogTimestamp; this.accumulator = new BatchAccumulator<>( snapshot.snapshotId().epoch(), @@ -69,7 +66,7 @@ private RecordsSnapshotWriter( * * @throws IllegalStateException if the snapshot is not empty */ - private void initializeSnapshotWithHeader() { + private void initializeSnapshotWithHeader(long lastContainedLogTimestamp) { if (snapshot.sizeInBytes() != 0) { String message = String.format( "Initializing writer with a non-empty snapshot: id = '%s'.", @@ -97,63 +94,6 @@ private void finalizeSnapshotWithFooter() { accumulator.forceDrain(); } - /** - * Create an instance of this class and initialize - * the underlying snapshot with {@link SnapshotHeaderRecord} - * - * @param supplier a lambda to create the low level snapshot writer - * @param maxBatchSize the maximum size in byte for a batch - * @param memoryPool the memory pool for buffer allocation - * @param snapshotTime the clock implementation - * @param lastContainedLogTimestamp The append time of the highest record contained in this snapshot - * @param compressionType the compression algorithm to use - * @param serde the record serialization and deserialization implementation - * @return {@link Optional}{@link RecordsSnapshotWriter} - */ - public static Optional> createWithHeader( - Supplier> supplier, - int maxBatchSize, - MemoryPool memoryPool, - Time snapshotTime, - long lastContainedLogTimestamp, - CompressionType compressionType, - RecordSerde serde - ) { - return supplier.get().map(writer -> - createWithHeader( - writer, - maxBatchSize, - memoryPool, - snapshotTime, - lastContainedLogTimestamp, - compressionType, - serde - ) - ); - } - - public static RecordsSnapshotWriter createWithHeader( - RawSnapshotWriter rawSnapshotWriter, - int maxBatchSize, - MemoryPool memoryPool, - Time snapshotTime, - long lastContainedLogTimestamp, - CompressionType compressionType, - RecordSerde serde - ) { - RecordsSnapshotWriter writer = new RecordsSnapshotWriter<>( - rawSnapshotWriter, - maxBatchSize, - memoryPool, - snapshotTime, - lastContainedLogTimestamp, - compressionType, - serde - ); - writer.initializeSnapshotWithHeader(); - return writer; - } - @Override public OffsetAndEpoch snapshotId() { return snapshot.snapshotId(); @@ -216,4 +156,78 @@ private void appendBatches(List> batches) { batches.forEach(CompletedBatch::release); } } + + final public static class Builder { + private long lastContainedLogTimestamp = 0; + private CompressionType compressionType = CompressionType.NONE; + private Time time = Time.SYSTEM; + private int maxBatchSize = 1024; + MemoryPool memoryPool = MemoryPool.NONE; + // TODO: implement this... + //private short kraftVersion = 0; + //private Optional voterSet = Optional.empty(); + private Optional rawSnapshotWriter = Optional.empty(); + + public Builder setLastContainedLogTimestamp(long lastContainedLogTimestamp) { + this.lastContainedLogTimestamp = lastContainedLogTimestamp; + return this; + } + + public Builder setCompressionType(CompressionType compressionType) { + this.compressionType = compressionType; + return this; + } + + public Builder setTime(Time time) { + this.time = time; + return this; + } + + public Builder setMaxBatchSize(int maxBatchSize) { + this.maxBatchSize = maxBatchSize; + return this; + } + + public Builder setMemoryPool(MemoryPool memoryPool) { + this.memoryPool = memoryPool; + return this; + } + + public Builder setRawSnapshotWriter(RawSnapshotWriter rawSnapshotWriter) { + this.rawSnapshotWriter = Optional.of(rawSnapshotWriter); + return this; + } + + /* + public Builder setKraftVersion(short kraftVersion) { + this.kraftVersion = kraftVersion; + return this; + } + + public Builder setVoterSet(VoterSet voterSet) { + this.voterSet = Optional.of(voterSet); + return this; + } + */ + + public RecordsSnapshotWriter build(RecordSerde serde) { + if (!rawSnapshotWriter.isPresent()) { + throw new IllegalStateException("Builder::build called without a RawSnapshotWriter"); + } + + RecordsSnapshotWriter writer = new RecordsSnapshotWriter<>( + rawSnapshotWriter.get(), + maxBatchSize, + memoryPool, + time, + compressionType, + serde + ); + + // TODO: implement writing kraft version and voter set to the snaopshot + writer.initializeSnapshotWithHeader(lastContainedLogTimestamp); + + return writer; + } + } } 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 07d8b5a0d6e6..8a537c64f5c3 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java @@ -18,12 +18,10 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; -import org.apache.kafka.common.memory.MemoryPool; import org.apache.kafka.common.message.FetchResponseData; import org.apache.kafka.common.message.FetchSnapshotRequestData; import org.apache.kafka.common.message.FetchSnapshotResponseData; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.UnalignedMemoryRecords; import org.apache.kafka.common.requests.FetchSnapshotRequest; @@ -1950,15 +1948,10 @@ private static Optional assertFetchS } private static SnapshotWriter snapshotWriter(RaftClientTestContext context, RawSnapshotWriter snapshot) { - return RecordsSnapshotWriter.createWithHeader( - snapshot, - 4 * 1024, - MemoryPool.NONE, - context.time, - 0, - CompressionType.NONE, - new StringSerde() - ); + return new RecordsSnapshotWriter.Builder() + .setTime(context.time) + .setRawSnapshotWriter(snapshot) + .build(new StringSerde()); } private final static class MemorySnapshotWriter implements RawSnapshotWriter { 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 1f1d6470e5b4..a07b65162e81 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 @@ -33,7 +33,6 @@ import net.jqwik.api.ForAll; import net.jqwik.api.Property; import org.apache.kafka.common.errors.CorruptRecordException; -import org.apache.kafka.common.memory.MemoryPool; import org.apache.kafka.common.message.KRaftVersionRecord; import org.apache.kafka.common.message.LeaderChangeMessage; import org.apache.kafka.common.message.SnapshotFooterRecord; @@ -154,16 +153,12 @@ public void testCrcValidation( @Test public void testControlRecordIteration() { AtomicReference buffer = new AtomicReference<>(null); - try (RecordsSnapshotWriter snapshot = RecordsSnapshotWriter.createWithHeader( - new MockRawSnapshotWriter(new OffsetAndEpoch(100, 10), snapshotBuf -> buffer.set(snapshotBuf)), - 4 * 1024, - MemoryPool.NONE, - new MockTime(), - 0, - CompressionType.NONE, - STRING_SERDE - ) - ) { + RecordsSnapshotWriter.Builder builder = new RecordsSnapshotWriter.Builder() + .setTime(new MockTime()) + .setRawSnapshotWriter( + new MockRawSnapshotWriter(new OffsetAndEpoch(100, 10), snapshotBuf -> buffer.set(snapshotBuf)) + ); + try (RecordsSnapshotWriter snapshot = builder.build(STRING_SERDE)) { snapshot.append(Arrays.asList("a", "b", "c")); snapshot.append(Arrays.asList("d", "e", "f")); snapshot.append(Arrays.asList("g", "h", "i")); From 89ac6987dd633d90345120905431ef5130b7a520 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Sun, 7 Apr 2024 15:52:10 -0700 Subject: [PATCH 14/34] KAFKA-16207; Include kraft.version and voter set in the snapshots --- .../apache/kafka/raft/KafkaRaftClient.java | 18 ++- .../raft/internals/BatchAccumulator.java | 84 +++++++++----- .../raft/internals/InternalLogListener.java | 64 ++++++++++- .../apache/kafka/raft/internals/VoterSet.java | 2 +- .../kafka/snapshot/RecordsSnapshotWriter.java | 105 ++++++++++++------ 5 files changed, 204 insertions(+), 69 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 7d1ad879c462..a1206d012164 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -170,6 +170,8 @@ final public class KafkaRaftClient implements RaftClient { private final ConcurrentLinkedQueue> pendingRegistrations = new ConcurrentLinkedQueue<>(); // Components that needs to be initialized because they depend on the voter set + // TODO: figure out how to enforce the requirement that the internal log listener must be more up to date then + // the external raft listener. This is a requirement for a consistent snapshot. private volatile InternalLogListener internalListener; private volatile KafkaRaftMetrics kafkaRaftMetrics; private volatile QuorumState quorum; @@ -2470,15 +2472,23 @@ public Optional> createSnapshot( OffsetAndEpoch snapshotId, long lastContainedLogTimestamp ) { - return log.createNewSnapshot(snapshotId).map(writer -> - new RecordsSnapshotWriter.Builder() + if (!isInitialized()) { + throw new IllegalStateException("Cannot create snapshot before the kraft client has been initialized"); + } + + return log.createNewSnapshot(snapshotId).map(writer -> { + long lastContainedLogOffset = snapshotId.offset() - 1; + + return new RecordsSnapshotWriter.Builder() .setLastContainedLogTimestamp(lastContainedLogTimestamp) .setTime(time) .setMaxBatchSize(MAX_BATCH_SIZE_BYTES) .setMemoryPool(memoryPool) .setRawSnapshotWriter(writer) - .build(serde) - ); + .setKraftVersion(internalListener.kraftVersionAtOffset(lastContainedLogOffset)) + .setVoterSet(internalListener.voterSetAtOffset(lastContainedLogOffset)) + .build(serde); + }); } @Override diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java b/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java index febe0969afbb..74db8fc99a32 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java @@ -213,20 +213,21 @@ private void completeCurrentBatch() { * * @param valueCreator a function that uses the passed buffer to create the control * batch that will be appended. The memory records returned must contain one - * control batch and that control batch have one record. + * control batch and that control batch have at least one record. */ - private void appendControlMessages(Function valueCreator) { + public void appendControlMessages(Function valueCreator) { appendLock.lock(); try { ByteBuffer buffer = memoryPool.tryAllocate(256); if (buffer != null) { try { forceDrain(); + CreatedRecords createdRecords = valueCreator.apply(buffer); completed.add( new CompletedBatch<>( nextOffset, - 1, - valueCreator.apply(buffer), + createdRecords.numberOfRecords(), + createdRecords.records(), memoryPool, buffer ) @@ -256,13 +257,18 @@ public void appendLeaderChangeMessage( LeaderChangeMessage leaderChangeMessage, long currentTimestamp ) { - appendControlMessages(buffer -> MemoryRecords.withLeaderChangeMessage( - this.nextOffset, - currentTimestamp, - this.epoch, - buffer, - leaderChangeMessage - )); + appendControlMessages(buffer -> + new CreatedRecords( + 1, + MemoryRecords.withLeaderChangeMessage( + this.nextOffset, + currentTimestamp, + this.epoch, + buffer, + leaderChangeMessage + ) + ) + ); } @@ -277,13 +283,18 @@ public void appendSnapshotHeaderRecord( SnapshotHeaderRecord snapshotHeaderRecord, long currentTimestamp ) { - appendControlMessages(buffer -> MemoryRecords.withSnapshotHeaderRecord( - this.nextOffset, - currentTimestamp, - this.epoch, - buffer, - snapshotHeaderRecord - )); + appendControlMessages(buffer -> + new CreatedRecords( + 1, + MemoryRecords.withSnapshotHeaderRecord( + this.nextOffset, + currentTimestamp, + this.epoch, + buffer, + snapshotHeaderRecord + ) + ) + ); } /** @@ -297,13 +308,18 @@ public void appendSnapshotFooterRecord( SnapshotFooterRecord snapshotFooterRecord, long currentTimestamp ) { - appendControlMessages(buffer -> MemoryRecords.withSnapshotFooterRecord( - this.nextOffset, - currentTimestamp, - this.epoch, - buffer, - snapshotFooterRecord - )); + appendControlMessages(buffer -> + new CreatedRecords( + 1, + MemoryRecords.withSnapshotFooterRecord( + this.nextOffset, + currentTimestamp, + this.epoch, + buffer, + snapshotFooterRecord + ) + ) + ); } public void forceDrain() { @@ -513,6 +529,24 @@ public long appendTimestamp() { } } + final public static class CreatedRecords { + private final int numberOfRecords; + private final MemoryRecords records; + + public CreatedRecords(int numberOfRecords, MemoryRecords records) { + this.numberOfRecords = numberOfRecords; + this.records = records; + } + + public int numberOfRecords() { + return numberOfRecords; + } + + public MemoryRecords records() { + return records; + } + } + private static class SimpleTimer { // We use an atomic long so that the Raft IO thread can query the linger // time without any locking diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java b/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java index 63ce637a924b..1634269615e1 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java @@ -40,10 +40,18 @@ final public class InternalLogListener { private final RecordSerde serde; private final BufferSupplier bufferSupplier; + // These are objects are synchronized using the perspective object monitor. The two actors + // are the KRaft driver and the RaftClient callers private final VoterSetHistory voterSetHistory; private final History kraftVersion = new TreeMapHistory<>(); - private long nextOffset = 0; + // This synchronization is enough because + // 1. The write operation updateListener only sets the value without reading and updates to + // voterSetHistory or kraftVersion are done before setting the nextOffset + // + // 2. The read operations lastVoterSet, voterSetAtOffset and kraftVersionAtOffset read + // the nextOffset first before reading voterSetHistory or kraftVersion + private volatile long nextOffset = 0; public InternalLogListener( Optional staticVoterSet, @@ -63,7 +71,43 @@ public void updateListener() { } public VoterSet lastVoterSet() { - return voterSetHistory.lastValue(); + synchronized (voterSetHistory) { + return voterSetHistory.lastValue(); + } + } + + public Optional voterSetAtOffset(long offset) { + long fixedNextOffset = nextOffset; + if (offset >= fixedNextOffset) { + throw new IllegalArgumentException( + String.format( + "Attempting the read the voter set at an offset (%d) which kraft hasn't seen (%d)", + offset, + fixedNextOffset - 1 + ) + ); + } + + synchronized (voterSetHistory) { + return voterSetHistory.valueAt(offset); + } + } + + public short kraftVersionAtOffset(long offset) { + long fixedNextOffset = nextOffset; + if (offset >= fixedNextOffset) { + throw new IllegalArgumentException( + String.format( + "Attempting the read the kraft.version at an offset (%d) which kraft hasn't seen (%d)", + offset, + fixedNextOffset - 1 + ) + ); + } + + synchronized (kraftVersion) { + return kraftVersion.valueAt(offset).orElse((short) 0); + } } private void maybeLoadLog() { @@ -90,8 +134,12 @@ private void maybeLoadSnapshot() { Optional rawSnapshot = log.latestSnapshot(); if (rawSnapshot.isPresent() && (nextOffset == 0 || nextOffset < log.startOffset())) { // Clear the current state - kraftVersion.clear(); - voterSetHistory.clear(); + synchronized (kraftVersion) { + kraftVersion.clear(); + } + synchronized (voterSetHistory) { + voterSetHistory.clear(); + } // Load the snapshot since the listener is at the start of the log or the log doesn't have the next entry. try (SnapshotReader reader = RecordsSnapshotReader.of( @@ -120,11 +168,15 @@ private void handleBatch(Batch batch, OptionalLong overrideOffset) { long currentOffset = overrideOffset.orElse(batch.baseOffset() + index); switch (record.type()) { case VOTERS: - voterSetHistory.addAt(currentOffset, VoterSet.fromVotersRecord((VotersRecord) record.message())); + synchronized (voterSetHistory) { + voterSetHistory.addAt(currentOffset, VoterSet.fromVotersRecord((VotersRecord) record.message())); + } break; case KRAFT_VERSION: - kraftVersion.addAt(currentOffset, ((KRaftVersionRecord) record.message()).kRaftVersion()); + synchronized (kraftVersion) { + kraftVersion.addAt(currentOffset, ((KRaftVersionRecord) record.message()).kRaftVersion()); + } break; default: diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java index 24191c4b129e..4b2a27c5e3b3 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java @@ -48,7 +48,7 @@ public Set voterIds() { return voters.keySet(); } - VotersRecord toVotersRecord(short version) { + public VotersRecord toVotersRecord(short version) { return new VotersRecord() .setVersion(version) .setVoters( diff --git a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java index bbfdc4d6001d..1061d25f55ab 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java @@ -18,19 +18,24 @@ package org.apache.kafka.snapshot; import org.apache.kafka.common.memory.MemoryPool; +import org.apache.kafka.common.message.KRaftVersionRecord; import org.apache.kafka.common.message.SnapshotFooterRecord; import org.apache.kafka.common.message.SnapshotHeaderRecord; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.ControlRecordUtils; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.MemoryRecordsBuilder; +import org.apache.kafka.common.record.RecordBatch; +import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.utils.Time; import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.raft.internals.BatchAccumulator.CompletedBatch; import org.apache.kafka.raft.internals.BatchAccumulator; -//import org.apache.kafka.raft.internals.VoterSet; +import org.apache.kafka.raft.internals.VoterSet; import org.apache.kafka.server.common.serialization.RecordSerde; -import java.util.Optional; import java.util.List; +import java.util.Optional; import java.util.OptionalLong; final public class RecordsSnapshotWriter implements SnapshotWriter { @@ -61,27 +66,6 @@ private RecordsSnapshotWriter( ); } - /** - * Adds a {@link SnapshotHeaderRecord} to snapshot - * - * @throws IllegalStateException if the snapshot is not empty - */ - private void initializeSnapshotWithHeader(long lastContainedLogTimestamp) { - if (snapshot.sizeInBytes() != 0) { - String message = String.format( - "Initializing writer with a non-empty snapshot: id = '%s'.", - snapshot.snapshotId() - ); - throw new IllegalStateException(message); - } - - SnapshotHeaderRecord headerRecord = new SnapshotHeaderRecord() - .setVersion(ControlRecordUtils.SNAPSHOT_HEADER_CURRENT_VERSION) - .setLastContainedLogTimestamp(lastContainedLogTimestamp); - accumulator.appendSnapshotHeaderRecord(headerRecord, time.milliseconds()); - accumulator.forceDrain(); - } - /** * Adds a {@link SnapshotFooterRecord} to the snapshot * @@ -157,15 +141,15 @@ private void appendBatches(List> batches) { } } + // TODO: Write test for this builder final public static class Builder { private long lastContainedLogTimestamp = 0; private CompressionType compressionType = CompressionType.NONE; private Time time = Time.SYSTEM; private int maxBatchSize = 1024; MemoryPool memoryPool = MemoryPool.NONE; - // TODO: implement this... - //private short kraftVersion = 0; - //private Optional voterSet = Optional.empty(); + private short kraftVersion = 0; + private Optional voterSet = Optional.empty(); private Optional rawSnapshotWriter = Optional.empty(); public Builder setLastContainedLogTimestamp(long lastContainedLogTimestamp) { @@ -194,25 +178,31 @@ public Builder setMemoryPool(MemoryPool memoryPool) { } public Builder setRawSnapshotWriter(RawSnapshotWriter rawSnapshotWriter) { - this.rawSnapshotWriter = Optional.of(rawSnapshotWriter); + this.rawSnapshotWriter = Optional.ofNullable(rawSnapshotWriter); return this; } - /* public Builder setKraftVersion(short kraftVersion) { this.kraftVersion = kraftVersion; return this; } - public Builder setVoterSet(VoterSet voterSet) { - this.voterSet = Optional.of(voterSet); + public Builder setVoterSet(Optional voterSet) { + this.voterSet = voterSet; return this; } - */ public RecordsSnapshotWriter build(RecordSerde serde) { if (!rawSnapshotWriter.isPresent()) { throw new IllegalStateException("Builder::build called without a RawSnapshotWriter"); + } else if (rawSnapshotWriter.get().sizeInBytes() != 0) { + throw new IllegalStateException( + String.format("Initializing writer with a non-empty snapshot: %s", rawSnapshotWriter.get().snapshotId()) + ); + } else if (kraftVersion == 0 && voterSet.isPresent()) { + throw new IllegalStateException( + String.format("Voter set (%s) not expected when the kraft.version is 0", voterSet.get()) + ); } RecordsSnapshotWriter writer = new RecordsSnapshotWriter<>( @@ -224,8 +214,57 @@ public RecordsSnapshotWriter build(RecordSerde serde) { serde ); - // TODO: implement writing kraft version and voter set to the snaopshot - writer.initializeSnapshotWithHeader(lastContainedLogTimestamp); + writer.accumulator.appendControlMessages(buffer -> { + long now = time.milliseconds(); + int numberOfRecords = 0; + try (MemoryRecordsBuilder builder = new MemoryRecordsBuilder( + buffer, + RecordBatch.CURRENT_MAGIC_VALUE, + compressionType, + TimestampType.CREATE_TIME, + 0, // initialOffset + now, + RecordBatch.NO_PRODUCER_ID, + RecordBatch.NO_PRODUCER_EPOCH, + RecordBatch.NO_SEQUENCE, + false, // isTransactional + true, // isControlBatch + rawSnapshotWriter.get().snapshotId().epoch(), + buffer.capacity() + ) + ) { + numberOfRecords++; + builder.appendSnapshotHeaderMessage( + now, + new SnapshotHeaderRecord() + .setVersion(ControlRecordUtils.SNAPSHOT_HEADER_CURRENT_VERSION) + .setLastContainedLogTimestamp(lastContainedLogTimestamp) + ); + + if (kraftVersion > 0) { + numberOfRecords++; + builder.appendKRaftVersionMessage( + now, + new KRaftVersionRecord() + .setVersion(ControlRecordUtils.KRAFT_VERSION_CURRENT_VERSION) + .setKRaftVersion(kraftVersion) + ); + + if (voterSet.isPresent()) { + numberOfRecords++; + builder.appendVotersMessage( + now, + voterSet.get().toVotersRecord(ControlRecordUtils.VOTERS_CURRENT_VERSION) + ); + } + } + } + + buffer.flip(); + return new BatchAccumulator.CreatedRecords(numberOfRecords, MemoryRecords.readableRecords(buffer)); + }); + + writer.accumulator.forceDrain(); return writer; } From 2889b42f46fce62777f4e7abe717afd4bcb2f351 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Mon, 8 Apr 2024 09:46:25 -0700 Subject: [PATCH 15/34] KAFKA-16207; Use SupportedVersionRange --- checkstyle/import-control.xml | 3 +- .../common/feature/BaseVersionRange.java | 5 +- .../common/feature/SupportedVersionRange.java | 2 +- .../feature/SupportedVersionRangeTest.java | 24 +++++----- .../apache/kafka/raft/internals/History.java | 24 +++++++++- .../apache/kafka/raft/internals/VoterSet.java | 47 ++++++++++++++----- .../raft/internals/VoterSetHistoryTest.java | 6 +-- 7 files changed, 78 insertions(+), 33 deletions(-) diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 616e0e519c63..bcd3496447fb 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -452,12 +452,13 @@ + + - diff --git a/clients/src/main/java/org/apache/kafka/common/feature/BaseVersionRange.java b/clients/src/main/java/org/apache/kafka/common/feature/BaseVersionRange.java index 2d6ce702e253..3395a7b83aee 100644 --- a/clients/src/main/java/org/apache/kafka/common/feature/BaseVersionRange.java +++ b/clients/src/main/java/org/apache/kafka/common/feature/BaseVersionRange.java @@ -60,10 +60,10 @@ class BaseVersionRange { * - minKeyLabel is empty, OR, minKeyLabel is empty. */ protected BaseVersionRange(String minKeyLabel, short minValue, String maxKeyLabel, short maxValue) { - if (minValue < 1 || maxValue < 1 || maxValue < minValue) { + if (minValue < 0 || maxValue < 0 || maxValue < minValue) { throw new IllegalArgumentException( String.format( - "Expected minValue >= 1, maxValue >= 1 and maxValue >= minValue, but received" + + "Expected minValue >= 0, maxValue >= 0 and maxValue >= minValue, but received" + " minValue: %d, maxValue: %d", minValue, maxValue)); } if (minKeyLabel.isEmpty()) { @@ -86,6 +86,7 @@ public short max() { return maxValue; } + @Override public String toString() { return String.format( "%s[%s]", diff --git a/clients/src/main/java/org/apache/kafka/common/feature/SupportedVersionRange.java b/clients/src/main/java/org/apache/kafka/common/feature/SupportedVersionRange.java index a864a9176263..b06263549032 100644 --- a/clients/src/main/java/org/apache/kafka/common/feature/SupportedVersionRange.java +++ b/clients/src/main/java/org/apache/kafka/common/feature/SupportedVersionRange.java @@ -33,7 +33,7 @@ public SupportedVersionRange(short minVersion, short maxVersion) { } public SupportedVersionRange(short maxVersion) { - this((short) 1, maxVersion); + this((short) 0, maxVersion); } public static SupportedVersionRange fromMap(Map versionRangeMap) { diff --git a/clients/src/test/java/org/apache/kafka/common/feature/SupportedVersionRangeTest.java b/clients/src/test/java/org/apache/kafka/common/feature/SupportedVersionRangeTest.java index a1d2af419fd2..47f312d9d902 100644 --- a/clients/src/test/java/org/apache/kafka/common/feature/SupportedVersionRangeTest.java +++ b/clients/src/test/java/org/apache/kafka/common/feature/SupportedVersionRangeTest.java @@ -36,18 +36,18 @@ public class SupportedVersionRangeTest { @Test public void testFailDueToInvalidParams() { - // min and max can't be < 1. + // min and max can't be < 0. assertThrows( IllegalArgumentException.class, - () -> new SupportedVersionRange((short) 0, (short) 0)); - // min can't be < 1. + () -> new SupportedVersionRange((short) -1, (short) -1)); + // min can't be < 0. assertThrows( IllegalArgumentException.class, - () -> new SupportedVersionRange((short) 0, (short) 1)); - // max can't be < 1. + () -> new SupportedVersionRange((short) -1, (short) 0)); + // max can't be < 0. assertThrows( IllegalArgumentException.class, - () -> new SupportedVersionRange((short) 1, (short) 0)); + () -> new SupportedVersionRange((short) 0, (short) -1)); // min can't be > max. assertThrows( IllegalArgumentException.class, @@ -73,23 +73,23 @@ public void testFromToMap() { @Test public void testFromMapFailure() { - // min_version can't be < 1. + // min_version can't be < 0. Map invalidWithBadMinVersion = - mkMap(mkEntry("min_version", (short) 0), mkEntry("max_version", (short) 1)); + mkMap(mkEntry("min_version", (short) -1), mkEntry("max_version", (short) 0)); assertThrows( IllegalArgumentException.class, () -> SupportedVersionRange.fromMap(invalidWithBadMinVersion)); - // max_version can't be < 1. + // max_version can't be < 0. Map invalidWithBadMaxVersion = - mkMap(mkEntry("min_version", (short) 1), mkEntry("max_version", (short) 0)); + mkMap(mkEntry("min_version", (short) 0), mkEntry("max_version", (short) -1)); assertThrows( IllegalArgumentException.class, () -> SupportedVersionRange.fromMap(invalidWithBadMaxVersion)); - // min_version and max_version can't be < 1. + // min_version and max_version can't be < 0. Map invalidWithBadMinMaxVersion = - mkMap(mkEntry("min_version", (short) 0), mkEntry("max_version", (short) 0)); + mkMap(mkEntry("min_version", (short) -1), mkEntry("max_version", (short) -1)); assertThrows( IllegalArgumentException.class, () -> SupportedVersionRange.fromMap(invalidWithBadMinMaxVersion)); diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/History.java b/raft/src/main/java/org/apache/kafka/raft/internals/History.java index d2b9cb4654ba..0380d1fcda37 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/History.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/History.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.raft.internals; +import java.util.Objects; import java.util.Optional; // TODO: document this type @@ -49,6 +50,27 @@ public T value() { return value; } - // TODO: override equals, hashCode and toString + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + Entry that = (Entry) o; + + if (offset != that.offset) return false; + if (value != that.value) return false; + + return true; + } + + @Override + public int hashCode() { + return Objects.hash(offset, value); + } + + @Override + public String toString() { + return String.format("Entry(offset=%d, value=%s)", offset, value); + } } } diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java index 4b2a27c5e3b3..86b3dea60064 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java @@ -29,6 +29,7 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.common.message.VotersRecord; import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.common.feature.SupportedVersionRange; // TODO: write unittest for VoterSet // TODO: Write documentation @@ -68,11 +69,15 @@ public VotersRecord toVotersRecord(short version) { ) .iterator(); + VotersRecord.KRaftVersionFeature kraftVersionFeature = new VotersRecord.KRaftVersionFeature() + .setMinSupportedVersion(voter.supportedKRaftVersion().min()) + .setMaxSupportedVersion(voter.supportedKRaftVersion().max()); + return new VotersRecord.Voter() .setVoterId(voter.id()) .setVoterUuid(voter.uuid().orElse(Uuid.ZERO_UUID)) .setEndpoints(new VotersRecord.EndpointCollection(endpoints)) - .setKRaftVersionFeature(voter.feature()); + .setKRaftVersionFeature(kraftVersionFeature); }) .collect(Collectors.toList()) ); @@ -109,19 +114,18 @@ final static class VoterNode { private final int id; private final Optional uuid; private final Map listeners; - // TODO: is there a better type for this? - private final VotersRecord.KRaftVersionFeature feature; + private final SupportedVersionRange supportedKRaftVersion; VoterNode( int id, Optional uuid, Map listeners, - VotersRecord.KRaftVersionFeature feature + SupportedVersionRange supportedKRaftVersion ) { this.id = id; this.uuid = uuid; this.listeners = listeners; - this.feature = feature; + this.supportedKRaftVersion = supportedKRaftVersion; } int id() { @@ -136,15 +140,15 @@ Map listeners() { return listeners; } - VotersRecord.KRaftVersionFeature feature() { - return feature; + SupportedVersionRange supportedKRaftVersion() { + return supportedKRaftVersion; } Optional address(String listener) { return Optional.ofNullable(listeners.get(listener)); } - + @Override public boolean equals(Object o) { if (this == o) return true; @@ -154,7 +158,7 @@ public boolean equals(Object o) { if (id != that.id) return false; if (!Objects.equals(uuid, that.uuid)) return false; - if (!Objects.equals(feature, that.feature)) return false; + if (!Objects.equals(supportedKRaftVersion, that.supportedKRaftVersion)) return false; if (!Objects.equals(listeners, that.listeners)) return false; return true; @@ -162,12 +166,18 @@ public boolean equals(Object o) { @Override public int hashCode() { - return Objects.hash(id, uuid, listeners, feature); + return Objects.hash(id, uuid, listeners, supportedKRaftVersion); } @Override public String toString() { - return String.format("VoterNode(id=%d, uuid=%s, listeners=%s, feature=%s)", id, uuid, listeners, feature); + return String.format( + "VoterNode(id=%d, uuid=%s, listeners=%s, supportedKRaftVersion=%s)", + id, + uuid, + listeners, + supportedKRaftVersion + ); } } @@ -186,7 +196,18 @@ public static VoterSet fromVotersRecord(VotersRecord voters) { listeners.put(endpoint.name(), InetSocketAddress.createUnresolved(endpoint.host(), endpoint.port())); } - voterNodes.put(voter.voterId(), new VoterNode(voter.voterId(), uuid, listeners, voter.kRaftVersionFeature())); + voterNodes.put( + voter.voterId(), + new VoterNode( + voter.voterId(), + uuid, + listeners, + new SupportedVersionRange( + voter.kRaftVersionFeature().minSupportedVersion(), + voter.kRaftVersionFeature().maxSupportedVersion() + ) + ) + ); } return new VoterSet(voterNodes); @@ -203,7 +224,7 @@ public static VoterSet fromAddressSpecs(String listener, Map replicas) { "LISTNER", InetSocketAddress.createUnresolved(String.format("replica-%d", replica), 1234) ), - new VotersRecord.KRaftVersionFeature() + new SupportedVersionRange((short) 0, (short) 0) ) ) ) From 1188b9479d56fdb16ae84cf4a335b9c985f1c9c2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Mon, 8 Apr 2024 09:57:11 -0700 Subject: [PATCH 16/34] KAFKA-16207; Fix version range documentation --- .../apache/kafka/clients/admin/SupportedVersionRange.java | 2 +- .../org/apache/kafka/common/feature/BaseVersionRange.java | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/SupportedVersionRange.java b/clients/src/main/java/org/apache/kafka/clients/admin/SupportedVersionRange.java index b85a392a658c..3ba6303eacd5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/SupportedVersionRange.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/SupportedVersionRange.java @@ -28,7 +28,7 @@ public class SupportedVersionRange { /** * Raises an exception unless the following conditions are met: - * 1 <= minVersion <= maxVersion. + * 0 <= minVersion <= maxVersion. * * @param minVersion The minimum version value. * @param maxVersion The maximum version value. diff --git a/clients/src/main/java/org/apache/kafka/common/feature/BaseVersionRange.java b/clients/src/main/java/org/apache/kafka/common/feature/BaseVersionRange.java index 3395a7b83aee..573e2c47a082 100644 --- a/clients/src/main/java/org/apache/kafka/common/feature/BaseVersionRange.java +++ b/clients/src/main/java/org/apache/kafka/common/feature/BaseVersionRange.java @@ -26,7 +26,7 @@ /** * Represents an immutable basic version range using 2 attributes: min and max, each of type short. * The min and max attributes need to satisfy 2 rules: - * - they are each expected to be >= 1, as we only consider positive version values to be valid. + * - they are each expected to be >= 0, as we only consider positive version values to be valid. * - max should be >= min. * * The class also provides API to convert the version range to a map. @@ -48,7 +48,7 @@ class BaseVersionRange { /** * Raises an exception unless the following condition is met: - * minValue >= 1 and maxValue >= 1 and maxValue >= minValue. + * minValue >= 0 and maxValue >= 0 and maxValue >= minValue. * * @param minKeyLabel Label for the min version key, that's used only to convert to/from a map. * @param minValue The minimum version value. @@ -56,7 +56,7 @@ class BaseVersionRange { * @param maxValue The maximum version value. * * @throws IllegalArgumentException If any of the following conditions are true: - * - (minValue < 1) OR (maxValue < 1) OR (maxValue < minValue). + * - (minValue < 0) OR (maxValue < 0) OR (maxValue < minValue). * - minKeyLabel is empty, OR, minKeyLabel is empty. */ protected BaseVersionRange(String minKeyLabel, short minValue, String maxKeyLabel, short maxValue) { From 92909fbfa413e4179bbe5d34a8bbedbcb41e7f28 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Mon, 8 Apr 2024 11:14:32 -0700 Subject: [PATCH 17/34] KAFKA-16207; Snapshots should return control records --- .../apache/kafka/raft/ReplicatedCounter.java | 36 ++++++++++++------- .../kafka/snapshot/RecordsSnapshotReader.java | 6 +--- .../kafka/raft/RaftEventSimulationTest.java | 35 ++++++++++++------ 3 files changed, 49 insertions(+), 28 deletions(-) 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 0b4fecd8d5ab..18a90a7b0bd3 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java +++ b/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java @@ -16,16 +16,14 @@ */ package org.apache.kafka.raft; +import java.util.Optional; +import java.util.OptionalInt; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.raft.errors.NotLeaderException; 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; public class ReplicatedCounter implements RaftClient.Listener { @@ -139,16 +137,30 @@ public synchronized void handleCommit(BatchReader reader) { public synchronized void handleLoadSnapshot(SnapshotReader reader) { try { log.debug("Loading snapshot {}", reader.snapshotId()); + // Since the state machine is only one value, expect only one data record + boolean foundDataRecord = false; 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() - ) - ); + if (!batch.records().isEmpty()) { + if (foundDataRecord) { + throw new AssertionError( + String.format( + "Expected the snapshot at %s to only one data batch %s", + reader.snapshotId(), + batch + ) + ); + } else 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() + ) + ); + } + + foundDataRecord = true; } for (Integer value : batch) { diff --git a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotReader.java b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotReader.java index c9c42a1da9ac..cd8cd159350c 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotReader.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotReader.java @@ -123,8 +123,6 @@ public static RecordsSnapshotReader of( * Returns the next non-control Batch */ private Optional> nextBatch() { - // TODO: Should this return control records? It may need to return control record if I decide to use - // this type in the internal KRaft listener. Another option is to use the RecordsIterator directly. while (iterator.hasNext()) { Batch batch = iterator.next(); @@ -149,9 +147,7 @@ private Optional> nextBatch() { ); } - if (!batch.records().isEmpty()) { - return Optional.of(batch); - } + return Optional.of(batch); } return Optional.empty(); 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 1ccd7dca5025..749d3e5da5fc 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java @@ -1124,22 +1124,33 @@ private void assertCommittedData(RaftNode node) { assertTrue(snapshotId.offset() <= highWatermark.getAsLong()); startOffset.set(snapshotId.offset()); - try (SnapshotReader snapshot = - RecordsSnapshotReader.of(log.readSnapshot(snapshotId).get(), node.intSerde, BufferSupplier.create(), Integer.MAX_VALUE, true)) { + try (SnapshotReader snapshot = RecordsSnapshotReader.of( + log.readSnapshot(snapshotId).get(), + node.intSerde, + BufferSupplier.create(), + Integer.MAX_VALUE, + true + ) + ) { + // Since the state machine is only on e value we only expect one data record in the snapshot // Expect only one batch with only one record - assertTrue(snapshot.hasNext()); - Batch batch = snapshot.next(); - assertFalse(snapshot.hasNext()); - assertEquals(1, batch.records().size()); + OptionalInt sequence = OptionalInt.empty(); + while (snapshot.hasNext()) { + Batch batch = snapshot.next(); + if (!batch.records().isEmpty()) { + assertEquals(1, batch.records().size()); + assertFalse(sequence.isPresent()); + sequence = OptionalInt.of(batch.records().get(0)); + } + } // The snapshotId offset is an "end offset" long offset = snapshotId.offset() - 1; - int sequence = batch.records().get(0); - committedSequenceNumbers.putIfAbsent(offset, sequence); + committedSequenceNumbers.putIfAbsent(offset, sequence.getAsInt()); assertEquals( committedSequenceNumbers.get(offset), - sequence, + sequence.getAsInt(), String.format("Committed sequence at offset %s changed on node %s", offset, nodeId) ); } @@ -1159,8 +1170,10 @@ private void assertCommittedData(RaftNode node) { int committedSequence = committedSequenceNumbers.get(offset); assertEquals( - committedSequence, sequence, - "Committed sequence at offset " + offset + " changed on node " + nodeId); + committedSequence, + sequence, + String.format("Committed sequence at offset %d changed on node %d", offset, nodeId) + ); } } } From dd28ef457a2d90dffe26c67927c7b7569c8128e2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Mon, 8 Apr 2024 14:41:34 -0700 Subject: [PATCH 18/34] KAFKA-16207; Document internal listener invariant --- .../apache/kafka/raft/KafkaRaftClient.java | 38 ++++++++++--------- .../raft/internals/InternalLogListener.java | 34 +++++++++++------ 2 files changed, 44 insertions(+), 28 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 a1206d012164..efd44418db45 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -169,9 +169,13 @@ final public class KafkaRaftClient implements RaftClient { private final Map, ListenerContext> listenerContexts = new IdentityHashMap<>(); private final ConcurrentLinkedQueue> pendingRegistrations = new ConcurrentLinkedQueue<>(); - // Components that needs to be initialized because they depend on the voter set - // TODO: figure out how to enforce the requirement that the internal log listener must be more up to date then - // the external raft listener. This is a requirement for a consistent snapshot. + // These components need to be initialized by the method initialize() because they depend on the voter set + /* + * The key invariant for the internal listener is that it has always read to the LEO. This is achived by: + * 1. reading the entire partition (snapshot and log) at start up, + * 2. updating the internal log listener when a snapshot is replaced, because of FETCH_SNAPSHOT, on the followers + * 3. updating the internal log listener when the leader (call to append()) or follower (FETCH) appends to the log + */ private volatile InternalLogListener internalListener; private volatile KafkaRaftMetrics kafkaRaftMetrics; private volatile QuorumState quorum; @@ -356,8 +360,6 @@ private void maybeFireLeaderChange() { } } - // TODO: need a way to verify that initialize has been called. - public void initialize( Map voterAddresses, String listenerName, @@ -368,7 +370,9 @@ public void initialize( Optional.of(VoterSet.fromAddressSpecs(listenerName, voterAddresses)), log, serde, - BufferSupplier.create() + BufferSupplier.create(), + MAX_BATCH_SIZE_BYTES, + logContext ); // Read the entire log logger.info("Reading KRaft snapshot and log as part of the initialization"); @@ -441,10 +445,10 @@ public void unregister(Listener listener) { @Override public LeaderAndEpoch leaderAndEpoch() { - if (!isInitialized()) { - return LeaderAndEpoch.UNKNOWN; - } else { + if (isInitialized()) { return quorum.leaderAndEpoch(); + } else { + return LeaderAndEpoch.UNKNOWN; } } @@ -2349,7 +2353,7 @@ public void handle(RaftRequest.Inbound request) { */ public void poll() { if (!isInitialized()) { - new IllegalStateException("KafkaRaftClient must be initialized before polling"); + throw new IllegalStateException("Replica needs to be initialized before polling"); } long startPollTimeMs = time.milliseconds(); @@ -2388,11 +2392,11 @@ public long scheduleAtomicAppend(int epoch, OptionalLong requiredBaseOffset, Lis private long append(int epoch, List records, OptionalLong requiredBaseOffset, boolean isAtomic) { if (!isInitialized()) { - new NotLeaderException("Append failed because the replication is not the current leader"); + throw new NotLeaderException("Append failed because the replica is not the current leader"); } LeaderState leaderState = quorum.maybeLeaderState().orElseThrow( - () -> new NotLeaderException("Append failed because the replication is not the current leader") + () -> new NotLeaderException("Append failed because the replica is not the current leader") ); BatchAccumulator accumulator = leaderState.accumulator(); @@ -2423,9 +2427,9 @@ public CompletableFuture shutdown(int timeoutMs) { public void resign(int epoch) { if (epoch < 0) { throw new IllegalArgumentException("Attempt to resign from an invalid negative epoch " + epoch); - } - - if (!quorum.isVoter()) { + } else if (!isInitialized()) { + throw new IllegalStateException("Replica needs to be initialized before resigning"); + } else if (!quorum.isVoter()) { throw new IllegalStateException("Attempt to resign by a non-voter"); } @@ -2473,7 +2477,7 @@ public Optional> createSnapshot( long lastContainedLogTimestamp ) { if (!isInitialized()) { - throw new IllegalStateException("Cannot create snapshot before the kraft client has been initialized"); + throw new IllegalStateException("Cannot create snapshot before the replica has been initialized"); } return log.createNewSnapshot(snapshotId).map(writer -> { @@ -2524,7 +2528,7 @@ public OptionalLong highWatermark() { // Visible only for test QuorumState quorum() { - // because this is only called by test is it okay to return null + // It's okay to return null since this method is only called by tests return quorum; } diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java b/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java index 1634269615e1..173c2eda79b0 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java @@ -16,21 +16,22 @@ */ package org.apache.kafka.raft.internals; +import java.util.Optional; +import java.util.OptionalLong; import org.apache.kafka.common.message.KRaftVersionRecord; import org.apache.kafka.common.message.VotersRecord; import org.apache.kafka.common.utils.BufferSupplier; +import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.raft.Batch; import org.apache.kafka.raft.ControlRecord; import org.apache.kafka.raft.Isolation; import org.apache.kafka.raft.LogFetchInfo; import org.apache.kafka.raft.ReplicatedLog; import org.apache.kafka.server.common.serialization.RecordSerde; -import java.util.OptionalLong; -import java.util.Optional; import org.apache.kafka.snapshot.RawSnapshotReader; -import org.apache.kafka.snapshot.SnapshotReader; import org.apache.kafka.snapshot.RecordsSnapshotReader; -import static org.apache.kafka.raft.KafkaRaftClient.MAX_BATCH_SIZE_BYTES; +import org.apache.kafka.snapshot.SnapshotReader; +import org.slf4j.Logger; // TODO: File an issue to remove the RecordSerde. The internal listener should just skip data record batches // TODO: Document this class and methods @@ -39,6 +40,8 @@ final public class InternalLogListener { private final ReplicatedLog log; private final RecordSerde serde; private final BufferSupplier bufferSupplier; + private final Logger logger; + private final int maxBatchSizeBytes; // These are objects are synchronized using the perspective object monitor. The two actors // are the KRaft driver and the RaftClient callers @@ -57,12 +60,16 @@ public InternalLogListener( Optional staticVoterSet, ReplicatedLog log, RecordSerde serde, - BufferSupplier bufferSupplier + BufferSupplier bufferSupplier, + int maxBatchSizeBytes, + LogContext logContext ) { this.log = log; this.voterSetHistory = new VoterSetHistory(staticVoterSet); this.serde = serde; this.bufferSupplier = bufferSupplier; + this.maxBatchSizeBytes = maxBatchSizeBytes; + this.logger = logContext.logger(this.getClass()); } public void updateListener() { @@ -117,7 +124,7 @@ private void maybeLoadLog() { info.records, serde, bufferSupplier, - MAX_BATCH_SIZE_BYTES, + maxBatchSizeBytes, true // Validate batch CRC ) ) { @@ -131,8 +138,8 @@ private void maybeLoadLog() { } private void maybeLoadSnapshot() { - Optional rawSnapshot = log.latestSnapshot(); - if (rawSnapshot.isPresent() && (nextOffset == 0 || nextOffset < log.startOffset())) { + if ((nextOffset == 0 || nextOffset < log.startOffset()) && log.latestSnapshot().isPresent()) { + RawSnapshotReader rawSnapshot = log.latestSnapshot().get(); // Clear the current state synchronized (kraftVersion) { kraftVersion.clear(); @@ -143,14 +150,19 @@ private void maybeLoadSnapshot() { // Load the snapshot since the listener is at the start of the log or the log doesn't have the next entry. try (SnapshotReader reader = RecordsSnapshotReader.of( - rawSnapshot.get(), + rawSnapshot, serde, bufferSupplier, - MAX_BATCH_SIZE_BYTES, + maxBatchSizeBytes, true // Validate batch CRC ) ) { - // TODO: log a message that we are loading a snapshot + logger.info( + "Loading snapshot ({}) since log start offset ({}) is greater than the internal listener's next offset ({})", + reader.snapshotId(), + log.startOffset(), + nextOffset + ); OptionalLong currentOffset = OptionalLong.of(reader.lastContainedLogOffset()); while (reader.hasNext()) { Batch batch = reader.next(); From deb79203dc3ac2312c2f4b5dadc78a1072437d19 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Mon, 8 Apr 2024 16:24:08 -0700 Subject: [PATCH 19/34] KAFKA-16207; Update some of the documentation --- .../apache/kafka/raft/KafkaRaftClient.java | 8 +++ .../raft/internals/InternalLogListener.java | 34 ++++++++++++- .../kafka/raft/internals/TreeMapHistory.java | 4 +- .../apache/kafka/raft/internals/VoterSet.java | 51 ++++++++++++++++++- .../kafka/raft/internals/VoterSetHistory.java | 28 ++++++++-- .../raft/internals/VoterSetHistoryTest.java | 3 +- 6 files changed, 118 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 efd44418db45..1e1a0391152a 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -175,6 +175,8 @@ final public class KafkaRaftClient implements RaftClient { * 1. reading the entire partition (snapshot and log) at start up, * 2. updating the internal log listener when a snapshot is replaced, because of FETCH_SNAPSHOT, on the followers * 3. updating the internal log listener when the leader (call to append()) or follower (FETCH) appends to the log + * 4. truncate the internal log listener when a follower truncates their log + * 5. trim the internal listener prefix when a snapshot gets generated */ private volatile InternalLogListener internalListener; private volatile KafkaRaftMetrics kafkaRaftMetrics; @@ -1180,6 +1182,7 @@ private boolean handleFetchResponse( truncationOffset, quorum.leaderIdOrSentinel() ); + // TODO: we should also truncate the internal listener } else if (partitionResponse.snapshotId().epoch() >= 0 || partitionResponse.snapshotId().endOffset() >= 0) { // The leader is asking us to fetch a snapshot @@ -1517,6 +1520,9 @@ private boolean handleFetchSnapshotResponse( quorum.leaderIdOrSentinel() ); + // TODO: are we guarantee to always load a snapshot? This always true, if the next offset is always less + // than the snapshot id. Is this true? I think this always true because the internal listener is always + // at the LEO and we now that the LEO is less that the snapshot id. internalListener.updateListener(); updateFollowerHighWatermark(state, OptionalLong.of(log.highWatermark().offset)); @@ -2480,6 +2486,8 @@ public Optional> createSnapshot( throw new IllegalStateException("Cannot create snapshot before the replica has been initialized"); } + // TODO: need to track when a snapshot has been frozen so that we can trim the prefix of the internal log listener + // this is currently track only by the KafkaMetadataLog implementation. We need to move this to this implementation. return log.createNewSnapshot(snapshotId).map(writer -> { long lastContainedLogOffset = snapshotId.offset() - 1; diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java b/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java index 173c2eda79b0..3d35622e7632 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java @@ -34,8 +34,12 @@ import org.slf4j.Logger; // TODO: File an issue to remove the RecordSerde. The internal listener should just skip data record batches -// TODO: Document this class and methods // TODO: Add unnitest for it +/** + * The KRaft state machine for tracking control records in the topic partition. + * + * This type keeps track of changes to the finalized kraft.version and the sets of voters. + */ final public class InternalLogListener { private final ReplicatedLog log; private final RecordSerde serde; @@ -56,6 +60,16 @@ final public class InternalLogListener { // the nextOffset first before reading voterSetHistory or kraftVersion private volatile long nextOffset = 0; + /** + * Constructs an internal log listener + * + * @param staticVoterSet the set of voter statically configured + * @param log the on disk topic partition + * @param serde the record decoder for data records + * @param bufferSupplier the supplier of byte buffers + * @param maxBatchSizeBytes the maximum size of record batch + * @param logContext the log context + */ public InternalLogListener( Optional staticVoterSet, ReplicatedLog log, @@ -72,17 +86,29 @@ public InternalLogListener( this.logger = logContext.logger(this.getClass()); } + /** + * Must be called whenever the {@code log} has changed. + */ public void updateListener() { maybeLoadSnapshot(); maybeLoadLog(); } + /** + * Returns the last voter set. + */ public VoterSet lastVoterSet() { synchronized (voterSetHistory) { return voterSetHistory.lastValue(); } } + /** + * Rturns the voter set at a given offset. + * + * @param offset the offset (inclusive) + * @return the voter set if one exist, otherwise {@code Optional.empty()} + */ public Optional voterSetAtOffset(long offset) { long fixedNextOffset = nextOffset; if (offset >= fixedNextOffset) { @@ -100,6 +126,12 @@ public Optional voterSetAtOffset(long offset) { } } + /** + * Returns the finalized kraft version at a given offset. + * + * @param offset the offset (inclusive) + * @return the finalized kraft version if one exist, otherwise 0 + */ public short kraftVersionAtOffset(long offset) { long fixedNextOffset = nextOffset; if (offset >= fixedNextOffset) { diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/TreeMapHistory.java b/raft/src/main/java/org/apache/kafka/raft/internals/TreeMapHistory.java index c4c41eaaf2d2..81e884277067 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/TreeMapHistory.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/TreeMapHistory.java @@ -21,8 +21,10 @@ import java.util.TreeMap; import java.util.Map; -// TODO: document this type // TODO: add unittest +/** + * A implementation for {@code History} which uses a red-black tree to store values sorted by offset. + */ final public class TreeMapHistory implements History { private final NavigableMap history = new TreeMap<>(); diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java index 86b3dea60064..d74c63b39572 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java @@ -32,7 +32,14 @@ import org.apache.kafka.common.feature.SupportedVersionRange; // TODO: write unittest for VoterSet -// TODO: Write documentation +/** + * A type for representing the set of voters for a topic partition. + * + * It encapsulates static information like a voter's endpoint and their supported kraft.version. + * + * It providees functionality for converting to and from {@code VotersRecord} and for converting + * from the static configuration. + */ final public class VoterSet { private final Map voters; @@ -40,15 +47,30 @@ final public class VoterSet { this.voters = voters; } + /** + * Returns the socket address for a given voter at a given listener. + * + * @param voter the id of the voter + * @param listener the name of the listener + * @return the socket address if it exist, otherwise {@code Optional.empty()} + */ public Optional voterAddress(int voter, String listener) { return Optional.ofNullable(voters.get(voter)) .flatMap(voterNode -> voterNode.address(listener)); } + /** + * Returns all of the voter ids. + */ public Set voterIds() { return voters.keySet(); } + /** + * Converts a voter set to a voters record for a given version. + * + * @param version the version of the voters record + */ public VotersRecord toVotersRecord(short version) { return new VotersRecord() .setVersion(version) @@ -83,7 +105,19 @@ public VotersRecord toVotersRecord(short version) { ); } - boolean hasOverlappingMajority(VoterSet that) { + /** + * Determines if two sets of voters have an overlapping majority. + * + * A overlapping majority means that for all majorities in {@code this} set of voters and for + * all majority in {@code that} voeter set they have at least one voter in common. + * + * This can be used to validate a change in the set of voters will get committed by both sets + * of voters. + * + * @param that the other voter set to compare + * @return true if they have an overlapping majority, false otherwise + */ + public boolean hasOverlappingMajority(VoterSet that) { if (Utils.diff(HashSet::new, voters.keySet(), that.voters.keySet()).size() > 2) return false; if (Utils.diff(HashSet::new, that.voters.keySet(), voters.keySet()).size() > 2) return false; @@ -181,6 +215,12 @@ public String toString() { } } + /** + * Converts a {@code VotersRecord} to a {@code VoterSet}. + * + * @param voters the set of voters control record + * @return the voter set + */ public static VoterSet fromVotersRecord(VotersRecord voters) { Map voterNodes = new HashMap<>(voters.voters().size()); for (VotersRecord.Voter voter: voters.voters()) { @@ -213,6 +253,13 @@ public static VoterSet fromVotersRecord(VotersRecord voters) { return new VoterSet(voterNodes); } + /** + * Creates a voter set from a map of socket addresses. + * + * @param listener the listener name for all of the endpoints + * @param voters the socket addresses by voter id + * @return the voter set + */ public static VoterSet fromAddressSpecs(String listener, Map voters) { Map voterNodes = voters .entrySet() diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java index 7fa98036f3b3..13b1fd61392e 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java @@ -19,7 +19,13 @@ import java.util.Optional; // TODO: write unittest for VoterSetHistory -// TODO: Write documentation +/** + * A type for storing the historical value of the set of voters. + * + * This type can be use to keep track in-memory the sets for voters stored in the latest snapshot + * and log. This is useful when both generating a new snapshot at a given offset or when evaulating + * the latest set of voters. + */ final public class VoterSetHistory implements History { private final Optional staticVoterSet; private final History votersHistory = new TreeMapHistory<>(); @@ -32,9 +38,10 @@ final public class VoterSetHistory implements History { public void addAt(long offset, VoterSet voters) { Optional> lastEntry = votersHistory.lastEntry(); if (lastEntry.isPresent() && lastEntry.get().offset() >= 0) { - // If the last voter set comes from the replicated log then the majorities must overlap. This ignores - // the static voter set and the bootstrapped voter set since they come from the configuration and the KRaft - // leader never guaranteed that they are the same across all replicas. + // If the last voter set comes from the replicated log then the majorities must overlap. + // This ignores the static voter set and the bootstrapped voter set since they come from + // the configuration and the KRaft leader never guaranteed that they are the same across + // all replicas. VoterSet lastVoterSet = lastEntry.get().value(); if (!lastVoterSet.hasOverlappingMajority(voters)) { throw new IllegalArgumentException( @@ -50,7 +57,15 @@ public void addAt(long offset, VoterSet voters) { votersHistory.addAt(offset, voters); } - // TODO: document that this doesn't include the static configuration + /** + * Computes the value of the voter set at a given offset. + * + * This function will only return values provided through {@code addAt} and it would never + * include the {@code staticVoterSet} provided through the constructoer. + * + * @param offset the offset (inclusive) + * @return the voter set if one exist, otherwise {@code Optional.empty()} + */ @Override public Optional valueAt(long offset) { return votersHistory.valueAt(offset); @@ -64,6 +79,9 @@ public Optional> lastEntry() { return staticVoterSet.map(value -> new History.Entry<>(-1, value)); } + /** + * Returns the latest set of voters. + */ public VoterSet lastValue() { return lastEntry().orElseThrow(() -> new IllegalStateException("No voter set found")).value(); } diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java index 486d28d16dbf..93fabeb6a548 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java @@ -21,6 +21,7 @@ import java.util.Collections; import java.util.List; import java.util.Optional; +import java.util.function.Function; import java.util.stream.Collectors; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.feature.SupportedVersionRange; @@ -56,7 +57,7 @@ private VoterSet createVoterSet(List replicas) { .stream() .collect( Collectors.toMap( - replica -> replica, + Function.identity(), replica -> new VoterSet.VoterNode( replica, Optional.of(Uuid.randomUuid()), From 0db1a0651fc510ba36fbe3d25a1c8be7ac7465a4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Mon, 8 Apr 2024 21:10:27 -0700 Subject: [PATCH 20/34] KAFKA-16207; Fix cluster snapshot test --- .../kafka/server/RaftClusterSnapshotTest.scala | 9 +++++---- .../org/apache/kafka/snapshot/RecordsSnapshotReader.java | 2 +- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala b/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala index e6abc04f17fb..bebd90e6b087 100644 --- a/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala +++ b/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala @@ -17,7 +17,6 @@ package kafka.server -import java.util.Collections import kafka.testkit.KafkaClusterTestKit import kafka.testkit.TestKitNodes import kafka.utils.TestUtils @@ -26,7 +25,6 @@ import org.apache.kafka.common.utils.BufferSupplier import org.apache.kafka.metadata.MetadataRecordSerde import org.apache.kafka.snapshot.RecordsSnapshotReader import org.junit.jupiter.api.Assertions.assertEquals -import org.junit.jupiter.api.Assertions.assertNotEquals import org.junit.jupiter.api.Assertions.assertTrue import org.junit.jupiter.api.Test import org.junit.jupiter.api.Timeout @@ -87,9 +85,12 @@ class RaftClusterSnapshotTest { // Check that we can read the entire snapshot while (snapshot.hasNext) { - val batch = snapshot.next() + val batch = snapshot.next assertTrue(batch.sizeInBytes > 0) - assertNotEquals(Collections.emptyList(), batch.records()) + assertTrue( + batch.records.isEmpty != batch.controlRecords.isEmpty, + s"data records = ${batch.records}; control records = ${batch.controlRecords}" + ) } } } diff --git a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotReader.java b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotReader.java index cd8cd159350c..3e6a9a5732db 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotReader.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotReader.java @@ -127,7 +127,7 @@ private Optional> nextBatch() { Batch batch = iterator.next(); if (!lastContainedLogTimestamp.isPresent()) { - // This must be the first batch which is expected to be a control batch with one record for + // This must be the first batch which is expected to be a control batch with at least one record for // the snapshot header. if (batch.controlRecords().isEmpty()) { throw new IllegalStateException( From ab1886aa0531b90a0db91f9a6dadfe7c02fb6ff1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Tue, 9 Apr 2024 11:24:23 -0700 Subject: [PATCH 21/34] KAFKA-16207; Implement truncating and prefix trimming the internal listener --- .../scala/kafka/raft/KafkaMetadataLog.scala | 19 ++++- .../kafka/raft/KafkaMetadataLogTest.scala | 53 ++++++------- .../apache/kafka/raft/KafkaRaftClient.java | 24 +++--- .../org/apache/kafka/raft/ReplicatedLog.java | 5 +- .../apache/kafka/raft/internals/History.java | 2 + .../raft/internals/InternalLogListener.java | 46 ++++++++--- .../kafka/raft/internals/TreeMapHistory.java | 4 +- .../kafka/snapshot/FileRawSnapshotWriter.java | 18 +---- .../snapshot/NotifyingRawSnapshotWriter.java | 77 +++++++++++++++++++ .../kafka/snapshot/RecordsSnapshotWriter.java | 2 +- .../java/org/apache/kafka/raft/MockLog.java | 4 +- .../org/apache/kafka/raft/MockLogTest.java | 26 +++---- .../kafka/raft/RaftClientTestContext.java | 2 +- .../kafka/snapshot/FileRawSnapshotTest.java | 3 +- .../apache/kafka/snapshot/SnapshotsTest.java | 2 +- 15 files changed, 196 insertions(+), 91 deletions(-) create mode 100644 raft/src/main/java/org/apache/kafka/snapshot/NotifyingRawSnapshotWriter.java diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index 514b08797a10..ab480f57339b 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -32,7 +32,13 @@ import org.apache.kafka.common.utils.Time import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid} import org.apache.kafka.raft.{Isolation, KafkaRaftClient, LogAppendInfo, LogFetchInfo, LogOffsetMetadata, OffsetAndEpoch, OffsetMetadata, ReplicatedLog, ValidOffsetAndEpoch} import org.apache.kafka.server.util.Scheduler -import org.apache.kafka.snapshot.{FileRawSnapshotReader, FileRawSnapshotWriter, RawSnapshotReader, RawSnapshotWriter, SnapshotPath, Snapshots} +import org.apache.kafka.snapshot.FileRawSnapshotReader +import org.apache.kafka.snapshot.FileRawSnapshotWriter +import org.apache.kafka.snapshot.NotifyingRawSnapshotWriter +import org.apache.kafka.snapshot.RawSnapshotReader +import org.apache.kafka.snapshot.RawSnapshotWriter +import org.apache.kafka.snapshot.SnapshotPath +import org.apache.kafka.snapshot.Snapshots import org.apache.kafka.storage.internals import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchIsolation, LogConfig, LogDirFailureChannel, LogStartOffsetIncrementReason, ProducerStateManagerConfig} @@ -264,10 +270,10 @@ final class KafkaMetadataLog private ( ) } - storeSnapshot(snapshotId) + createNewSnapshotUnchecked(snapshotId) } - override def storeSnapshot(snapshotId: OffsetAndEpoch): Optional[RawSnapshotWriter] = { + override def createNewSnapshotUnchecked(snapshotId: OffsetAndEpoch): Optional[RawSnapshotWriter] = { val containsSnapshotId = snapshots synchronized { snapshots.contains(snapshotId) } @@ -275,7 +281,12 @@ final class KafkaMetadataLog private ( if (containsSnapshotId) { Optional.empty() } else { - Optional.of(FileRawSnapshotWriter.create(log.dir.toPath, snapshotId, Optional.of(this))) + Optional.of( + new NotifyingRawSnapshotWriter( + FileRawSnapshotWriter.create(log.dir.toPath, snapshotId), + onSnapshotFrozen + ) + ) } } diff --git a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala index 508a82a511c4..0016b9258f35 100644 --- a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala +++ b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala @@ -323,7 +323,7 @@ final class KafkaMetadataLogTest { append(log, numberOfRecords, epoch) - TestUtils.resource(log.storeSnapshot(sameEpochSnapshotId).get()) { snapshot => + TestUtils.resource(log.createNewSnapshotUnchecked(sameEpochSnapshotId).get()) { snapshot => snapshot.freeze() } @@ -337,7 +337,7 @@ final class KafkaMetadataLogTest { append(log, numberOfRecords, epoch) - TestUtils.resource(log.storeSnapshot(greaterEpochSnapshotId).get()) { snapshot => + TestUtils.resource(log.createNewSnapshotUnchecked(greaterEpochSnapshotId).get()) { snapshot => snapshot.freeze() } @@ -356,25 +356,25 @@ final class KafkaMetadataLogTest { append(log, 1, epoch - 1) val oldSnapshotId1 = new OffsetAndEpoch(1, epoch - 1) - TestUtils.resource(log.storeSnapshot(oldSnapshotId1).get()) { snapshot => + TestUtils.resource(log.createNewSnapshotUnchecked(oldSnapshotId1).get()) { snapshot => snapshot.freeze() } append(log, 1, epoch) val oldSnapshotId2 = new OffsetAndEpoch(2, epoch) - TestUtils.resource(log.storeSnapshot(oldSnapshotId2).get()) { snapshot => + TestUtils.resource(log.createNewSnapshotUnchecked(oldSnapshotId2).get()) { snapshot => snapshot.freeze() } append(log, numberOfRecords - 2, epoch) val oldSnapshotId3 = new OffsetAndEpoch(numberOfRecords, epoch) - TestUtils.resource(log.storeSnapshot(oldSnapshotId3).get()) { snapshot => + TestUtils.resource(log.createNewSnapshotUnchecked(oldSnapshotId3).get()) { snapshot => snapshot.freeze() } val greaterSnapshotId = new OffsetAndEpoch(3 * numberOfRecords, epoch) append(log, numberOfRecords, epoch) - TestUtils.resource(log.storeSnapshot(greaterSnapshotId).get()) { snapshot => + TestUtils.resource(log.createNewSnapshotUnchecked(greaterSnapshotId).get()) { snapshot => snapshot.freeze() } @@ -467,12 +467,7 @@ final class KafkaMetadataLogTest { metadataDir: File, snapshotId: OffsetAndEpoch ): Unit = { - val writer = FileRawSnapshotWriter.create( - metadataDir.toPath, - snapshotId, - Optional.empty() - ) - TestUtils.resource(writer)(_.freeze()) + TestUtils.resource(FileRawSnapshotWriter.create(metadataDir.toPath, snapshotId))(_.freeze()) } @Test @@ -484,7 +479,7 @@ final class KafkaMetadataLogTest { append(log, numberOfRecords, epoch) val olderEpochSnapshotId = new OffsetAndEpoch(numberOfRecords, epoch - 1) - TestUtils.resource(log.storeSnapshot(olderEpochSnapshotId).get()) { snapshot => + TestUtils.resource(log.createNewSnapshotUnchecked(olderEpochSnapshotId).get()) { snapshot => snapshot.freeze() } @@ -493,7 +488,7 @@ final class KafkaMetadataLogTest { append(log, numberOfRecords, epoch) val olderOffsetSnapshotId = new OffsetAndEpoch(numberOfRecords, epoch) - TestUtils.resource(log.storeSnapshot(olderOffsetSnapshotId).get()) { snapshot => + TestUtils.resource(log.createNewSnapshotUnchecked(olderOffsetSnapshotId).get()) { snapshot => snapshot.freeze() } @@ -508,7 +503,7 @@ final class KafkaMetadataLogTest { val snapshotId = new OffsetAndEpoch(1, epoch) append(log, numberOfRecords, epoch) - TestUtils.resource(log.storeSnapshot(snapshotId).get()) { snapshot => + TestUtils.resource(log.createNewSnapshotUnchecked(snapshotId).get()) { snapshot => snapshot.freeze() } @@ -545,25 +540,25 @@ final class KafkaMetadataLogTest { append(log, 1, epoch - 1) val oldSnapshotId1 = new OffsetAndEpoch(1, epoch - 1) - TestUtils.resource(log.storeSnapshot(oldSnapshotId1).get()) { snapshot => + TestUtils.resource(log.createNewSnapshotUnchecked(oldSnapshotId1).get()) { snapshot => snapshot.freeze() } append(log, 1, epoch) val oldSnapshotId2 = new OffsetAndEpoch(2, epoch) - TestUtils.resource(log.storeSnapshot(oldSnapshotId2).get()) { snapshot => + TestUtils.resource(log.createNewSnapshotUnchecked(oldSnapshotId2).get()) { snapshot => snapshot.freeze() } append(log, numberOfRecords - 2, epoch) val oldSnapshotId3 = new OffsetAndEpoch(numberOfRecords, epoch) - TestUtils.resource(log.storeSnapshot(oldSnapshotId3).get()) { snapshot => + TestUtils.resource(log.createNewSnapshotUnchecked(oldSnapshotId3).get()) { snapshot => snapshot.freeze() } val greaterSnapshotId = new OffsetAndEpoch(3 * numberOfRecords, epoch) append(log, numberOfRecords, epoch) - TestUtils.resource(log.storeSnapshot(greaterSnapshotId).get()) { snapshot => + TestUtils.resource(log.createNewSnapshotUnchecked(greaterSnapshotId).get()) { snapshot => snapshot.freeze() } @@ -594,7 +589,7 @@ final class KafkaMetadataLogTest { val snapshotId = new OffsetAndEpoch(numberOfRecords + 1, epoch + 1) append(log, numberOfRecords, epoch) - TestUtils.resource(log.storeSnapshot(snapshotId).get()) { snapshot => + TestUtils.resource(log.createNewSnapshotUnchecked(snapshotId).get()) { snapshot => snapshot.freeze() } @@ -752,7 +747,7 @@ final class KafkaMetadataLogTest { val log = buildMetadataLog(tempDir, mockTime) log.updateHighWatermark(new LogOffsetMetadata(offset)) val snapshotId = new OffsetAndEpoch(offset, 1) - TestUtils.resource(log.storeSnapshot(snapshotId).get()) { snapshot => + TestUtils.resource(log.createNewSnapshotUnchecked(snapshotId).get()) { snapshot => snapshot.freeze() } log.truncateToLatestSnapshot() @@ -776,7 +771,7 @@ final class KafkaMetadataLogTest { val log = buildMetadataLog(tempDir, mockTime) log.updateHighWatermark(new LogOffsetMetadata(offset)) val snapshotId = new OffsetAndEpoch(offset, 1) - TestUtils.resource(log.storeSnapshot(snapshotId).get()) { snapshot => + TestUtils.resource(log.createNewSnapshotUnchecked(snapshotId).get()) { snapshot => snapshot.freeze() } log.truncateToLatestSnapshot() @@ -858,13 +853,13 @@ final class KafkaMetadataLogTest { assertFalse(log.maybeClean(), "Should not clean since no snapshots exist") val snapshotId1 = new OffsetAndEpoch(1000, 1) - TestUtils.resource(log.storeSnapshot(snapshotId1).get()) { snapshot => + TestUtils.resource(log.createNewSnapshotUnchecked(snapshotId1).get()) { snapshot => append(snapshot, 100) snapshot.freeze() } val snapshotId2 = new OffsetAndEpoch(2000, 1) - TestUtils.resource(log.storeSnapshot(snapshotId2).get()) { snapshot => + TestUtils.resource(log.createNewSnapshotUnchecked(snapshotId2).get()) { snapshot => append(snapshot, 100) snapshot.freeze() } @@ -896,7 +891,7 @@ final class KafkaMetadataLogTest { for (offset <- Seq(100, 200, 300, 400, 500, 600)) { val snapshotId = new OffsetAndEpoch(offset, 1) - TestUtils.resource(log.storeSnapshot(snapshotId).get()) { snapshot => + TestUtils.resource(log.createNewSnapshotUnchecked(snapshotId).get()) { snapshot => append(snapshot, 10) snapshot.freeze() } @@ -931,14 +926,14 @@ final class KafkaMetadataLogTest { // Then generate two snapshots val snapshotId1 = new OffsetAndEpoch(1000, 1) - TestUtils.resource(log.storeSnapshot(snapshotId1).get()) { snapshot => + TestUtils.resource(log.createNewSnapshotUnchecked(snapshotId1).get()) { snapshot => append(snapshot, 500) snapshot.freeze() } // Then generate a snapshot val snapshotId2 = new OffsetAndEpoch(2000, 1) - TestUtils.resource(log.storeSnapshot(snapshotId2).get()) { snapshot => + TestUtils.resource(log.createNewSnapshotUnchecked(snapshotId2).get()) { snapshot => append(snapshot, 500) snapshot.freeze() } @@ -978,7 +973,7 @@ final class KafkaMetadataLogTest { log.log.logSegments.asScala.drop(1).head.baseOffset, 1 ) - TestUtils.resource(log.storeSnapshot(snapshotId1).get()) { snapshot => + TestUtils.resource(log.createNewSnapshotUnchecked(snapshotId1).get()) { snapshot => snapshot.freeze() } // Generate second snapshots that includes the second segment by using the base offset of the third segment @@ -986,7 +981,7 @@ final class KafkaMetadataLogTest { log.log.logSegments.asScala.drop(2).head.baseOffset, 1 ) - TestUtils.resource(log.storeSnapshot(snapshotId2).get()) { snapshot => + TestUtils.resource(log.createNewSnapshotUnchecked(snapshotId2).get()) { snapshot => snapshot.freeze() } 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 1e1a0391152a..4232a782513c 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -72,12 +72,13 @@ import org.apache.kafka.raft.internals.ThresholdPurgatory; import org.apache.kafka.raft.internals.VoterSet; import org.apache.kafka.server.common.serialization.RecordSerde; +import org.apache.kafka.snapshot.NotifyingRawSnapshotWriter; import org.apache.kafka.snapshot.RawSnapshotReader; import org.apache.kafka.snapshot.RawSnapshotWriter; import org.apache.kafka.snapshot.RecordsSnapshotReader; import org.apache.kafka.snapshot.RecordsSnapshotWriter; -import org.apache.kafka.snapshot.SnapshotWriter; import org.apache.kafka.snapshot.SnapshotReader; +import org.apache.kafka.snapshot.SnapshotWriter; import org.slf4j.Logger; import java.net.InetSocketAddress; @@ -1182,7 +1183,9 @@ private boolean handleFetchResponse( truncationOffset, quorum.leaderIdOrSentinel() ); - // TODO: we should also truncate the internal listener + + // Update the internal listener to the new end offset + internalListener.truncateTo(truncationOffset); } else if (partitionResponse.snapshotId().epoch() >= 0 || partitionResponse.snapshotId().endOffset() >= 0) { // The leader is asking us to fetch a snapshot @@ -1207,10 +1210,10 @@ private boolean handleFetchResponse( partitionResponse.snapshotId().epoch() ); - // Do not validate the snapshot id against the local replicated log - // since this snapshot is expected to reference offsets and epochs - // greater than the log end offset and high-watermark - state.setFetchingSnapshot(log.storeSnapshot(snapshotId)); + // Do not validate the snapshot id against the local replicated log since this + // snapshot is expected to reference offsets and epochs greater than the log + // end offset and high-watermark. + state.setFetchingSnapshot(log.createNewSnapshotUnchecked(snapshotId)); logger.info( "Fetching snapshot {} from Fetch response from leader {}", snapshotId, @@ -2486,17 +2489,20 @@ public Optional> createSnapshot( throw new IllegalStateException("Cannot create snapshot before the replica has been initialized"); } - // TODO: need to track when a snapshot has been frozen so that we can trim the prefix of the internal log listener - // this is currently track only by the KafkaMetadataLog implementation. We need to move this to this implementation. return log.createNewSnapshot(snapshotId).map(writer -> { long lastContainedLogOffset = snapshotId.offset() - 1; + RawSnapshotWriter wrappedWriter = new NotifyingRawSnapshotWriter(writer, offsetAndEpoch -> { + // Trim the state in the internal starting with the new starting offset + internalListener.trimPrefixTo(offsetAndEpoch.offset()); + }); + return new RecordsSnapshotWriter.Builder() .setLastContainedLogTimestamp(lastContainedLogTimestamp) .setTime(time) .setMaxBatchSize(MAX_BATCH_SIZE_BYTES) .setMemoryPool(memoryPool) - .setRawSnapshotWriter(writer) + .setRawSnapshotWriter(wrappedWriter) .setKraftVersion(internalListener.kraftVersionAtOffset(lastContainedLogOffset)) .setVoterSet(internalListener.voterSetAtOffset(lastContainedLogOffset)) .build(serde); 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 6f174e1f7d43..0a30f3fc235f 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ReplicatedLog.java +++ b/raft/src/main/java/org/apache/kafka/raft/ReplicatedLog.java @@ -16,14 +16,13 @@ */ package org.apache.kafka.raft; +import java.util.Optional; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.record.Records; import org.apache.kafka.snapshot.RawSnapshotReader; import org.apache.kafka.snapshot.RawSnapshotWriter; -import java.util.Optional; - public interface ReplicatedLog extends AutoCloseable { /** @@ -261,7 +260,7 @@ default long truncateToEndOffset(OffsetAndEpoch endOffset) { * @param snapshotId the end offset and epoch that identifies the snapshot * @return a writable snapshot if it doesn't already exist */ - Optional storeSnapshot(OffsetAndEpoch snapshotId); + Optional createNewSnapshotUnchecked(OffsetAndEpoch snapshotId); /** * Opens a readable snapshot for the given snapshot id. diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/History.java b/raft/src/main/java/org/apache/kafka/raft/internals/History.java index 0380d1fcda37..f5d42f848870 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/History.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/History.java @@ -27,8 +27,10 @@ public interface History { public Optional> lastEntry(); + // TODO: the delete is inclusive, meaning that the entries removed include the endOffset. public void truncateTo(long endOffset); + // TODO: the delete is exclusive, meaning that the entries removed don't include the startOffset. public void trimPrefixTo(long startOffset); public void clear(); diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java b/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java index 3d35622e7632..46c55ae06e05 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java @@ -50,14 +50,14 @@ final public class InternalLogListener { // These are objects are synchronized using the perspective object monitor. The two actors // are the KRaft driver and the RaftClient callers private final VoterSetHistory voterSetHistory; - private final History kraftVersion = new TreeMapHistory<>(); + private final History kraftVersionHistory = new TreeMapHistory<>(); // This synchronization is enough because // 1. The write operation updateListener only sets the value without reading and updates to - // voterSetHistory or kraftVersion are done before setting the nextOffset + // voterSetHistory or kraftVersionHistory are done before setting the nextOffset // // 2. The read operations lastVoterSet, voterSetAtOffset and kraftVersionAtOffset read - // the nextOffset first before reading voterSetHistory or kraftVersion + // the nextOffset first before reading voterSetHistory or kraftVersionHistory private volatile long nextOffset = 0; /** @@ -94,6 +94,34 @@ public void updateListener() { maybeLoadLog(); } + /** + * Remove the head of the log until the given offset. + * + * @param endOffset the end offset (exclusive) + */ + public void truncateTo(long endOffset) { + synchronized (voterSetHistory) { + voterSetHistory.truncateTo(endOffset); + } + synchronized (kraftVersionHistory) { + kraftVersionHistory.truncateTo(endOffset); + } + } + + /** + * Remove the tail of the log until the given offset. + * + * @param @startOffset the start offset (inclusive) + */ + public void trimPrefixTo(long startOffset) { + synchronized (voterSetHistory) { + voterSetHistory.trimPrefixTo(startOffset); + } + synchronized (kraftVersionHistory) { + kraftVersionHistory.trimPrefixTo(startOffset); + } + } + /** * Returns the last voter set. */ @@ -144,8 +172,8 @@ public short kraftVersionAtOffset(long offset) { ); } - synchronized (kraftVersion) { - return kraftVersion.valueAt(offset).orElse((short) 0); + synchronized (kraftVersionHistory) { + return kraftVersionHistory.valueAt(offset).orElse((short) 0); } } @@ -173,8 +201,8 @@ private void maybeLoadSnapshot() { if ((nextOffset == 0 || nextOffset < log.startOffset()) && log.latestSnapshot().isPresent()) { RawSnapshotReader rawSnapshot = log.latestSnapshot().get(); // Clear the current state - synchronized (kraftVersion) { - kraftVersion.clear(); + synchronized (kraftVersionHistory) { + kraftVersionHistory.clear(); } synchronized (voterSetHistory) { voterSetHistory.clear(); @@ -218,8 +246,8 @@ private void handleBatch(Batch batch, OptionalLong overrideOffset) { break; case KRAFT_VERSION: - synchronized (kraftVersion) { - kraftVersion.addAt(currentOffset, ((KRaftVersionRecord) record.message()).kRaftVersion()); + synchronized (kraftVersionHistory) { + kraftVersionHistory.addAt(currentOffset, ((KRaftVersionRecord) record.message()).kRaftVersion()); } break; diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/TreeMapHistory.java b/raft/src/main/java/org/apache/kafka/raft/internals/TreeMapHistory.java index 81e884277067..dcabef2eb1bf 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/TreeMapHistory.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/TreeMapHistory.java @@ -63,12 +63,12 @@ public Optional> lastEntry() { @Override public void truncateTo(long endOffset) { - history.tailMap(endOffset, false).clear(); + history.tailMap(endOffset, true).clear(); } @Override public void trimPrefixTo(long startOffset) { - NavigableMap lesserValues = history.headMap(startOffset, true); + NavigableMap lesserValues = history.headMap(startOffset, false); while (lesserValues.size() > 1) { // Poll and ignore the entry to remove the first entry lesserValues.pollFirstEntry(); 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 4dc1fe7da03d..1f4af6949810 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotWriter.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotWriter.java @@ -20,7 +20,6 @@ import org.apache.kafka.common.record.UnalignedMemoryRecords; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.raft.OffsetAndEpoch; -import org.apache.kafka.raft.ReplicatedLog; import java.io.IOException; import java.io.UncheckedIOException; @@ -28,25 +27,21 @@ import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.StandardOpenOption; -import java.util.Optional; public final class FileRawSnapshotWriter implements RawSnapshotWriter { private final Path tempSnapshotPath; private final FileChannel channel; private final OffsetAndEpoch snapshotId; - private final Optional replicatedLog; private long frozenSize; private FileRawSnapshotWriter( Path tempSnapshotPath, FileChannel channel, - OffsetAndEpoch snapshotId, - Optional replicatedLog + OffsetAndEpoch snapshotId ) { this.tempSnapshotPath = tempSnapshotPath; this.channel = channel; this.snapshotId = snapshotId; - this.replicatedLog = replicatedLog; this.frozenSize = -1L; } @@ -123,8 +118,6 @@ public void freeze() { Path destination = Snapshots.moveRename(tempSnapshotPath, snapshotId); Utils.atomicMoveWithFallback(tempSnapshotPath, destination); - - replicatedLog.ifPresent(log -> log.onSnapshotFrozen(snapshotId)); } catch (IOException e) { throw new UncheckedIOException( String.format("Error freezing file snapshot, " + @@ -178,19 +171,14 @@ void checkIfFrozen(String operation) { * @param logDir the directory for the topic partition * @param snapshotId the end offset and epoch for the snapshotId */ - public static FileRawSnapshotWriter create( - Path logDir, - OffsetAndEpoch snapshotId, - Optional replicatedLog - ) { + public static FileRawSnapshotWriter create(Path logDir, OffsetAndEpoch snapshotId) { Path path = Snapshots.createTempFile(logDir, snapshotId); try { return new FileRawSnapshotWriter( path, FileChannel.open(path, StandardOpenOption.WRITE, StandardOpenOption.APPEND), - snapshotId, - replicatedLog + snapshotId ); } catch (IOException e) { throw new UncheckedIOException( diff --git a/raft/src/main/java/org/apache/kafka/snapshot/NotifyingRawSnapshotWriter.java b/raft/src/main/java/org/apache/kafka/snapshot/NotifyingRawSnapshotWriter.java new file mode 100644 index 000000000000..e22322f0e85e --- /dev/null +++ b/raft/src/main/java/org/apache/kafka/snapshot/NotifyingRawSnapshotWriter.java @@ -0,0 +1,77 @@ +/* + * 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.util.function.Consumer; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.UnalignedMemoryRecords; +import org.apache.kafka.raft.OffsetAndEpoch; + +// TODO: documen this +// TODO: write tests +public final class NotifyingRawSnapshotWriter implements RawSnapshotWriter { + private final RawSnapshotWriter writer; + private final Consumer callback; + + // TODO: document that this type owns the write and will close it. + public NotifyingRawSnapshotWriter(RawSnapshotWriter writer, Consumer callback) { + this.writer = writer; + this.callback = callback; + } + + @Override + public OffsetAndEpoch snapshotId() { + return writer.snapshotId(); + } + + @Override + public long sizeInBytes() { + return writer.sizeInBytes(); + } + + @Override + public void append(UnalignedMemoryRecords records) { + writer.append(records); + } + + @Override + public void append(MemoryRecords records) { + writer.append(records); + } + + @Override + public boolean isFrozen() { + return writer.isFrozen(); + } + + @Override + public void freeze() { + writer.freeze(); + // Only notify the callback on success + callback.accept(writer.snapshotId()); + } + + @Override + public void close() { + writer.close(); + } + + @Override + public String toString() { + return writer.toString(); + } +} diff --git a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java index 1061d25f55ab..e3483d0a0063 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java @@ -147,7 +147,7 @@ final public static class Builder { private CompressionType compressionType = CompressionType.NONE; private Time time = Time.SYSTEM; private int maxBatchSize = 1024; - MemoryPool memoryPool = MemoryPool.NONE; + private MemoryPool memoryPool = MemoryPool.NONE; private short kraftVersion = 0; private Optional voterSet = Optional.empty(); private Optional rawSnapshotWriter = Optional.empty(); 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 6beec93d974c..6ad22dd9e90a 100644 --- a/raft/src/test/java/org/apache/kafka/raft/MockLog.java +++ b/raft/src/test/java/org/apache/kafka/raft/MockLog.java @@ -489,11 +489,11 @@ public Optional createNewSnapshot(OffsetAndEpoch snapshotId) ); } - return storeSnapshot(snapshotId); + return createNewSnapshotUnchecked(snapshotId); } @Override - public Optional storeSnapshot(OffsetAndEpoch snapshotId) { + public Optional createNewSnapshotUnchecked(OffsetAndEpoch snapshotId) { if (snapshots.containsKey(snapshotId)) { return Optional.empty(); } else { 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 d70c051275dc..12ac9eb18734 100644 --- a/raft/src/test/java/org/apache/kafka/raft/MockLogTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/MockLogTest.java @@ -222,7 +222,7 @@ public void testAppendAsFollower() throws IOException { final int epoch = 3; SimpleRecord recordFoo = new SimpleRecord("foo".getBytes()); - try (RawSnapshotWriter snapshot = log.storeSnapshot(new OffsetAndEpoch(initialOffset, 0)).get()) { + try (RawSnapshotWriter snapshot = log.createNewSnapshotUnchecked(new OffsetAndEpoch(initialOffset, 0)).get()) { snapshot.freeze(); } log.truncateToLatestSnapshot(); @@ -386,7 +386,7 @@ public void testReadOutOfRangeOffset() throws IOException { final int epoch = 3; SimpleRecord recordFoo = new SimpleRecord("foo".getBytes()); - try (RawSnapshotWriter snapshot = log.storeSnapshot(new OffsetAndEpoch(initialOffset, 0)).get()) { + try (RawSnapshotWriter snapshot = log.createNewSnapshotUnchecked(new OffsetAndEpoch(initialOffset, 0)).get()) { snapshot.freeze(); } log.truncateToLatestSnapshot(); @@ -642,7 +642,7 @@ public void testFailToIncreaseLogStartPastHighWatermark() throws IOException { appendBatch(3 * offset, epoch); log.updateHighWatermark(new LogOffsetMetadata(offset)); - try (RawSnapshotWriter snapshot = log.storeSnapshot(snapshotId).get()) { + try (RawSnapshotWriter snapshot = log.createNewSnapshotUnchecked(snapshotId).get()) { snapshot.freeze(); } @@ -660,7 +660,7 @@ public void testTruncateFullyToLatestSnapshot() throws IOException { appendBatch(numberOfRecords, epoch); - try (RawSnapshotWriter snapshot = log.storeSnapshot(sameEpochSnapshotId).get()) { + try (RawSnapshotWriter snapshot = log.createNewSnapshotUnchecked(sameEpochSnapshotId).get()) { snapshot.freeze(); } @@ -674,7 +674,7 @@ public void testTruncateFullyToLatestSnapshot() throws IOException { appendBatch(numberOfRecords, epoch); - try (RawSnapshotWriter snapshot = log.storeSnapshot(greaterEpochSnapshotId).get()) { + try (RawSnapshotWriter snapshot = log.createNewSnapshotUnchecked(greaterEpochSnapshotId).get()) { snapshot.freeze(); } @@ -693,7 +693,7 @@ public void testDoesntTruncateFully() throws IOException { appendBatch(numberOfRecords, epoch); OffsetAndEpoch olderEpochSnapshotId = new OffsetAndEpoch(numberOfRecords, epoch - 1); - try (RawSnapshotWriter snapshot = log.storeSnapshot(olderEpochSnapshotId).get()) { + try (RawSnapshotWriter snapshot = log.createNewSnapshotUnchecked(olderEpochSnapshotId).get()) { snapshot.freeze(); } @@ -702,7 +702,7 @@ public void testDoesntTruncateFully() throws IOException { appendBatch(numberOfRecords, epoch); OffsetAndEpoch olderOffsetSnapshotId = new OffsetAndEpoch(numberOfRecords, epoch); - try (RawSnapshotWriter snapshot = log.storeSnapshot(olderOffsetSnapshotId).get()) { + try (RawSnapshotWriter snapshot = log.createNewSnapshotUnchecked(olderOffsetSnapshotId).get()) { snapshot.freeze(); } @@ -725,7 +725,7 @@ public void testTruncateWillRemoveOlderSnapshot() throws IOException { OffsetAndEpoch greaterEpochSnapshotId = new OffsetAndEpoch(2 * numberOfRecords, epoch + 1); appendBatch(numberOfRecords, epoch); - try (RawSnapshotWriter snapshot = log.storeSnapshot(greaterEpochSnapshotId).get()) { + try (RawSnapshotWriter snapshot = log.createNewSnapshotUnchecked(greaterEpochSnapshotId).get()) { snapshot.freeze(); } @@ -776,7 +776,7 @@ public void testValidateEpochLessThanOldestSnapshotEpoch() throws IOException { int epoch = 1; OffsetAndEpoch olderEpochSnapshotId = new OffsetAndEpoch(offset, epoch); - try (RawSnapshotWriter snapshot = log.storeSnapshot(olderEpochSnapshotId).get()) { + try (RawSnapshotWriter snapshot = log.createNewSnapshotUnchecked(olderEpochSnapshotId).get()) { snapshot.freeze(); } log.truncateToLatestSnapshot(); @@ -791,7 +791,7 @@ public void testValidateOffsetLessThanOldestSnapshotOffset() throws IOException int epoch = 1; OffsetAndEpoch olderEpochSnapshotId = new OffsetAndEpoch(offset, epoch); - try (RawSnapshotWriter snapshot = log.storeSnapshot(olderEpochSnapshotId).get()) { + try (RawSnapshotWriter snapshot = log.createNewSnapshotUnchecked(olderEpochSnapshotId).get()) { snapshot.freeze(); } log.truncateToLatestSnapshot(); @@ -806,7 +806,7 @@ public void testValidateOffsetEqualToOldestSnapshotOffset() throws IOException { int epoch = 1; OffsetAndEpoch olderEpochSnapshotId = new OffsetAndEpoch(offset, epoch); - try (RawSnapshotWriter snapshot = log.storeSnapshot(olderEpochSnapshotId).get()) { + try (RawSnapshotWriter snapshot = log.createNewSnapshotUnchecked(olderEpochSnapshotId).get()) { snapshot.freeze(); } log.truncateToLatestSnapshot(); @@ -821,7 +821,7 @@ public void testValidateUnknownEpochLessThanLastKnownGreaterThanOldestSnapshot() int offset = 10; OffsetAndEpoch olderEpochSnapshotId = new OffsetAndEpoch(offset, 1); - try (RawSnapshotWriter snapshot = log.storeSnapshot(olderEpochSnapshotId).get()) { + try (RawSnapshotWriter snapshot = log.createNewSnapshotUnchecked(olderEpochSnapshotId).get()) { snapshot.freeze(); } log.truncateToLatestSnapshot(); @@ -841,7 +841,7 @@ public void testValidateEpochLessThanFirstEpochInLog() throws IOException { int offset = 10; OffsetAndEpoch olderEpochSnapshotId = new OffsetAndEpoch(offset, 1); - try (RawSnapshotWriter snapshot = log.storeSnapshot(olderEpochSnapshotId).get()) { + try (RawSnapshotWriter snapshot = log.createNewSnapshotUnchecked(olderEpochSnapshotId).get()) { snapshot.freeze(); } log.truncateToLatestSnapshot(); 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 a246c4ae0d1b..93cb5e5076f0 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -203,7 +203,7 @@ public Builder appendToLog(int epoch, List records) { } Builder withEmptySnapshot(OffsetAndEpoch snapshotId) throws IOException { - try (RawSnapshotWriter snapshot = log.storeSnapshot(snapshotId).get()) { + try (RawSnapshotWriter snapshot = log.createNewSnapshotUnchecked(snapshotId).get()) { snapshot.freeze(); } return this; 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 c230e6017a86..9a6fe2e0e130 100644 --- a/raft/src/test/java/org/apache/kafka/snapshot/FileRawSnapshotTest.java +++ b/raft/src/test/java/org/apache/kafka/snapshot/FileRawSnapshotTest.java @@ -37,7 +37,6 @@ import java.nio.file.Path; import java.util.Arrays; import java.util.Iterator; -import java.util.Optional; import java.util.stream.IntStream; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -353,6 +352,6 @@ private static FileRawSnapshotWriter createSnapshotWriter( Path dir, OffsetAndEpoch snapshotId ) { - return FileRawSnapshotWriter.create(dir, snapshotId, Optional.empty()); + return FileRawSnapshotWriter.create(dir, snapshotId); } } diff --git a/raft/src/test/java/org/apache/kafka/snapshot/SnapshotsTest.java b/raft/src/test/java/org/apache/kafka/snapshot/SnapshotsTest.java index 57aca28220f2..4fcf25b7a0e6 100644 --- a/raft/src/test/java/org/apache/kafka/snapshot/SnapshotsTest.java +++ b/raft/src/test/java/org/apache/kafka/snapshot/SnapshotsTest.java @@ -108,7 +108,7 @@ public void testDeleteSnapshot(boolean renameBeforeDeleting) throws IOException ); Path logDirPath = TestUtils.tempDirectory().toPath(); - try (FileRawSnapshotWriter snapshot = FileRawSnapshotWriter.create(logDirPath, snapshotId, Optional.empty())) { + try (FileRawSnapshotWriter snapshot = FileRawSnapshotWriter.create(logDirPath, snapshotId)) { snapshot.freeze(); Path snapshotPath = Snapshots.snapshotPath(logDirPath, snapshotId); From a9621880303afaec64e886e18a89382a75179ae6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Wed, 10 Apr 2024 10:07:11 -0700 Subject: [PATCH 22/34] KAFKA-16207; Finish documenting all of the type and methods --- .../apache/kafka/raft/internals/History.java | 59 ++++++++++++++++--- .../raft/internals/InternalLogListener.java | 4 +- .../kafka/raft/internals/TreeMapHistory.java | 25 ++++---- .../kafka/raft/internals/VoterSetHistory.java | 4 +- .../snapshot/NotifyingRawSnapshotWriter.java | 14 ++++- .../raft/internals/VoterSetHistoryTest.java | 4 +- 6 files changed, 78 insertions(+), 32 deletions(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/History.java b/raft/src/main/java/org/apache/kafka/raft/internals/History.java index f5d42f848870..c2ba7166f40d 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/History.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/History.java @@ -19,20 +19,61 @@ import java.util.Objects; import java.util.Optional; -// TODO: document this type +/** + * A object tracks values of {@code T} at different offsets. + */ public interface History { - public void addAt(long offset, T newValue); - - public Optional valueAt(long offset); - + /** + * Add a new value at a given offset. + * + * The provided {@code offset} must be greater than or equal to 0 and must be greater than the + * offset of all previous calls to this method. + * + * @param offset the offset + * @param value the value to store + * @throws IllegalArgumentException if the offset is not greater that all previous offsets + */ + public void addAt(long offset, T value); + + /** + * Returns the value that has the largest offset that is less than or equals to the provided + * offset. + * + * @param offset the offset + * @return the value if it exist, otherwise {@code Optional.empty()} + */ + public Optional valueAtOrBefore(long offset); + + /** + * Returns the value with the largest offset. + * + * @return the value if it exist, otherwise {@code Optional.empty()} + */ public Optional> lastEntry(); - // TODO: the delete is inclusive, meaning that the entries removed include the endOffset. + /** + * Removes all entries with an offset greater than or equal to {@code endOffset}. + * + * @param endOffset the ending offset + */ public void truncateTo(long endOffset); - // TODO: the delete is exclusive, meaning that the entries removed don't include the startOffset. + /** + * Removes all entries but the largest entry that has an offset that is less than or equal to + * {@code startOffset}. + * + * This operation does not remove the entry with the largest offset that is less than or equal + * to {@code startOffset}. This is needed so that calls to {@code valueAtOrBefore} and + * {@code lastEntry} always return a non-empty value if a value was previously added to this + * object. + * + * @param startOffset the starting offset + */ public void trimPrefixTo(long startOffset); + /** + * Removes all of the values from this object. + */ public void clear(); final public static class Entry { @@ -57,10 +98,10 @@ public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; - Entry that = (Entry) o; + Entry that = (Entry) o; if (offset != that.offset) return false; - if (value != that.value) return false; + if (!Objects.equals(value, that.value)) return false; return true; } diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java b/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java index 46c55ae06e05..d62bf1ab8985 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java @@ -150,7 +150,7 @@ public Optional voterSetAtOffset(long offset) { } synchronized (voterSetHistory) { - return voterSetHistory.valueAt(offset); + return voterSetHistory.valueAtOrBefore(offset); } } @@ -173,7 +173,7 @@ public short kraftVersionAtOffset(long offset) { } synchronized (kraftVersionHistory) { - return kraftVersionHistory.valueAt(offset).orElse((short) 0); + return kraftVersionHistory.valueAtOrBefore(offset).orElse((short) 0); } } diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/TreeMapHistory.java b/raft/src/main/java/org/apache/kafka/raft/internals/TreeMapHistory.java index dcabef2eb1bf..9fa1054f74f3 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/TreeMapHistory.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/TreeMapHistory.java @@ -29,7 +29,13 @@ final public class TreeMapHistory implements History { private final NavigableMap history = new TreeMap<>(); @Override - public void addAt(long offset, T newValue) { + public void addAt(long offset, T value) { + if (offset < 0) { + throw new IllegalArgumentException( + String.format("Next offset %d must be greater than or equal to 0", offset) + ); + } + Map.Entry lastEntry = history.lastEntry(); if (lastEntry != null && offset <= lastEntry.getKey()) { throw new IllegalArgumentException( @@ -37,22 +43,11 @@ public void addAt(long offset, T newValue) { ); } - history.compute( - offset, - (key, oldValue) -> { - if (oldValue != null) { - throw new IllegalArgumentException( - String.format("Rejected %s since a value already exist at %d: %s", newValue, offset, oldValue) - ); - } - - return newValue; - } - ); + history.put(offset, value); } @Override - public Optional valueAt(long offset) { + public Optional valueAtOrBefore(long offset) { return Optional.ofNullable(history.floorEntry(offset)).map(Map.Entry::getValue); } @@ -68,7 +63,7 @@ public void truncateTo(long endOffset) { @Override public void trimPrefixTo(long startOffset) { - NavigableMap lesserValues = history.headMap(startOffset, false); + NavigableMap lesserValues = history.headMap(startOffset, true); while (lesserValues.size() > 1) { // Poll and ignore the entry to remove the first entry lesserValues.pollFirstEntry(); diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java index 13b1fd61392e..57c47c31b42a 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java @@ -67,8 +67,8 @@ public void addAt(long offset, VoterSet voters) { * @return the voter set if one exist, otherwise {@code Optional.empty()} */ @Override - public Optional valueAt(long offset) { - return votersHistory.valueAt(offset); + public Optional valueAtOrBefore(long offset) { + return votersHistory.valueAtOrBefore(offset); } @Override diff --git a/raft/src/main/java/org/apache/kafka/snapshot/NotifyingRawSnapshotWriter.java b/raft/src/main/java/org/apache/kafka/snapshot/NotifyingRawSnapshotWriter.java index e22322f0e85e..f0085e5fbcfc 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/NotifyingRawSnapshotWriter.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/NotifyingRawSnapshotWriter.java @@ -21,13 +21,20 @@ import org.apache.kafka.common.record.UnalignedMemoryRecords; import org.apache.kafka.raft.OffsetAndEpoch; -// TODO: documen this // TODO: write tests +/** + * A thin facade for a {@code RawSnapshotWriter} that notifies a callback when freeze completes successfully. + */ public final class NotifyingRawSnapshotWriter implements RawSnapshotWriter { private final RawSnapshotWriter writer; private final Consumer callback; - // TODO: document that this type owns the write and will close it. + /** + * Constructs a {@code RawSnapshotWriter}. + * + * @param writer the raw snapshot writer + * @param callback the consumer to call when freeze succeeds + */ public NotifyingRawSnapshotWriter(RawSnapshotWriter writer, Consumer callback) { this.writer = writer; this.callback = callback; @@ -58,6 +65,9 @@ public boolean isFrozen() { return writer.isFrozen(); } + /** + * Delegates the call to the internal writer and invokes the callback on success. + */ @Override public void freeze() { writer.freeze(); diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java index 93fabeb6a548..143a7aae05ab 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java @@ -46,8 +46,8 @@ void testStaicVoterSet() { } private void validateStaticVoterSet(VoterSet expected, VoterSetHistory votersHistory) { - assertEquals(Optional.empty(), votersHistory.valueAt(0)); - assertEquals(Optional.empty(), votersHistory.valueAt(100)); + assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(0)); + assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(100)); assertEquals(expected, votersHistory.lastValue()); } From 1829e06d29690722811259d500994a2b1a880458 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Sun, 14 Apr 2024 15:08:56 -0700 Subject: [PATCH 23/34] KAFKA-16207; Implement some of the unittest --- .../main/scala/kafka/server/KafkaServer.scala | 3 +- .../apache/kafka/raft/KafkaRaftClient.java | 9 +- .../raft/internals/InternalLogListener.java | 1 - .../kafka/raft/internals/TreeMapHistory.java | 1 - .../apache/kafka/raft/internals/VoterSet.java | 51 +++++++- .../raft/internals/TreeMapHistoryTest.java | 102 ++++++++++++++++ .../raft/internals/VoterSetHistoryTest.java | 30 +---- .../kafka/raft/internals/VoterSetTest.java | 111 ++++++++++++++++++ 8 files changed, 266 insertions(+), 42 deletions(-) create mode 100644 raft/src/test/java/org/apache/kafka/raft/internals/TreeMapHistoryTest.java create mode 100644 raft/src/test/java/org/apache/kafka/raft/internals/VoterSetTest.java diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 0e545ea9633c..bc90134bc89d 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -331,7 +331,7 @@ class KafkaServer( config.brokerId, config.interBrokerProtocolVersion, brokerFeatures, - kraftControllerNodes, // TODO: This needs to use the raft manager + kraftControllerNodes, config.migrationEnabled) val controllerNodeProvider = new MetadataCacheControllerNodeProvider(metadataCache, config) @@ -436,7 +436,6 @@ class KafkaServer( fatalFaultHandler = new LoggingFaultHandler("raftManager", () => shutdown()) ) val controllerNodes = RaftConfig.voterConnectionsToNodes(quorumVoters).asScala - // TODO: This needs to use the raft manager val quorumControllerNodeProvider = RaftControllerNodeProvider(raftManager, config, controllerNodes) val brokerToQuorumChannelManager = new NodeToControllerChannelManagerImpl( controllerNodeProvider = quorumControllerNodeProvider, 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 4232a782513c..72e1d2cfad67 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -381,7 +381,6 @@ public void initialize( logger.info("Reading KRaft snapshot and log as part of the initialization"); internalListener.updateListener(); - // TODO: Fix this to use internal listener requestManager = new RequestManager( internalListener.lastVoterSet().voterIds(), raftConfig.retryBackoffMs(), @@ -389,7 +388,6 @@ public void initialize( random ); - // TODO: Fix this to use internal listener quorum = new QuorumState( nodeId, internalListener.lastVoterSet().voterIds(), @@ -406,8 +404,6 @@ public void initialize( // so there are no unknown voter connections. Report this metric as 0. kafkaRaftMetrics.updateNumUnknownVoterConnections(0); - // TODO: Create issue to fix this. RaftRequest.Outbound should contain the node not just the id - // we need this because the controller.quorum.bootstrap.server doesn't contain an id. VoterSet lastVoterSet = internalListener.lastVoterSet(); for (Integer voterId : lastVoterSet.voterIds()) { channel.updateEndpoint(voterId, lastVoterSet.voterAddress(voterId, listenerName).get()); @@ -1523,9 +1519,8 @@ private boolean handleFetchSnapshotResponse( quorum.leaderIdOrSentinel() ); - // TODO: are we guarantee to always load a snapshot? This always true, if the next offset is always less - // than the snapshot id. Is this true? I think this always true because the internal listener is always - // at the LEO and we now that the LEO is less that the snapshot id. + // This will aways reload the snapshot because the internal listener's next offset + // is always less than the snapshot id just downloaded. internalListener.updateListener(); updateFollowerHighWatermark(state, OptionalLong.of(log.highWatermark().offset)); diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java b/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java index d62bf1ab8985..32367562b488 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java @@ -33,7 +33,6 @@ import org.apache.kafka.snapshot.SnapshotReader; import org.slf4j.Logger; -// TODO: File an issue to remove the RecordSerde. The internal listener should just skip data record batches // TODO: Add unnitest for it /** * The KRaft state machine for tracking control records in the topic partition. diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/TreeMapHistory.java b/raft/src/main/java/org/apache/kafka/raft/internals/TreeMapHistory.java index 9fa1054f74f3..32894be79fe7 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/TreeMapHistory.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/TreeMapHistory.java @@ -21,7 +21,6 @@ import java.util.TreeMap; import java.util.Map; -// TODO: add unittest /** * A implementation for {@code History} which uses a red-black tree to store values sorted by offset. */ diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java index d74c63b39572..4fdaa950d28c 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java @@ -31,7 +31,6 @@ import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.feature.SupportedVersionRange; -// TODO: write unittest for VoterSet /** * A type for representing the set of voters for a topic partition. * @@ -44,6 +43,10 @@ final public class VoterSet { private final Map voters; VoterSet(Map voters) { + if (voters.isEmpty()) { + throw new IllegalArgumentException("Voters cannot be empty"); + } + this.voters = voters; } @@ -66,6 +69,50 @@ public Set voterIds() { return voters.keySet(); } + /** + * Adds a voter to the voter set. + * + * This object is immutable. A new voter set is returned if the voter was added. + * + * A new voter can be added to a voter set if its id doesn't already exist in the voter set. + * + * @param voter the new voter to add + * @return a new voter set if the voter was added, otherwise {@code Optional.empty()} + */ + public Optional addVoter(VoterNode voter) { + if (voters.containsKey(voter.id())) { + return Optional.empty(); + } + + HashMap newVoters = new HashMap<>(voters); + newVoters.put(voter.id(), voter); + + return Optional.of(new VoterSet(newVoters)); + } + + /** + * Removew a voter from the voter set. + * + * This object is immutable. A new voter set is returned if the voter was removed. + * + * A voter can be removed from the voter set if its id and uuid match. + * + * @param voterId the voter id + * @param voterUuid the voter uuid + * @return a new voter set if the voter was remove, otherwise {@code Optional.empty()} + */ + public Optional removeVoter(int voterId, Optional voterUuid) { + VoterNode oldVoter = voters.get(voterId); + if (oldVoter != null && Objects.equals(oldVoter.uuid(), voterUuid)) { + HashMap newVoters = new HashMap<>(voters); + newVoters.remove(voterId); + + return Optional.of(new VoterSet(newVoters)); + } + + return Optional.empty(); + } + /** * Converts a voter set to a voters record for a given version. * @@ -222,7 +269,7 @@ public String toString() { * @return the voter set */ public static VoterSet fromVotersRecord(VotersRecord voters) { - Map voterNodes = new HashMap<>(voters.voters().size()); + HashMap voterNodes = new HashMap<>(voters.voters().size()); for (VotersRecord.Voter voter: voters.voters()) { final Optional uuid; if (!voter.voterUuid().equals(Uuid.ZERO_UUID)) { diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/TreeMapHistoryTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/TreeMapHistoryTest.java new file mode 100644 index 000000000000..4537ff2e1fb9 --- /dev/null +++ b/raft/src/test/java/org/apache/kafka/raft/internals/TreeMapHistoryTest.java @@ -0,0 +1,102 @@ +/* + * 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.util.Optional; +import org.junit.jupiter.api.Test; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +final public class TreeMapHistoryTest { + @Test + void testEmpty() { + TreeMapHistory history = new TreeMapHistory<>(); + assertEquals(Optional.empty(), history.valueAtOrBefore(100)); + assertEquals(Optional.empty(), history.lastEntry()); + } + + @Test + void testAddAt() { + TreeMapHistory history = new TreeMapHistory<>(); + assertThrows(IllegalArgumentException.class, () -> history.addAt(-1, "")); + assertEquals(Optional.empty(), history.lastEntry()); + + history.addAt(100, "100"); + assertThrows(IllegalArgumentException.class, () -> history.addAt(99, "")); + assertThrows(IllegalArgumentException.class, () -> history.addAt(100, "")); + + history.addAt(200, "200"); + assertEquals(Optional.empty(), history.valueAtOrBefore(99)); + assertEquals(Optional.of("100"), history.valueAtOrBefore(100)); + assertEquals(Optional.of("100"), history.valueAtOrBefore(101)); + assertEquals(Optional.of("100"), history.valueAtOrBefore(199)); + assertEquals(Optional.of("200"), history.valueAtOrBefore(200)); + assertEquals(Optional.of("200"), history.valueAtOrBefore(201)); + assertEquals(Optional.of(new History.Entry<>(200, "200")), history.lastEntry()); + } + + @Test + void testTruncateTo() { + TreeMapHistory history = new TreeMapHistory<>(); + history.addAt(100, "100"); + history.addAt(200, "200"); + + history.truncateTo(201); + assertEquals(Optional.of(new History.Entry<>(200, "200")), history.lastEntry()); + + history.truncateTo(200); + assertEquals(Optional.of(new History.Entry<>(100, "100")), history.lastEntry()); + + history.truncateTo(101); + assertEquals(Optional.of(new History.Entry<>(100, "100")), history.lastEntry()); + + history.truncateTo(100); + assertEquals(Optional.empty(), history.lastEntry()); + } + + @Test + void testTrimPrefixTo() { + TreeMapHistory history = new TreeMapHistory<>(); + history.addAt(100, "100"); + history.addAt(200, "200"); + + history.trimPrefixTo(99); + assertEquals(Optional.empty(), history.valueAtOrBefore(99)); + assertEquals(Optional.of("100"), history.valueAtOrBefore(100)); + + history.trimPrefixTo(100); + assertEquals(Optional.empty(), history.valueAtOrBefore(99)); + assertEquals(Optional.of("100"), history.valueAtOrBefore(100)); + + history.trimPrefixTo(101); + assertEquals(Optional.empty(), history.valueAtOrBefore(99)); + assertEquals(Optional.of("100"), history.valueAtOrBefore(100)); + + history.trimPrefixTo(200); + assertEquals(Optional.empty(), history.valueAtOrBefore(199)); + assertEquals(Optional.of("200"), history.valueAtOrBefore(200)); + } + + @Test + void testClear() { + TreeMapHistory history = new TreeMapHistory<>(); + history.addAt(100, "100"); + history.addAt(200, "200"); + history.clear(); + assertEquals(Optional.empty(), history.lastEntry()); + } +} diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java index 143a7aae05ab..5830c50dfd5e 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java @@ -16,22 +16,15 @@ */ package org.apache.kafka.raft.internals; -import java.net.InetSocketAddress; import java.util.Arrays; -import java.util.Collections; -import java.util.List; import java.util.Optional; -import java.util.function.Function; -import java.util.stream.Collectors; -import org.apache.kafka.common.Uuid; -import org.apache.kafka.common.feature.SupportedVersionRange; import org.junit.jupiter.api.Test; import static org.junit.jupiter.api.Assertions.assertEquals; final public class VoterSetHistoryTest { @Test void testStaicVoterSet() { - VoterSet staticVoterSet = createVoterSet(Arrays.asList(1, 2, 3)); + VoterSet staticVoterSet = new VoterSet(VoterSetTest.voterMap(Arrays.asList(1, 2, 3))); VoterSetHistory votersHistory = new VoterSetHistory(Optional.of(staticVoterSet)); validateStaticVoterSet(staticVoterSet, votersHistory); @@ -50,25 +43,4 @@ private void validateStaticVoterSet(VoterSet expected, VoterSetHistory votersHis assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(100)); assertEquals(expected, votersHistory.lastValue()); } - - private VoterSet createVoterSet(List replicas) { - return new VoterSet( - replicas - .stream() - .collect( - Collectors.toMap( - Function.identity(), - replica -> new VoterSet.VoterNode( - replica, - Optional.of(Uuid.randomUuid()), - Collections.singletonMap( - "LISTNER", - InetSocketAddress.createUnresolved(String.format("replica-%d", replica), 1234) - ), - new SupportedVersionRange((short) 0, (short) 0) - ) - ) - ) - ); - } } diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetTest.java new file mode 100644 index 000000000000..2c5b5575958f --- /dev/null +++ b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetTest.java @@ -0,0 +1,111 @@ +/* + * 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.net.InetSocketAddress; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.feature.SupportedVersionRange; +import org.junit.jupiter.api.Test; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +final public class VoterSetTest { + @Test + void testEmptyVoterSet() { + assertThrows(IllegalArgumentException.class, () -> new VoterSet(Collections.emptyMap())); + } + + @Test + void testVoterAddress() { + VoterSet voterSet = new VoterSet(voterMap(Arrays.asList(1, 2, 3))); + assertEquals(Optional.of(new InetSocketAddress("replica-1", 1234)), voterSet.voterAddress(1, "LISTENER")); + assertEquals(Optional.empty(), voterSet.voterAddress(1, "MISSING")); + assertEquals(Optional.empty(), voterSet.voterAddress(4, "LISTENER")); + } + + @Test + void testVoterIds() { + VoterSet voterSet = new VoterSet(voterMap(Arrays.asList(1, 2, 3))); + assertEquals(new HashSet<>(Arrays.asList(1, 2, 3)), voterSet.voterIds()); + } + + @Test + void testAddVoter() { + Map aVoterMap = voterMap(Arrays.asList(1, 2, 3)); + VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap)); + + assertEquals(Optional.empty(), voterSet.addVoter(voterNode(1))); + + VoterSet.VoterNode voter4 = voterNode(4); + aVoterMap.put(voter4.id(), voter4); + assertEquals(Optional.of(new VoterSet(new HashMap<>(aVoterMap))), voterSet.addVoter(voter4)); + } + + @Test + void testRemoveVoter() { + Map aVoterMap = voterMap(Arrays.asList(1, 2, 3)); + VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap)); + + assertEquals(Optional.empty(), voterSet.removeVoter(4, Optional.empty())); + assertEquals(Optional.empty(), voterSet.removeVoter(4, Optional.of(Uuid.randomUuid()))); + + VoterSet.VoterNode voter3 = aVoterMap.remove(3); + assertEquals( + Optional.of(new VoterSet(new HashMap<>(aVoterMap))), + voterSet.removeVoter(voter3.id(), voter3.uuid()) + ); + } + + @Test + void testRecordRoundTrip() { + VoterSet voterSet = new VoterSet(voterMap(Arrays.asList(1, 2, 3))); + + assertEquals(voterSet, VoterSet.fromVotersRecord(voterSet.toVotersRecord((short) 0))); + } + + static Map voterMap(List replicas) { + return replicas + .stream() + .collect( + Collectors.toMap( + Function.identity(), + VoterSetTest::voterNode + ) + ); + } + + static VoterSet.VoterNode voterNode(int id) { + return new VoterSet.VoterNode( + id, + Optional.of(Uuid.randomUuid()), + Collections.singletonMap( + "LISTENER", + InetSocketAddress.createUnresolved(String.format("replica-%d", id), 1234) + ), + new SupportedVersionRange((short) 0, (short) 0) + ); + } +} From a108e9caae2cde7df9147bdfbe7349ca5a96c584 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Mon, 15 Apr 2024 13:37:40 -0700 Subject: [PATCH 24/34] KAFKA-16207; Add 2 more test suites --- .../kafka/raft/internals/VoterSetHistory.java | 1 - .../snapshot/NotifyingRawSnapshotWriter.java | 1 - .../raft/internals/TreeMapHistoryTest.java | 2 + .../raft/internals/VoterSetHistoryTest.java | 131 +++++++++++++++++- .../NotifyingRawSnapshotWriterTest.java | 123 ++++++++++++++++ 5 files changed, 251 insertions(+), 7 deletions(-) create mode 100644 raft/src/test/java/org/apache/kafka/snapshot/NotifyingRawSnapshotWriterTest.java diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java index 57c47c31b42a..3deef3e25281 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java @@ -18,7 +18,6 @@ import java.util.Optional; -// TODO: write unittest for VoterSetHistory /** * A type for storing the historical value of the set of voters. * diff --git a/raft/src/main/java/org/apache/kafka/snapshot/NotifyingRawSnapshotWriter.java b/raft/src/main/java/org/apache/kafka/snapshot/NotifyingRawSnapshotWriter.java index f0085e5fbcfc..0ffba500953e 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/NotifyingRawSnapshotWriter.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/NotifyingRawSnapshotWriter.java @@ -21,7 +21,6 @@ import org.apache.kafka.common.record.UnalignedMemoryRecords; import org.apache.kafka.raft.OffsetAndEpoch; -// TODO: write tests /** * A thin facade for a {@code RawSnapshotWriter} that notifies a callback when freeze completes successfully. */ diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/TreeMapHistoryTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/TreeMapHistoryTest.java index 4537ff2e1fb9..5503dd2b2879 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/TreeMapHistoryTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/TreeMapHistoryTest.java @@ -38,6 +38,8 @@ void testAddAt() { history.addAt(100, "100"); assertThrows(IllegalArgumentException.class, () -> history.addAt(99, "")); assertThrows(IllegalArgumentException.class, () -> history.addAt(100, "")); + assertEquals(Optional.of("100"), history.valueAtOrBefore(100)); + assertEquals(Optional.of("100"), history.valueAtOrBefore(201)); history.addAt(200, "200"); assertEquals(Optional.empty(), history.valueAtOrBefore(99)); diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java index 5830c50dfd5e..29df600efd1b 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java @@ -17,9 +17,12 @@ package org.apache.kafka.raft.internals; import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; import java.util.Optional; import org.junit.jupiter.api.Test; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; final public class VoterSetHistoryTest { @Test @@ -27,20 +30,138 @@ void testStaicVoterSet() { VoterSet staticVoterSet = new VoterSet(VoterSetTest.voterMap(Arrays.asList(1, 2, 3))); VoterSetHistory votersHistory = new VoterSetHistory(Optional.of(staticVoterSet)); - validateStaticVoterSet(staticVoterSet, votersHistory); + assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(0)); + assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(100)); + assertEquals(staticVoterSet, votersHistory.lastValue()); // Should be a no-op votersHistory.truncateTo(100); - validateStaticVoterSet(staticVoterSet, votersHistory); + assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(0)); + assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(100)); + assertEquals(staticVoterSet, votersHistory.lastValue()); // Should be a no-op votersHistory.trimPrefixTo(100); - validateStaticVoterSet(staticVoterSet, votersHistory); + assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(0)); + assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(100)); + assertEquals(staticVoterSet, votersHistory.lastValue()); } - private void validateStaticVoterSet(VoterSet expected, VoterSetHistory votersHistory) { + @Test + void TestNoStaticVoterSet() { + VoterSetHistory votersHistory = new VoterSetHistory(Optional.empty()); + assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(0)); assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(100)); - assertEquals(expected, votersHistory.lastValue()); + assertThrows(IllegalStateException.class, votersHistory::lastValue); + } + + @Test + void testAddAt() { + Map voterMap = VoterSetTest.voterMap(Arrays.asList(1, 2, 3)); + VoterSet staticVoterSet = new VoterSet(new HashMap<>(voterMap)); + VoterSetHistory votersHistory = new VoterSetHistory(Optional.of(staticVoterSet)); + + assertThrows( + IllegalArgumentException.class, + () -> votersHistory.addAt(-1, new VoterSet(VoterSetTest.voterMap(Arrays.asList(1, 2, 3)))) + ); + assertEquals(staticVoterSet, votersHistory.lastValue()); + + voterMap.put(4, VoterSetTest.voterNode(4)); + VoterSet addedVoterSet = new VoterSet(new HashMap<>(voterMap)); + votersHistory.addAt(100, addedVoterSet); + + assertEquals(addedVoterSet, votersHistory.lastValue()); + assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(99)); + assertEquals(Optional.of(addedVoterSet), votersHistory.valueAtOrBefore(100)); + + VoterSet removedVoterSet = new VoterSet(new HashMap<>(voterMap)); + votersHistory.addAt(200, removedVoterSet); + + assertEquals(removedVoterSet, votersHistory.lastValue()); + assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(99)); + assertEquals(Optional.of(addedVoterSet), votersHistory.valueAtOrBefore(199)); + assertEquals(Optional.of(removedVoterSet), votersHistory.valueAtOrBefore(200)); + } + + @Test + void testTruncateTo() { + Map voterMap = VoterSetTest.voterMap(Arrays.asList(1, 2, 3)); + VoterSet staticVoterSet = new VoterSet(new HashMap<>(voterMap)); + VoterSetHistory votersHistory = new VoterSetHistory(Optional.of(staticVoterSet)); + + // Add voter 4 to the voter set and voter set history + voterMap.put(4, VoterSetTest.voterNode(4)); + VoterSet voterSet1234 = new VoterSet(new HashMap<>(voterMap)); + votersHistory.addAt(100, voterSet1234); + + // Add voter 5 to the voter set and voter set history + voterMap.put(5, VoterSetTest.voterNode(5)); + VoterSet voterSet12345 = new VoterSet(new HashMap<>(voterMap)); + votersHistory.addAt(200, voterSet12345); + + votersHistory.truncateTo(201); + assertEquals(Optional.of(new History.Entry<>(200, voterSet12345)), votersHistory.lastEntry()); + votersHistory.truncateTo(200); + assertEquals(Optional.of(new History.Entry<>(100, voterSet1234)), votersHistory.lastEntry()); + votersHistory.truncateTo(101); + assertEquals(Optional.of(new History.Entry<>(100, voterSet1234)), votersHistory.lastEntry()); + votersHistory.truncateTo(100); + assertEquals(Optional.of(new History.Entry<>(-1, staticVoterSet)), votersHistory.lastEntry()); + } + + @Test + void testTrimPrefixTo() { + Map voterMap = VoterSetTest.voterMap(Arrays.asList(1, 2, 3)); + VoterSet staticVoterSet = new VoterSet(new HashMap<>(voterMap)); + VoterSetHistory votersHistory = new VoterSetHistory(Optional.of(staticVoterSet)); + + // Add voter 4 to the voter set and voter set history + voterMap.put(4, VoterSetTest.voterNode(4)); + VoterSet voterSet1234 = new VoterSet(new HashMap<>(voterMap)); + votersHistory.addAt(100, voterSet1234); + + // Add voter 5 to the voter set and voter set history + voterMap.put(5, VoterSetTest.voterNode(5)); + VoterSet voterSet12345 = new VoterSet(new HashMap<>(voterMap)); + votersHistory.addAt(200, voterSet12345); + + votersHistory.trimPrefixTo(99); + assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(99)); + assertEquals(Optional.of(voterSet1234), votersHistory.valueAtOrBefore(100)); + + votersHistory.trimPrefixTo(100); + assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(99)); + assertEquals(Optional.of(voterSet1234), votersHistory.valueAtOrBefore(100)); + + votersHistory.trimPrefixTo(101); + assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(99)); + assertEquals(Optional.of(voterSet1234), votersHistory.valueAtOrBefore(100)); + + votersHistory.trimPrefixTo(200); + assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(199)); + assertEquals(Optional.of(voterSet12345), votersHistory.valueAtOrBefore(200)); + } + + @Test + void testClear() { + Map voterMap = VoterSetTest.voterMap(Arrays.asList(1, 2, 3)); + VoterSet staticVoterSet = new VoterSet(new HashMap<>(voterMap)); + VoterSetHistory votersHistory = new VoterSetHistory(Optional.of(staticVoterSet)); + + // Add voter 4 to the voter set and voter set history + voterMap.put(4, VoterSetTest.voterNode(4)); + VoterSet voterSet1234 = new VoterSet(new HashMap<>(voterMap)); + votersHistory.addAt(100, voterSet1234); + + // Add voter 5 to the voter set and voter set history + voterMap.put(5, VoterSetTest.voterNode(5)); + VoterSet voterSet12345 = new VoterSet(new HashMap<>(voterMap)); + votersHistory.addAt(200, voterSet12345); + + votersHistory.clear(); + + assertEquals(Optional.of(new History.Entry<>(-1, staticVoterSet)), votersHistory.lastEntry()); } } diff --git a/raft/src/test/java/org/apache/kafka/snapshot/NotifyingRawSnapshotWriterTest.java b/raft/src/test/java/org/apache/kafka/snapshot/NotifyingRawSnapshotWriterTest.java new file mode 100644 index 000000000000..4b9232134cc8 --- /dev/null +++ b/raft/src/test/java/org/apache/kafka/snapshot/NotifyingRawSnapshotWriterTest.java @@ -0,0 +1,123 @@ +/* + * 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.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Consumer; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.UnalignedMemoryRecords; +import org.apache.kafka.raft.OffsetAndEpoch; +import org.junit.jupiter.api.Test; +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; + +final class NotifyingRawSnapshotWriterTest { + @Test + void testFreezeClose() { + NoopRawSnapshotWriter rawWriter = new NoopRawSnapshotWriter(); + + AtomicBoolean called = new AtomicBoolean(false); + Consumer consumer = offset -> { + assertEquals(offset, rawWriter.snapshotId()); + called.set(true); + }; + + try (RawSnapshotWriter writer = new NotifyingRawSnapshotWriter(rawWriter, consumer)) { + writer.freeze(); + } + + assertTrue(called.get()); + assertTrue(rawWriter.isFrozen()); + assertTrue(rawWriter.closed); + } + + @Test + void testFailingFreeze() { + NoopRawSnapshotWriter rawWriter = new NoopRawSnapshotWriter() { + @Override + public void freeze() { + throw new IllegalStateException(); + } + }; + + AtomicBoolean called = new AtomicBoolean(false); + Consumer consumer = ignored -> called.set(true); + + try (RawSnapshotWriter writer = new NotifyingRawSnapshotWriter(rawWriter, consumer)) { + assertThrows(IllegalStateException.class, writer::freeze); + } + + assertFalse(called.get()); + assertFalse(rawWriter.isFrozen()); + assertTrue(rawWriter.closed); + } + + @Test + void testCloseWithoutFreeze() { + NoopRawSnapshotWriter rawWriter = new NoopRawSnapshotWriter(); + + AtomicBoolean called = new AtomicBoolean(false); + Consumer consumer = offset -> called.set(true); + + try (RawSnapshotWriter writer = new NotifyingRawSnapshotWriter(rawWriter, consumer)) { + } + + assertFalse(called.get()); + assertFalse(rawWriter.isFrozen()); + assertTrue(rawWriter.closed); + } + + class NoopRawSnapshotWriter implements RawSnapshotWriter { + boolean frozen = false; + boolean closed = false; + + @Override + public OffsetAndEpoch snapshotId() { + return new OffsetAndEpoch(100, 10); + } + + @Override + public long sizeInBytes() { + return 255; + } + + @Override + public void append(UnalignedMemoryRecords records) { + } + + @Override + public void append(MemoryRecords records) { + } + + @Override + public boolean isFrozen() { + return frozen; + } + + @Override + public void freeze() { + frozen = true; + } + + @Override + public void close() { + closed = true; + } + } +} From 795858f319b497af3148e782341b8225c69f5774 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Mon, 15 Apr 2024 19:04:28 -0700 Subject: [PATCH 25/34] KAFKA-16207; Make the control batch buffer configurable --- .../raft/internals/BatchAccumulator.java | 2 +- .../kafka/raft/KafkaRaftClientTest.java | 25 +++---- .../raft/internals/BatchAccumulatorTest.java | 4 +- .../raft/internals/RecordsIteratorTest.java | 66 ++++++++++++++++++- 4 files changed, 76 insertions(+), 21 deletions(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java b/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java index 74db8fc99a32..d966ae2ac50a 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java @@ -218,7 +218,7 @@ private void completeCurrentBatch() { public void appendControlMessages(Function valueCreator) { appendLock.lock(); try { - ByteBuffer buffer = memoryPool.tryAllocate(256); + ByteBuffer buffer = memoryPool.tryAllocate(maxBatchSize); if (buffer != null) { try { forceDrain(); 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 f18a0ff2e882..1b1d8fc9db75 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -303,12 +303,11 @@ public void testAppendFailedWithBufferAllocationException() throws Exception { Set voters = Utils.mkSet(localId, otherNodeId); MemoryPool memoryPool = Mockito.mock(MemoryPool.class); - ByteBuffer leaderBuffer = ByteBuffer.allocate(256); + ByteBuffer buffer = ByteBuffer.allocate(KafkaRaftClient.MAX_BATCH_SIZE_BYTES); // Return null when allocation error Mockito.when(memoryPool.tryAllocate(KafkaRaftClient.MAX_BATCH_SIZE_BYTES)) - .thenReturn(null); - Mockito.when(memoryPool.tryAllocate(256)) - .thenReturn(leaderBuffer); + .thenReturn(buffer) // Buffer for the leader message control record + .thenReturn(null); // Buffer for the scheduleAppend call RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withMemoryPool(memoryPool) @@ -319,6 +318,7 @@ public void testAppendFailedWithBufferAllocationException() throws Exception { int epoch = context.currentEpoch(); assertThrows(BufferAllocationException.class, () -> context.client.scheduleAppend(epoch, singletonList("a"))); + Mockito.verify(memoryPool).release(buffer); } @Test @@ -881,11 +881,8 @@ public void testAccumulatorClearedAfterBecomingFollower() throws Exception { MemoryPool memoryPool = Mockito.mock(MemoryPool.class); ByteBuffer buffer = ByteBuffer.allocate(KafkaRaftClient.MAX_BATCH_SIZE_BYTES); - ByteBuffer leaderBuffer = ByteBuffer.allocate(256); Mockito.when(memoryPool.tryAllocate(KafkaRaftClient.MAX_BATCH_SIZE_BYTES)) .thenReturn(buffer); - Mockito.when(memoryPool.tryAllocate(256)) - .thenReturn(leaderBuffer); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withAppendLingerMs(lingerMs) @@ -901,7 +898,8 @@ public void testAccumulatorClearedAfterBecomingFollower() throws Exception { context.pollUntilResponse(); context.assertElectedLeader(epoch + 1, otherNodeId); - Mockito.verify(memoryPool).release(buffer); + // Expect two calls one for the leader change control batch and one for the data batch + Mockito.verify(memoryPool, Mockito.times(2)).release(buffer); } @Test @@ -913,11 +911,8 @@ public void testAccumulatorClearedAfterBecomingVoted() throws Exception { MemoryPool memoryPool = Mockito.mock(MemoryPool.class); ByteBuffer buffer = ByteBuffer.allocate(KafkaRaftClient.MAX_BATCH_SIZE_BYTES); - ByteBuffer leaderBuffer = ByteBuffer.allocate(256); Mockito.when(memoryPool.tryAllocate(KafkaRaftClient.MAX_BATCH_SIZE_BYTES)) .thenReturn(buffer); - Mockito.when(memoryPool.tryAllocate(256)) - .thenReturn(leaderBuffer); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withAppendLingerMs(lingerMs) @@ -934,7 +929,7 @@ public void testAccumulatorClearedAfterBecomingVoted() throws Exception { context.pollUntilResponse(); context.assertVotedCandidate(epoch + 1, otherNodeId); - Mockito.verify(memoryPool).release(buffer); + Mockito.verify(memoryPool, Mockito.times(2)).release(buffer); } @Test @@ -946,11 +941,8 @@ public void testAccumulatorClearedAfterBecomingUnattached() throws Exception { MemoryPool memoryPool = Mockito.mock(MemoryPool.class); ByteBuffer buffer = ByteBuffer.allocate(KafkaRaftClient.MAX_BATCH_SIZE_BYTES); - ByteBuffer leaderBuffer = ByteBuffer.allocate(256); Mockito.when(memoryPool.tryAllocate(KafkaRaftClient.MAX_BATCH_SIZE_BYTES)) .thenReturn(buffer); - Mockito.when(memoryPool.tryAllocate(256)) - .thenReturn(leaderBuffer); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withAppendLingerMs(lingerMs) @@ -966,7 +958,8 @@ public void testAccumulatorClearedAfterBecomingUnattached() throws Exception { context.pollUntilResponse(); context.assertUnknownLeader(epoch + 1); - Mockito.verify(memoryPool).release(buffer); + // Expect two calls one for the leader change control batch and one for the data batch + Mockito.verify(memoryPool, Mockito.times(2)).release(buffer); } @Test diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/BatchAccumulatorTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/BatchAccumulatorTest.java index 70ae8cc3e209..8cc7de59273a 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/BatchAccumulatorTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/BatchAccumulatorTest.java @@ -78,8 +78,8 @@ public void testLeaderChangeMessageWritten() { int lingerMs = 50; int maxBatchSize = 512; - ByteBuffer buffer = ByteBuffer.allocate(256); - Mockito.when(memoryPool.tryAllocate(256)) + ByteBuffer buffer = ByteBuffer.allocate(maxBatchSize); + Mockito.when(memoryPool.tryAllocate(maxBatchSize)) .thenReturn(buffer); BatchAccumulator acc = buildAccumulator( 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 a07b65162e81..78cce7bf639e 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,10 +21,12 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.IdentityHashMap; import java.util.List; import java.util.NoSuchElementException; import java.util.Objects; +import java.util.Optional; import java.util.Random; import java.util.Set; import java.util.concurrent.atomic.AtomicReference; @@ -151,10 +153,13 @@ public void testCrcValidation( } @Test - public void testControlRecordIteration() { + public void testControlRecordIterationWithKraftVerion0() { AtomicReference buffer = new AtomicReference<>(null); + VoterSet voterSet = new VoterSet(new HashMap<>(VoterSetTest.voterMap(Arrays.asList(1, 2, 3)))); RecordsSnapshotWriter.Builder builder = new RecordsSnapshotWriter.Builder() .setTime(new MockTime()) + .setKraftVersion((short) 0) + .setVoterSet(Optional.empty()) .setRawSnapshotWriter( new MockRawSnapshotWriter(new OffsetAndEpoch(100, 10), snapshotBuf -> buffer.set(snapshotBuf)) ); @@ -171,13 +176,70 @@ public void testControlRecordIteration() { true ) ) { - // Check snapshot header control record + // Consume the control record batch Batch batch = iterator.next(); + assertEquals(1, batch.controlRecords().size()); + + // Check snapshot header control record + assertEquals(ControlRecordType.SNAPSHOT_HEADER, batch.controlRecords().get(0).type()); + assertEquals(new SnapshotHeaderRecord(), batch.controlRecords().get(0).message()); + + // Consume the iterator until we find a control record + do { + batch = iterator.next(); + } + while (batch.controlRecords().isEmpty()); + // Check snapshot footer control record assertEquals(1, batch.controlRecords().size()); + assertEquals(ControlRecordType.SNAPSHOT_FOOTER, batch.controlRecords().get(0).type()); + assertEquals(new SnapshotFooterRecord(), batch.controlRecords().get(0).message()); + + // Snapshot footer must be last record + assertFalse(iterator.hasNext()); + } + } + + @Test + public void testControlRecordIterationWithKraftVerion1() { + AtomicReference buffer = new AtomicReference<>(null); + VoterSet voterSet = new VoterSet(new HashMap<>(VoterSetTest.voterMap(Arrays.asList(1, 2, 3)))); + RecordsSnapshotWriter.Builder builder = new RecordsSnapshotWriter.Builder() + .setTime(new MockTime()) + .setKraftVersion((short) 1) + .setVoterSet(Optional.of(voterSet)) + .setRawSnapshotWriter( + new MockRawSnapshotWriter(new OffsetAndEpoch(100, 10), snapshotBuf -> buffer.set(snapshotBuf)) + ); + try (RecordsSnapshotWriter snapshot = builder.build(STRING_SERDE)) { + snapshot.append(Arrays.asList("a", "b", "c")); + snapshot.append(Arrays.asList("d", "e", "f")); + snapshot.append(Arrays.asList("g", "h", "i")); + snapshot.freeze(); + } + + try (RecordsIterator iterator = createIterator( + MemoryRecords.readableRecords(buffer.get()), + BufferSupplier.NO_CACHING, + true + ) + ) { + // Consume the control record batch + Batch batch = iterator.next(); + assertEquals(3, batch.controlRecords().size()); + + // Check snapshot header control record assertEquals(ControlRecordType.SNAPSHOT_HEADER, batch.controlRecords().get(0).type()); assertEquals(new SnapshotHeaderRecord(), batch.controlRecords().get(0).message()); + // Check kraft version control record + assertEquals(ControlRecordType.KRAFT_VERSION, batch.controlRecords().get(1).type()); + assertEquals(new KRaftVersionRecord().setKRaftVersion((short) 1), batch.controlRecords().get(1).message()); + + // Check the voters control record + assertEquals(ControlRecordType.VOTERS, batch.controlRecords().get(2).type()); + assertEquals(voterSet.toVotersRecord((short) 0), batch.controlRecords().get(2).message()); + // Consume the iterator until we find a control record do { batch = iterator.next(); From a79660eb2361906d7170a3b8d4b26c1945dfffdb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Tue, 16 Apr 2024 09:49:25 -0700 Subject: [PATCH 26/34] KAFKA-16207; Add tests for snapshot builder --- .../kafka/snapshot/RecordsSnapshotWriter.java | 1 - .../raft/internals/RecordsIteratorTest.java | 1 - .../kafka/raft/internals/VoterSetTest.java | 6 +- .../snapshot/RecordsSnapshotWriterTest.java | 223 ++++++++++++++++++ 4 files changed, 228 insertions(+), 3 deletions(-) create mode 100644 raft/src/test/java/org/apache/kafka/snapshot/RecordsSnapshotWriterTest.java diff --git a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java index e3483d0a0063..d67ff9a194cd 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java @@ -141,7 +141,6 @@ private void appendBatches(List> batches) { } } - // TODO: Write test for this builder final public static class Builder { private long lastContainedLogTimestamp = 0; private CompressionType compressionType = CompressionType.NONE; 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 78cce7bf639e..d1dc2ccf6bd7 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 @@ -155,7 +155,6 @@ public void testCrcValidation( @Test public void testControlRecordIterationWithKraftVerion0() { AtomicReference buffer = new AtomicReference<>(null); - VoterSet voterSet = new VoterSet(new HashMap<>(VoterSetTest.voterMap(Arrays.asList(1, 2, 3)))); RecordsSnapshotWriter.Builder builder = new RecordsSnapshotWriter.Builder() .setTime(new MockTime()) .setKraftVersion((short) 0) diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetTest.java index 2c5b5575958f..94481fc968fd 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetTest.java @@ -86,7 +86,7 @@ void testRecordRoundTrip() { assertEquals(voterSet, VoterSet.fromVotersRecord(voterSet.toVotersRecord((short) 0))); } - static Map voterMap(List replicas) { + public static Map voterMap(List replicas) { return replicas .stream() .collect( @@ -108,4 +108,8 @@ static VoterSet.VoterNode voterNode(int id) { new SupportedVersionRange((short) 0, (short) 0) ); } + + public static VoterSet voterSet(Map voters) { + return new VoterSet(voters); + } } diff --git a/raft/src/test/java/org/apache/kafka/snapshot/RecordsSnapshotWriterTest.java b/raft/src/test/java/org/apache/kafka/snapshot/RecordsSnapshotWriterTest.java new file mode 100644 index 000000000000..5bed77d3b88d --- /dev/null +++ b/raft/src/test/java/org/apache/kafka/snapshot/RecordsSnapshotWriterTest.java @@ -0,0 +1,223 @@ +/* + * 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.nio.ByteBuffer; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.kafka.common.message.KRaftVersionRecord; +import org.apache.kafka.common.message.SnapshotFooterRecord; +import org.apache.kafka.common.message.SnapshotHeaderRecord; +import org.apache.kafka.common.record.ControlRecordType; +import org.apache.kafka.common.utils.BufferSupplier; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.raft.Batch; +import org.apache.kafka.raft.OffsetAndEpoch; +import org.apache.kafka.raft.internals.StringSerde; +import org.apache.kafka.raft.internals.VoterSet; +import org.apache.kafka.raft.internals.VoterSetTest; +import org.apache.kafka.server.common.serialization.RecordSerde; +import org.junit.jupiter.api.Test; +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 class RecordsSnapshotWriterTest { + private static final RecordSerde STRING_SERDE = new StringSerde(); + + @Test + void testBuilderKRaftVersion0() { + OffsetAndEpoch snapshotId = new OffsetAndEpoch(100, 10); + int maxBatchSize = 1024; + AtomicReference buffer = new AtomicReference<>(null); + RecordsSnapshotWriter.Builder builder = new RecordsSnapshotWriter.Builder() + .setKraftVersion((short) 0) + .setVoterSet(Optional.empty()) + .setTime(new MockTime()) + .setMaxBatchSize(maxBatchSize) + .setRawSnapshotWriter( + new MockRawSnapshotWriter(snapshotId, snapshotBuf -> buffer.set(snapshotBuf)) + ); + try (RecordsSnapshotWriter snapshot = builder.build(STRING_SERDE)) { + snapshot.freeze(); + } + + try (RecordsSnapshotReader reader = RecordsSnapshotReader.of( + new MockRawSnapshotReader(snapshotId, buffer.get()), + STRING_SERDE, + BufferSupplier.NO_CACHING, + maxBatchSize, + true + ) + ) { + // Consume the control record batch + Batch batch = reader.next(); + assertEquals(1, batch.controlRecords().size()); + + // Check snapshot header control record + assertEquals(ControlRecordType.SNAPSHOT_HEADER, batch.controlRecords().get(0).type()); + assertEquals(new SnapshotHeaderRecord(), batch.controlRecords().get(0).message()); + + // Consume the reader until we find a control record + do { + batch = reader.next(); + } + while (batch.controlRecords().isEmpty()); + + // Check snapshot footer control record + assertEquals(1, batch.controlRecords().size()); + assertEquals(ControlRecordType.SNAPSHOT_FOOTER, batch.controlRecords().get(0).type()); + assertEquals(new SnapshotFooterRecord(), batch.controlRecords().get(0).message()); + + // Snapshot footer must be last record + assertFalse(reader.hasNext()); + } + } + + @Test + void testBuilderKRaftVersion0WithVoterSet() { + OffsetAndEpoch snapshotId = new OffsetAndEpoch(100, 10); + int maxBatchSize = 1024; + VoterSet voterSet = VoterSetTest.voterSet(new HashMap<>(VoterSetTest.voterMap(Arrays.asList(1, 2, 3)))); + AtomicReference buffer = new AtomicReference<>(null); + RecordsSnapshotWriter.Builder builder = new RecordsSnapshotWriter.Builder() + .setKraftVersion((short) 0) + .setVoterSet(Optional.of(voterSet)) + .setTime(new MockTime()) + .setMaxBatchSize(maxBatchSize) + .setRawSnapshotWriter( + new MockRawSnapshotWriter(snapshotId, snapshotBuf -> buffer.set(snapshotBuf)) + ); + + assertThrows(IllegalStateException.class, () -> builder.build(STRING_SERDE)); + } + + @Test + void testKBuilderRaftVersion1WithVoterSet() { + OffsetAndEpoch snapshotId = new OffsetAndEpoch(100, 10); + int maxBatchSize = 1024; + VoterSet voterSet = VoterSetTest.voterSet(new HashMap<>(VoterSetTest.voterMap(Arrays.asList(1, 2, 3)))); + AtomicReference buffer = new AtomicReference<>(null); + RecordsSnapshotWriter.Builder builder = new RecordsSnapshotWriter.Builder() + .setKraftVersion((short) 1) + .setVoterSet(Optional.of(voterSet)) + .setTime(new MockTime()) + .setMaxBatchSize(maxBatchSize) + .setRawSnapshotWriter( + new MockRawSnapshotWriter(snapshotId, snapshotBuf -> buffer.set(snapshotBuf)) + ); + try (RecordsSnapshotWriter snapshot = builder.build(STRING_SERDE)) { + snapshot.freeze(); + } + + try (RecordsSnapshotReader reader = RecordsSnapshotReader.of( + new MockRawSnapshotReader(snapshotId, buffer.get()), + STRING_SERDE, + BufferSupplier.NO_CACHING, + maxBatchSize, + true + ) + ) { + // Consume the control record batch + Batch batch = reader.next(); + assertEquals(3, batch.controlRecords().size()); + + // Check snapshot header control record + assertEquals(ControlRecordType.SNAPSHOT_HEADER, batch.controlRecords().get(0).type()); + assertEquals(new SnapshotHeaderRecord(), batch.controlRecords().get(0).message()); + + // Check kraft version control record + assertEquals(ControlRecordType.KRAFT_VERSION, batch.controlRecords().get(1).type()); + assertEquals(new KRaftVersionRecord().setKRaftVersion((short) 1), batch.controlRecords().get(1).message()); + + // Check the voters control record + assertEquals(ControlRecordType.VOTERS, batch.controlRecords().get(2).type()); + assertEquals(voterSet.toVotersRecord((short) 0), batch.controlRecords().get(2).message()); + + // Consume the reader until we find a control record + do { + batch = reader.next(); + } + while (batch.controlRecords().isEmpty()); + + // Check snapshot footer control record + assertEquals(1, batch.controlRecords().size()); + assertEquals(ControlRecordType.SNAPSHOT_FOOTER, batch.controlRecords().get(0).type()); + assertEquals(new SnapshotFooterRecord(), batch.controlRecords().get(0).message()); + + // Snapshot footer must be last record + assertFalse(reader.hasNext()); + } + } + + @Test + void testBuilderKRaftVersion1WithoutVoterSet() { + OffsetAndEpoch snapshotId = new OffsetAndEpoch(100, 10); + int maxBatchSize = 1024; + AtomicReference buffer = new AtomicReference<>(null); + RecordsSnapshotWriter.Builder builder = new RecordsSnapshotWriter.Builder() + .setKraftVersion((short) 1) + .setVoterSet(Optional.empty()) + .setTime(new MockTime()) + .setMaxBatchSize(maxBatchSize) + .setRawSnapshotWriter( + new MockRawSnapshotWriter(snapshotId, snapshotBuf -> buffer.set(snapshotBuf)) + ); + try (RecordsSnapshotWriter snapshot = builder.build(STRING_SERDE)) { + snapshot.freeze(); + } + + try (RecordsSnapshotReader reader = RecordsSnapshotReader.of( + new MockRawSnapshotReader(snapshotId, buffer.get()), + STRING_SERDE, + BufferSupplier.NO_CACHING, + maxBatchSize, + true + ) + ) { + // Consume the control record batch + Batch batch = reader.next(); + assertEquals(2, batch.controlRecords().size()); + + // Check snapshot header control record + assertEquals(ControlRecordType.SNAPSHOT_HEADER, batch.controlRecords().get(0).type()); + assertEquals(new SnapshotHeaderRecord(), batch.controlRecords().get(0).message()); + + // Check kraft version control record + assertEquals(ControlRecordType.KRAFT_VERSION, batch.controlRecords().get(1).type()); + assertEquals(new KRaftVersionRecord().setKRaftVersion((short) 1), batch.controlRecords().get(1).message()); + + // Consume the reader until we find a control record + do { + batch = reader.next(); + } + while (batch.controlRecords().isEmpty()); + + // Check snapshot footer control record + assertEquals(1, batch.controlRecords().size()); + assertEquals(ControlRecordType.SNAPSHOT_FOOTER, batch.controlRecords().get(0).type()); + assertEquals(new SnapshotFooterRecord(), batch.controlRecords().get(0).message()); + + // Snapshot footer must be last record + assertFalse(reader.hasNext()); + } + } +} From 625aabdd7629f1d84da11d2601db4b2645b9dadf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Tue, 16 Apr 2024 14:33:38 -0700 Subject: [PATCH 27/34] KAFKA-16207; Add test for internal partition listener --- .../apache/kafka/raft/KafkaRaftClient.java | 30 +- ...ogListener.java => PartitionListener.java} | 5 +- .../raft/internals/PartitionListenerTest.java | 375 ++++++++++++++++++ 3 files changed, 392 insertions(+), 18 deletions(-) rename raft/src/main/java/org/apache/kafka/raft/internals/{InternalLogListener.java => PartitionListener.java} (98%) create mode 100644 raft/src/test/java/org/apache/kafka/raft/internals/PartitionListenerTest.java 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 72e1d2cfad67..a71ac6196872 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -65,7 +65,7 @@ import org.apache.kafka.raft.internals.BlockingMessageQueue; import org.apache.kafka.raft.internals.CloseListener; import org.apache.kafka.raft.internals.FuturePurgatory; -import org.apache.kafka.raft.internals.InternalLogListener; +import org.apache.kafka.raft.internals.PartitionListener; import org.apache.kafka.raft.internals.KafkaRaftMetrics; import org.apache.kafka.raft.internals.MemoryBatchReader; import org.apache.kafka.raft.internals.RecordsBatchReader; @@ -179,7 +179,7 @@ final public class KafkaRaftClient implements RaftClient { * 4. truncate the internal log listener when a follower truncates their log * 5. trim the internal listener prefix when a snapshot gets generated */ - private volatile InternalLogListener internalListener; + private volatile PartitionListener partitionListener; private volatile KafkaRaftMetrics kafkaRaftMetrics; private volatile QuorumState quorum; private volatile RequestManager requestManager; @@ -369,7 +369,7 @@ public void initialize( QuorumStateStore quorumStateStore, Metrics metrics ) { - internalListener = new InternalLogListener( + partitionListener = new PartitionListener( Optional.of(VoterSet.fromAddressSpecs(listenerName, voterAddresses)), log, serde, @@ -379,10 +379,10 @@ public void initialize( ); // Read the entire log logger.info("Reading KRaft snapshot and log as part of the initialization"); - internalListener.updateListener(); + partitionListener.updateListener(); requestManager = new RequestManager( - internalListener.lastVoterSet().voterIds(), + partitionListener.lastVoterSet().voterIds(), raftConfig.retryBackoffMs(), raftConfig.requestTimeoutMs(), random @@ -390,7 +390,7 @@ public void initialize( quorum = new QuorumState( nodeId, - internalListener.lastVoterSet().voterIds(), + partitionListener.lastVoterSet().voterIds(), raftConfig.electionTimeoutMs(), raftConfig.fetchTimeoutMs(), quorumStateStore, @@ -404,7 +404,7 @@ public void initialize( // so there are no unknown voter connections. Report this metric as 0. kafkaRaftMetrics.updateNumUnknownVoterConnections(0); - VoterSet lastVoterSet = internalListener.lastVoterSet(); + VoterSet lastVoterSet = partitionListener.lastVoterSet(); for (Integer voterId : lastVoterSet.voterIds()) { channel.updateEndpoint(voterId, lastVoterSet.voterAddress(voterId, listenerName).get()); } @@ -1181,7 +1181,7 @@ private boolean handleFetchResponse( ); // Update the internal listener to the new end offset - internalListener.truncateTo(truncationOffset); + partitionListener.truncateTo(truncationOffset); } else if (partitionResponse.snapshotId().epoch() >= 0 || partitionResponse.snapshotId().endOffset() >= 0) { // The leader is asking us to fetch a snapshot @@ -1244,7 +1244,7 @@ private void appendAsFollower( log.flush(false); } - internalListener.updateListener(); + partitionListener.updateListener(); OffsetAndEpoch endOffset = endOffset(); kafkaRaftMetrics.updateFetchedRecords(info.lastOffset - info.firstOffset + 1); @@ -1257,7 +1257,7 @@ private LogAppendInfo appendAsLeader( ) { LogAppendInfo info = log.appendAsLeader(records, quorum.epoch()); - internalListener.updateListener(); + partitionListener.updateListener(); OffsetAndEpoch endOffset = endOffset(); kafkaRaftMetrics.updateAppendRecords(info.lastOffset - info.firstOffset + 1); @@ -1521,7 +1521,7 @@ private boolean handleFetchSnapshotResponse( // This will aways reload the snapshot because the internal listener's next offset // is always less than the snapshot id just downloaded. - internalListener.updateListener(); + partitionListener.updateListener(); updateFollowerHighWatermark(state, OptionalLong.of(log.highWatermark().offset)); } else { @@ -2489,7 +2489,7 @@ public Optional> createSnapshot( RawSnapshotWriter wrappedWriter = new NotifyingRawSnapshotWriter(writer, offsetAndEpoch -> { // Trim the state in the internal starting with the new starting offset - internalListener.trimPrefixTo(offsetAndEpoch.offset()); + partitionListener.trimPrefixTo(offsetAndEpoch.offset()); }); return new RecordsSnapshotWriter.Builder() @@ -2498,8 +2498,8 @@ public Optional> createSnapshot( .setMaxBatchSize(MAX_BATCH_SIZE_BYTES) .setMemoryPool(memoryPool) .setRawSnapshotWriter(wrappedWriter) - .setKraftVersion(internalListener.kraftVersionAtOffset(lastContainedLogOffset)) - .setVoterSet(internalListener.voterSetAtOffset(lastContainedLogOffset)) + .setKraftVersion(partitionListener.kraftVersionAtOffset(lastContainedLogOffset)) + .setVoterSet(partitionListener.voterSetAtOffset(lastContainedLogOffset)) .build(serde); }); } @@ -2542,7 +2542,7 @@ QuorumState quorum() { } private boolean isInitialized() { - return internalListener != null && quorum != null && requestManager != null && kafkaRaftMetrics != null; + return partitionListener != null && quorum != null && requestManager != null && kafkaRaftMetrics != null; } private class GracefulShutdown { diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java b/raft/src/main/java/org/apache/kafka/raft/internals/PartitionListener.java similarity index 98% rename from raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java rename to raft/src/main/java/org/apache/kafka/raft/internals/PartitionListener.java index 32367562b488..43d55339843c 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/InternalLogListener.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/PartitionListener.java @@ -33,13 +33,12 @@ import org.apache.kafka.snapshot.SnapshotReader; import org.slf4j.Logger; -// TODO: Add unnitest for it /** * The KRaft state machine for tracking control records in the topic partition. * * This type keeps track of changes to the finalized kraft.version and the sets of voters. */ -final public class InternalLogListener { +final public class PartitionListener { private final ReplicatedLog log; private final RecordSerde serde; private final BufferSupplier bufferSupplier; @@ -69,7 +68,7 @@ final public class InternalLogListener { * @param maxBatchSizeBytes the maximum size of record batch * @param logContext the log context */ - public InternalLogListener( + public PartitionListener( Optional staticVoterSet, ReplicatedLog log, RecordSerde serde, diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/PartitionListenerTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/PartitionListenerTest.java new file mode 100644 index 000000000000..6a08d57921e1 --- /dev/null +++ b/raft/src/test/java/org/apache/kafka/raft/internals/PartitionListenerTest.java @@ -0,0 +1,375 @@ +/* + * 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.util.Arrays; +import java.util.Optional; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.message.KRaftVersionRecord; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.utils.BufferSupplier; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.raft.MockLog; +import org.apache.kafka.raft.OffsetAndEpoch; +import org.apache.kafka.server.common.serialization.RecordSerde; +import org.apache.kafka.snapshot.RecordsSnapshotWriter; +import org.junit.jupiter.api.Test; +import static org.junit.jupiter.api.Assertions.assertEquals; + +final class PartitionListenerTest { + private static final RecordSerde STRING_SERDE = new StringSerde(); + + private static MockLog buildLog() { + return new MockLog(new TopicPartition("partition", 0), Uuid.randomUuid(), new LogContext()); + } + + private static PartitionListener buildPartitionListener(MockLog log, Optional staticVoterSet) { + return new PartitionListener( + staticVoterSet, + log, + STRING_SERDE, + BufferSupplier.NO_CACHING, + 1024, + new LogContext() + ); + } + + @Test + void testEmptyParition() { + MockLog log = buildLog(); + VoterSet voterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(1, 2, 3))); + + PartitionListener partitionListener = buildPartitionListener(log, Optional.of(voterSet)); + + // This should be a no-op operation + partitionListener.updateListener(); + + assertEquals(voterSet, partitionListener.lastVoterSet()); + } + + @Test + void testUpdateWithoutSnapshot() { + MockLog log = buildLog(); + VoterSet staticVoterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(1, 2, 3))); + BufferSupplier bufferSupplier = BufferSupplier.NO_CACHING; + int epoch = 1; + + PartitionListener partitionListener = buildPartitionListener(log, Optional.of(staticVoterSet)); + + // Append the kraft.version control record + short kraftVersion = 1; + log.appendAsLeader( + MemoryRecords.withKRaftVersionRecord( + log.endOffset().offset, + 0, + epoch, + bufferSupplier.get(300), + new KRaftVersionRecord().setKRaftVersion(kraftVersion) + ), + epoch + ); + + // Append the voter set control record + VoterSet voterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(4, 5, 6))); + log.appendAsLeader( + MemoryRecords.withVotersRecord( + log.endOffset().offset, + 0, + epoch, + bufferSupplier.get(300), + voterSet.toVotersRecord((short) 0) + ), + epoch + ); + + // Read the entire partition + partitionListener.updateListener(); + + assertEquals(voterSet, partitionListener.lastVoterSet()); + assertEquals(Optional.of(voterSet), partitionListener.voterSetAtOffset(log.endOffset().offset - 1)); + assertEquals(kraftVersion, partitionListener.kraftVersionAtOffset(log.endOffset().offset - 1)); + } + + @Test + void testUpdateWithEmptySnapshot() { + MockLog log = buildLog(); + VoterSet staticVoterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(1, 2, 3))); + BufferSupplier bufferSupplier = BufferSupplier.NO_CACHING; + int epoch = 1; + + PartitionListener partitionListener = buildPartitionListener(log, Optional.of(staticVoterSet)); + + // Create a snapshot that doesn't have any kraft.version or voter set control records + RecordsSnapshotWriter.Builder builder = new RecordsSnapshotWriter.Builder() + .setRawSnapshotWriter(log.createNewSnapshotUnchecked(new OffsetAndEpoch(10, epoch)).get()); + try (RecordsSnapshotWriter writer = builder.build(STRING_SERDE)) { + writer.freeze(); + } + log.truncateToLatestSnapshot(); + + // Append the kraft.version control record + short kraftVersion = 1; + log.appendAsLeader( + MemoryRecords.withKRaftVersionRecord( + log.endOffset().offset, + 0, + epoch, + bufferSupplier.get(300), + new KRaftVersionRecord().setKRaftVersion(kraftVersion) + ), + epoch + ); + + // Append the voter set control record + VoterSet voterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(4, 5, 6))); + log.appendAsLeader( + MemoryRecords.withVotersRecord( + log.endOffset().offset, + 0, + epoch, + bufferSupplier.get(300), + voterSet.toVotersRecord((short) 0) + ), + epoch + ); + + // Read the entire partition + partitionListener.updateListener(); + + assertEquals(voterSet, partitionListener.lastVoterSet()); + assertEquals(Optional.of(voterSet), partitionListener.voterSetAtOffset(log.endOffset().offset - 1)); + assertEquals(kraftVersion, partitionListener.kraftVersionAtOffset(log.endOffset().offset - 1)); + } + + @Test + void testUpdateWithSnapshot() { + MockLog log = buildLog(); + VoterSet staticVoterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(1, 2, 3))); + int epoch = 1; + + PartitionListener partitionListener = buildPartitionListener(log, Optional.of(staticVoterSet)); + + // Create a snapshot that has kraft.version and voter set control records + short kraftVersion = 1; + VoterSet voterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(4, 5, 6))); + + RecordsSnapshotWriter.Builder builder = new RecordsSnapshotWriter.Builder() + .setRawSnapshotWriter(log.createNewSnapshotUnchecked(new OffsetAndEpoch(10, epoch)).get()) + .setKraftVersion(kraftVersion) + .setVoterSet(Optional.of(voterSet)); + try (RecordsSnapshotWriter writer = builder.build(STRING_SERDE)) { + writer.freeze(); + } + log.truncateToLatestSnapshot(); + + // Read the entire partition + partitionListener.updateListener(); + + assertEquals(voterSet, partitionListener.lastVoterSet()); + assertEquals(Optional.of(voterSet), partitionListener.voterSetAtOffset(log.endOffset().offset - 1)); + assertEquals(kraftVersion, partitionListener.kraftVersionAtOffset(log.endOffset().offset - 1)); + } + + @Test + void testUpdateWithSnapshotAndLogOverride() { + MockLog log = buildLog(); + VoterSet staticVoterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(1, 2, 3))); + BufferSupplier bufferSupplier = BufferSupplier.NO_CACHING; + int epoch = 1; + + PartitionListener partitionListener = buildPartitionListener(log, Optional.of(staticVoterSet)); + + // Create a snapshot that has kraft.version and voter set control records + short kraftVersion = 1; + VoterSet snapshotVoterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(4, 5, 6))); + + OffsetAndEpoch snapshotId = new OffsetAndEpoch(10, epoch); + RecordsSnapshotWriter.Builder builder = new RecordsSnapshotWriter.Builder() + .setRawSnapshotWriter(log.createNewSnapshotUnchecked(snapshotId).get()) + .setKraftVersion(kraftVersion) + .setVoterSet(Optional.of(snapshotVoterSet)); + try (RecordsSnapshotWriter writer = builder.build(STRING_SERDE)) { + writer.freeze(); + } + log.truncateToLatestSnapshot(); + + // Append the voter set control record + VoterSet voterSet = snapshotVoterSet.addVoter(VoterSetTest.voterNode(7)).get(); + log.appendAsLeader( + MemoryRecords.withVotersRecord( + log.endOffset().offset, + 0, + epoch, + bufferSupplier.get(300), + voterSet.toVotersRecord((short) 0) + ), + epoch + ); + + // Read the entire partition + partitionListener.updateListener(); + + assertEquals(voterSet, partitionListener.lastVoterSet()); + assertEquals(Optional.of(voterSet), partitionListener.voterSetAtOffset(log.endOffset().offset - 1)); + assertEquals(kraftVersion, partitionListener.kraftVersionAtOffset(log.endOffset().offset - 1)); + + // Check the voter set at the snapshot + assertEquals(Optional.of(snapshotVoterSet), partitionListener.voterSetAtOffset(snapshotId.offset() - 1)); + } + + @Test + void testTruncateTo() { + MockLog log = buildLog(); + VoterSet staticVoterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(1, 2, 3))); + BufferSupplier bufferSupplier = BufferSupplier.NO_CACHING; + int epoch = 1; + + PartitionListener partitionListener = buildPartitionListener(log, Optional.of(staticVoterSet)); + + // Append the kraft.version control record + short kraftVersion = 1; + log.appendAsLeader( + MemoryRecords.withKRaftVersionRecord( + log.endOffset().offset, + 0, + epoch, + bufferSupplier.get(300), + new KRaftVersionRecord().setKRaftVersion(kraftVersion) + ), + epoch + ); + + // Append the voter set control record + long firstVoterSetOffset = log.endOffset().offset; + VoterSet firstVoterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(4, 5, 6))); + log.appendAsLeader( + MemoryRecords.withVotersRecord( + firstVoterSetOffset, + 0, + epoch, + bufferSupplier.get(300), + firstVoterSet.toVotersRecord((short) 0) + ), + epoch + ); + + // Append another voter set control record + long voterSetOffset = log.endOffset().offset; + VoterSet voterSet = firstVoterSet.addVoter(VoterSetTest.voterNode(7)).get(); + log.appendAsLeader( + MemoryRecords.withVotersRecord( + voterSetOffset, + 0, + epoch, + bufferSupplier.get(300), + voterSet.toVotersRecord((short) 0) + ), + epoch + ); + + // Read the entire partition + partitionListener.updateListener(); + + assertEquals(voterSet, partitionListener.lastVoterSet()); + + // Truncate log and listener + log.truncateTo(voterSetOffset); + partitionListener.truncateTo(voterSetOffset); + + assertEquals(firstVoterSet, partitionListener.lastVoterSet()); + + // Truncate the entire log + log.truncateTo(0); + partitionListener.truncateTo(0); + + assertEquals(staticVoterSet, partitionListener.lastVoterSet()); + } + + @Test + void testTrimPrefixTo() { + MockLog log = buildLog(); + VoterSet staticVoterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(1, 2, 3))); + BufferSupplier bufferSupplier = BufferSupplier.NO_CACHING; + int epoch = 1; + + PartitionListener partitionListener = buildPartitionListener(log, Optional.of(staticVoterSet)); + + // Append the kraft.version control record + long kraftVersionOffset = log.endOffset().offset; + short kraftVersion = 1; + log.appendAsLeader( + MemoryRecords.withKRaftVersionRecord( + kraftVersionOffset, + 0, + epoch, + bufferSupplier.get(300), + new KRaftVersionRecord().setKRaftVersion(kraftVersion) + ), + epoch + ); + + // Append the voter set control record + long firstVoterSetOffset = log.endOffset().offset; + VoterSet firstVoterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(4, 5, 6))); + log.appendAsLeader( + MemoryRecords.withVotersRecord( + firstVoterSetOffset, + 0, + epoch, + bufferSupplier.get(300), + firstVoterSet.toVotersRecord((short) 0) + ), + epoch + ); + + // Append another voter set control record + long voterSetOffset = log.endOffset().offset; + VoterSet voterSet = firstVoterSet.addVoter(VoterSetTest.voterNode(7)).get(); + log.appendAsLeader( + MemoryRecords.withVotersRecord( + voterSetOffset, + 0, + epoch, + bufferSupplier.get(300), + voterSet.toVotersRecord((short) 0) + ), + epoch + ); + + // Read the entire partition + partitionListener.updateListener(); + + assertEquals(voterSet, partitionListener.lastVoterSet()); + assertEquals(kraftVersion, partitionListener.kraftVersionAtOffset(kraftVersionOffset)); + + // Trim the prefix for the partition listener up to the kraft.version + partitionListener.trimPrefixTo(kraftVersionOffset); + assertEquals(kraftVersion, partitionListener.kraftVersionAtOffset(kraftVersionOffset)); + + // Trim the prefix for the partition listener up to the first voter set + partitionListener.trimPrefixTo(firstVoterSetOffset); + assertEquals(kraftVersion, partitionListener.kraftVersionAtOffset(kraftVersionOffset)); + assertEquals(Optional.of(firstVoterSet), partitionListener.voterSetAtOffset(firstVoterSetOffset)); + + // Trim the prefix for the partition listener up to the second voter set + partitionListener.trimPrefixTo(voterSetOffset); + assertEquals(kraftVersion, partitionListener.kraftVersionAtOffset(kraftVersionOffset)); + assertEquals(Optional.empty(), partitionListener.voterSetAtOffset(firstVoterSetOffset)); + assertEquals(Optional.of(voterSet), partitionListener.voterSetAtOffset(voterSetOffset)); + } +} From b6fc7f6ce16c90f51d4c90b5b470c75c3a74a05a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Fri, 26 Apr 2024 09:34:30 -0700 Subject: [PATCH 28/34] KAFKA-16207; Use directory id instead of uuid --- .../common/record/ControlRecordType.java | 4 +- .../common/record/ControlRecordUtils.java | 6 +- .../kafka/common/record/MemoryRecords.java | 36 ++++++++--- .../common/record/MemoryRecordsBuilder.java | 4 +- .../common/message/VotersRecord.json | 2 +- .../common/record/ControlRecordTypeTest.java | 60 +++---------------- .../common/record/ControlRecordUtilsTest.java | 2 +- .../scala/kafka/tools/DumpLogSegments.scala | 2 +- .../org/apache/kafka/raft/ControlRecord.java | 4 +- .../raft/internals/PartitionListener.java | 2 +- .../kafka/raft/internals/RecordsIterator.java | 2 +- .../apache/kafka/raft/internals/VoterSet.java | 38 ++++++------ .../kafka/snapshot/RecordsSnapshotWriter.java | 2 +- .../apache/kafka/raft/ControlRecordTest.java | 2 +- .../raft/internals/RecordsIteratorTest.java | 10 ++-- .../kafka/raft/internals/VoterSetTest.java | 2 +- .../snapshot/RecordsSnapshotWriterTest.java | 2 +- 17 files changed, 78 insertions(+), 102 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/record/ControlRecordType.java b/clients/src/main/java/org/apache/kafka/common/record/ControlRecordType.java index 9e52777bfb40..8d0bbdba9eaa 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/ControlRecordType.java +++ b/clients/src/main/java/org/apache/kafka/common/record/ControlRecordType.java @@ -51,7 +51,7 @@ public enum ControlRecordType { // KRaft membership changes messages KRAFT_VERSION((short) 5), - VOTERS((short) 6), + KRAFT_VOTERS((short) 6), // UNKNOWN is used to indicate a control type which the client is not aware of and should be ignored UNKNOWN((short) -1); @@ -115,7 +115,7 @@ public static ControlRecordType fromTypeId(short typeId) { case 5: return KRAFT_VERSION; case 6: - return VOTERS; + return KRAFT_VOTERS; default: return UNKNOWN; diff --git a/clients/src/main/java/org/apache/kafka/common/record/ControlRecordUtils.java b/clients/src/main/java/org/apache/kafka/common/record/ControlRecordUtils.java index 3f8d4a56943b..1e78448643e1 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/ControlRecordUtils.java +++ b/clients/src/main/java/org/apache/kafka/common/record/ControlRecordUtils.java @@ -33,7 +33,7 @@ public class ControlRecordUtils { public static final short LEADER_CHANGE_CURRENT_VERSION = 0; public static final short SNAPSHOT_FOOTER_CURRENT_VERSION = 0; public static final short SNAPSHOT_HEADER_CURRENT_VERSION = 0; - public static final short VOTERS_CURRENT_VERSION = 0; + public static final short KRAFT_VOTERS_CURRENT_VERSION = 0; public static LeaderChangeMessage deserializeLeaderChangeMessage(Record record) { ControlRecordType recordType = ControlRecordType.parse(record.key()); @@ -81,13 +81,13 @@ public static KRaftVersionRecord deserializeKRaftVersionRecord(ByteBuffer data) public static VotersRecord deserializeVotersRecord(Record record) { ControlRecordType recordType = ControlRecordType.parse(record.key()); - validateControlRecordType(ControlRecordType.VOTERS, recordType); + validateControlRecordType(ControlRecordType.KRAFT_VOTERS, recordType); return deserializeVotersRecord(record.value()); } public static VotersRecord deserializeVotersRecord(ByteBuffer data) { - return new VotersRecord(new ByteBufferAccessor(data.slice()), VOTERS_CURRENT_VERSION); + return new VotersRecord(new ByteBufferAccessor(data.slice()), KRAFT_VOTERS_CURRENT_VERSION); } private static void validateControlRecordType(ControlRecordType expected, ControlRecordType actual) { diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index b07d54ad8ef8..d264f576e681 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -830,10 +830,20 @@ private static void writeKRaftVersionRecord( KRaftVersionRecord kraftVersionRecord ) { try (MemoryRecordsBuilder builder = new MemoryRecordsBuilder( - buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, - TimestampType.CREATE_TIME, initialOffset, timestamp, - RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, - false, true, leaderEpoch, buffer.capacity()) + buffer, + RecordBatch.CURRENT_MAGIC_VALUE, + CompressionType.NONE, + TimestampType.CREATE_TIME, + initialOffset, + timestamp, + RecordBatch.NO_PRODUCER_ID, + RecordBatch.NO_PRODUCER_EPOCH, + RecordBatch.NO_SEQUENCE, + false, + true, + leaderEpoch, + buffer.capacity() + ) ) { builder.appendKRaftVersionMessage(timestamp, kraftVersionRecord); } @@ -859,10 +869,20 @@ private static void writeVotersRecord( VotersRecord votersRecord ) { try (MemoryRecordsBuilder builder = new MemoryRecordsBuilder( - buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, - TimestampType.CREATE_TIME, initialOffset, timestamp, - RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, - false, true, leaderEpoch, buffer.capacity()) + buffer, + RecordBatch.CURRENT_MAGIC_VALUE, + CompressionType.NONE, + TimestampType.CREATE_TIME, + initialOffset, + timestamp, + RecordBatch.NO_PRODUCER_ID, + RecordBatch.NO_PRODUCER_EPOCH, + RecordBatch.NO_SEQUENCE, + false, + true, + leaderEpoch, + buffer.capacity() + ) ) { builder.appendVotersMessage(timestamp, votersRecord); } diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java index 556ebae254ca..1afa91b5487f 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java @@ -664,8 +664,8 @@ public void appendKRaftVersionMessage(long timestamp, KRaftVersionRecord kraftVe public void appendVotersMessage(long timestamp, VotersRecord votersRecord) { appendControlRecord( timestamp, - ControlRecordType.VOTERS, - MessageUtil.toByteBuffer(votersRecord, ControlRecordUtils.VOTERS_CURRENT_VERSION) + ControlRecordType.KRAFT_VOTERS, + MessageUtil.toByteBuffer(votersRecord, ControlRecordUtils.KRAFT_VOTERS_CURRENT_VERSION) ); } diff --git a/clients/src/main/resources/common/message/VotersRecord.json b/clients/src/main/resources/common/message/VotersRecord.json index 2f040424a5c8..e9df56ad4f14 100644 --- a/clients/src/main/resources/common/message/VotersRecord.json +++ b/clients/src/main/resources/common/message/VotersRecord.json @@ -24,7 +24,7 @@ { "name": "Voters", "type": "[]Voter", "versions": "0+", "fields": [ { "name": "VoterId", "type": "int32", "versions": "0+", "entityType": "brokerId", "about": "The replica id of the voter in the topic partition" }, - { "name": "VoterUuid", "type": "uuid", "versions": "0+", + { "name": "VoterDirectoryId", "type": "uuid", "versions": "0+", "about": "The directory id of the voter in the topic partition" }, { "name": "Endpoints", "type": "[]Endpoint", "versions": "0+", "about": "The endpoint that can be used to communicate with the voter", "fields": [ diff --git a/clients/src/test/java/org/apache/kafka/common/record/ControlRecordTypeTest.java b/clients/src/test/java/org/apache/kafka/common/record/ControlRecordTypeTest.java index fc0be11b127a..d23fe45932f6 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/ControlRecordTypeTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/ControlRecordTypeTest.java @@ -16,10 +16,10 @@ */ package org.apache.kafka.common.record; -import org.junit.jupiter.api.Test; - import java.nio.ByteBuffer; - +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; import static org.junit.jupiter.api.Assertions.assertEquals; public class ControlRecordTypeTest { @@ -45,58 +45,14 @@ public void testParseUnknownVersion() { assertEquals(ControlRecordType.ABORT, type); } - @Test - public void testLeaderChange() { - ByteBuffer buffer = ByteBuffer.allocate(32); - buffer.putShort(ControlRecordType.CURRENT_CONTROL_RECORD_KEY_VERSION); - buffer.putShort((short) 2); - buffer.flip(); - - ControlRecordType type = ControlRecordType.parse(buffer); - assertEquals(ControlRecordType.LEADER_CHANGE, type); - } - - @Test - public void testSnapshotHeader() { + @ParameterizedTest + @EnumSource(value = ControlRecordType.class) + public void testRoundTrip(ControlRecordType expected) { ByteBuffer buffer = ByteBuffer.allocate(32); buffer.putShort(ControlRecordType.CURRENT_CONTROL_RECORD_KEY_VERSION); - buffer.putShort((short) 3); + buffer.putShort(expected.type()); buffer.flip(); - ControlRecordType type = ControlRecordType.parse(buffer); - assertEquals(ControlRecordType.SNAPSHOT_HEADER, type); - } - - @Test - public void testSnapshotFooter() { - ByteBuffer buffer = ByteBuffer.allocate(32); - buffer.putShort(ControlRecordType.CURRENT_CONTROL_RECORD_KEY_VERSION); - buffer.putShort((short) 4); - buffer.flip(); - - ControlRecordType type = ControlRecordType.parse(buffer); - assertEquals(ControlRecordType.SNAPSHOT_FOOTER, type); - } - - @Test - public void testKRaftVersion() { - ByteBuffer buffer = ByteBuffer.allocate(32); - buffer.putShort(ControlRecordType.CURRENT_CONTROL_RECORD_KEY_VERSION); - buffer.putShort((short) 5); - buffer.flip(); - - ControlRecordType type = ControlRecordType.parse(buffer); - assertEquals(ControlRecordType.KRAFT_VERSION, type); - } - - @Test - public void testVoters() { - ByteBuffer buffer = ByteBuffer.allocate(32); - buffer.putShort(ControlRecordType.CURRENT_CONTROL_RECORD_KEY_VERSION); - buffer.putShort((short) 6); - buffer.flip(); - - ControlRecordType type = ControlRecordType.parse(buffer); - assertEquals(ControlRecordType.VOTERS, type); + assertEquals(expected, ControlRecordType.parse(buffer)); } } diff --git a/clients/src/test/java/org/apache/kafka/common/record/ControlRecordUtilsTest.java b/clients/src/test/java/org/apache/kafka/common/record/ControlRecordUtilsTest.java index 643c69b687a9..462660bdc58c 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/ControlRecordUtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/ControlRecordUtilsTest.java @@ -54,7 +54,7 @@ public void testCurrentVersions() { ); assertEquals( VotersRecord.HIGHEST_SUPPORTED_VERSION, - ControlRecordUtils.VOTERS_CURRENT_VERSION + ControlRecordUtils.KRAFT_VOTERS_CURRENT_VERSION ); } diff --git a/core/src/main/scala/kafka/tools/DumpLogSegments.scala b/core/src/main/scala/kafka/tools/DumpLogSegments.scala index f54b8717a550..1a7bbba66659 100755 --- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala +++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala @@ -313,7 +313,7 @@ object DumpLogSegments { case ControlRecordType.KRAFT_VERSION => val kraftVersion = ControlRecordUtils.deserializeKRaftVersionRecord(record) print(s" KRaftVersion ${KRaftVersionRecordJsonConverter.write(kraftVersion, kraftVersion.version())}") - case ControlRecordType.VOTERS=> + case ControlRecordType.KRAFT_VOTERS=> val voters = ControlRecordUtils.deserializeVotersRecord(record) print(s" Votes ${VotersRecordJsonConverter.write(voters, voters.version())}") case controlType => diff --git a/raft/src/main/java/org/apache/kafka/raft/ControlRecord.java b/raft/src/main/java/org/apache/kafka/raft/ControlRecord.java index f0b999cc0f78..8e2c26198303 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ControlRecord.java +++ b/raft/src/main/java/org/apache/kafka/raft/ControlRecord.java @@ -61,7 +61,7 @@ public ControlRecord(ControlRecordType recordType, ApiMessage message) { throwIllegalArgument(recordType, message); } break; - case VOTERS: + case KRAFT_VOTERS: if (!(message instanceof VotersRecord)) { throwIllegalArgument(recordType, message); } @@ -88,7 +88,7 @@ public short version() { return ((SnapshotFooterRecord) message).version(); case KRAFT_VERSION: return ((KRaftVersionRecord) message).version(); - case VOTERS: + case KRAFT_VOTERS: return ((VotersRecord) message).version(); default: throw new IllegalStateException(String.format("Unknown control record type %s", recordType)); diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/PartitionListener.java b/raft/src/main/java/org/apache/kafka/raft/internals/PartitionListener.java index 43d55339843c..5a566de70c0d 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/PartitionListener.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/PartitionListener.java @@ -237,7 +237,7 @@ private void handleBatch(Batch batch, OptionalLong overrideOffset) { for (ControlRecord record : batch.controlRecords()) { long currentOffset = overrideOffset.orElse(batch.baseOffset() + index); switch (record.type()) { - case VOTERS: + case KRAFT_VOTERS: synchronized (voterSetHistory) { voterSetHistory.addAt(currentOffset, VoterSet.fromVotersRecord((VotersRecord) record.message())); } 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 3dc53cdf2294..6123f5188ed3 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 @@ -373,7 +373,7 @@ private static ControlRecord decodeControlRecord(Optional key, Optio case KRAFT_VERSION: message = ControlRecordUtils.deserializeKRaftVersionRecord(value.get()); break; - case VOTERS: + case KRAFT_VOTERS: message = ControlRecordUtils.deserializeVotersRecord(value.get()); break; default: diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java index 4fdaa950d28c..6a602e8ef981 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java @@ -95,15 +95,15 @@ public Optional addVoter(VoterNode voter) { * * This object is immutable. A new voter set is returned if the voter was removed. * - * A voter can be removed from the voter set if its id and uuid match. + * A voter can be removed from the voter set if its id and directory id match. * * @param voterId the voter id - * @param voterUuid the voter uuid + * @param voterDirectoryId the voter directory id * @return a new voter set if the voter was remove, otherwise {@code Optional.empty()} */ - public Optional removeVoter(int voterId, Optional voterUuid) { + public Optional removeVoter(int voterId, Optional voterDirectoryId) { VoterNode oldVoter = voters.get(voterId); - if (oldVoter != null && Objects.equals(oldVoter.uuid(), voterUuid)) { + if (oldVoter != null && Objects.equals(oldVoter.directoryId(), voterDirectoryId)) { HashMap newVoters = new HashMap<>(voters); newVoters.remove(voterId); @@ -144,7 +144,7 @@ public VotersRecord toVotersRecord(short version) { return new VotersRecord.Voter() .setVoterId(voter.id()) - .setVoterUuid(voter.uuid().orElse(Uuid.ZERO_UUID)) + .setVoterDirectoryId(voter.directoryId().orElse(Uuid.ZERO_UUID)) .setEndpoints(new VotersRecord.EndpointCollection(endpoints)) .setKRaftVersionFeature(kraftVersionFeature); }) @@ -193,18 +193,18 @@ public String toString() { final static class VoterNode { private final int id; - private final Optional uuid; + private final Optional directoryId; private final Map listeners; private final SupportedVersionRange supportedKRaftVersion; VoterNode( int id, - Optional uuid, + Optional directoryId, Map listeners, SupportedVersionRange supportedKRaftVersion ) { this.id = id; - this.uuid = uuid; + this.directoryId = directoryId; this.listeners = listeners; this.supportedKRaftVersion = supportedKRaftVersion; } @@ -213,8 +213,8 @@ int id() { return id; } - Optional uuid() { - return uuid; + Optional directoryId() { + return directoryId; } Map listeners() { @@ -238,7 +238,7 @@ public boolean equals(Object o) { VoterNode that = (VoterNode) o; if (id != that.id) return false; - if (!Objects.equals(uuid, that.uuid)) return false; + if (!Objects.equals(directoryId, that.directoryId)) return false; if (!Objects.equals(supportedKRaftVersion, that.supportedKRaftVersion)) return false; if (!Objects.equals(listeners, that.listeners)) return false; @@ -247,15 +247,15 @@ public boolean equals(Object o) { @Override public int hashCode() { - return Objects.hash(id, uuid, listeners, supportedKRaftVersion); + return Objects.hash(id, directoryId, listeners, supportedKRaftVersion); } @Override public String toString() { return String.format( - "VoterNode(id=%d, uuid=%s, listeners=%s, supportedKRaftVersion=%s)", + "VoterNode(id=%d, directoryId=%s, listeners=%s, supportedKRaftVersion=%s)", id, - uuid, + directoryId, listeners, supportedKRaftVersion ); @@ -271,11 +271,11 @@ public String toString() { public static VoterSet fromVotersRecord(VotersRecord voters) { HashMap voterNodes = new HashMap<>(voters.voters().size()); for (VotersRecord.Voter voter: voters.voters()) { - final Optional uuid; - if (!voter.voterUuid().equals(Uuid.ZERO_UUID)) { - uuid = Optional.of(voter.voterUuid()); + final Optional directoryId; + if (!voter.voterDirectoryId().equals(Uuid.ZERO_UUID)) { + directoryId = Optional.of(voter.voterDirectoryId()); } else { - uuid = Optional.empty(); + directoryId = Optional.empty(); } Map listeners = new HashMap<>(voter.endpoints().size()); @@ -287,7 +287,7 @@ public static VoterSet fromVotersRecord(VotersRecord voters) { voter.voterId(), new VoterNode( voter.voterId(), - uuid, + directoryId, listeners, new SupportedVersionRange( voter.kRaftVersionFeature().minSupportedVersion(), diff --git a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java index d67ff9a194cd..20ad51946c1d 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java @@ -253,7 +253,7 @@ public RecordsSnapshotWriter build(RecordSerde serde) { numberOfRecords++; builder.appendVotersMessage( now, - voterSet.get().toVotersRecord(ControlRecordUtils.VOTERS_CURRENT_VERSION) + voterSet.get().toVotersRecord(ControlRecordUtils.KRAFT_VOTERS_CURRENT_VERSION) ); } } diff --git a/raft/src/test/java/org/apache/kafka/raft/ControlRecordTest.java b/raft/src/test/java/org/apache/kafka/raft/ControlRecordTest.java index de03aa8785b9..fc5c670beb07 100644 --- a/raft/src/test/java/org/apache/kafka/raft/ControlRecordTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/ControlRecordTest.java @@ -36,7 +36,7 @@ void testCtr() { new ControlRecord(ControlRecordType.SNAPSHOT_HEADER, new SnapshotHeaderRecord()); new ControlRecord(ControlRecordType.SNAPSHOT_FOOTER, new SnapshotFooterRecord()); new ControlRecord(ControlRecordType.KRAFT_VERSION, new KRaftVersionRecord()); - new ControlRecord(ControlRecordType.VOTERS, new VotersRecord()); + new ControlRecord(ControlRecordType.KRAFT_VOTERS, new VotersRecord()); // Invalid constructions assertThrows( 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 d1dc2ccf6bd7..580f50951917 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 @@ -236,7 +236,7 @@ public void testControlRecordIterationWithKraftVerion1() { assertEquals(new KRaftVersionRecord().setKRaftVersion((short) 1), batch.controlRecords().get(1).message()); // Check the voters control record - assertEquals(ControlRecordType.VOTERS, batch.controlRecords().get(2).type()); + assertEquals(ControlRecordType.KRAFT_VOTERS, batch.controlRecords().get(2).type()); assertEquals(voterSet.toVotersRecord((short) 0), batch.controlRecords().get(2).message()); // Consume the iterator until we find a control record @@ -258,7 +258,7 @@ public void testControlRecordIterationWithKraftVerion1() { @ParameterizedTest @EnumSource( value = ControlRecordType.class, - names = {"LEADER_CHANGE", "SNAPSHOT_HEADER", "SNAPSHOT_FOOTER", "KRAFT_VERSION", "VOTERS"} + names = {"LEADER_CHANGE", "SNAPSHOT_HEADER", "SNAPSHOT_FOOTER", "KRAFT_VERSION", "KRAFT_VOTERS"} ) void testWithAllSupportedControlRecords(ControlRecordType type) { MemoryRecords records = buildControlRecords(type); @@ -472,7 +472,7 @@ private static ApiMessage defaultControlRecord(ControlRecordType type) { return new SnapshotFooterRecord(); case KRAFT_VERSION: return new KRaftVersionRecord(); - case VOTERS: + case KRAFT_VOTERS: return new VotersRecord(); default: throw new RuntimeException("Should not happen. Poorly configured test"); @@ -489,8 +489,8 @@ private static short defaultControlRecordVersion(ControlRecordType type) { return ControlRecordUtils.SNAPSHOT_FOOTER_CURRENT_VERSION; case KRAFT_VERSION: return ControlRecordUtils.KRAFT_VERSION_CURRENT_VERSION; - case VOTERS: - return ControlRecordUtils.VOTERS_CURRENT_VERSION; + case KRAFT_VOTERS: + return ControlRecordUtils.KRAFT_VOTERS_CURRENT_VERSION; default: throw new RuntimeException("Should not happen. Poorly configured test"); } diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetTest.java index 94481fc968fd..a9cc09751393 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetTest.java @@ -75,7 +75,7 @@ void testRemoveVoter() { VoterSet.VoterNode voter3 = aVoterMap.remove(3); assertEquals( Optional.of(new VoterSet(new HashMap<>(aVoterMap))), - voterSet.removeVoter(voter3.id(), voter3.uuid()) + voterSet.removeVoter(voter3.id(), voter3.directoryId()) ); } diff --git a/raft/src/test/java/org/apache/kafka/snapshot/RecordsSnapshotWriterTest.java b/raft/src/test/java/org/apache/kafka/snapshot/RecordsSnapshotWriterTest.java index 5bed77d3b88d..fe8654f8c4d4 100644 --- a/raft/src/test/java/org/apache/kafka/snapshot/RecordsSnapshotWriterTest.java +++ b/raft/src/test/java/org/apache/kafka/snapshot/RecordsSnapshotWriterTest.java @@ -149,7 +149,7 @@ void testKBuilderRaftVersion1WithVoterSet() { assertEquals(new KRaftVersionRecord().setKRaftVersion((short) 1), batch.controlRecords().get(1).message()); // Check the voters control record - assertEquals(ControlRecordType.VOTERS, batch.controlRecords().get(2).type()); + assertEquals(ControlRecordType.KRAFT_VOTERS, batch.controlRecords().get(2).type()); assertEquals(voterSet.toVotersRecord((short) 0), batch.controlRecords().get(2).message()); // Consume the reader until we find a control record From e55564d4510f305cf2b6d5ac71472d440e5ab0b9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Fri, 26 Apr 2024 13:45:15 -0700 Subject: [PATCH 29/34] KAFKA-16207; Many fixes and improvements --- .../scala/kafka/tools/DumpLogSegments.scala | 2 +- .../server/RaftClusterSnapshotTest.scala | 7 +- .../unit/kafka/server/KafkaConfigTest.scala | 20 ++- .../kafka/tools/DumpLogSegmentsTest.scala | 33 ++-- .../apache/kafka/raft/KafkaRaftClient.java | 6 +- .../org/apache/kafka/raft/ReplicatedLog.java | 19 ++- .../apache/kafka/raft/internals/History.java | 12 +- .../raft/internals/PartitionListener.java | 12 +- .../kafka/raft/internals/TreeMapHistory.java | 4 +- .../apache/kafka/raft/internals/VoterSet.java | 156 ++++++++++++------ .../kafka/raft/internals/VoterSetHistory.java | 12 +- .../raft/internals/PartitionListenerTest.java | 10 +- .../raft/internals/TreeMapHistoryTest.java | 16 +- .../raft/internals/VoterSetHistoryTest.java | 20 +-- .../kafka/raft/internals/VoterSetTest.java | 70 ++++++++ 15 files changed, 273 insertions(+), 126 deletions(-) diff --git a/core/src/main/scala/kafka/tools/DumpLogSegments.scala b/core/src/main/scala/kafka/tools/DumpLogSegments.scala index 1a7bbba66659..613bd0d95e13 100755 --- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala +++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala @@ -315,7 +315,7 @@ object DumpLogSegments { print(s" KRaftVersion ${KRaftVersionRecordJsonConverter.write(kraftVersion, kraftVersion.version())}") case ControlRecordType.KRAFT_VOTERS=> val voters = ControlRecordUtils.deserializeVotersRecord(record) - print(s" Votes ${VotersRecordJsonConverter.write(voters, voters.version())}") + print(s" KRaftVoters ${VotersRecordJsonConverter.write(voters, voters.version())}") case controlType => print(s" controlType: $controlType($controlTypeId)") } diff --git a/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala b/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala index bebd90e6b087..d961b8150135 100644 --- a/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala +++ b/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala @@ -25,6 +25,7 @@ import org.apache.kafka.common.utils.BufferSupplier import org.apache.kafka.metadata.MetadataRecordSerde import org.apache.kafka.snapshot.RecordsSnapshotReader import org.junit.jupiter.api.Assertions.assertEquals +import org.junit.jupiter.api.Assertions.assertNotEquals import org.junit.jupiter.api.Assertions.assertTrue import org.junit.jupiter.api.Test import org.junit.jupiter.api.Timeout @@ -87,8 +88,10 @@ class RaftClusterSnapshotTest { while (snapshot.hasNext) { val batch = snapshot.next assertTrue(batch.sizeInBytes > 0) - assertTrue( - batch.records.isEmpty != batch.controlRecords.isEmpty, + // A batch must have at least one control records or at least one data records, but not both + assertNotEquals( + batch.records.isEmpty, + batch.controlRecords.isEmpty, s"data records = ${batch.records}; control records = ${batch.controlRecords}" ) } diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 929150031998..932f2628eff5 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -1338,21 +1338,27 @@ class KafkaConfigTest { } @Test - def testValidQuorumVotersParsing(): Unit = { - val expected = new util.HashMap[Integer, InetSocketAddress]() - assertValidQuorumVoters("", expected) + def testValidEmptyQuorumVotersParsing(): Unit = { + assertValidQuorumVoters(new util.HashMap[Integer, InetSocketAddress](), "") + } + @Test + def testValidQuorumVotersParsingWithIpAddress(): Unit = { + val expected = new util.HashMap[Integer, InetSocketAddress]() expected.put(1, new InetSocketAddress("127.0.0.1", 9092)) - assertValidQuorumVoters("1@127.0.0.1:9092", expected) + assertValidQuorumVoters(expected, "1@127.0.0.1:9092") + } - expected.clear() + @Test + def testValidQuorumVotersParsingWithMultipleHost(): Unit = { + val expected = new util.HashMap[Integer, InetSocketAddress]() expected.put(1, new InetSocketAddress("kafka1", 9092)) expected.put(2, new InetSocketAddress("kafka2", 9092)) expected.put(3, new InetSocketAddress("kafka3", 9092)) - assertValidQuorumVoters("1@kafka1:9092,2@kafka2:9092,3@kafka3:9092", expected) + assertValidQuorumVoters(expected, "1@kafka1:9092,2@kafka2:9092,3@kafka3:9092") } - private def assertValidQuorumVoters(value: String, expectedVoters: util.Map[Integer, InetSocketAddress]): Unit = { + private def assertValidQuorumVoters(expectedVoters: util.Map[Integer, InetSocketAddress], value: String): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect) props.setProperty(KafkaConfig.QuorumVotersProp, value) val addresses = QuorumConfig.parseVoterConnections(KafkaConfig.fromProps(props).quorumVoters) diff --git a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala index 9a15db3a3347..1d710be7aa5b 100644 --- a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala +++ b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala @@ -20,6 +20,8 @@ package kafka.tools import java.io.{ByteArrayOutputStream, File, PrintWriter} import java.nio.ByteBuffer import java.util +import java.util.Optional +import java.util.Arrays import java.util.Properties import kafka.log.{LogTestUtils, UnifiedLog} import kafka.raft.{KafkaMetadataLog, MetadataLogConfig} @@ -35,6 +37,7 @@ import org.apache.kafka.common.utils.Utils import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.metadata.MetadataRecordSerde import org.apache.kafka.raft.{KafkaRaftClient, OffsetAndEpoch} +import org.apache.kafka.raft.internals.VoterSetTest import org.apache.kafka.server.common.ApiMessageAndVersion import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.util.MockTime @@ -327,6 +330,8 @@ class DumpLogSegmentsTest { .setTime(new MockTime) .setLastContainedLogTimestamp(lastContainedLogTimestamp) .setRawSnapshotWriter(metadataLog.createNewSnapshot(new OffsetAndEpoch(0, 0)).get) + .setKraftVersion(1) + .setVoterSet(Optional.of(VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(1, 2, 3))))) .build(MetadataRecordSerde.INSTANCE) ) { snapshotWriter => snapshotWriter.append(metadataRecords.asJava) @@ -334,20 +339,24 @@ class DumpLogSegmentsTest { } var output = runDumpLogSegments(Array("--cluster-metadata-decoder", "--files", snapshotPath)) - assertTrue(output.contains("Snapshot end offset: 0, epoch: 0")) - assertTrue(output.contains("TOPIC_RECORD")) - assertTrue(output.contains("BROKER_RECORD")) - assertTrue(output.contains("SnapshotHeader")) - assertTrue(output.contains("SnapshotFooter")) - assertTrue(output.contains(s""""lastContainedLogTimestamp":$lastContainedLogTimestamp""")) + assertTrue(output.contains("Snapshot end offset: 0, epoch: 0"), output) + assertTrue(output.contains("TOPIC_RECORD"), output) + assertTrue(output.contains("BROKER_RECORD"), output) + assertTrue(output.contains("SnapshotHeader"), output) + assertTrue(output.contains("SnapshotFooter"), output) + assertTrue(output.contains("KRaftVersion"), output) + assertTrue(output.contains("KRaftVoters"), output) + assertTrue(output.contains(s""""lastContainedLogTimestamp":$lastContainedLogTimestamp"""), output) output = runDumpLogSegments(Array("--cluster-metadata-decoder", "--skip-record-metadata", "--files", snapshotPath)) - assertTrue(output.contains("Snapshot end offset: 0, epoch: 0")) - assertTrue(output.contains("TOPIC_RECORD")) - assertTrue(output.contains("BROKER_RECORD")) - assertFalse(output.contains("SnapshotHeader")) - assertFalse(output.contains("SnapshotFooter")) - assertFalse(output.contains(s""""lastContainedLogTimestamp": $lastContainedLogTimestamp""")) + assertTrue(output.contains("Snapshot end offset: 0, epoch: 0"), output) + assertTrue(output.contains("TOPIC_RECORD"), output) + assertTrue(output.contains("BROKER_RECORD"), output) + assertFalse(output.contains("SnapshotHeader"), output) + assertFalse(output.contains("SnapshotFooter"), output) + assertFalse(output.contains("KRaftVersion"), output) + assertFalse(output.contains("KRaftVoters"), output) + assertFalse(output.contains(s""""lastContainedLogTimestamp": $lastContainedLogTimestamp"""), output) } @Test 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 51f89fd56a52..2f9175767817 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -1181,7 +1181,7 @@ private boolean handleFetchResponse( ); // Update the internal listener to the new end offset - partitionListener.truncateTo(truncationOffset); + partitionListener.truncateNewEntries(truncationOffset); } else if (partitionResponse.snapshotId().epoch() >= 0 || partitionResponse.snapshotId().endOffset() >= 0) { // The leader is asking us to fetch a snapshot @@ -2486,8 +2486,8 @@ public Optional> createSnapshot( long lastContainedLogOffset = snapshotId.offset() - 1; RawSnapshotWriter wrappedWriter = new NotifyingRawSnapshotWriter(writer, offsetAndEpoch -> { - // Trim the state in the internal starting with the new starting offset - partitionListener.trimPrefixTo(offsetAndEpoch.offset()); + // Trim the state in the internal listener up to the new snapshot + partitionListener.truncateOldEntries(offsetAndEpoch.offset()); }); return new RecordsSnapshotWriter.Builder() 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 0a30f3fc235f..a8bd7f13d9a2 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ReplicatedLog.java +++ b/raft/src/main/java/org/apache/kafka/raft/ReplicatedLog.java @@ -235,14 +235,14 @@ default long truncateToEndOffset(OffsetAndEpoch endOffset) { * snapshot already exists or it is less than log start offset then return an * {@link Optional#empty()}. * - * Snapshots created using this method will be validated against the existing snapshots - * and the replicated log. + * The snapshot id will be validated against the existing snapshots and the log. The snapshot id + * must not alread exist, it must be greater than the log start offset, it must be less than + * the high-watermark and it must exist in the log. * * @param snapshotId the end offset and epoch that identifies the snapshot - * @return a writable snapshot if it doesn't already exist and greater than the log start - * offset - * @throws IllegalArgumentException if validate is true and end offset is greater than the - * high-watermark + * @return a writable snapshot + * @throws IllegalArgumentException if the snapshot id is greater than the high-watermark or not + * a valid epoch and offset in the log */ Optional createNewSnapshot(OffsetAndEpoch snapshotId); @@ -253,9 +253,10 @@ default long truncateToEndOffset(OffsetAndEpoch endOffset) { * this method is responsible for invoking {@link RawSnapshotWriter#close()}. If a * snapshot already exists then return an {@link Optional#empty()}. * - * Snapshots created using this method will not be validated against the existing snapshots - * and the replicated log. This is useful when creating snapshot from a trusted source like - * the quorum leader. + * The snapshot id will not be validated against the log. The snapshot id is not checked against + * the log start offset, the high-watermark or against existing epochs and offsets in the log. + * + * This is useful when creating snapshots from a trusted source like the quorum leader. * * @param snapshotId the end offset and epoch that identifies the snapshot * @return a writable snapshot if it doesn't already exist diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/History.java b/raft/src/main/java/org/apache/kafka/raft/internals/History.java index c2ba7166f40d..200dd9a0bb94 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/History.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/History.java @@ -36,18 +36,18 @@ public interface History { public void addAt(long offset, T value); /** - * Returns the value that has the largest offset that is less than or equals to the provided + * Returns the value that has the largest offset that is less than or equal to the provided * offset. * * @param offset the offset - * @return the value if it exist, otherwise {@code Optional.empty()} + * @return the value if it exists, otherwise {@code Optional.empty()} */ public Optional valueAtOrBefore(long offset); /** * Returns the value with the largest offset. * - * @return the value if it exist, otherwise {@code Optional.empty()} + * @return the value if it exists, otherwise {@code Optional.empty()} */ public Optional> lastEntry(); @@ -56,10 +56,10 @@ public interface History { * * @param endOffset the ending offset */ - public void truncateTo(long endOffset); + public void truncateNewEntries(long endOffset); /** - * Removes all entries but the largest entry that has an offset that is less than or equal to + * Removes all entries but the last entry that has an offset that is less than or equal to * {@code startOffset}. * * This operation does not remove the entry with the largest offset that is less than or equal @@ -69,7 +69,7 @@ public interface History { * * @param startOffset the starting offset */ - public void trimPrefixTo(long startOffset); + public void truncateOldEntries(long startOffset); /** * Removes all of the values from this object. diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/PartitionListener.java b/raft/src/main/java/org/apache/kafka/raft/internals/PartitionListener.java index 5a566de70c0d..d2e58420ec94 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/PartitionListener.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/PartitionListener.java @@ -97,12 +97,12 @@ public void updateListener() { * * @param endOffset the end offset (exclusive) */ - public void truncateTo(long endOffset) { + public void truncateNewEntries(long endOffset) { synchronized (voterSetHistory) { - voterSetHistory.truncateTo(endOffset); + voterSetHistory.truncateNewEntries(endOffset); } synchronized (kraftVersionHistory) { - kraftVersionHistory.truncateTo(endOffset); + kraftVersionHistory.truncateNewEntries(endOffset); } } @@ -111,12 +111,12 @@ public void truncateTo(long endOffset) { * * @param @startOffset the start offset (inclusive) */ - public void trimPrefixTo(long startOffset) { + public void truncateOldEntries(long startOffset) { synchronized (voterSetHistory) { - voterSetHistory.trimPrefixTo(startOffset); + voterSetHistory.truncateOldEntries(startOffset); } synchronized (kraftVersionHistory) { - kraftVersionHistory.trimPrefixTo(startOffset); + kraftVersionHistory.truncateOldEntries(startOffset); } } diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/TreeMapHistory.java b/raft/src/main/java/org/apache/kafka/raft/internals/TreeMapHistory.java index 32894be79fe7..8509737d9c19 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/TreeMapHistory.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/TreeMapHistory.java @@ -56,12 +56,12 @@ public Optional> lastEntry() { } @Override - public void truncateTo(long endOffset) { + public void truncateNewEntries(long endOffset) { history.tailMap(endOffset, true).clear(); } @Override - public void trimPrefixTo(long startOffset) { + public void truncateOldEntries(long startOffset) { NavigableMap lesserValues = history.headMap(startOffset, true); while (lesserValues.size() > 1) { // Poll and ignore the entry to remove the first entry diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java index 6a602e8ef981..78cb8638c5a2 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java @@ -21,15 +21,17 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; +import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.Set; +import java.util.function.Function; import java.util.stream.Collectors; import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.feature.SupportedVersionRange; import org.apache.kafka.common.message.VotersRecord; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.common.feature.SupportedVersionRange; /** * A type for representing the set of voters for a topic partition. @@ -55,7 +57,7 @@ final public class VoterSet { * * @param voter the id of the voter * @param listener the name of the listener - * @return the socket address if it exist, otherwise {@code Optional.empty()} + * @return the socket address if it exists, otherwise {@code Optional.empty()} */ public Optional voterAddress(int voter, String listener) { return Optional.ofNullable(voters.get(voter)) @@ -91,7 +93,7 @@ public Optional addVoter(VoterNode voter) { } /** - * Removew a voter from the voter set. + * Remove a voter from the voter set. * * This object is immutable. A new voter set is returned if the voter was removed. * @@ -99,7 +101,7 @@ public Optional addVoter(VoterNode voter) { * * @param voterId the voter id * @param voterDirectoryId the voter directory id - * @return a new voter set if the voter was remove, otherwise {@code Optional.empty()} + * @return a new voter set if the voter was removed, otherwise {@code Optional.empty()} */ public Optional removeVoter(int voterId, Optional voterDirectoryId) { VoterNode oldVoter = voters.get(voterId); @@ -119,54 +121,68 @@ public Optional removeVoter(int voterId, Optional voterDirectory * @param version the version of the voters record */ public VotersRecord toVotersRecord(short version) { + Function voterConvertor = voter -> { + Iterator endpoints = voter + .listeners() + .entrySet() + .stream() + .map(entry -> + new VotersRecord.Endpoint() + .setName(entry.getKey()) + .setHost(entry.getValue().getHostString()) + .setPort(entry.getValue().getPort()) + ) + .iterator(); + + VotersRecord.KRaftVersionFeature kraftVersionFeature = new VotersRecord.KRaftVersionFeature() + .setMinSupportedVersion(voter.supportedKRaftVersion().min()) + .setMaxSupportedVersion(voter.supportedKRaftVersion().max()); + + return new VotersRecord.Voter() + .setVoterId(voter.id()) + .setVoterDirectoryId(voter.directoryId().orElse(Uuid.ZERO_UUID)) + .setEndpoints(new VotersRecord.EndpointCollection(endpoints)) + .setKRaftVersionFeature(kraftVersionFeature); + }; + + List voterRecordVoters = voters + .values() + .stream() + .map(voterConvertor) + .collect(Collectors.toList()); + return new VotersRecord() .setVersion(version) - .setVoters( - voters - .values() - .stream() - .map(voter -> { - Iterator endpoints = voter - .listeners() - .entrySet() - .stream() - .map(entry -> - new VotersRecord.Endpoint() - .setName(entry.getKey()) - .setHost(entry.getValue().getHostString()) - .setPort(entry.getValue().getPort()) - ) - .iterator(); - - VotersRecord.KRaftVersionFeature kraftVersionFeature = new VotersRecord.KRaftVersionFeature() - .setMinSupportedVersion(voter.supportedKRaftVersion().min()) - .setMaxSupportedVersion(voter.supportedKRaftVersion().max()); - - return new VotersRecord.Voter() - .setVoterId(voter.id()) - .setVoterDirectoryId(voter.directoryId().orElse(Uuid.ZERO_UUID)) - .setEndpoints(new VotersRecord.EndpointCollection(endpoints)) - .setKRaftVersionFeature(kraftVersionFeature); - }) - .collect(Collectors.toList()) - ); + .setVoters(voterRecordVoters); } /** * Determines if two sets of voters have an overlapping majority. * - * A overlapping majority means that for all majorities in {@code this} set of voters and for - * all majority in {@code that} voeter set they have at least one voter in common. + * An overlapping majority means that for all majorities in {@code this} set of voters and for + * all majority in {@code that} set of voters, they have at least one voter in common. * - * This can be used to validate a change in the set of voters will get committed by both sets - * of voters. + * If this function returns true is means that one of the voter set commits an offset, it means + * that the other voter set cannot commit a conflicting offset. * * @param that the other voter set to compare * @return true if they have an overlapping majority, false otherwise */ public boolean hasOverlappingMajority(VoterSet that) { - if (Utils.diff(HashSet::new, voters.keySet(), that.voters.keySet()).size() > 2) return false; - if (Utils.diff(HashSet::new, that.voters.keySet(), voters.keySet()).size() > 2) return false; + Set thisVoterKeys = voters + .values() + .stream() + .map(VoterNode::voterKey) + .collect(Collectors.toSet()); + + Set thatVoterKeys = that.voters + .values() + .stream() + .map(VoterNode::voterKey) + .collect(Collectors.toSet()); + + if (Utils.diff(HashSet::new, thisVoterKeys, thatVoterKeys).size() > 1) return false; + if (Utils.diff(HashSet::new, thatVoterKeys, thisVoterKeys).size() > 1) return false; return true; } @@ -191,9 +207,48 @@ public String toString() { return String.format("VoterSet(voters=%s)", voters); } - final static class VoterNode { + final static class VoterKey { private final int id; private final Optional directoryId; + + VoterKey(int id, Optional directoryId) { + this.id = id; + this.directoryId = directoryId; + } + int id() { + return id; + } + + Optional directoryId() { + return directoryId; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + VoterKey that = (VoterKey) o; + + if (id != that.id) return false; + if (!Objects.equals(directoryId, that.directoryId)) return false; + + return true; + } + + @Override + public int hashCode() { + return Objects.hash(id, directoryId); + } + + @Override + public String toString() { + return String.format("VoterKey(id=%d, directoryId=%s)", id, directoryId); + } + } + + final static class VoterNode { + private final VoterKey voterKey; private final Map listeners; private final SupportedVersionRange supportedKRaftVersion; @@ -203,18 +258,21 @@ final static class VoterNode { Map listeners, SupportedVersionRange supportedKRaftVersion ) { - this.id = id; - this.directoryId = directoryId; + this.voterKey = new VoterKey(id, directoryId); this.listeners = listeners; this.supportedKRaftVersion = supportedKRaftVersion; } int id() { - return id; + return voterKey.id; } Optional directoryId() { - return directoryId; + return voterKey.directoryId; + } + + VoterKey voterKey() { + return voterKey; } Map listeners() { @@ -237,8 +295,7 @@ public boolean equals(Object o) { VoterNode that = (VoterNode) o; - if (id != that.id) return false; - if (!Objects.equals(directoryId, that.directoryId)) return false; + if (!Objects.equals(voterKey, that.voterKey)) return false; if (!Objects.equals(supportedKRaftVersion, that.supportedKRaftVersion)) return false; if (!Objects.equals(listeners, that.listeners)) return false; @@ -247,15 +304,14 @@ public boolean equals(Object o) { @Override public int hashCode() { - return Objects.hash(id, directoryId, listeners, supportedKRaftVersion); + return Objects.hash(voterKey, listeners, supportedKRaftVersion); } @Override public String toString() { return String.format( - "VoterNode(id=%d, directoryId=%s, listeners=%s, supportedKRaftVersion=%s)", - id, - directoryId, + "VoterNode(voterKey=%s, listeners=%s, supportedKRaftVersion=%s)", + voterKey, listeners, supportedKRaftVersion ); diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java index 3deef3e25281..11d0279918a1 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java @@ -82,17 +82,19 @@ public Optional> lastEntry() { * Returns the latest set of voters. */ public VoterSet lastValue() { - return lastEntry().orElseThrow(() -> new IllegalStateException("No voter set found")).value(); + return lastEntry() + .orElseThrow(() -> new IllegalStateException("No voter set found")) + .value(); } @Override - public void truncateTo(long endOffset) { - votersHistory.truncateTo(endOffset); + public void truncateNewEntries(long endOffset) { + votersHistory.truncateNewEntries(endOffset); } @Override - public void trimPrefixTo(long startOffset) { - votersHistory.trimPrefixTo(startOffset); + public void truncateOldEntries(long startOffset) { + votersHistory.truncateOldEntries(startOffset); } @Override diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/PartitionListenerTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/PartitionListenerTest.java index 6a08d57921e1..ba3500825aef 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/PartitionListenerTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/PartitionListenerTest.java @@ -289,13 +289,13 @@ void testTruncateTo() { // Truncate log and listener log.truncateTo(voterSetOffset); - partitionListener.truncateTo(voterSetOffset); + partitionListener.truncateNewEntries(voterSetOffset); assertEquals(firstVoterSet, partitionListener.lastVoterSet()); // Truncate the entire log log.truncateTo(0); - partitionListener.truncateTo(0); + partitionListener.truncateNewEntries(0); assertEquals(staticVoterSet, partitionListener.lastVoterSet()); } @@ -358,16 +358,16 @@ void testTrimPrefixTo() { assertEquals(kraftVersion, partitionListener.kraftVersionAtOffset(kraftVersionOffset)); // Trim the prefix for the partition listener up to the kraft.version - partitionListener.trimPrefixTo(kraftVersionOffset); + partitionListener.truncateOldEntries(kraftVersionOffset); assertEquals(kraftVersion, partitionListener.kraftVersionAtOffset(kraftVersionOffset)); // Trim the prefix for the partition listener up to the first voter set - partitionListener.trimPrefixTo(firstVoterSetOffset); + partitionListener.truncateOldEntries(firstVoterSetOffset); assertEquals(kraftVersion, partitionListener.kraftVersionAtOffset(kraftVersionOffset)); assertEquals(Optional.of(firstVoterSet), partitionListener.voterSetAtOffset(firstVoterSetOffset)); // Trim the prefix for the partition listener up to the second voter set - partitionListener.trimPrefixTo(voterSetOffset); + partitionListener.truncateOldEntries(voterSetOffset); assertEquals(kraftVersion, partitionListener.kraftVersionAtOffset(kraftVersionOffset)); assertEquals(Optional.empty(), partitionListener.voterSetAtOffset(firstVoterSetOffset)); assertEquals(Optional.of(voterSet), partitionListener.voterSetAtOffset(voterSetOffset)); diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/TreeMapHistoryTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/TreeMapHistoryTest.java index 5503dd2b2879..b8bcaf06f1d8 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/TreeMapHistoryTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/TreeMapHistoryTest.java @@ -57,16 +57,16 @@ void testTruncateTo() { history.addAt(100, "100"); history.addAt(200, "200"); - history.truncateTo(201); + history.truncateNewEntries(201); assertEquals(Optional.of(new History.Entry<>(200, "200")), history.lastEntry()); - history.truncateTo(200); + history.truncateNewEntries(200); assertEquals(Optional.of(new History.Entry<>(100, "100")), history.lastEntry()); - history.truncateTo(101); + history.truncateNewEntries(101); assertEquals(Optional.of(new History.Entry<>(100, "100")), history.lastEntry()); - history.truncateTo(100); + history.truncateNewEntries(100); assertEquals(Optional.empty(), history.lastEntry()); } @@ -76,19 +76,19 @@ void testTrimPrefixTo() { history.addAt(100, "100"); history.addAt(200, "200"); - history.trimPrefixTo(99); + history.truncateOldEntries(99); assertEquals(Optional.empty(), history.valueAtOrBefore(99)); assertEquals(Optional.of("100"), history.valueAtOrBefore(100)); - history.trimPrefixTo(100); + history.truncateOldEntries(100); assertEquals(Optional.empty(), history.valueAtOrBefore(99)); assertEquals(Optional.of("100"), history.valueAtOrBefore(100)); - history.trimPrefixTo(101); + history.truncateOldEntries(101); assertEquals(Optional.empty(), history.valueAtOrBefore(99)); assertEquals(Optional.of("100"), history.valueAtOrBefore(100)); - history.trimPrefixTo(200); + history.truncateOldEntries(200); assertEquals(Optional.empty(), history.valueAtOrBefore(199)); assertEquals(Optional.of("200"), history.valueAtOrBefore(200)); } diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java index 29df600efd1b..039239233c72 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java @@ -35,13 +35,13 @@ void testStaicVoterSet() { assertEquals(staticVoterSet, votersHistory.lastValue()); // Should be a no-op - votersHistory.truncateTo(100); + votersHistory.truncateNewEntries(100); assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(0)); assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(100)); assertEquals(staticVoterSet, votersHistory.lastValue()); // Should be a no-op - votersHistory.trimPrefixTo(100); + votersHistory.truncateOldEntries(100); assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(0)); assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(100)); assertEquals(staticVoterSet, votersHistory.lastValue()); @@ -101,13 +101,13 @@ void testTruncateTo() { VoterSet voterSet12345 = new VoterSet(new HashMap<>(voterMap)); votersHistory.addAt(200, voterSet12345); - votersHistory.truncateTo(201); + votersHistory.truncateNewEntries(201); assertEquals(Optional.of(new History.Entry<>(200, voterSet12345)), votersHistory.lastEntry()); - votersHistory.truncateTo(200); + votersHistory.truncateNewEntries(200); assertEquals(Optional.of(new History.Entry<>(100, voterSet1234)), votersHistory.lastEntry()); - votersHistory.truncateTo(101); + votersHistory.truncateNewEntries(101); assertEquals(Optional.of(new History.Entry<>(100, voterSet1234)), votersHistory.lastEntry()); - votersHistory.truncateTo(100); + votersHistory.truncateNewEntries(100); assertEquals(Optional.of(new History.Entry<>(-1, staticVoterSet)), votersHistory.lastEntry()); } @@ -127,19 +127,19 @@ void testTrimPrefixTo() { VoterSet voterSet12345 = new VoterSet(new HashMap<>(voterMap)); votersHistory.addAt(200, voterSet12345); - votersHistory.trimPrefixTo(99); + votersHistory.truncateOldEntries(99); assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(99)); assertEquals(Optional.of(voterSet1234), votersHistory.valueAtOrBefore(100)); - votersHistory.trimPrefixTo(100); + votersHistory.truncateOldEntries(100); assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(99)); assertEquals(Optional.of(voterSet1234), votersHistory.valueAtOrBefore(100)); - votersHistory.trimPrefixTo(101); + votersHistory.truncateOldEntries(101); assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(99)); assertEquals(Optional.of(voterSet1234), votersHistory.valueAtOrBefore(100)); - votersHistory.trimPrefixTo(200); + votersHistory.truncateOldEntries(200); assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(199)); assertEquals(Optional.of(voterSet12345), votersHistory.valueAtOrBefore(200)); } diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetTest.java index a9cc09751393..d1afd59361ed 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetTest.java @@ -86,6 +86,76 @@ void testRecordRoundTrip() { assertEquals(voterSet, VoterSet.fromVotersRecord(voterSet.toVotersRecord((short) 0))); } + @Test + void testOverlappingMajority() { + Map startingVoterMap = voterMap(Arrays.asList(1, 2, 3)); + VoterSet startingVoterSet = voterSet(startingVoterMap); + + VoterSet biggerVoterSet = startingVoterSet + .addVoter(voterNode(4)) + .get(); + assertMajorities(true, startingVoterSet, biggerVoterSet); + + VoterSet smallerVoterSet = startingVoterSet + .removeVoter(1, startingVoterMap.get(1).directoryId()) + .get(); + assertMajorities(true, startingVoterSet, smallerVoterSet); + + VoterSet replacedVoterSet = startingVoterSet + .removeVoter(1, startingVoterMap.get(1).directoryId()) + .get() + .addVoter(voterNode(1)) + .get(); + assertMajorities(true, startingVoterSet, replacedVoterSet); + } + + @Test + void testNonoverlappingMajority() { + Map startingVoterMap = voterMap(Arrays.asList(1, 2, 3, 4, 5)); + VoterSet startingVoterSet = voterSet(startingVoterMap); + + // Two additions don't have an overlapping majority + VoterSet biggerVoterSet = startingVoterSet + .addVoter(voterNode(6)) + .get() + .addVoter(voterNode(7)) + .get(); + assertMajorities(false, startingVoterSet, biggerVoterSet); + + // Two removals don't have an overlapping majority + VoterSet smallerVoterSet = startingVoterSet + .removeVoter(1, startingVoterMap.get(1).directoryId()) + .get() + .removeVoter(2, startingVoterMap.get(2).directoryId()) + .get(); + assertMajorities(false, startingVoterSet, smallerVoterSet); + + // Two replacements don't have an overlapping majority + VoterSet replacedVoterSet = startingVoterSet + .removeVoter(1, startingVoterMap.get(1).directoryId()) + .get() + .addVoter(voterNode(1)) + .get() + .removeVoter(2, startingVoterMap.get(2).directoryId()) + .get() + .addVoter(voterNode(2)) + .get(); + assertMajorities(false, startingVoterSet, replacedVoterSet); + } + + private void assertMajorities(boolean overlap, VoterSet a, VoterSet b) { + assertEquals( + overlap, + a.hasOverlappingMajority(b), + String.format("a = %s, b = %s", a, b) + ); + assertEquals( + overlap, + b.hasOverlappingMajority(a), + String.format("b = %s, a = %s", b, a) + ); + } + public static Map voterMap(List replicas) { return replicas .stream() From de1906278e485299dae7bb61276905d90ad4492c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Sat, 27 Apr 2024 14:37:06 -0700 Subject: [PATCH 30/34] KAFKA-16207; Rename PartitionListener --- .../apache/kafka/raft/KafkaRaftClient.java | 42 ++++----- .../apache/kafka/raft/internals/History.java | 18 ++-- ...va => KRaftControlRecordStateMachine.java} | 54 ++++++----- .../kafka/raft/internals/VoterSetHistory.java | 56 +++++++---- ...> KRaftControlRecordStateMachineTest.java} | 94 +++++++++---------- .../raft/internals/VoterSetHistoryTest.java | 10 +- 6 files changed, 148 insertions(+), 126 deletions(-) rename raft/src/main/java/org/apache/kafka/raft/internals/{PartitionListener.java => KRaftControlRecordStateMachine.java} (87%) rename raft/src/test/java/org/apache/kafka/raft/internals/{PartitionListenerTest.java => KRaftControlRecordStateMachineTest.java} (74%) 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 2f9175767817..e9acc3ad820c 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -65,7 +65,7 @@ import org.apache.kafka.raft.internals.BlockingMessageQueue; import org.apache.kafka.raft.internals.CloseListener; import org.apache.kafka.raft.internals.FuturePurgatory; -import org.apache.kafka.raft.internals.PartitionListener; +import org.apache.kafka.raft.internals.KRaftControlRecordStateMachine; import org.apache.kafka.raft.internals.KafkaRaftMetrics; import org.apache.kafka.raft.internals.MemoryBatchReader; import org.apache.kafka.raft.internals.RecordsBatchReader; @@ -172,14 +172,14 @@ final public class KafkaRaftClient implements RaftClient { // These components need to be initialized by the method initialize() because they depend on the voter set /* - * The key invariant for the internal listener is that it has always read to the LEO. This is achived by: + * The key invariant for the kraft control record state machine is that it has always read to the LEO. This is achived by: * 1. reading the entire partition (snapshot and log) at start up, - * 2. updating the internal log listener when a snapshot is replaced, because of FETCH_SNAPSHOT, on the followers - * 3. updating the internal log listener when the leader (call to append()) or follower (FETCH) appends to the log - * 4. truncate the internal log listener when a follower truncates their log - * 5. trim the internal listener prefix when a snapshot gets generated + * 2. updating the state when a snapshot is replaced, because of FETCH_SNAPSHOT, on the followers + * 3. updating the state when the leader (call to append()) or follower (FETCH) appends to the log + * 4. truncate new entries when a follower truncates their log + * 5. truncate old entries when a snapshot gets generated */ - private volatile PartitionListener partitionListener; + private volatile KRaftControlRecordStateMachine partitionState; private volatile KafkaRaftMetrics kafkaRaftMetrics; private volatile QuorumState quorum; private volatile RequestManager requestManager; @@ -369,7 +369,7 @@ public void initialize( QuorumStateStore quorumStateStore, Metrics metrics ) { - partitionListener = new PartitionListener( + partitionState = new KRaftControlRecordStateMachine( Optional.of(VoterSet.fromAddressSpecs(listenerName, voterAddresses)), log, serde, @@ -379,10 +379,10 @@ public void initialize( ); // Read the entire log logger.info("Reading KRaft snapshot and log as part of the initialization"); - partitionListener.updateListener(); + partitionState.updateState(); requestManager = new RequestManager( - partitionListener.lastVoterSet().voterIds(), + partitionState.lastVoterSet().voterIds(), quorumConfig.retryBackoffMs(), quorumConfig.requestTimeoutMs(), random @@ -390,7 +390,7 @@ public void initialize( quorum = new QuorumState( nodeId, - partitionListener.lastVoterSet().voterIds(), + partitionState.lastVoterSet().voterIds(), quorumConfig.electionTimeoutMs(), quorumConfig.fetchTimeoutMs(), quorumStateStore, @@ -404,7 +404,7 @@ public void initialize( // so there are no unknown voter connections. Report this metric as 0. kafkaRaftMetrics.updateNumUnknownVoterConnections(0); - VoterSet lastVoterSet = partitionListener.lastVoterSet(); + VoterSet lastVoterSet = partitionState.lastVoterSet(); for (Integer voterId : lastVoterSet.voterIds()) { channel.updateEndpoint(voterId, lastVoterSet.voterAddress(voterId, listenerName).get()); } @@ -1181,7 +1181,7 @@ private boolean handleFetchResponse( ); // Update the internal listener to the new end offset - partitionListener.truncateNewEntries(truncationOffset); + partitionState.truncateNewEntries(truncationOffset); } else if (partitionResponse.snapshotId().epoch() >= 0 || partitionResponse.snapshotId().endOffset() >= 0) { // The leader is asking us to fetch a snapshot @@ -1244,7 +1244,7 @@ private void appendAsFollower( log.flush(false); } - partitionListener.updateListener(); + partitionState.updateState(); OffsetAndEpoch endOffset = endOffset(); kafkaRaftMetrics.updateFetchedRecords(info.lastOffset - info.firstOffset + 1); @@ -1257,7 +1257,7 @@ private LogAppendInfo appendAsLeader( ) { LogAppendInfo info = log.appendAsLeader(records, quorum.epoch()); - partitionListener.updateListener(); + partitionState.updateState(); OffsetAndEpoch endOffset = endOffset(); kafkaRaftMetrics.updateAppendRecords(info.lastOffset - info.firstOffset + 1); @@ -1519,9 +1519,9 @@ private boolean handleFetchSnapshotResponse( quorum.leaderIdOrSentinel() ); - // This will aways reload the snapshot because the internal listener's next offset + // This will aways reload the snapshot because the internal next offset // is always less than the snapshot id just downloaded. - partitionListener.updateListener(); + partitionState.updateState(); updateFollowerHighWatermark(state, OptionalLong.of(log.highWatermark().offset)); } else { @@ -2487,7 +2487,7 @@ public Optional> createSnapshot( RawSnapshotWriter wrappedWriter = new NotifyingRawSnapshotWriter(writer, offsetAndEpoch -> { // Trim the state in the internal listener up to the new snapshot - partitionListener.truncateOldEntries(offsetAndEpoch.offset()); + partitionState.truncateOldEntries(offsetAndEpoch.offset()); }); return new RecordsSnapshotWriter.Builder() @@ -2496,8 +2496,8 @@ public Optional> createSnapshot( .setMaxBatchSize(MAX_BATCH_SIZE_BYTES) .setMemoryPool(memoryPool) .setRawSnapshotWriter(wrappedWriter) - .setKraftVersion(partitionListener.kraftVersionAtOffset(lastContainedLogOffset)) - .setVoterSet(partitionListener.voterSetAtOffset(lastContainedLogOffset)) + .setKraftVersion(partitionState.kraftVersionAtOffset(lastContainedLogOffset)) + .setVoterSet(partitionState.voterSetAtOffset(lastContainedLogOffset)) .build(serde); }); } @@ -2540,7 +2540,7 @@ QuorumState quorum() { } private boolean isInitialized() { - return partitionListener != null && quorum != null && requestManager != null && kafkaRaftMetrics != null; + return partitionState != null && quorum != null && requestManager != null && kafkaRaftMetrics != null; } private class GracefulShutdown { diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/History.java b/raft/src/main/java/org/apache/kafka/raft/internals/History.java index 200dd9a0bb94..8c846fe8ee91 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/History.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/History.java @@ -20,7 +20,7 @@ import java.util.Optional; /** - * A object tracks values of {@code T} at different offsets. + * An object that tracks values of {@code T} at different offsets. */ public interface History { /** @@ -31,9 +31,9 @@ public interface History { * * @param offset the offset * @param value the value to store - * @throws IllegalArgumentException if the offset is not greater that all previous offsets + * @throws IllegalArgumentException if the offset is not greater than all previous offsets */ - public void addAt(long offset, T value); + void addAt(long offset, T value); /** * Returns the value that has the largest offset that is less than or equal to the provided @@ -42,21 +42,21 @@ public interface History { * @param offset the offset * @return the value if it exists, otherwise {@code Optional.empty()} */ - public Optional valueAtOrBefore(long offset); + Optional valueAtOrBefore(long offset); /** * Returns the value with the largest offset. * * @return the value if it exists, otherwise {@code Optional.empty()} */ - public Optional> lastEntry(); + Optional> lastEntry(); /** * Removes all entries with an offset greater than or equal to {@code endOffset}. * * @param endOffset the ending offset */ - public void truncateNewEntries(long endOffset); + void truncateNewEntries(long endOffset); /** * Removes all entries but the last entry that has an offset that is less than or equal to @@ -69,14 +69,14 @@ public interface History { * * @param startOffset the starting offset */ - public void truncateOldEntries(long startOffset); + void truncateOldEntries(long startOffset); /** * Removes all of the values from this object. */ - public void clear(); + void clear(); - final public static class Entry { + final static class Entry { private final long offset; private final T value; diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/PartitionListener.java b/raft/src/main/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachine.java similarity index 87% rename from raft/src/main/java/org/apache/kafka/raft/internals/PartitionListener.java rename to raft/src/main/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachine.java index d2e58420ec94..b544231bf371 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/PartitionListener.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachine.java @@ -36,22 +36,28 @@ /** * The KRaft state machine for tracking control records in the topic partition. * - * This type keeps track of changes to the finalized kraft.version and the sets of voters. + * This type keeps track of changes to the finalized kraft.version and the sets of voters between + * the latest snasphot and the log end offset. + * + * The are two actors/threads for this type. One is the KRaft driver which indirectly call a lot of + * the public methods. The other are the callers of {@code RaftClient::createSnapshot} which + * indirectly call {@code voterSetAtOffset} and {@code kraftVersionAtOffset} when freezing a snapshot. */ -final public class PartitionListener { +final public class KRaftControlRecordStateMachine { private final ReplicatedLog log; private final RecordSerde serde; private final BufferSupplier bufferSupplier; private final Logger logger; private final int maxBatchSizeBytes; - // These are objects are synchronized using the perspective object monitor. The two actors - // are the KRaft driver and the RaftClient callers + // These objects are synchronized using their respective object monitor. The two actors + // are the KRaft driver when calling updateState and the RaftClient callers when freezing + // snapshots private final VoterSetHistory voterSetHistory; private final History kraftVersionHistory = new TreeMapHistory<>(); // This synchronization is enough because - // 1. The write operation updateListener only sets the value without reading and updates to + // 1. The write operation updateState only sets the value without reading it and updates to // voterSetHistory or kraftVersionHistory are done before setting the nextOffset // // 2. The read operations lastVoterSet, voterSetAtOffset and kraftVersionAtOffset read @@ -68,7 +74,7 @@ final public class PartitionListener { * @param maxBatchSizeBytes the maximum size of record batch * @param logContext the log context */ - public PartitionListener( + public KRaftControlRecordStateMachine( Optional staticVoterSet, ReplicatedLog log, RecordSerde serde, @@ -87,7 +93,7 @@ public PartitionListener( /** * Must be called whenever the {@code log} has changed. */ - public void updateListener() { + public void updateState() { maybeLoadSnapshot(); maybeLoadLog(); } @@ -130,22 +136,13 @@ public VoterSet lastVoterSet() { } /** - * Rturns the voter set at a given offset. + * Returns the voter set at a given offset. * * @param offset the offset (inclusive) * @return the voter set if one exist, otherwise {@code Optional.empty()} */ public Optional voterSetAtOffset(long offset) { - long fixedNextOffset = nextOffset; - if (offset >= fixedNextOffset) { - throw new IllegalArgumentException( - String.format( - "Attempting the read the voter set at an offset (%d) which kraft hasn't seen (%d)", - offset, - fixedNextOffset - 1 - ) - ); - } + checkOffsetIsValid(offset); synchronized (voterSetHistory) { return voterSetHistory.valueAtOrBefore(offset); @@ -159,20 +156,25 @@ public Optional voterSetAtOffset(long offset) { * @return the finalized kraft version if one exist, otherwise 0 */ public short kraftVersionAtOffset(long offset) { + checkOffsetIsValid(offset); + + synchronized (kraftVersionHistory) { + return kraftVersionHistory.valueAtOrBefore(offset).orElse((short) 0); + } + } + + private void checkOffsetIsValid(long offset) { long fixedNextOffset = nextOffset; if (offset >= fixedNextOffset) { throw new IllegalArgumentException( String.format( - "Attempting the read the kraft.version at an offset (%d) which kraft hasn't seen (%d)", + "Attempting the read a value at an offset (%d) which is greater than or " + + "equal to the largest known offset (%d)", offset, fixedNextOffset - 1 ) ); } - - synchronized (kraftVersionHistory) { - return kraftVersionHistory.valueAtOrBefore(offset).orElse((short) 0); - } } private void maybeLoadLog() { @@ -233,9 +235,9 @@ private void maybeLoadSnapshot() { } private void handleBatch(Batch batch, OptionalLong overrideOffset) { - int index = 0; + int offsetDelta = 0; for (ControlRecord record : batch.controlRecords()) { - long currentOffset = overrideOffset.orElse(batch.baseOffset() + index); + long currentOffset = overrideOffset.orElse(batch.baseOffset() + offsetDelta); switch (record.type()) { case KRAFT_VOTERS: synchronized (voterSetHistory) { @@ -253,7 +255,7 @@ private void handleBatch(Batch batch, OptionalLong overrideOffset) { // Skip the rest of the control records break; } - ++index; + ++offsetDelta; } } } diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java index 11d0279918a1..a1f0b34ab67d 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java @@ -22,10 +22,10 @@ * A type for storing the historical value of the set of voters. * * This type can be use to keep track in-memory the sets for voters stored in the latest snapshot - * and log. This is useful when both generating a new snapshot at a given offset or when evaulating + * and log. This is useful when generating a new snapshot at a given offset or when evaulating * the latest set of voters. */ -final public class VoterSetHistory implements History { +final public class VoterSetHistory { private final Optional staticVoterSet; private final History votersHistory = new TreeMapHistory<>(); @@ -33,7 +33,16 @@ final public class VoterSetHistory implements History { this.staticVoterSet = staticVoterSet; } - @Override + /** + * Add a new value at a given offset. + * + * The provided {@code offset} must be greater than or equal to 0 and must be greater than the + * offset of all previous calls to this method. + * + * @param offset the offset + * @param value the value to store + * @throws IllegalArgumentException if the offset is not greater than all previous offsets + */ public void addAt(long offset, VoterSet voters) { Optional> lastEntry = votersHistory.lastEntry(); if (lastEntry.isPresent() && lastEntry.get().offset() >= 0) { @@ -65,39 +74,50 @@ public void addAt(long offset, VoterSet voters) { * @param offset the offset (inclusive) * @return the voter set if one exist, otherwise {@code Optional.empty()} */ - @Override public Optional valueAtOrBefore(long offset) { return votersHistory.valueAtOrBefore(offset); } - @Override - public Optional> lastEntry() { - Optional> result = votersHistory.lastEntry(); - if (result.isPresent()) return result; - - return staticVoterSet.map(value -> new History.Entry<>(-1, value)); - } - /** * Returns the latest set of voters. */ public VoterSet lastValue() { - return lastEntry() - .orElseThrow(() -> new IllegalStateException("No voter set found")) - .value(); + Optional> result = votersHistory.lastEntry(); + if (result.isPresent()) { + return result.get().value(); + } + + return staticVoterSet + .orElseThrow(() -> new IllegalStateException("No voter set found")); } - @Override + /** + * Removes all entries with an offset greater than or equal to {@code endOffset}. + * + * @param endOffset the ending offset + */ public void truncateNewEntries(long endOffset) { votersHistory.truncateNewEntries(endOffset); } - @Override + /** + * Removes all entries but the last entry that has an offset that is less than or equal to + * {@code startOffset}. + * + * This operation does not remove the entry with the largest offset that is less than or equal + * to {@code startOffset}. This is needed so that calls to {@code valueAtOrBefore} and + * {@code lastEntry} always return a non-empty value if a value was previously added to this + * object. + * + * @param startOffset the starting offset + */ public void truncateOldEntries(long startOffset) { votersHistory.truncateOldEntries(startOffset); } - @Override + /** + * Removes all of the values from this object. + */ public void clear() { votersHistory.clear(); } diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/PartitionListenerTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachineTest.java similarity index 74% rename from raft/src/test/java/org/apache/kafka/raft/internals/PartitionListenerTest.java rename to raft/src/test/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachineTest.java index ba3500825aef..b8bbf1e8e2ef 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/PartitionListenerTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachineTest.java @@ -31,15 +31,15 @@ import org.junit.jupiter.api.Test; import static org.junit.jupiter.api.Assertions.assertEquals; -final class PartitionListenerTest { +final class KRaftControlRecordStateMachineTest { private static final RecordSerde STRING_SERDE = new StringSerde(); private static MockLog buildLog() { return new MockLog(new TopicPartition("partition", 0), Uuid.randomUuid(), new LogContext()); } - private static PartitionListener buildPartitionListener(MockLog log, Optional staticVoterSet) { - return new PartitionListener( + private static KRaftControlRecordStateMachine buildPartitionListener(MockLog log, Optional staticVoterSet) { + return new KRaftControlRecordStateMachine( staticVoterSet, log, STRING_SERDE, @@ -54,12 +54,12 @@ void testEmptyParition() { MockLog log = buildLog(); VoterSet voterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(1, 2, 3))); - PartitionListener partitionListener = buildPartitionListener(log, Optional.of(voterSet)); + KRaftControlRecordStateMachine partitionState = buildPartitionListener(log, Optional.of(voterSet)); // This should be a no-op operation - partitionListener.updateListener(); + partitionState.updateState(); - assertEquals(voterSet, partitionListener.lastVoterSet()); + assertEquals(voterSet, partitionState.lastVoterSet()); } @Test @@ -69,7 +69,7 @@ void testUpdateWithoutSnapshot() { BufferSupplier bufferSupplier = BufferSupplier.NO_CACHING; int epoch = 1; - PartitionListener partitionListener = buildPartitionListener(log, Optional.of(staticVoterSet)); + KRaftControlRecordStateMachine partitionState = buildPartitionListener(log, Optional.of(staticVoterSet)); // Append the kraft.version control record short kraftVersion = 1; @@ -98,11 +98,11 @@ void testUpdateWithoutSnapshot() { ); // Read the entire partition - partitionListener.updateListener(); + partitionState.updateState(); - assertEquals(voterSet, partitionListener.lastVoterSet()); - assertEquals(Optional.of(voterSet), partitionListener.voterSetAtOffset(log.endOffset().offset - 1)); - assertEquals(kraftVersion, partitionListener.kraftVersionAtOffset(log.endOffset().offset - 1)); + assertEquals(voterSet, partitionState.lastVoterSet()); + assertEquals(Optional.of(voterSet), partitionState.voterSetAtOffset(log.endOffset().offset - 1)); + assertEquals(kraftVersion, partitionState.kraftVersionAtOffset(log.endOffset().offset - 1)); } @Test @@ -112,7 +112,7 @@ void testUpdateWithEmptySnapshot() { BufferSupplier bufferSupplier = BufferSupplier.NO_CACHING; int epoch = 1; - PartitionListener partitionListener = buildPartitionListener(log, Optional.of(staticVoterSet)); + KRaftControlRecordStateMachine partitionState = buildPartitionListener(log, Optional.of(staticVoterSet)); // Create a snapshot that doesn't have any kraft.version or voter set control records RecordsSnapshotWriter.Builder builder = new RecordsSnapshotWriter.Builder() @@ -149,11 +149,11 @@ void testUpdateWithEmptySnapshot() { ); // Read the entire partition - partitionListener.updateListener(); + partitionState.updateState(); - assertEquals(voterSet, partitionListener.lastVoterSet()); - assertEquals(Optional.of(voterSet), partitionListener.voterSetAtOffset(log.endOffset().offset - 1)); - assertEquals(kraftVersion, partitionListener.kraftVersionAtOffset(log.endOffset().offset - 1)); + assertEquals(voterSet, partitionState.lastVoterSet()); + assertEquals(Optional.of(voterSet), partitionState.voterSetAtOffset(log.endOffset().offset - 1)); + assertEquals(kraftVersion, partitionState.kraftVersionAtOffset(log.endOffset().offset - 1)); } @Test @@ -162,7 +162,7 @@ void testUpdateWithSnapshot() { VoterSet staticVoterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(1, 2, 3))); int epoch = 1; - PartitionListener partitionListener = buildPartitionListener(log, Optional.of(staticVoterSet)); + KRaftControlRecordStateMachine partitionState = buildPartitionListener(log, Optional.of(staticVoterSet)); // Create a snapshot that has kraft.version and voter set control records short kraftVersion = 1; @@ -178,11 +178,11 @@ void testUpdateWithSnapshot() { log.truncateToLatestSnapshot(); // Read the entire partition - partitionListener.updateListener(); + partitionState.updateState(); - assertEquals(voterSet, partitionListener.lastVoterSet()); - assertEquals(Optional.of(voterSet), partitionListener.voterSetAtOffset(log.endOffset().offset - 1)); - assertEquals(kraftVersion, partitionListener.kraftVersionAtOffset(log.endOffset().offset - 1)); + assertEquals(voterSet, partitionState.lastVoterSet()); + assertEquals(Optional.of(voterSet), partitionState.voterSetAtOffset(log.endOffset().offset - 1)); + assertEquals(kraftVersion, partitionState.kraftVersionAtOffset(log.endOffset().offset - 1)); } @Test @@ -192,7 +192,7 @@ void testUpdateWithSnapshotAndLogOverride() { BufferSupplier bufferSupplier = BufferSupplier.NO_CACHING; int epoch = 1; - PartitionListener partitionListener = buildPartitionListener(log, Optional.of(staticVoterSet)); + KRaftControlRecordStateMachine partitionState = buildPartitionListener(log, Optional.of(staticVoterSet)); // Create a snapshot that has kraft.version and voter set control records short kraftVersion = 1; @@ -222,14 +222,14 @@ void testUpdateWithSnapshotAndLogOverride() { ); // Read the entire partition - partitionListener.updateListener(); + partitionState.updateState(); - assertEquals(voterSet, partitionListener.lastVoterSet()); - assertEquals(Optional.of(voterSet), partitionListener.voterSetAtOffset(log.endOffset().offset - 1)); - assertEquals(kraftVersion, partitionListener.kraftVersionAtOffset(log.endOffset().offset - 1)); + assertEquals(voterSet, partitionState.lastVoterSet()); + assertEquals(Optional.of(voterSet), partitionState.voterSetAtOffset(log.endOffset().offset - 1)); + assertEquals(kraftVersion, partitionState.kraftVersionAtOffset(log.endOffset().offset - 1)); // Check the voter set at the snapshot - assertEquals(Optional.of(snapshotVoterSet), partitionListener.voterSetAtOffset(snapshotId.offset() - 1)); + assertEquals(Optional.of(snapshotVoterSet), partitionState.voterSetAtOffset(snapshotId.offset() - 1)); } @Test @@ -239,7 +239,7 @@ void testTruncateTo() { BufferSupplier bufferSupplier = BufferSupplier.NO_CACHING; int epoch = 1; - PartitionListener partitionListener = buildPartitionListener(log, Optional.of(staticVoterSet)); + KRaftControlRecordStateMachine partitionState = buildPartitionListener(log, Optional.of(staticVoterSet)); // Append the kraft.version control record short kraftVersion = 1; @@ -283,21 +283,21 @@ void testTruncateTo() { ); // Read the entire partition - partitionListener.updateListener(); + partitionState.updateState(); - assertEquals(voterSet, partitionListener.lastVoterSet()); + assertEquals(voterSet, partitionState.lastVoterSet()); // Truncate log and listener log.truncateTo(voterSetOffset); - partitionListener.truncateNewEntries(voterSetOffset); + partitionState.truncateNewEntries(voterSetOffset); - assertEquals(firstVoterSet, partitionListener.lastVoterSet()); + assertEquals(firstVoterSet, partitionState.lastVoterSet()); // Truncate the entire log log.truncateTo(0); - partitionListener.truncateNewEntries(0); + partitionState.truncateNewEntries(0); - assertEquals(staticVoterSet, partitionListener.lastVoterSet()); + assertEquals(staticVoterSet, partitionState.lastVoterSet()); } @Test @@ -307,7 +307,7 @@ void testTrimPrefixTo() { BufferSupplier bufferSupplier = BufferSupplier.NO_CACHING; int epoch = 1; - PartitionListener partitionListener = buildPartitionListener(log, Optional.of(staticVoterSet)); + KRaftControlRecordStateMachine partitionState = buildPartitionListener(log, Optional.of(staticVoterSet)); // Append the kraft.version control record long kraftVersionOffset = log.endOffset().offset; @@ -352,24 +352,24 @@ void testTrimPrefixTo() { ); // Read the entire partition - partitionListener.updateListener(); + partitionState.updateState(); - assertEquals(voterSet, partitionListener.lastVoterSet()); - assertEquals(kraftVersion, partitionListener.kraftVersionAtOffset(kraftVersionOffset)); + assertEquals(voterSet, partitionState.lastVoterSet()); + assertEquals(kraftVersion, partitionState.kraftVersionAtOffset(kraftVersionOffset)); // Trim the prefix for the partition listener up to the kraft.version - partitionListener.truncateOldEntries(kraftVersionOffset); - assertEquals(kraftVersion, partitionListener.kraftVersionAtOffset(kraftVersionOffset)); + partitionState.truncateOldEntries(kraftVersionOffset); + assertEquals(kraftVersion, partitionState.kraftVersionAtOffset(kraftVersionOffset)); // Trim the prefix for the partition listener up to the first voter set - partitionListener.truncateOldEntries(firstVoterSetOffset); - assertEquals(kraftVersion, partitionListener.kraftVersionAtOffset(kraftVersionOffset)); - assertEquals(Optional.of(firstVoterSet), partitionListener.voterSetAtOffset(firstVoterSetOffset)); + partitionState.truncateOldEntries(firstVoterSetOffset); + assertEquals(kraftVersion, partitionState.kraftVersionAtOffset(kraftVersionOffset)); + assertEquals(Optional.of(firstVoterSet), partitionState.voterSetAtOffset(firstVoterSetOffset)); // Trim the prefix for the partition listener up to the second voter set - partitionListener.truncateOldEntries(voterSetOffset); - assertEquals(kraftVersion, partitionListener.kraftVersionAtOffset(kraftVersionOffset)); - assertEquals(Optional.empty(), partitionListener.voterSetAtOffset(firstVoterSetOffset)); - assertEquals(Optional.of(voterSet), partitionListener.voterSetAtOffset(voterSetOffset)); + partitionState.truncateOldEntries(voterSetOffset); + assertEquals(kraftVersion, partitionState.kraftVersionAtOffset(kraftVersionOffset)); + assertEquals(Optional.empty(), partitionState.voterSetAtOffset(firstVoterSetOffset)); + assertEquals(Optional.of(voterSet), partitionState.voterSetAtOffset(voterSetOffset)); } } diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java index 039239233c72..405bb7d026d9 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java @@ -102,13 +102,13 @@ void testTruncateTo() { votersHistory.addAt(200, voterSet12345); votersHistory.truncateNewEntries(201); - assertEquals(Optional.of(new History.Entry<>(200, voterSet12345)), votersHistory.lastEntry()); + assertEquals(voterSet12345, votersHistory.lastValue()); votersHistory.truncateNewEntries(200); - assertEquals(Optional.of(new History.Entry<>(100, voterSet1234)), votersHistory.lastEntry()); + assertEquals(voterSet1234, votersHistory.lastValue()); votersHistory.truncateNewEntries(101); - assertEquals(Optional.of(new History.Entry<>(100, voterSet1234)), votersHistory.lastEntry()); + assertEquals(voterSet1234, votersHistory.lastValue()); votersHistory.truncateNewEntries(100); - assertEquals(Optional.of(new History.Entry<>(-1, staticVoterSet)), votersHistory.lastEntry()); + assertEquals(staticVoterSet, votersHistory.lastValue()); } @Test @@ -162,6 +162,6 @@ void testClear() { votersHistory.clear(); - assertEquals(Optional.of(new History.Entry<>(-1, staticVoterSet)), votersHistory.lastEntry()); + assertEquals(staticVoterSet, votersHistory.lastValue()); } } From c821a838903235fc273b434e4086e4a16cf2d79c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Wed, 1 May 2024 15:05:46 -0700 Subject: [PATCH 31/34] KAFKA-16207; Review feedback changes --- .../kafka/common/record/MemoryRecords.java | 147 ++++++------------ .../main/scala/kafka/raft/RaftManager.scala | 2 +- .../kafka/raft/FileBasedStateStore.java | 14 +- .../apache/kafka/raft/KafkaRaftClient.java | 15 +- .../org/apache/kafka/raft/QuorumConfig.java | 7 +- .../org/apache/kafka/raft/QuorumState.java | 10 +- .../apache/kafka/raft/QuorumStateStore.java | 7 + .../raft/internals/BatchAccumulator.java | 100 ++++++------ .../KRaftControlRecordStateMachine.java | 2 +- .../{History.java => LogHistory.java} | 2 +- ...MapHistory.java => TreeMapLogHistory.java} | 4 +- .../apache/kafka/raft/internals/VoterSet.java | 54 +++---- .../kafka/raft/internals/VoterSetHistory.java | 6 +- .../kafka/snapshot/RecordsSnapshotWriter.java | 10 +- .../kafka/raft/MockQuorumStateStore.java | 8 + .../KRaftControlRecordStateMachineTest.java | 2 +- ...ryTest.java => TreeMapLogHistoryTest.java} | 20 +-- .../raft/internals/VoterSetHistoryTest.java | 49 ++++++ .../kafka/raft/internals/VoterSetTest.java | 23 ++- 19 files changed, 253 insertions(+), 229 deletions(-) rename raft/src/main/java/org/apache/kafka/raft/internals/{History.java => LogHistory.java} (99%) rename raft/src/main/java/org/apache/kafka/raft/internals/{TreeMapHistory.java => TreeMapLogHistory.java} (93%) rename raft/src/test/java/org/apache/kafka/raft/internals/{TreeMapHistoryTest.java => TreeMapLogHistoryTest.java} (82%) diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index d264f576e681..4eab7f7b658f 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -730,25 +730,15 @@ public static MemoryRecords withLeaderChangeMessage( ByteBuffer buffer, LeaderChangeMessage leaderChangeMessage ) { - writeLeaderChangeMessage(buffer, initialOffset, timestamp, leaderEpoch, leaderChangeMessage); - buffer.flip(); - return MemoryRecords.readableRecords(buffer); - } - - private static void writeLeaderChangeMessage( - ByteBuffer buffer, - long initialOffset, - long timestamp, - int leaderEpoch, - LeaderChangeMessage leaderChangeMessage - ) { - try (MemoryRecordsBuilder builder = new MemoryRecordsBuilder( - buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, - TimestampType.CREATE_TIME, initialOffset, timestamp, - RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, - false, true, leaderEpoch, buffer.capacity()) + try (MemoryRecordsBuilder builder = createKraftControlReccordBuilder( + initialOffset, + timestamp, + leaderEpoch, + buffer + ) ) { builder.appendLeaderChangeMessage(timestamp, leaderChangeMessage); + return builder.build(); } } @@ -759,25 +749,15 @@ public static MemoryRecords withSnapshotHeaderRecord( ByteBuffer buffer, SnapshotHeaderRecord snapshotHeaderRecord ) { - writeSnapshotHeaderRecord(buffer, initialOffset, timestamp, leaderEpoch, snapshotHeaderRecord); - buffer.flip(); - return MemoryRecords.readableRecords(buffer); - } - - private static void writeSnapshotHeaderRecord( - ByteBuffer buffer, - long initialOffset, - long timestamp, - int leaderEpoch, - SnapshotHeaderRecord snapshotHeaderRecord - ) { - try (MemoryRecordsBuilder builder = new MemoryRecordsBuilder( - buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, - TimestampType.CREATE_TIME, initialOffset, timestamp, - RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, - false, true, leaderEpoch, buffer.capacity()) + try (MemoryRecordsBuilder builder = createKraftControlReccordBuilder( + initialOffset, + timestamp, + leaderEpoch, + buffer + ) ) { builder.appendSnapshotHeaderMessage(timestamp, snapshotHeaderRecord); + return builder.build(); } } @@ -788,25 +768,15 @@ public static MemoryRecords withSnapshotFooterRecord( ByteBuffer buffer, SnapshotFooterRecord snapshotFooterRecord ) { - writeSnapshotFooterRecord(buffer, initialOffset, timestamp, leaderEpoch, snapshotFooterRecord); - buffer.flip(); - return MemoryRecords.readableRecords(buffer); - } - - private static void writeSnapshotFooterRecord( - ByteBuffer buffer, - long initialOffset, - long timestamp, - int leaderEpoch, - SnapshotFooterRecord snapshotFooterRecord - ) { - try (MemoryRecordsBuilder builder = new MemoryRecordsBuilder( - buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, - TimestampType.CREATE_TIME, initialOffset, timestamp, - RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, - false, true, leaderEpoch, buffer.capacity()) + try (MemoryRecordsBuilder builder = createKraftControlReccordBuilder( + initialOffset, + timestamp, + leaderEpoch, + buffer + ) ) { builder.appendSnapshotFooterMessage(timestamp, snapshotFooterRecord); + return builder.build(); } } @@ -817,35 +787,15 @@ public static MemoryRecords withKRaftVersionRecord( ByteBuffer buffer, KRaftVersionRecord kraftVersionRecord ) { - writeKRaftVersionRecord(buffer, initialOffset, timestamp, leaderEpoch, kraftVersionRecord); - buffer.flip(); - return MemoryRecords.readableRecords(buffer); - } - - private static void writeKRaftVersionRecord( - ByteBuffer buffer, - long initialOffset, - long timestamp, - int leaderEpoch, - KRaftVersionRecord kraftVersionRecord - ) { - try (MemoryRecordsBuilder builder = new MemoryRecordsBuilder( - buffer, - RecordBatch.CURRENT_MAGIC_VALUE, - CompressionType.NONE, - TimestampType.CREATE_TIME, + try (MemoryRecordsBuilder builder = createKraftControlReccordBuilder( initialOffset, timestamp, - RecordBatch.NO_PRODUCER_ID, - RecordBatch.NO_PRODUCER_EPOCH, - RecordBatch.NO_SEQUENCE, - false, - true, leaderEpoch, - buffer.capacity() + buffer ) ) { builder.appendKRaftVersionMessage(timestamp, kraftVersionRecord); + return builder.build(); } } @@ -856,35 +806,38 @@ public static MemoryRecords withVotersRecord( ByteBuffer buffer, VotersRecord votersRecord ) { - writeVotersRecord(buffer, initialOffset, timestamp, leaderEpoch, votersRecord); - buffer.flip(); - return MemoryRecords.readableRecords(buffer); - } - - private static void writeVotersRecord( - ByteBuffer buffer, - long initialOffset, - long timestamp, - int leaderEpoch, - VotersRecord votersRecord - ) { - try (MemoryRecordsBuilder builder = new MemoryRecordsBuilder( - buffer, - RecordBatch.CURRENT_MAGIC_VALUE, - CompressionType.NONE, - TimestampType.CREATE_TIME, + try (MemoryRecordsBuilder builder = createKraftControlReccordBuilder( initialOffset, timestamp, - RecordBatch.NO_PRODUCER_ID, - RecordBatch.NO_PRODUCER_EPOCH, - RecordBatch.NO_SEQUENCE, - false, - true, leaderEpoch, - buffer.capacity() + buffer ) ) { builder.appendVotersMessage(timestamp, votersRecord); + return builder.build(); } } + + private static MemoryRecordsBuilder createKraftControlReccordBuilder( + long initialOffset, + long timestamp, + int leaderEpoch, + ByteBuffer buffer + ) { + return new MemoryRecordsBuilder( + buffer, + RecordBatch.CURRENT_MAGIC_VALUE, + CompressionType.NONE, + TimestampType.CREATE_TIME, + initialOffset, + timestamp, + RecordBatch.NO_PRODUCER_ID, + RecordBatch.NO_PRODUCER_EPOCH, + RecordBatch.NO_SEQUENCE, + false, + true, + leaderEpoch, + buffer.capacity() + ); + } } diff --git a/core/src/main/scala/kafka/raft/RaftManager.scala b/core/src/main/scala/kafka/raft/RaftManager.scala index 63ec0d041055..0430952e17cb 100644 --- a/core/src/main/scala/kafka/raft/RaftManager.scala +++ b/core/src/main/scala/kafka/raft/RaftManager.scala @@ -184,7 +184,7 @@ class KafkaRaftManager[T]( client.initialize( controllerQuorumVotersFuture.get(), config.controllerListenerNames.head, - new FileBasedStateStore(new File(dataDir, "quorum-state")), + new FileBasedStateStore(new File(dataDir, FileBasedStateStore.DEFAULT_FILE_NAME)), metrics ) netChannel.start() diff --git a/raft/src/main/java/org/apache/kafka/raft/FileBasedStateStore.java b/raft/src/main/java/org/apache/kafka/raft/FileBasedStateStore.java index 0d543767fa37..a805d6394328 100644 --- a/raft/src/main/java/org/apache/kafka/raft/FileBasedStateStore.java +++ b/raft/src/main/java/org/apache/kafka/raft/FileBasedStateStore.java @@ -38,6 +38,7 @@ import java.io.OutputStreamWriter; import java.nio.charset.StandardCharsets; import java.nio.file.Files; +import java.nio.file.Path; import java.util.List; import java.util.OptionalInt; import java.util.Set; @@ -60,12 +61,14 @@ * */ public class FileBasedStateStore implements QuorumStateStore { private static final Logger log = LoggerFactory.getLogger(FileBasedStateStore.class); + private static final String DATA_VERSION = "data_version"; - private final File stateFile; - - static final String DATA_VERSION = "data_version"; static final short HIGHEST_SUPPORTED_VERSION = 0; + public static final String DEFAULT_FILE_NAME = "quorum-state"; + + private final File stateFile; + public FileBasedStateStore(final File stateFile) { this.stateFile = stateFile; } @@ -134,6 +137,11 @@ public void writeElectionState(ElectionState latest) { writeElectionStateToFile(stateFile, data); } + @Override + public Path path() { + return stateFile.toPath(); + } + private List voters(Set votersId) { return votersId.stream().map( voterId -> new Voter().setVoterId(voterId)).collect(Collectors.toList()); 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 e9acc3ad820c..0662af6875c4 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -170,12 +170,17 @@ final public class KafkaRaftClient implements RaftClient { private final Map, ListenerContext> listenerContexts = new IdentityHashMap<>(); private final ConcurrentLinkedQueue> pendingRegistrations = new ConcurrentLinkedQueue<>(); - // These components need to be initialized by the method initialize() because they depend on the voter set + // These components need to be initialized by the method initialize() because they depend on + // the voter set /* - * The key invariant for the kraft control record state machine is that it has always read to the LEO. This is achived by: + * The key invariant for the kraft control record state machine is that it has always read to + * the LEO. This is achieved by: + * * 1. reading the entire partition (snapshot and log) at start up, - * 2. updating the state when a snapshot is replaced, because of FETCH_SNAPSHOT, on the followers - * 3. updating the state when the leader (call to append()) or follower (FETCH) appends to the log + * 2. updating the state when a snapshot is replaced, because of FETCH_SNAPSHOT, on the + * followers + * 3. updating the state when the leader (call to append()) or follower (FETCH) appends to the + * log * 4. truncate new entries when a follower truncates their log * 5. truncate old entries when a snapshot gets generated */ @@ -370,7 +375,7 @@ public void initialize( Metrics metrics ) { partitionState = new KRaftControlRecordStateMachine( - Optional.of(VoterSet.fromAddressSpecs(listenerName, voterAddresses)), + Optional.of(VoterSet.fromInetSocketAddresses(listenerName, voterAddresses)), log, serde, BufferSupplier.create(), diff --git a/raft/src/main/java/org/apache/kafka/raft/QuorumConfig.java b/raft/src/main/java/org/apache/kafka/raft/QuorumConfig.java index 4e3c455b485c..fbfaa82d4649 100644 --- a/raft/src/main/java/org/apache/kafka/raft/QuorumConfig.java +++ b/raft/src/main/java/org/apache/kafka/raft/QuorumConfig.java @@ -159,7 +159,10 @@ public static Set parseVoterIds(List voterEntries) { return parseVoterConnections(voterEntries, false).keySet(); } - private static Map parseVoterConnections(List voterEntries, boolean routableOnly) { + private static Map parseVoterConnections( + List voterEntries, + boolean requireRoutableAddresses + ) { Map voterMap = new HashMap<>(); for (String voterMapEntry : voterEntries) { String[] idAndAddress = voterMapEntry.split("@"); @@ -184,7 +187,7 @@ private static Map parseVoterConnections(List valueCreator) { + public void appendControlMessages(Function valueCreator) { appendLock.lock(); try { ByteBuffer buffer = memoryPool.tryAllocate(maxBatchSize); if (buffer != null) { try { forceDrain(); - CreatedRecords createdRecords = valueCreator.apply(buffer); + MemoryRecords memoryRecords = valueCreator.apply(buffer); + + int numberOfRecords = validateMemoryRecordAndReturnCount(memoryRecords); + completed.add( new CompletedBatch<>( nextOffset, - createdRecords.numberOfRecords(), - createdRecords.records(), + numberOfRecords, + memoryRecords, memoryPool, buffer ) ); - nextOffset += 1; + nextOffset += numberOfRecords; } catch (Exception e) { // Release the buffer now since the buffer was not stored in completed for a delayed release memoryPool.release(buffer); @@ -246,6 +251,28 @@ public void appendControlMessages(Function valueCrea } } + private int validateMemoryRecordAndReturnCount(MemoryRecords memoryRecord) { + // Confirm that it is at most one batch and it is a control record + Iterator batches = memoryRecord.batches().iterator(); + if (!batches.hasNext()) { + throw new IllegalArgumentException("valueCreator didn't create a batch"); + } + + MutableRecordBatch batch = batches.next(); + if (!batch.isControlBatch()) { + throw new IllegalArgumentException("valueCreator didn't creatte a control batch"); + } + + Integer numberOfRecords = batch.countOrNull(); + if (numberOfRecords == null) { + throw new IllegalArgumentException("valueCreator didn't create a batch with the count"); + } else if (batches.hasNext()) { + throw new IllegalArgumentException("valueCreator created more than one batch"); + } + + return numberOfRecords; + } + /** * Append a {@link LeaderChangeMessage} record to the batch * @@ -258,15 +285,12 @@ public void appendLeaderChangeMessage( long currentTimestamp ) { appendControlMessages(buffer -> - new CreatedRecords( - 1, - MemoryRecords.withLeaderChangeMessage( - this.nextOffset, - currentTimestamp, - this.epoch, - buffer, - leaderChangeMessage - ) + MemoryRecords.withLeaderChangeMessage( + this.nextOffset, + currentTimestamp, + this.epoch, + buffer, + leaderChangeMessage ) ); } @@ -284,15 +308,12 @@ public void appendSnapshotHeaderRecord( long currentTimestamp ) { appendControlMessages(buffer -> - new CreatedRecords( - 1, - MemoryRecords.withSnapshotHeaderRecord( - this.nextOffset, - currentTimestamp, - this.epoch, - buffer, - snapshotHeaderRecord - ) + MemoryRecords.withSnapshotHeaderRecord( + this.nextOffset, + currentTimestamp, + this.epoch, + buffer, + snapshotHeaderRecord ) ); } @@ -309,15 +330,12 @@ public void appendSnapshotFooterRecord( long currentTimestamp ) { appendControlMessages(buffer -> - new CreatedRecords( - 1, - MemoryRecords.withSnapshotFooterRecord( - this.nextOffset, - currentTimestamp, - this.epoch, - buffer, - snapshotFooterRecord - ) + MemoryRecords.withSnapshotFooterRecord( + this.nextOffset, + currentTimestamp, + this.epoch, + buffer, + snapshotFooterRecord ) ); } @@ -529,24 +547,6 @@ public long appendTimestamp() { } } - final public static class CreatedRecords { - private final int numberOfRecords; - private final MemoryRecords records; - - public CreatedRecords(int numberOfRecords, MemoryRecords records) { - this.numberOfRecords = numberOfRecords; - this.records = records; - } - - public int numberOfRecords() { - return numberOfRecords; - } - - public MemoryRecords records() { - return records; - } - } - private static class SimpleTimer { // We use an atomic long so that the Raft IO thread can query the linger // time without any locking diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachine.java b/raft/src/main/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachine.java index b544231bf371..4dab03bf7980 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachine.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachine.java @@ -54,7 +54,7 @@ final public class KRaftControlRecordStateMachine { // are the KRaft driver when calling updateState and the RaftClient callers when freezing // snapshots private final VoterSetHistory voterSetHistory; - private final History kraftVersionHistory = new TreeMapHistory<>(); + private final LogHistory kraftVersionHistory = new TreeMapLogHistory<>(); // This synchronization is enough because // 1. The write operation updateState only sets the value without reading it and updates to diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/History.java b/raft/src/main/java/org/apache/kafka/raft/internals/LogHistory.java similarity index 99% rename from raft/src/main/java/org/apache/kafka/raft/internals/History.java rename to raft/src/main/java/org/apache/kafka/raft/internals/LogHistory.java index 8c846fe8ee91..6751400678e1 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/History.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/LogHistory.java @@ -22,7 +22,7 @@ /** * An object that tracks values of {@code T} at different offsets. */ -public interface History { +public interface LogHistory { /** * Add a new value at a given offset. * diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/TreeMapHistory.java b/raft/src/main/java/org/apache/kafka/raft/internals/TreeMapLogHistory.java similarity index 93% rename from raft/src/main/java/org/apache/kafka/raft/internals/TreeMapHistory.java rename to raft/src/main/java/org/apache/kafka/raft/internals/TreeMapLogHistory.java index 8509737d9c19..2a27f96aeece 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/TreeMapHistory.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/TreeMapLogHistory.java @@ -22,9 +22,9 @@ import java.util.Map; /** - * A implementation for {@code History} which uses a red-black tree to store values sorted by offset. + * A implementation for {@code LogHistory} which uses a red-black tree to store values sorted by offset. */ -final public class TreeMapHistory implements History { +final public class TreeMapLogHistory implements LogHistory { private final NavigableMap history = new TreeMap<>(); @Override diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java index 78cb8638c5a2..ae0ec4d4b669 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java @@ -82,12 +82,12 @@ public Set voterIds() { * @return a new voter set if the voter was added, otherwise {@code Optional.empty()} */ public Optional addVoter(VoterNode voter) { - if (voters.containsKey(voter.id())) { + if (voters.containsKey(voter.voterKey().id())) { return Optional.empty(); } HashMap newVoters = new HashMap<>(voters); - newVoters.put(voter.id(), voter); + newVoters.put(voter.voterKey().id(), voter); return Optional.of(new VoterSet(newVoters)); } @@ -99,15 +99,14 @@ public Optional addVoter(VoterNode voter) { * * A voter can be removed from the voter set if its id and directory id match. * - * @param voterId the voter id - * @param voterDirectoryId the voter directory id + * @param voterKey the voter key * @return a new voter set if the voter was removed, otherwise {@code Optional.empty()} */ - public Optional removeVoter(int voterId, Optional voterDirectoryId) { - VoterNode oldVoter = voters.get(voterId); - if (oldVoter != null && Objects.equals(oldVoter.directoryId(), voterDirectoryId)) { + public Optional removeVoter(VoterKey voterKey) { + VoterNode oldVoter = voters.get(voterKey.id()); + if (oldVoter != null && Objects.equals(oldVoter.voterKey(), voterKey)) { HashMap newVoters = new HashMap<>(voters); - newVoters.remove(voterId); + newVoters.remove(voterKey.id()); return Optional.of(new VoterSet(newVoters)); } @@ -139,8 +138,8 @@ public VotersRecord toVotersRecord(short version) { .setMaxSupportedVersion(voter.supportedKRaftVersion().max()); return new VotersRecord.Voter() - .setVoterId(voter.id()) - .setVoterDirectoryId(voter.directoryId().orElse(Uuid.ZERO_UUID)) + .setVoterId(voter.voterKey().id()) + .setVoterDirectoryId(voter.voterKey().directoryId().orElse(Uuid.ZERO_UUID)) .setEndpoints(new VotersRecord.EndpointCollection(endpoints)) .setKRaftVersionFeature(kraftVersionFeature); }; @@ -207,19 +206,20 @@ public String toString() { return String.format("VoterSet(voters=%s)", voters); } - final static class VoterKey { + public final static class VoterKey { private final int id; private final Optional directoryId; - VoterKey(int id, Optional directoryId) { + private VoterKey(int id, Optional directoryId) { this.id = id; this.directoryId = directoryId; } - int id() { + + public int id() { return id; } - Optional directoryId() { + public Optional directoryId() { return directoryId; } @@ -245,6 +245,10 @@ public int hashCode() { public String toString() { return String.format("VoterKey(id=%d, directoryId=%s)", id, directoryId); } + + public static VoterKey of(int id, Optional directoryId) { + return new VoterKey(id, directoryId); + } } final static class VoterNode { @@ -253,24 +257,15 @@ final static class VoterNode { private final SupportedVersionRange supportedKRaftVersion; VoterNode( - int id, - Optional directoryId, + VoterKey voterKey, Map listeners, SupportedVersionRange supportedKRaftVersion ) { - this.voterKey = new VoterKey(id, directoryId); + this.voterKey = voterKey; this.listeners = listeners; this.supportedKRaftVersion = supportedKRaftVersion; } - int id() { - return voterKey.id; - } - - Optional directoryId() { - return voterKey.directoryId; - } - VoterKey voterKey() { return voterKey; } @@ -342,8 +337,7 @@ public static VoterSet fromVotersRecord(VotersRecord voters) { voterNodes.put( voter.voterId(), new VoterNode( - voter.voterId(), - directoryId, + VoterKey.of(voter.voterId(), directoryId), listeners, new SupportedVersionRange( voter.kRaftVersionFeature().minSupportedVersion(), @@ -363,7 +357,7 @@ public static VoterSet fromVotersRecord(VotersRecord voters) { * @param voters the socket addresses by voter id * @return the voter set */ - public static VoterSet fromAddressSpecs(String listener, Map voters) { + public static VoterSet fromInetSocketAddresses(String listener, Map voters) { Map voterNodes = voters .entrySet() .stream() @@ -371,13 +365,13 @@ public static VoterSet fromAddressSpecs(String listener, Map new VoterNode( - entry.getKey(), - Optional.empty(), + VoterKey.of(entry.getKey(), Optional.empty()), Collections.singletonMap(listener, entry.getValue()), new SupportedVersionRange((short) 0, (short) 0) ) ) ); + return new VoterSet(voterNodes); } } diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java index a1f0b34ab67d..3a15d62c5a91 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java @@ -27,7 +27,7 @@ */ final public class VoterSetHistory { private final Optional staticVoterSet; - private final History votersHistory = new TreeMapHistory<>(); + private final LogHistory votersHistory = new TreeMapLogHistory<>(); VoterSetHistory(Optional staticVoterSet) { this.staticVoterSet = staticVoterSet; @@ -44,7 +44,7 @@ final public class VoterSetHistory { * @throws IllegalArgumentException if the offset is not greater than all previous offsets */ public void addAt(long offset, VoterSet voters) { - Optional> lastEntry = votersHistory.lastEntry(); + Optional> lastEntry = votersHistory.lastEntry(); if (lastEntry.isPresent() && lastEntry.get().offset() >= 0) { // If the last voter set comes from the replicated log then the majorities must overlap. // This ignores the static voter set and the bootstrapped voter set since they come from @@ -82,7 +82,7 @@ public Optional valueAtOrBefore(long offset) { * Returns the latest set of voters. */ public VoterSet lastValue() { - Optional> result = votersHistory.lastEntry(); + Optional> result = votersHistory.lastEntry(); if (result.isPresent()) { return result.get().value(); } diff --git a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java index 20ad51946c1d..c587d47530f2 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java @@ -23,7 +23,6 @@ import org.apache.kafka.common.message.SnapshotHeaderRecord; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.ControlRecordUtils; -import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecordsBuilder; import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.record.TimestampType; @@ -215,7 +214,6 @@ public RecordsSnapshotWriter build(RecordSerde serde) { writer.accumulator.appendControlMessages(buffer -> { long now = time.milliseconds(); - int numberOfRecords = 0; try (MemoryRecordsBuilder builder = new MemoryRecordsBuilder( buffer, RecordBatch.CURRENT_MAGIC_VALUE, @@ -232,7 +230,6 @@ public RecordsSnapshotWriter build(RecordSerde serde) { buffer.capacity() ) ) { - numberOfRecords++; builder.appendSnapshotHeaderMessage( now, new SnapshotHeaderRecord() @@ -241,7 +238,6 @@ public RecordsSnapshotWriter build(RecordSerde serde) { ); if (kraftVersion > 0) { - numberOfRecords++; builder.appendKRaftVersionMessage( now, new KRaftVersionRecord() @@ -250,17 +246,15 @@ public RecordsSnapshotWriter build(RecordSerde serde) { ); if (voterSet.isPresent()) { - numberOfRecords++; builder.appendVotersMessage( now, voterSet.get().toVotersRecord(ControlRecordUtils.KRAFT_VOTERS_CURRENT_VERSION) ); } } - } - buffer.flip(); - return new BatchAccumulator.CreatedRecords(numberOfRecords, MemoryRecords.readableRecords(buffer)); + return builder.build(); + } }); writer.accumulator.forceDrain(); diff --git a/raft/src/test/java/org/apache/kafka/raft/MockQuorumStateStore.java b/raft/src/test/java/org/apache/kafka/raft/MockQuorumStateStore.java index 02e3bc82f59a..87f7c0d91b51 100644 --- a/raft/src/test/java/org/apache/kafka/raft/MockQuorumStateStore.java +++ b/raft/src/test/java/org/apache/kafka/raft/MockQuorumStateStore.java @@ -16,6 +16,9 @@ */ package org.apache.kafka.raft; +import java.nio.file.FileSystems; +import java.nio.file.Path; + public class MockQuorumStateStore implements QuorumStateStore { private ElectionState current; @@ -29,6 +32,11 @@ public void writeElectionState(ElectionState update) { this.current = update; } + @Override + public Path path() { + return FileSystems.getDefault().getPath("mock-file"); + } + @Override public void clear() { current = null; diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachineTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachineTest.java index b8bbf1e8e2ef..355085c95f17 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachineTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachineTest.java @@ -50,7 +50,7 @@ private static KRaftControlRecordStateMachine buildPartitionListener(MockLog log } @Test - void testEmptyParition() { + void testEmptyPartition() { MockLog log = buildLog(); VoterSet voterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(1, 2, 3))); diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/TreeMapHistoryTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/TreeMapLogHistoryTest.java similarity index 82% rename from raft/src/test/java/org/apache/kafka/raft/internals/TreeMapHistoryTest.java rename to raft/src/test/java/org/apache/kafka/raft/internals/TreeMapLogHistoryTest.java index b8bcaf06f1d8..596d89f337c2 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/TreeMapHistoryTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/TreeMapLogHistoryTest.java @@ -21,17 +21,17 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; -final public class TreeMapHistoryTest { +final public class TreeMapLogHistoryTest { @Test void testEmpty() { - TreeMapHistory history = new TreeMapHistory<>(); + TreeMapLogHistory history = new TreeMapLogHistory<>(); assertEquals(Optional.empty(), history.valueAtOrBefore(100)); assertEquals(Optional.empty(), history.lastEntry()); } @Test void testAddAt() { - TreeMapHistory history = new TreeMapHistory<>(); + TreeMapLogHistory history = new TreeMapLogHistory<>(); assertThrows(IllegalArgumentException.class, () -> history.addAt(-1, "")); assertEquals(Optional.empty(), history.lastEntry()); @@ -48,23 +48,23 @@ void testAddAt() { assertEquals(Optional.of("100"), history.valueAtOrBefore(199)); assertEquals(Optional.of("200"), history.valueAtOrBefore(200)); assertEquals(Optional.of("200"), history.valueAtOrBefore(201)); - assertEquals(Optional.of(new History.Entry<>(200, "200")), history.lastEntry()); + assertEquals(Optional.of(new LogHistory.Entry<>(200, "200")), history.lastEntry()); } @Test void testTruncateTo() { - TreeMapHistory history = new TreeMapHistory<>(); + TreeMapLogHistory history = new TreeMapLogHistory<>(); history.addAt(100, "100"); history.addAt(200, "200"); history.truncateNewEntries(201); - assertEquals(Optional.of(new History.Entry<>(200, "200")), history.lastEntry()); + assertEquals(Optional.of(new LogHistory.Entry<>(200, "200")), history.lastEntry()); history.truncateNewEntries(200); - assertEquals(Optional.of(new History.Entry<>(100, "100")), history.lastEntry()); + assertEquals(Optional.of(new LogHistory.Entry<>(100, "100")), history.lastEntry()); history.truncateNewEntries(101); - assertEquals(Optional.of(new History.Entry<>(100, "100")), history.lastEntry()); + assertEquals(Optional.of(new LogHistory.Entry<>(100, "100")), history.lastEntry()); history.truncateNewEntries(100); assertEquals(Optional.empty(), history.lastEntry()); @@ -72,7 +72,7 @@ void testTruncateTo() { @Test void testTrimPrefixTo() { - TreeMapHistory history = new TreeMapHistory<>(); + TreeMapLogHistory history = new TreeMapLogHistory<>(); history.addAt(100, "100"); history.addAt(200, "200"); @@ -95,7 +95,7 @@ void testTrimPrefixTo() { @Test void testClear() { - TreeMapHistory history = new TreeMapHistory<>(); + TreeMapLogHistory history = new TreeMapLogHistory<>(); history.addAt(100, "100"); history.addAt(200, "200"); history.clear(); diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java index 405bb7d026d9..c5c26e3dfb3f 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java @@ -85,6 +85,55 @@ void testAddAt() { assertEquals(Optional.of(removedVoterSet), votersHistory.valueAtOrBefore(200)); } + @Test + void testAddAtNonOverlapping() { + VoterSetHistory votersHistory = new VoterSetHistory(Optional.empty()); + + Map voterMap = VoterSetTest.voterMap(Arrays.asList(1, 2, 3)); + VoterSet voterSet = new VoterSet(new HashMap<>(voterMap)); + + // Add a starting voter to the history + votersHistory.addAt(100, voterSet); + + // Remove voter so that it doesn't overlap + VoterSet nonoverlappingRemovedSet = voterSet + .removeVoter(voterMap.get(1).voterKey()).get() + .removeVoter(voterMap.get(2).voterKey()).get(); + + assertThrows( + IllegalArgumentException.class, + () -> votersHistory.addAt(200, nonoverlappingRemovedSet) + ); + assertEquals(voterSet, votersHistory.lastValue()); + + + // Add voters so that it doesn't overlap + VoterSet nonoverlappingAddSet = voterSet + .addVoter(VoterSetTest.voterNode(4)).get() + .addVoter(VoterSetTest.voterNode(5)).get(); + + assertThrows( + IllegalArgumentException.class, + () -> votersHistory.addAt(200, nonoverlappingAddSet) + ); + assertEquals(voterSet, votersHistory.lastValue()); + } + + @Test + void testNonoverlappingFromStaticVoterSet() { + Map voterMap = VoterSetTest.voterMap(Arrays.asList(1, 2, 3)); + VoterSet staticVoterSet = new VoterSet(new HashMap<>(voterMap)); + VoterSetHistory votersHistory = new VoterSetHistory(Optional.empty()); + + // Remove voter so that it doesn't overlap + VoterSet nonoverlappingRemovedSet = staticVoterSet + .removeVoter(voterMap.get(1).voterKey()).get() + .removeVoter(voterMap.get(2).voterKey()).get(); + + votersHistory.addAt(100, nonoverlappingRemovedSet); + assertEquals(nonoverlappingRemovedSet, votersHistory.lastValue()); + } + @Test void testTruncateTo() { Map voterMap = VoterSetTest.voterMap(Arrays.asList(1, 2, 3)); diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetTest.java index d1afd59361ed..6226a88c790d 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetTest.java @@ -60,7 +60,7 @@ void testAddVoter() { assertEquals(Optional.empty(), voterSet.addVoter(voterNode(1))); VoterSet.VoterNode voter4 = voterNode(4); - aVoterMap.put(voter4.id(), voter4); + aVoterMap.put(voter4.voterKey().id(), voter4); assertEquals(Optional.of(new VoterSet(new HashMap<>(aVoterMap))), voterSet.addVoter(voter4)); } @@ -69,13 +69,13 @@ void testRemoveVoter() { Map aVoterMap = voterMap(Arrays.asList(1, 2, 3)); VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap)); - assertEquals(Optional.empty(), voterSet.removeVoter(4, Optional.empty())); - assertEquals(Optional.empty(), voterSet.removeVoter(4, Optional.of(Uuid.randomUuid()))); + assertEquals(Optional.empty(), voterSet.removeVoter(VoterSet.VoterKey.of(4, Optional.empty()))); + assertEquals(Optional.empty(), voterSet.removeVoter(VoterSet.VoterKey.of(4, Optional.of(Uuid.randomUuid())))); VoterSet.VoterNode voter3 = aVoterMap.remove(3); assertEquals( Optional.of(new VoterSet(new HashMap<>(aVoterMap))), - voterSet.removeVoter(voter3.id(), voter3.directoryId()) + voterSet.removeVoter(voter3.voterKey()) ); } @@ -97,12 +97,12 @@ void testOverlappingMajority() { assertMajorities(true, startingVoterSet, biggerVoterSet); VoterSet smallerVoterSet = startingVoterSet - .removeVoter(1, startingVoterMap.get(1).directoryId()) + .removeVoter(startingVoterMap.get(1).voterKey()) .get(); assertMajorities(true, startingVoterSet, smallerVoterSet); VoterSet replacedVoterSet = startingVoterSet - .removeVoter(1, startingVoterMap.get(1).directoryId()) + .removeVoter(startingVoterMap.get(1).voterKey()) .get() .addVoter(voterNode(1)) .get(); @@ -124,19 +124,19 @@ void testNonoverlappingMajority() { // Two removals don't have an overlapping majority VoterSet smallerVoterSet = startingVoterSet - .removeVoter(1, startingVoterMap.get(1).directoryId()) + .removeVoter(startingVoterMap.get(1).voterKey()) .get() - .removeVoter(2, startingVoterMap.get(2).directoryId()) + .removeVoter(startingVoterMap.get(2).voterKey()) .get(); assertMajorities(false, startingVoterSet, smallerVoterSet); // Two replacements don't have an overlapping majority VoterSet replacedVoterSet = startingVoterSet - .removeVoter(1, startingVoterMap.get(1).directoryId()) + .removeVoter(startingVoterMap.get(1).voterKey()) .get() .addVoter(voterNode(1)) .get() - .removeVoter(2, startingVoterMap.get(2).directoryId()) + .removeVoter(startingVoterMap.get(2).voterKey()) .get() .addVoter(voterNode(2)) .get(); @@ -169,8 +169,7 @@ public static Map voterMap(List replicas) static VoterSet.VoterNode voterNode(int id) { return new VoterSet.VoterNode( - id, - Optional.of(Uuid.randomUuid()), + VoterSet.VoterKey.of(id, Optional.of(Uuid.randomUuid())), Collections.singletonMap( "LISTENER", InetSocketAddress.createUnresolved(String.format("replica-%d", id), 1234) From c2a7a9a506adfc15ebeb49d290aac29439558f3e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Thu, 2 May 2024 10:29:39 -0700 Subject: [PATCH 32/34] KAFKA-16207; Better validate the control records in the accumulator --- .../raft/internals/BatchAccumulator.java | 51 ++++-- .../kafka/snapshot/RecordsSnapshotWriter.java | 8 +- .../raft/internals/BatchAccumulatorTest.java | 161 +++++++++++++++++- 3 files changed, 194 insertions(+), 26 deletions(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java b/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java index 4e98298ce4d8..5d3feec24e47 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java @@ -41,13 +41,17 @@ import java.util.Optional; import java.util.OptionalInt; import java.util.OptionalLong; -import java.util.function.Function; import java.util.Iterator; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantLock; public class BatchAccumulator implements Closeable { + @FunctionalInterface + public interface MemoryRecordsCreator { + MemoryRecords create(long baseOffset, int epoch, ByteBuffer byteBuffer); + } + private final int epoch; private final Time time; private final SimpleTimer lingerTimer; @@ -217,14 +221,14 @@ private void completeCurrentBatch() { * batch that will be appended. The memory records returned must contain one * control batch and that control batch have at least one record. */ - public void appendControlMessages(Function valueCreator) { + public void appendControlMessages(MemoryRecordsCreator valueCreator) { appendLock.lock(); try { ByteBuffer buffer = memoryPool.tryAllocate(maxBatchSize); if (buffer != null) { try { forceDrain(); - MemoryRecords memoryRecords = valueCreator.apply(buffer); + MemoryRecords memoryRecords = valueCreator.create(nextOffset, epoch, buffer); int numberOfRecords = validateMemoryRecordAndReturnCount(memoryRecords); @@ -259,12 +263,26 @@ private int validateMemoryRecordAndReturnCount(MemoryRecords memoryRecord) { } MutableRecordBatch batch = batches.next(); + Integer numberOfRecords = batch.countOrNull(); if (!batch.isControlBatch()) { throw new IllegalArgumentException("valueCreator didn't creatte a control batch"); - } - - Integer numberOfRecords = batch.countOrNull(); - if (numberOfRecords == null) { + } else if (batch.baseOffset() != nextOffset) { + throw new IllegalArgumentException( + String.format( + "Expected a base offset of {} but got {}", + nextOffset, + batch.baseOffset() + ) + ); + } else if (batch.partitionLeaderEpoch() != epoch) { + throw new IllegalArgumentException( + String.format( + "Expected a partition leader epoch of {} but got {}", + epoch, + batch.partitionLeaderEpoch() + ) + ); + } else if (numberOfRecords == null) { throw new IllegalArgumentException("valueCreator didn't create a batch with the count"); } else if (batches.hasNext()) { throw new IllegalArgumentException("valueCreator created more than one batch"); @@ -284,11 +302,11 @@ public void appendLeaderChangeMessage( LeaderChangeMessage leaderChangeMessage, long currentTimestamp ) { - appendControlMessages(buffer -> + appendControlMessages((baseOffset, epoch, buffer) -> MemoryRecords.withLeaderChangeMessage( - this.nextOffset, + baseOffset, currentTimestamp, - this.epoch, + epoch, buffer, leaderChangeMessage ) @@ -307,11 +325,11 @@ public void appendSnapshotHeaderRecord( SnapshotHeaderRecord snapshotHeaderRecord, long currentTimestamp ) { - appendControlMessages(buffer -> + appendControlMessages((baseOffset, epoch, buffer) -> MemoryRecords.withSnapshotHeaderRecord( - this.nextOffset, + nextOffset, currentTimestamp, - this.epoch, + epoch, buffer, snapshotHeaderRecord ) @@ -329,11 +347,11 @@ public void appendSnapshotFooterRecord( SnapshotFooterRecord snapshotFooterRecord, long currentTimestamp ) { - appendControlMessages(buffer -> + appendControlMessages((baseOffset, epoch, buffer) -> MemoryRecords.withSnapshotFooterRecord( - this.nextOffset, + nextOffset, currentTimestamp, - this.epoch, + epoch, buffer, snapshotFooterRecord ) @@ -564,5 +582,4 @@ long remainingMs(long currentTimeMs) { return Math.max(0, deadlineMs.get() - currentTimeMs); } } - } diff --git a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java index c587d47530f2..18cf7397c28b 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java @@ -212,21 +212,21 @@ public RecordsSnapshotWriter build(RecordSerde serde) { serde ); - writer.accumulator.appendControlMessages(buffer -> { + writer.accumulator.appendControlMessages((baseOffset, epoch, buffer) -> { long now = time.milliseconds(); try (MemoryRecordsBuilder builder = new MemoryRecordsBuilder( buffer, RecordBatch.CURRENT_MAGIC_VALUE, compressionType, TimestampType.CREATE_TIME, - 0, // initialOffset + baseOffset, now, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, false, // isTransactional true, // isControlBatch - rawSnapshotWriter.get().snapshotId().epoch(), + epoch, buffer.capacity() ) ) { @@ -257,8 +257,6 @@ public RecordsSnapshotWriter build(RecordSerde serde) { } }); - writer.accumulator.forceDrain(); - return writer; } } diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/BatchAccumulatorTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/BatchAccumulatorTest.java index 8dac0dacdfbe..cf63f4384f9e 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/BatchAccumulatorTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/BatchAccumulatorTest.java @@ -17,13 +17,18 @@ package org.apache.kafka.raft.internals; import org.apache.kafka.common.memory.MemoryPool; +import org.apache.kafka.common.message.KRaftVersionRecord; import org.apache.kafka.common.message.LeaderChangeMessage; +import org.apache.kafka.common.message.SnapshotHeaderRecord; import org.apache.kafka.common.protocol.ObjectSerializationCache; import org.apache.kafka.common.protocol.Writable; import org.apache.kafka.common.record.AbstractRecords; import org.apache.kafka.common.record.CompressionType; +import org.apache.kafka.common.record.ControlRecordUtils; import org.apache.kafka.common.record.DefaultRecord; +import org.apache.kafka.common.record.MemoryRecordsBuilder; import org.apache.kafka.common.record.RecordBatch; +import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.utils.ByteUtils; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Utils; @@ -120,7 +125,7 @@ public void testForceDrain() { List records = asList("a", "b", "c", "d", "e", "f", "g", "h", "i"); - // Append records + // Append records assertEquals(baseOffset, appender.call(acc, leaderEpoch, records.subList(0, 1))); assertEquals(baseOffset + 2, appender.call(acc, leaderEpoch, records.subList(1, 3))); assertEquals(baseOffset + 5, appender.call(acc, leaderEpoch, records.subList(3, 6))); @@ -131,7 +136,7 @@ public void testForceDrain() { acc.forceDrain(); assertTrue(acc.needsDrain(time.milliseconds())); assertEquals(0, acc.timeUntilDrain(time.milliseconds())); - + // Drain completed batches List> batches = acc.drain(); @@ -168,7 +173,7 @@ public void testForceDrainBeforeAppendLeaderChangeMessage() { List records = asList("a", "b", "c", "d", "e", "f", "g", "h", "i"); - // Append records + // Append records assertEquals(baseOffset, appender.call(acc, leaderEpoch, records.subList(0, 1))); assertEquals(baseOffset + 2, appender.call(acc, leaderEpoch, records.subList(1, 3))); assertEquals(baseOffset + 5, appender.call(acc, leaderEpoch, records.subList(3, 6))); @@ -176,7 +181,7 @@ public void testForceDrainBeforeAppendLeaderChangeMessage() { assertEquals(baseOffset + 8, appender.call(acc, leaderEpoch, records.subList(8, 9))); assertFalse(acc.needsDrain(time.milliseconds())); - + // Append a leader change message acc.appendLeaderChangeMessage(new LeaderChangeMessage(), time.milliseconds()); @@ -491,6 +496,8 @@ public void testDrainDoesNotBlockWithConcurrentAppend() throws Exception { completedBatch.data.batches().forEach(recordBatch -> { assertEquals(leaderEpoch, recordBatch.partitionLeaderEpoch()); }); }); + + acc.close(); } int recordSizeInBytes(String record, int numberOfRecords) { @@ -552,4 +559,150 @@ public void testAppendWithRequiredBaseOffset(boolean correctOffset) { } acc.close(); } + + @Test + public void testMultipleControlRecords() { + int leaderEpoch = 17; + long baseOffset = 157; + int lingerMs = 50; + int maxBatchSize = 512; + + ByteBuffer buffer = ByteBuffer.allocate(maxBatchSize); + Mockito.when(memoryPool.tryAllocate(maxBatchSize)) + .thenReturn(buffer); + + try (BatchAccumulator acc = buildAccumulator( + leaderEpoch, + baseOffset, + lingerMs, + maxBatchSize + ) + ) { + acc.appendControlMessages((offset, epoch, buf) -> { + long now = 1234; + try (MemoryRecordsBuilder builder = controlRecordsBuilder(offset, epoch, now, buf)) { + builder.appendSnapshotHeaderMessage( + now, + new SnapshotHeaderRecord() + .setVersion(ControlRecordUtils.SNAPSHOT_HEADER_CURRENT_VERSION) + .setLastContainedLogTimestamp(now) + ); + + builder.appendKRaftVersionMessage( + now, + new KRaftVersionRecord() + .setVersion(ControlRecordUtils.KRAFT_VERSION_CURRENT_VERSION) + .setKRaftVersion((short) 0) + ); + + return builder.build(); + } + }); + + List> batches = acc.drain(); + assertEquals(1, batches.size()); + + BatchAccumulator.CompletedBatch batch = batches.get(0); + assertEquals(baseOffset, batch.baseOffset); + assertEquals(2, batch.numRecords); + assertEquals(buffer.duplicate().flip(), batch.data.buffer()); + + batch.release(); + } + } + + @Test + public void testInvalidControlRecordOffset() { + int leaderEpoch = 17; + long baseOffset = 157; + int lingerMs = 50; + int maxBatchSize = 512; + + ByteBuffer buffer = ByteBuffer.allocate(maxBatchSize); + Mockito.when(memoryPool.tryAllocate(maxBatchSize)) + .thenReturn(buffer); + + BatchAccumulator.MemoryRecordsCreator creator = (offset, epoch, buf) -> { + long now = 1234; + try (MemoryRecordsBuilder builder = controlRecordsBuilder(offset + 1, epoch, now, buf)) { + builder.appendSnapshotHeaderMessage( + now, + new SnapshotHeaderRecord() + .setVersion(ControlRecordUtils.SNAPSHOT_HEADER_CURRENT_VERSION) + .setLastContainedLogTimestamp(now) + ); + + return builder.build(); + } + }; + + try (BatchAccumulator acc = buildAccumulator( + leaderEpoch, + baseOffset, + lingerMs, + maxBatchSize + ) + ) { + assertThrows(IllegalArgumentException.class, () -> acc.appendControlMessages(creator)); + } + } + + @Test + public void testInvalidControlRecordEpoch() { + int leaderEpoch = 17; + long baseOffset = 157; + int lingerMs = 50; + int maxBatchSize = 512; + + ByteBuffer buffer = ByteBuffer.allocate(maxBatchSize); + Mockito.when(memoryPool.tryAllocate(maxBatchSize)) + .thenReturn(buffer); + + BatchAccumulator.MemoryRecordsCreator creator = (offset, epoch, buf) -> { + long now = 1234; + try (MemoryRecordsBuilder builder = controlRecordsBuilder(offset, epoch + 1, now, buf)) { + builder.appendSnapshotHeaderMessage( + now, + new SnapshotHeaderRecord() + .setVersion(ControlRecordUtils.SNAPSHOT_HEADER_CURRENT_VERSION) + .setLastContainedLogTimestamp(now) + ); + + return builder.build(); + } + }; + + try (BatchAccumulator acc = buildAccumulator( + leaderEpoch, + baseOffset, + lingerMs, + maxBatchSize + ) + ) { + assertThrows(IllegalArgumentException.class, () -> acc.appendControlMessages(creator)); + } + } + + private static MemoryRecordsBuilder controlRecordsBuilder( + long baseOffset, + int epoch, + long now, + ByteBuffer buffer + ) { + return new MemoryRecordsBuilder( + buffer, + RecordBatch.CURRENT_MAGIC_VALUE, + CompressionType.NONE, + TimestampType.CREATE_TIME, + baseOffset, + now, + RecordBatch.NO_PRODUCER_ID, + RecordBatch.NO_PRODUCER_EPOCH, + RecordBatch.NO_SEQUENCE, + false, // isTransactional + true, // isControlBatch + epoch, + buffer.capacity() + ); + } } From bc80295d76950f3d3803c3a2476c6bc5e8229588 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Thu, 2 May 2024 14:08:58 -0700 Subject: [PATCH 33/34] KAFKA-16207; Minor fix --- .../org/apache/kafka/raft/internals/BatchAccumulator.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java b/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java index 5d3feec24e47..1163d68b47f2 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java @@ -327,7 +327,7 @@ public void appendSnapshotHeaderRecord( ) { appendControlMessages((baseOffset, epoch, buffer) -> MemoryRecords.withSnapshotHeaderRecord( - nextOffset, + baseOffset, currentTimestamp, epoch, buffer, @@ -349,7 +349,7 @@ public void appendSnapshotFooterRecord( ) { appendControlMessages((baseOffset, epoch, buffer) -> MemoryRecords.withSnapshotFooterRecord( - nextOffset, + baseOffset, currentTimestamp, epoch, buffer, From c363bf199ca9e72ee2a29a8056555374555c4e65 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Fri, 3 May 2024 13:34:49 -0700 Subject: [PATCH 34/34] KAFKA-16207; kraft version default of 1 for tests --- .../java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java index 18cf7397c28b..489dfc2008b0 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java @@ -146,7 +146,7 @@ final public static class Builder { private Time time = Time.SYSTEM; private int maxBatchSize = 1024; private MemoryPool memoryPool = MemoryPool.NONE; - private short kraftVersion = 0; + private short kraftVersion = 1; private Optional voterSet = Optional.empty(); private Optional rawSnapshotWriter = Optional.empty();