mirror of https://github.com/apache/kafka.git
KAFKA-16207; KRaft's internal log listener to update voter set (#15671)
Adds support for the KafkaRaftClient to read the control records KRaftVersionRecord and VotersRecord in the snapshot and log. As the control records in the KRaft partition are read, the replica's known set of voters are updated. This change also contains the necessary changes to include the control records when a snapshot is generated by the KRaft state machine. It is important to note that this commit changes the code and the in-memory state to track the sets of voters but it doesn't change any data that is externally exposed. It doesn't change the RPCs, data stored on disk or configuration. When the KRaft replica starts the PartitionListener reads the latest snapshot and then log segments up to the LEO, updating the in-memory state as it reads KRaftVersionRecord and VotersRecord. When the replica (leader and follower) appends to the log, the PartitionListener catches up to the new LEO. When the replica truncates the log because of a diverging epoch, the PartitionListener also truncates the in-memory state to the new LEO. When the state machine generate a new snapshot the PartitionListener trims any prefix entries that are not needed. This is all done to minimize the amount of data tracked in-memory and to make sure that it matches the state on disk. To implement the functionality described above this commit also makes the following changes: Adds control records for KRaftVersionRecord and VotersRecord. KRaftVersionRecord describes the finalized kraft.version supported by all of the replicas. VotersRecords describes the set of voters at a specific offset. Changes Kafka's feature version to support 0 as the smallest valid value. This is needed because the default value for kraft.version is 0. Refactors FileRawSnapshotWriter so that it doesn't directly call the onSnapshotFrozen callback. It adds NotifyingRawSnapshotWriter for calling such callbacks. This reorganization is needed because in this change both the KafkaMetadataLog and the KafkaRaftClient need to react to snapshots getting frozen. Cleans up KafkaRaftClient's initialization. Removes initialize from RaftClient - this is an implementation detail that doesn't need to be exposed in the interface. Removes RaftConfig.AddressSpec and simplifies the bootstrapping of the static voter's address. The bootstrapping of the address is delayed because of tests. We should be able to simplify this further in future commits. Update the DumpLogSegment CLI to support the new control records KRaftVersionRecord and VotersRecord. Fix the RecordsSnapshotReader implementations so that the iterator includes control records. RecordsIterator is extended to support reading the new control records. Improve the BatchAccumulator implementation to allow multiple control records in one control batch. This is needed so that KRaft can make sure that VotersRecord is included in the same batch as the control record (KRaftVersionRecord) that upgrades the kraft.version to 1. Add a History interface and default implementation TreeMapHistory. This is used to track all of the sets of voters between the latest snapshot and the LEO. This is needed so that KafkaRaftClient can query for the latest set of voters and so that KafkaRaftClient can include the correct set of voters when the state machine generates a new snapshot at a given offset. Add a builder pattern for RecordsSnapshotWriter. The new builder pattern also implements including the KRaftVersionRecord and VotersRecord control records in the snapshot as necessary. A KRaftVersionRecord should be appended if the kraft.version is greater than 0 at the snapshot's offset. Similarly, a VotersRecord should be appended to the snapshot with the latest value up to the snapshot's offset. Reviewers: Jason Gustafson <jason@confluent.io>
This commit is contained in:
parent
9b8aac22ec
commit
bfe81d6229
|
@ -456,12 +456,13 @@
|
|||
<allow pkg="org.apache.kafka.snapshot" />
|
||||
<allow pkg="org.apache.kafka.clients" />
|
||||
<allow pkg="org.apache.kafka.common.config" />
|
||||
<allow pkg="org.apache.kafka.common.feature" />
|
||||
<allow pkg="org.apache.kafka.common.message" />
|
||||
<allow pkg="org.apache.kafka.common.metadata" />
|
||||
<allow pkg="org.apache.kafka.common.metrics" />
|
||||
<allow pkg="org.apache.kafka.common.protocol" />
|
||||
<allow pkg="org.apache.kafka.common.record" />
|
||||
<allow pkg="org.apache.kafka.common.requests" />
|
||||
<allow pkg="org.apache.kafka.common.protocol" />
|
||||
<allow pkg="org.apache.kafka.server.common" />
|
||||
<allow pkg="org.apache.kafka.server.common.serialization" />
|
||||
<allow pkg="org.apache.kafka.server.fault"/>
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -26,7 +26,7 @@ import org.apache.kafka.common.utils.Utils;
|
|||
/**
|
||||
* 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,14 +56,14 @@ 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) {
|
||||
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 @@ class BaseVersionRange {
|
|||
return maxValue;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return String.format(
|
||||
"%s[%s]",
|
||||
|
|
|
@ -33,7 +33,7 @@ public class SupportedVersionRange extends BaseVersionRange {
|
|||
}
|
||||
|
||||
public SupportedVersionRange(short maxVersion) {
|
||||
this((short) 1, maxVersion);
|
||||
this((short) 0, maxVersion);
|
||||
}
|
||||
|
||||
public static SupportedVersionRange fromMap(Map<String, Short> versionRangeMap) {
|
||||
|
|
|
@ -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),
|
||||
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);
|
||||
|
||||
|
@ -108,6 +112,10 @@ public enum ControlRecordType {
|
|||
return SNAPSHOT_HEADER;
|
||||
case 4:
|
||||
return SNAPSHOT_FOOTER;
|
||||
case 5:
|
||||
return KRAFT_VERSION;
|
||||
case 6:
|
||||
return KRAFT_VOTERS;
|
||||
|
||||
default:
|
||||
return UNKNOWN;
|
||||
|
|
|
@ -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 @@ import java.nio.ByteBuffer;
|
|||
* 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 KRAFT_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.KRAFT_VOTERS, recordType);
|
||||
|
||||
return deserializeVotersRecord(record.value());
|
||||
}
|
||||
|
||||
public static VotersRecord deserializeVotersRecord(ByteBuffer data) {
|
||||
return new VotersRecord(new ByteBufferAccessor(data.slice()), KRAFT_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
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,9 +18,11 @@ package org.apache.kafka.common.record;
|
|||
|
||||
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;
|
||||
|
@ -728,25 +730,15 @@ public class MemoryRecords extends AbstractRecords {
|
|||
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();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -757,25 +749,15 @@ public class MemoryRecords extends AbstractRecords {
|
|||
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();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -786,25 +768,76 @@ public class MemoryRecords extends AbstractRecords {
|
|||
ByteBuffer buffer,
|
||||
SnapshotFooterRecord snapshotFooterRecord
|
||||
) {
|
||||
writeSnapshotFooterRecord(buffer, initialOffset, timestamp, leaderEpoch, snapshotFooterRecord);
|
||||
buffer.flip();
|
||||
return MemoryRecords.readableRecords(buffer);
|
||||
try (MemoryRecordsBuilder builder = createKraftControlReccordBuilder(
|
||||
initialOffset,
|
||||
timestamp,
|
||||
leaderEpoch,
|
||||
buffer
|
||||
)
|
||||
) {
|
||||
builder.appendSnapshotFooterMessage(timestamp, snapshotFooterRecord);
|
||||
return builder.build();
|
||||
}
|
||||
}
|
||||
|
||||
private static void writeSnapshotFooterRecord(
|
||||
ByteBuffer buffer,
|
||||
public static MemoryRecords withKRaftVersionRecord(
|
||||
long initialOffset,
|
||||
long timestamp,
|
||||
int leaderEpoch,
|
||||
SnapshotFooterRecord snapshotFooterRecord
|
||||
ByteBuffer buffer,
|
||||
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())
|
||||
try (MemoryRecordsBuilder builder = createKraftControlReccordBuilder(
|
||||
initialOffset,
|
||||
timestamp,
|
||||
leaderEpoch,
|
||||
buffer
|
||||
)
|
||||
) {
|
||||
builder.appendSnapshotFooterMessage(timestamp, snapshotFooterRecord);
|
||||
builder.appendKRaftVersionMessage(timestamp, kraftVersionRecord);
|
||||
return builder.build();
|
||||
}
|
||||
}
|
||||
|
||||
public static MemoryRecords withVotersRecord(
|
||||
long initialOffset,
|
||||
long timestamp,
|
||||
int leaderEpoch,
|
||||
ByteBuffer buffer,
|
||||
VotersRecord votersRecord
|
||||
) {
|
||||
try (MemoryRecordsBuilder builder = createKraftControlReccordBuilder(
|
||||
initialOffset,
|
||||
timestamp,
|
||||
leaderEpoch,
|
||||
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()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,9 +18,11 @@ package org.apache.kafka.common.record;
|
|||
|
||||
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;
|
||||
|
@ -602,11 +604,12 @@ public class MemoryRecordsBuilder implements AutoCloseable {
|
|||
|
||||
/**
|
||||
* 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);
|
||||
|
@ -650,6 +653,22 @@ public class MemoryRecordsBuilder implements AutoCloseable {
|
|||
);
|
||||
}
|
||||
|
||||
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.KRAFT_VOTERS,
|
||||
MessageUtil.toByteBuffer(votersRecord, ControlRecordUtils.KRAFT_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
|
||||
|
|
|
@ -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" }
|
||||
]
|
||||
}
|
|
@ -0,0 +1,47 @@
|
|||
// 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": "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": [
|
||||
{ "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": "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" }
|
||||
]}
|
||||
]}
|
||||
]
|
||||
}
|
|
@ -36,18 +36,18 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
|
|||
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 class SupportedVersionRangeTest {
|
|||
|
||||
@Test
|
||||
public void testFromMapFailure() {
|
||||
// min_version can't be < 1.
|
||||
// min_version can't be < 0.
|
||||
Map<String, Short> 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<String, Short> 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<String, Short> 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));
|
||||
|
|
|
@ -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,4 +45,14 @@ public class ControlRecordTypeTest {
|
|||
assertEquals(ControlRecordType.ABORT, type);
|
||||
}
|
||||
|
||||
@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(expected.type());
|
||||
buffer.flip();
|
||||
|
||||
assertEquals(expected, ControlRecordType.parse(buffer));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,9 +19,11 @@ package org.apache.kafka.common.record;
|
|||
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 class ControlRecordUtilsTest {
|
|||
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.KRAFT_VOTERS_CURRENT_VERSION
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -32,7 +32,13 @@ 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.config.{KRaftConfigs, ServerLogConfigs}
|
||||
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
|
||||
)
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
@ -40,7 +41,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, Utils}
|
||||
import org.apache.kafka.raft.QuorumConfig.{AddressSpec, InetAddressSpec, NON_ROUTABLE_ADDRESS, UnknownAddressSpec}
|
||||
import org.apache.kafka.raft.{FileBasedStateStore, KafkaNetworkChannel, KafkaRaftClient, KafkaRaftClientDriver, LeaderAndEpoch, RaftClient, QuorumConfig, ReplicatedLog}
|
||||
import org.apache.kafka.server.ProcessRole
|
||||
import org.apache.kafka.server.common.serialization.RecordSerde
|
||||
|
@ -144,7 +144,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 {
|
||||
|
||||
|
@ -181,20 +181,12 @@ 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(),
|
||||
config.controllerListenerNames.head,
|
||||
new FileBasedStateStore(new File(dataDir, FileBasedStateStore.DEFAULT_FILE_NAME)),
|
||||
metrics
|
||||
)
|
||||
netChannel.start()
|
||||
clientDriver.start()
|
||||
}
|
||||
|
@ -224,23 +216,17 @@ 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(
|
||||
OptionalInt.of(config.nodeId),
|
||||
recordSerde,
|
||||
netChannel,
|
||||
replicatedLog,
|
||||
quorumStateStore,
|
||||
time,
|
||||
metrics,
|
||||
expirationService,
|
||||
logContext,
|
||||
clusterId,
|
||||
nodeId,
|
||||
raftConfig
|
||||
)
|
||||
client.initialize()
|
||||
client
|
||||
}
|
||||
|
||||
|
@ -267,7 +253,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,
|
||||
|
|
|
@ -1346,14 +1346,14 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
|
|||
val advertisedListenerNames = effectiveAdvertisedListeners.map(_.listenerName).toSet
|
||||
|
||||
// validate KRaft-related configs
|
||||
val voterAddressSpecsByNodeId = QuorumConfig.parseVoterConnections(quorumVoters)
|
||||
val voterIds = QuorumConfig.parseVoterIds(quorumVoters)
|
||||
def validateNonEmptyQuorumVotersForKRaft(): Unit = {
|
||||
if (voterAddressSpecsByNodeId.isEmpty) {
|
||||
if (voterIds.isEmpty) {
|
||||
throw new ConfigException(s"If using ${KRaftConfigs.PROCESS_ROLES_CONFIG}, ${QuorumConfig.QUORUM_VOTERS_CONFIG} must contain a parseable set of voters.")
|
||||
}
|
||||
}
|
||||
def validateNonEmptyQuorumVotersForMigration(): Unit = {
|
||||
if (voterAddressSpecsByNodeId.isEmpty) {
|
||||
if (voterIds.isEmpty) {
|
||||
throw new ConfigException(s"If using ${KRaftConfigs.MIGRATION_ENABLED_CONFIG}, ${QuorumConfig.QUORUM_VOTERS_CONFIG} must contain a parseable set of voters.")
|
||||
}
|
||||
}
|
||||
|
@ -1366,8 +1366,8 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
|
|||
s"The advertised.listeners config must not contain KRaft controller listeners from ${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG} when ${KRaftConfigs.PROCESS_ROLES_CONFIG} 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 ${KRaftConfigs.PROCESS_ROLES_CONFIG} contains the 'controller' role, the node id $nodeId must be included in the set of voters ${QuorumConfig.QUORUM_VOTERS_CONFIG}=${voterAddressSpecsByNodeId.asScala.keySet.toSet}")
|
||||
require(voterIds.contains(nodeId),
|
||||
s"If ${KRaftConfigs.PROCESS_ROLES_CONFIG} contains the 'controller' role, the node id $nodeId must be included in the set of voters ${QuorumConfig.QUORUM_VOTERS_CONFIG}=${voterIds.asScala.toSet}")
|
||||
}
|
||||
def validateControllerListenerExistsForKRaftController(): Unit = {
|
||||
require(controllerListeners.nonEmpty,
|
||||
|
@ -1389,8 +1389,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 ${KRaftConfigs.PROCESS_ROLES_CONFIG} contains just the 'broker' role, the node id $nodeId must not be included in the set of voters ${QuorumConfig.QUORUM_VOTERS_CONFIG}=${voterAddressSpecsByNodeId.asScala.keySet.toSet}")
|
||||
require(!voterIds.contains(nodeId),
|
||||
s"If ${KRaftConfigs.PROCESS_ROLES_CONFIG} contains just the 'broker' role, the node id $nodeId must not be included in the set of voters ${QuorumConfig.QUORUM_VOTERS_CONFIG}=${voterIds.asScala.toSet}")
|
||||
// controller.listener.names must be non-empty...
|
||||
require(controllerListenerNames.nonEmpty,
|
||||
s"${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG} must contain at least one value when running KRaft with just the broker role")
|
||||
|
|
|
@ -65,15 +65,12 @@ class KafkaRaftServer(
|
|||
metaPropsEnsemble.clusterId().get()
|
||||
)
|
||||
|
||||
private val controllerQuorumVotersFuture = CompletableFuture.completedFuture(
|
||||
QuorumConfig.parseVoterConnections(config.quorumVoters))
|
||||
|
||||
private val sharedServer = new SharedServer(
|
||||
config,
|
||||
metaPropsEnsemble,
|
||||
time,
|
||||
metrics,
|
||||
controllerQuorumVotersFuture,
|
||||
CompletableFuture.completedFuture(QuorumConfig.parseVoterConnections(config.quorumVoters)),
|
||||
new StandardFaultHandlerFactory(),
|
||||
)
|
||||
|
||||
|
|
|
@ -323,7 +323,8 @@ class KafkaServer(
|
|||
|
||||
if (config.migrationEnabled) {
|
||||
kraftControllerNodes = QuorumConfig.voterConnectionsToNodes(
|
||||
QuorumConfig.parseVoterConnections(config.quorumVoters)).asScala
|
||||
QuorumConfig.parseVoterConnections(config.quorumVoters)
|
||||
).asScala
|
||||
} else {
|
||||
kraftControllerNodes = Seq.empty
|
||||
}
|
||||
|
@ -427,8 +428,7 @@ class KafkaServer(
|
|||
logger.info("Successfully deleted local metadata log. It will be re-created.")
|
||||
|
||||
// If the ZK broker is in migration mode, start up a RaftManager to learn about the new KRaft controller
|
||||
val controllerQuorumVotersFuture = CompletableFuture.completedFuture(
|
||||
QuorumConfig.parseVoterConnections(config.quorumVoters))
|
||||
val quorumVoters = QuorumConfig.parseVoterConnections(config.quorumVoters)
|
||||
raftManager = new KafkaRaftManager[ApiMessageAndVersion](
|
||||
metaPropsEnsemble.clusterId().get(),
|
||||
config,
|
||||
|
@ -438,10 +438,10 @@ class KafkaServer(
|
|||
time,
|
||||
metrics,
|
||||
threadNamePrefix,
|
||||
controllerQuorumVotersFuture,
|
||||
CompletableFuture.completedFuture(quorumVoters),
|
||||
fatalFaultHandler = new LoggingFaultHandler("raftManager", () => shutdown())
|
||||
)
|
||||
val controllerNodes = QuorumConfig.voterConnectionsToNodes(controllerQuorumVotersFuture.get()).asScala
|
||||
val controllerNodes = QuorumConfig.voterConnectionsToNodes(quorumVoters).asScala
|
||||
val quorumControllerNodeProvider = RaftControllerNodeProvider(raftManager, config, controllerNodes)
|
||||
val brokerToQuorumChannelManager = new NodeToControllerChannelManagerImpl(
|
||||
controllerNodeProvider = quorumControllerNodeProvider,
|
||||
|
|
|
@ -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.QuorumConfig.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)
|
||||
|
|
|
@ -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.KRAFT_VOTERS=>
|
||||
val voters = ControlRecordUtils.deserializeVotersRecord(record)
|
||||
print(s" KRaftVoters ${VotersRecordJsonConverter.write(voters, voters.version())}")
|
||||
case controlType =>
|
||||
print(s" controlType: $controlType($controlTypeId)")
|
||||
}
|
||||
|
|
|
@ -90,7 +90,7 @@ public class KafkaClusterTestKit implements AutoCloseable {
|
|||
*/
|
||||
private static class ControllerQuorumVotersFutureManager implements AutoCloseable {
|
||||
private final int expectedControllers;
|
||||
private final CompletableFuture<Map<Integer, QuorumConfig.AddressSpec>> future = new CompletableFuture<>();
|
||||
private final CompletableFuture<Map<Integer, InetSocketAddress>> future = new CompletableFuture<>();
|
||||
private final Map<Integer, Integer> controllerPorts = new TreeMap<>();
|
||||
|
||||
ControllerQuorumVotersFutureManager(int expectedControllers) {
|
||||
|
@ -100,11 +100,17 @@ public class KafkaClusterTestKit implements AutoCloseable {
|
|||
synchronized void registerPort(int nodeId, int port) {
|
||||
controllerPorts.put(nodeId, port);
|
||||
if (controllerPorts.size() >= expectedControllers) {
|
||||
future.complete(controllerPorts.entrySet().stream().
|
||||
collect(Collectors.toMap(
|
||||
future.complete(
|
||||
controllerPorts
|
||||
.entrySet()
|
||||
.stream()
|
||||
.collect(
|
||||
Collectors.toMap(
|
||||
Map.Entry::getKey,
|
||||
entry -> new QuorumConfig.InetAddressSpec(new InetSocketAddress("localhost", entry.getValue()))
|
||||
)));
|
||||
entry -> new InetSocketAddress("localhost", entry.getValue())
|
||||
)
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -452,7 +458,8 @@ public class KafkaClusterTestKit implements AutoCloseable {
|
|||
|
||||
public String quorumVotersConfig() throws ExecutionException, InterruptedException {
|
||||
Collection<Node> controllerNodes = QuorumConfig.voterConnectionsToNodes(
|
||||
controllerQuorumVotersFutureManager.future.get());
|
||||
controllerQuorumVotersFutureManager.future.get()
|
||||
);
|
||||
StringBuilder bld = new StringBuilder();
|
||||
String prefix = "";
|
||||
for (Node node : controllerNodes) {
|
||||
|
|
|
@ -26,13 +26,13 @@ import javax.security.auth.login.Configuration
|
|||
import kafka.utils.{CoreUtils, Logging, TestInfoUtils, TestUtils}
|
||||
import kafka.zk.{AdminZkClient, EmbeddedZookeeper, KafkaZkClient}
|
||||
import org.apache.kafka.clients.consumer.GroupProtocol
|
||||
import org.apache.kafka.common.metadata.FeatureLevelRecord
|
||||
import org.apache.kafka.common.metrics.Metrics
|
||||
import org.apache.kafka.common.{DirectoryId, Uuid}
|
||||
import org.apache.kafka.common.security.JaasUtils
|
||||
import org.apache.kafka.common.security.auth.SecurityProtocol
|
||||
import org.apache.kafka.common.utils.{Exit, Time}
|
||||
import org.apache.kafka.common.{DirectoryId, Uuid}
|
||||
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.network.SocketServerConfigs
|
||||
|
@ -90,7 +90,7 @@ class KRaftQuorumImplementation(
|
|||
val controllerServer: ControllerServer,
|
||||
val faultHandlerFactory: FaultHandlerFactory,
|
||||
val metadataDir: File,
|
||||
val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, QuorumConfig.AddressSpec]],
|
||||
val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, InetSocketAddress]],
|
||||
val clusterId: String,
|
||||
val log: Logging,
|
||||
val faultHandler: FaultHandler
|
||||
|
@ -355,7 +355,7 @@ abstract class QuorumTestHarness extends Logging {
|
|||
props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
|
||||
props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, s"$nodeId@localhost:0")
|
||||
val config = new KafkaConfig(props)
|
||||
val controllerQuorumVotersFuture = new CompletableFuture[util.Map[Integer, QuorumConfig.AddressSpec]]
|
||||
val controllerQuorumVotersFuture = new CompletableFuture[util.Map[Integer, InetSocketAddress]]
|
||||
val metaPropertiesEnsemble = new MetaPropertiesEnsemble.Loader().
|
||||
addMetadataLogDir(metadataDir.getAbsolutePath).
|
||||
load()
|
||||
|
@ -380,8 +380,9 @@ abstract class QuorumTestHarness extends Logging {
|
|||
error("Error completing controller socket server future", e)
|
||||
controllerQuorumVotersFuture.completeExceptionally(e)
|
||||
} else {
|
||||
controllerQuorumVotersFuture.complete(Collections.singletonMap(nodeId,
|
||||
new QuorumConfig.InetAddressSpec(new InetSocketAddress("localhost", port))))
|
||||
controllerQuorumVotersFuture.complete(
|
||||
Collections.singletonMap(nodeId, new InetSocketAddress("localhost", port))
|
||||
)
|
||||
}
|
||||
})
|
||||
controllerServer.startup()
|
||||
|
@ -391,13 +392,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 = {
|
||||
|
|
|
@ -17,7 +17,6 @@
|
|||
|
||||
package kafka.server
|
||||
|
||||
import java.util.Collections
|
||||
import kafka.testkit.KafkaClusterTestKit
|
||||
import kafka.testkit.TestKitNodes
|
||||
import kafka.utils.TestUtils
|
||||
|
@ -88,9 +87,14 @@ 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())
|
||||
// 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}"
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
}
|
||||
|
||||
|
|
|
@ -17,37 +17,35 @@
|
|||
|
||||
package kafka.server
|
||||
|
||||
import java.net.InetSocketAddress
|
||||
import java.util
|
||||
import java.util.{Collections, Properties}
|
||||
import kafka.cluster.EndPoint
|
||||
import kafka.security.authorizer.AclAuthorizer
|
||||
import kafka.utils.TestUtils.assertBadConfigContainingMessage
|
||||
import kafka.utils.{CoreUtils, TestUtils}
|
||||
import org.apache.kafka.common.Node
|
||||
import org.apache.kafka.common.config.{ConfigException, TopicConfig}
|
||||
import org.apache.kafka.common.metrics.Sensor
|
||||
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.QuorumConfig
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
import java.net.InetSocketAddress
|
||||
import java.util
|
||||
import java.util.{Collections, Properties}
|
||||
import org.apache.kafka.common.Node
|
||||
import org.apache.kafka.coordinator.group.ConsumerGroupMigrationPolicy
|
||||
import org.apache.kafka.coordinator.group.Group.GroupType
|
||||
import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs}
|
||||
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
|
||||
import org.apache.kafka.security.PasswordEncoderConfigs
|
||||
import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs}
|
||||
import org.apache.kafka.network.SocketServerConfigs
|
||||
import org.apache.kafka.raft.QuorumConfig
|
||||
import org.apache.kafka.security.PasswordEncoderConfigs
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.apache.kafka.server.common.MetadataVersion.{IBP_0_8_2, IBP_3_0_IV1}
|
||||
import org.apache.kafka.server.config.{KRaftConfigs, KafkaSecurityConfigs, QuotaConfigs, ReplicationConfigs, ServerLogConfigs, ServerTopicConfigSynonyms, ZkConfigs}
|
||||
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
|
||||
import org.apache.kafka.server.metrics.MetricConfigs
|
||||
import org.apache.kafka.storage.internals.log.CleanerConfig
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.Test
|
||||
import org.junit.jupiter.api.function.Executable
|
||||
|
||||
import scala.annotation.nowarn
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
||||
|
@ -1341,29 +1339,31 @@ class KafkaConfigTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
def testValidQuorumVotersConfig(): Unit = {
|
||||
val expected = new util.HashMap[Integer, QuorumConfig.AddressSpec]()
|
||||
assertValidQuorumVoters("", expected)
|
||||
|
||||
expected.put(1, new QuorumConfig.InetAddressSpec(new InetSocketAddress("127.0.0.1", 9092)))
|
||||
assertValidQuorumVoters("1@127.0.0.1:9092", expected)
|
||||
|
||||
expected.clear()
|
||||
expected.put(1, QuorumConfig.UNKNOWN_ADDRESS_SPEC_INSTANCE)
|
||||
assertValidQuorumVoters("1@0.0.0.0:0", expected)
|
||||
|
||||
expected.clear()
|
||||
expected.put(1, new QuorumConfig.InetAddressSpec(new InetSocketAddress("kafka1", 9092)))
|
||||
expected.put(2, new QuorumConfig.InetAddressSpec(new InetSocketAddress("kafka2", 9092)))
|
||||
expected.put(3, new QuorumConfig.InetAddressSpec(new InetSocketAddress("kafka3", 9092)))
|
||||
assertValidQuorumVoters("1@kafka1:9092,2@kafka2:9092,3@kafka3:9092", expected)
|
||||
def testValidEmptyQuorumVotersParsing(): Unit = {
|
||||
assertValidQuorumVoters(new util.HashMap[Integer, InetSocketAddress](), "")
|
||||
}
|
||||
|
||||
private def assertValidQuorumVoters(value: String, expectedVoters: util.Map[Integer, QuorumConfig.AddressSpec]): Unit = {
|
||||
@Test
|
||||
def testValidQuorumVotersParsingWithIpAddress(): Unit = {
|
||||
val expected = new util.HashMap[Integer, InetSocketAddress]()
|
||||
expected.put(1, new InetSocketAddress("127.0.0.1", 9092))
|
||||
assertValidQuorumVoters(expected, "1@127.0.0.1:9092")
|
||||
}
|
||||
|
||||
@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(expected, "1@kafka1:9092,2@kafka2:9092,3@kafka3:9092")
|
||||
}
|
||||
|
||||
private def assertValidQuorumVoters(expectedVoters: util.Map[Integer, InetSocketAddress], value: String): Unit = {
|
||||
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect)
|
||||
props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, value)
|
||||
val raftConfig = new QuorumConfig(KafkaConfig.fromProps(props))
|
||||
assertEquals(expectedVoters, raftConfig.quorumVoterConnections())
|
||||
val addresses = QuorumConfig.parseVoterConnections(KafkaConfig.fromProps(props).quorumVoters)
|
||||
assertEquals(expectedVoters, addresses)
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -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}
|
||||
|
@ -28,7 +30,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}
|
||||
|
@ -36,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
|
||||
|
@ -324,35 +326,37 @@ 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)
|
||||
.setKraftVersion(1)
|
||||
.setVoterSet(Optional.of(VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(1, 2, 3)))))
|
||||
.build(MetadataRecordSerde.INSTANCE)
|
||||
) { snapshotWriter =>
|
||||
snapshotWriter.append(metadataRecords.asJava)
|
||||
snapshotWriter.freeze()
|
||||
}
|
||||
|
||||
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
|
||||
|
|
|
@ -46,11 +46,6 @@ public class SnapshotEmitterTest {
|
|||
static class MockRaftClient implements RaftClient<ApiMessageAndVersion> {
|
||||
TreeMap<OffsetAndEpoch, FakeSnapshotWriter> writers = new TreeMap<>();
|
||||
|
||||
@Override
|
||||
public void initialize() {
|
||||
// nothing to do
|
||||
}
|
||||
|
||||
@Override
|
||||
public void register(Listener<ApiMessageAndVersion> listener) {
|
||||
// nothing to do
|
||||
|
|
|
@ -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;
|
||||
|
@ -484,11 +482,6 @@ public final class LocalLogManager implements RaftClient<ApiMessageAndVersion>,
|
|||
*/
|
||||
private final EventQueue eventQueue;
|
||||
|
||||
/**
|
||||
* Whether this LocalLogManager has been initialized.
|
||||
*/
|
||||
private boolean initialized = false;
|
||||
|
||||
/**
|
||||
* Whether this LocalLogManager has been shut down.
|
||||
*/
|
||||
|
@ -633,7 +626,7 @@ public final class LocalLogManager implements RaftClient<ApiMessageAndVersion>,
|
|||
@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 +672,6 @@ public final class LocalLogManager implements RaftClient<ApiMessageAndVersion>,
|
|||
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<ApiMessageAndVersion> listener) {
|
||||
CompletableFuture<Void> future = new CompletableFuture<>();
|
||||
|
@ -695,7 +680,7 @@ public final class LocalLogManager implements RaftClient<ApiMessageAndVersion>,
|
|||
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 +690,6 @@ public final class LocalLogManager implements RaftClient<ApiMessageAndVersion>,
|
|||
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 {
|
||||
|
@ -835,22 +815,19 @@ public final class LocalLogManager implements RaftClient<ApiMessageAndVersion>,
|
|||
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<RawSnapshotWriter> 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))
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -129,9 +129,6 @@ public class LocalLogManagerTestEnv implements AutoCloseable {
|
|||
shared,
|
||||
String.format("LocalLogManager-%d_", nodeId)));
|
||||
}
|
||||
for (LocalLogManager logManager : newLogManagers) {
|
||||
logManager.initialize();
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
for (LocalLogManager logManager : newLogManagers) {
|
||||
logManager.close();
|
||||
|
|
|
@ -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 final class ControlRecord {
|
|||
throwIllegalArgument(recordType, message);
|
||||
}
|
||||
break;
|
||||
case KRAFT_VERSION:
|
||||
if (!(message instanceof KRaftVersionRecord)) {
|
||||
throwIllegalArgument(recordType, message);
|
||||
}
|
||||
break;
|
||||
case KRAFT_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 final class ControlRecord {
|
|||
return ((SnapshotHeaderRecord) message).version();
|
||||
case SNAPSHOT_FOOTER:
|
||||
return ((SnapshotFooterRecord) message).version();
|
||||
case KRAFT_VERSION:
|
||||
return ((KRaftVersionRecord) message).version();
|
||||
case KRAFT_VOTERS:
|
||||
return ((VotersRecord) message).version();
|
||||
default:
|
||||
throw new IllegalStateException(String.format("Unknown control record type %s", recordType));
|
||||
}
|
||||
|
|
|
@ -38,6 +38,7 @@ import java.io.IOException;
|
|||
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 @@ import java.util.stream.Collectors;
|
|||
* */
|
||||
public class FileBasedStateStore implements QuorumStateStore {
|
||||
private static final Logger log = LoggerFactory.getLogger(FileBasedStateStore.class);
|
||||
private 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;
|
||||
|
||||
static final String DATA_VERSION = "data_version";
|
||||
static final short HIGHEST_SUPPORTED_VERSION = 0;
|
||||
|
||||
public FileBasedStateStore(final File stateFile) {
|
||||
this.stateFile = stateFile;
|
||||
}
|
||||
|
@ -134,6 +137,11 @@ public class FileBasedStateStore implements QuorumStateStore {
|
|||
writeElectionStateToFile(stateFile, data);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Path path() {
|
||||
return stateFile.toPath();
|
||||
}
|
||||
|
||||
private List<Voter> voters(Set<Integer> votersId) {
|
||||
return votersId.stream().map(
|
||||
voterId -> new Voter().setVoterId(voterId)).collect(Collectors.toList());
|
||||
|
|
|
@ -39,6 +39,7 @@ import org.apache.kafka.server.util.RequestAndCompletionHandler;
|
|||
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 @@ public class KafkaNetworkChannel implements NetworkChannel {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void updateEndpoint(int id, QuorumConfig.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);
|
||||
}
|
||||
|
||||
|
|
|
@ -65,19 +65,23 @@ import org.apache.kafka.raft.internals.BatchMemoryPool;
|
|||
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.KRaftControlRecordStateMachine;
|
||||
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.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;
|
||||
import java.util.Collections;
|
||||
import java.util.IdentityHashMap;
|
||||
import java.util.Iterator;
|
||||
|
@ -139,13 +143,15 @@ import static org.apache.kafka.raft.RaftUtil.hasValidTopicPartition;
|
|||
* as FileRecords, but we use {@link UnalignedRecords} in FetchSnapshotResponse because the records
|
||||
* are not necessarily offset-aligned.
|
||||
*/
|
||||
public class KafkaRaftClient<T> implements RaftClient<T> {
|
||||
final public class KafkaRaftClient<T> implements RaftClient<T> {
|
||||
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 OptionalInt nodeId;
|
||||
private final AtomicReference<GracefulShutdown> shutdown = new AtomicReference<>();
|
||||
private final LogContext logContext;
|
||||
private final Logger logger;
|
||||
private final Time time;
|
||||
private final int fetchMaxWaitMs;
|
||||
|
@ -159,14 +165,30 @@ public class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
private final MemoryPool memoryPool;
|
||||
private final RaftMessageQueue messageQueue;
|
||||
private final QuorumConfig quorumConfig;
|
||||
private final KafkaRaftMetrics kafkaRaftMetrics;
|
||||
private final QuorumState quorum;
|
||||
private final RequestManager requestManager;
|
||||
private final RaftMetadataLogCleanerManager snapshotCleaner;
|
||||
|
||||
private final Map<Listener<T>, ListenerContext> listenerContexts = new IdentityHashMap<>();
|
||||
private final ConcurrentLinkedQueue<Registration<T>> pendingRegistrations = new ConcurrentLinkedQueue<>();
|
||||
|
||||
// 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 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
|
||||
* 4. truncate new entries when a follower truncates their log
|
||||
* 5. truncate old entries when a snapshot gets generated
|
||||
*/
|
||||
private volatile KRaftControlRecordStateMachine partitionState;
|
||||
private volatile KafkaRaftMetrics kafkaRaftMetrics;
|
||||
private volatile QuorumState quorum;
|
||||
private volatile RequestManager requestManager;
|
||||
|
||||
/**
|
||||
* Create a new instance.
|
||||
*
|
||||
|
@ -174,52 +196,50 @@ public class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
* non-participating observer.
|
||||
*/
|
||||
public KafkaRaftClient(
|
||||
OptionalInt nodeId,
|
||||
RecordSerde<T> serde,
|
||||
NetworkChannel channel,
|
||||
ReplicatedLog log,
|
||||
QuorumStateStore quorumStateStore,
|
||||
Time time,
|
||||
Metrics metrics,
|
||||
ExpirationService expirationService,
|
||||
LogContext logContext,
|
||||
String clusterId,
|
||||
OptionalInt nodeId,
|
||||
QuorumConfig quorumConfig
|
||||
) {
|
||||
this(serde,
|
||||
this(
|
||||
nodeId,
|
||||
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(),
|
||||
quorumConfig);
|
||||
quorumConfig
|
||||
);
|
||||
}
|
||||
|
||||
KafkaRaftClient(
|
||||
OptionalInt nodeId,
|
||||
RecordSerde<T> serde,
|
||||
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,
|
||||
QuorumConfig quorumConfig
|
||||
) {
|
||||
this.nodeId = nodeId;
|
||||
this.logContext = logContext;
|
||||
this.serde = serde;
|
||||
this.channel = channel;
|
||||
this.messageQueue = messageQueue;
|
||||
|
@ -234,28 +254,6 @@ public class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
this.random = random;
|
||||
this.quorumConfig = quorumConfig;
|
||||
this.snapshotCleaner = new RaftMetadataLogCleanerManager(logger, time, 60000, log::maybeClean);
|
||||
Set<Integer> quorumVoterIds = quorumConfig.quorumVoterIds();
|
||||
this.requestManager = new RequestManager(quorumVoterIds, quorumConfig.retryBackoffMs(),
|
||||
quorumConfig.requestTimeoutMs(), random);
|
||||
this.quorum = new QuorumState(
|
||||
nodeId,
|
||||
quorumVoterIds,
|
||||
quorumConfig.electionTimeoutMs(),
|
||||
quorumConfig.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<Integer, QuorumConfig.AddressSpec> voterAddresses = quorumConfig.quorumVoterConnections();
|
||||
voterAddresses.entrySet().stream()
|
||||
.filter(e -> e.getValue() instanceof QuorumConfig.InetAddressSpec)
|
||||
.forEach(e -> this.channel.updateEndpoint(e.getKey(), (QuorumConfig.InetAddressSpec) e.getValue()));
|
||||
}
|
||||
|
||||
private void updateFollowerHighWatermark(
|
||||
|
@ -370,8 +368,52 @@ public class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void initialize() {
|
||||
public void initialize(
|
||||
Map<Integer, InetSocketAddress> voterAddresses,
|
||||
String listenerName,
|
||||
QuorumStateStore quorumStateStore,
|
||||
Metrics metrics
|
||||
) {
|
||||
partitionState = new KRaftControlRecordStateMachine(
|
||||
Optional.of(VoterSet.fromInetSocketAddresses(listenerName, voterAddresses)),
|
||||
log,
|
||||
serde,
|
||||
BufferSupplier.create(),
|
||||
MAX_BATCH_SIZE_BYTES,
|
||||
logContext
|
||||
);
|
||||
// Read the entire log
|
||||
logger.info("Reading KRaft snapshot and log as part of the initialization");
|
||||
partitionState.updateState();
|
||||
|
||||
requestManager = new RequestManager(
|
||||
partitionState.lastVoterSet().voterIds(),
|
||||
quorumConfig.retryBackoffMs(),
|
||||
quorumConfig.requestTimeoutMs(),
|
||||
random
|
||||
);
|
||||
|
||||
quorum = new QuorumState(
|
||||
nodeId,
|
||||
partitionState.lastVoterSet().voterIds(),
|
||||
quorumConfig.electionTimeoutMs(),
|
||||
quorumConfig.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);
|
||||
|
||||
VoterSet lastVoterSet = partitionState.lastVoterSet();
|
||||
for (Integer voterId : lastVoterSet.voterIds()) {
|
||||
channel.updateEndpoint(voterId, lastVoterSet.voterAddress(voterId, listenerName).get());
|
||||
}
|
||||
|
||||
quorum.initialize(new OffsetAndEpoch(log.endOffset().offset, log.lastFetchedEpoch()));
|
||||
|
||||
long currentTimeMs = time.milliseconds();
|
||||
|
@ -407,12 +449,16 @@ public class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
|
||||
@Override
|
||||
public LeaderAndEpoch leaderAndEpoch() {
|
||||
if (isInitialized()) {
|
||||
return quorum.leaderAndEpoch();
|
||||
} else {
|
||||
return LeaderAndEpoch.UNKNOWN;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public OptionalInt nodeId() {
|
||||
return quorum.localId();
|
||||
return nodeId;
|
||||
}
|
||||
|
||||
private OffsetAndEpoch endOffset() {
|
||||
|
@ -1138,6 +1184,9 @@ public class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
truncationOffset,
|
||||
quorum.leaderIdOrSentinel()
|
||||
);
|
||||
|
||||
// Update the internal listener to the new end offset
|
||||
partitionState.truncateNewEntries(truncationOffset);
|
||||
} else if (partitionResponse.snapshotId().epoch() >= 0 ||
|
||||
partitionResponse.snapshotId().endOffset() >= 0) {
|
||||
// The leader is asking us to fetch a snapshot
|
||||
|
@ -1162,10 +1211,10 @@ public class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
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,
|
||||
|
@ -1200,6 +1249,8 @@ public class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
log.flush(false);
|
||||
}
|
||||
|
||||
partitionState.updateState();
|
||||
|
||||
OffsetAndEpoch endOffset = endOffset();
|
||||
kafkaRaftMetrics.updateFetchedRecords(info.lastOffset - info.firstOffset + 1);
|
||||
kafkaRaftMetrics.updateLogEnd(endOffset);
|
||||
|
@ -1210,6 +1261,9 @@ public class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
Records records
|
||||
) {
|
||||
LogAppendInfo info = log.appendAsLeader(records, quorum.epoch());
|
||||
|
||||
partitionState.updateState();
|
||||
|
||||
OffsetAndEpoch endOffset = endOffset();
|
||||
kafkaRaftMetrics.updateAppendRecords(info.lastOffset - info.firstOffset + 1);
|
||||
kafkaRaftMetrics.updateLogEnd(endOffset);
|
||||
|
@ -1470,6 +1524,10 @@ public class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
quorum.leaderIdOrSentinel()
|
||||
);
|
||||
|
||||
// This will aways reload the snapshot because the internal next offset
|
||||
// is always less than the snapshot id just downloaded.
|
||||
partitionState.updateState();
|
||||
|
||||
updateFollowerHighWatermark(state, OptionalLong.of(log.highWatermark().offset));
|
||||
} else {
|
||||
throw new IllegalStateException(
|
||||
|
@ -1924,12 +1982,12 @@ public class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
|
||||
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);
|
||||
});
|
||||
|
@ -2301,6 +2359,10 @@ public class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
* requests and send any needed outbound requests.
|
||||
*/
|
||||
public void poll() {
|
||||
if (!isInitialized()) {
|
||||
throw new IllegalStateException("Replica needs to be initialized before polling");
|
||||
}
|
||||
|
||||
long startPollTimeMs = time.milliseconds();
|
||||
if (maybeCompleteShutdown(startPollTimeMs)) {
|
||||
return;
|
||||
|
@ -2336,8 +2398,12 @@ public class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
}
|
||||
|
||||
private long append(int epoch, List<T> records, OptionalLong requiredBaseOffset, boolean isAtomic) {
|
||||
if (!isInitialized()) {
|
||||
throw new NotLeaderException("Append failed because the replica is not the current leader");
|
||||
}
|
||||
|
||||
LeaderState<T> leaderState = quorum.<T>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<T> accumulator = leaderState.accumulator();
|
||||
|
@ -2368,9 +2434,9 @@ public class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
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");
|
||||
}
|
||||
|
||||
|
@ -2415,17 +2481,30 @@ public class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
@Override
|
||||
public Optional<SnapshotWriter<T>> createSnapshot(
|
||||
OffsetAndEpoch snapshotId,
|
||||
long lastContainedLogTime
|
||||
long lastContainedLogTimestamp
|
||||
) {
|
||||
return RecordsSnapshotWriter.createWithHeader(
|
||||
() -> log.createNewSnapshot(snapshotId),
|
||||
MAX_BATCH_SIZE_BYTES,
|
||||
memoryPool,
|
||||
time,
|
||||
lastContainedLogTime,
|
||||
CompressionType.NONE,
|
||||
serde
|
||||
);
|
||||
if (!isInitialized()) {
|
||||
throw new IllegalStateException("Cannot create snapshot before the replica has been initialized");
|
||||
}
|
||||
|
||||
return log.createNewSnapshot(snapshotId).map(writer -> {
|
||||
long lastContainedLogOffset = snapshotId.offset() - 1;
|
||||
|
||||
RawSnapshotWriter wrappedWriter = new NotifyingRawSnapshotWriter(writer, offsetAndEpoch -> {
|
||||
// Trim the state in the internal listener up to the new snapshot
|
||||
partitionState.truncateOldEntries(offsetAndEpoch.offset());
|
||||
});
|
||||
|
||||
return new RecordsSnapshotWriter.Builder()
|
||||
.setLastContainedLogTimestamp(lastContainedLogTimestamp)
|
||||
.setTime(time)
|
||||
.setMaxBatchSize(MAX_BATCH_SIZE_BYTES)
|
||||
.setMemoryPool(memoryPool)
|
||||
.setRawSnapshotWriter(wrappedWriter)
|
||||
.setKraftVersion(partitionState.kraftVersionAtOffset(lastContainedLogOffset))
|
||||
.setVoterSet(partitionState.voterSetAtOffset(lastContainedLogOffset))
|
||||
.build(serde);
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -2450,18 +2529,25 @@ public class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
}
|
||||
}
|
||||
|
||||
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() {
|
||||
// It's okay to return null since this method is only called by tests
|
||||
return quorum;
|
||||
}
|
||||
|
||||
private boolean isInitialized() {
|
||||
return partitionState != null && quorum != null && requestManager != null && kafkaRaftMetrics != null;
|
||||
}
|
||||
|
||||
private class GracefulShutdown {
|
||||
final Timer finishTimer;
|
||||
final CompletableFuture<Void> completeFuture;
|
||||
|
|
|
@ -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, QuorumConfig.InetAddressSpec address);
|
||||
void updateEndpoint(int id, InetSocketAddress address);
|
||||
|
||||
default void close() throws InterruptedException {}
|
||||
|
||||
|
|
|
@ -33,13 +33,7 @@ import java.util.Set;
|
|||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* RaftConfig encapsulates configuration specific to the Raft quorum 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
|
||||
* QuorumConfig 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
|
||||
|
@ -52,8 +46,7 @@ public class QuorumConfig {
|
|||
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 UnknownAddressSpec UNKNOWN_ADDRESS_SPEC_INSTANCE = new UnknownAddressSpec();
|
||||
public static final String NON_ROUTABLE_HOST = "0.0.0.0";
|
||||
|
||||
public static final String QUORUM_VOTERS_CONFIG = QUORUM_PREFIX + "voters";
|
||||
public static final String QUORUM_VOTERS_DOC = "Map of id/endpoint information for " +
|
||||
|
@ -98,58 +91,19 @@ public class QuorumConfig {
|
|||
private final int electionBackoffMaxMs;
|
||||
private final int fetchTimeoutMs;
|
||||
private final int appendLingerMs;
|
||||
private final Map<Integer, AddressSpec> voterConnections;
|
||||
|
||||
public interface AddressSpec {
|
||||
}
|
||||
|
||||
public static class InetAddressSpec implements AddressSpec {
|
||||
public final InetSocketAddress address;
|
||||
|
||||
public InetAddressSpec(InetSocketAddress address) {
|
||||
if (address == null || address.equals(NON_ROUTABLE_ADDRESS)) {
|
||||
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 QuorumConfig(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 QuorumConfig(
|
||||
Map<Integer, AddressSpec> voterConnections,
|
||||
int requestTimeoutMs,
|
||||
int retryBackoffMs,
|
||||
int electionTimeoutMs,
|
||||
|
@ -157,7 +111,6 @@ public class QuorumConfig {
|
|||
int fetchTimeoutMs,
|
||||
int appendLingerMs
|
||||
) {
|
||||
this.voterConnections = voterConnections;
|
||||
this.requestTimeoutMs = requestTimeoutMs;
|
||||
this.retryBackoffMs = retryBackoffMs;
|
||||
this.electionTimeoutMs = electionTimeoutMs;
|
||||
|
@ -190,14 +143,6 @@ public class QuorumConfig {
|
|||
return appendLingerMs;
|
||||
}
|
||||
|
||||
public Set<Integer> quorumVoterIds() {
|
||||
return quorumVoterConnections().keySet();
|
||||
}
|
||||
|
||||
public Map<Integer, AddressSpec> quorumVoterConnections() {
|
||||
return voterConnections;
|
||||
}
|
||||
|
||||
private static Integer parseVoterId(String idString) {
|
||||
try {
|
||||
return Integer.parseInt(idString);
|
||||
|
@ -206,8 +151,19 @@ public class QuorumConfig {
|
|||
}
|
||||
}
|
||||
|
||||
public static Map<Integer, AddressSpec> parseVoterConnections(List<String> voterEntries) {
|
||||
Map<Integer, AddressSpec> voterMap = new HashMap<>();
|
||||
public static Map<Integer, InetSocketAddress> parseVoterConnections(List<String> voterEntries) {
|
||||
return parseVoterConnections(voterEntries, true);
|
||||
}
|
||||
|
||||
public static Set<Integer> parseVoterIds(List<String> voterEntries) {
|
||||
return parseVoterConnections(voterEntries, false).keySet();
|
||||
}
|
||||
|
||||
private static Map<Integer, InetSocketAddress> parseVoterConnections(
|
||||
List<String> voterEntries,
|
||||
boolean requireRoutableAddresses
|
||||
) {
|
||||
Map<Integer, InetSocketAddress> voterMap = new HashMap<>();
|
||||
for (String voterMapEntry : voterEntries) {
|
||||
String[] idAndAddress = voterMapEntry.split("@");
|
||||
if (idAndAddress.length != 2) {
|
||||
|
@ -231,10 +187,12 @@ public class QuorumConfig {
|
|||
}
|
||||
|
||||
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) && requireRoutableAddresses) {
|
||||
throw new ConfigException(
|
||||
String.format("Host string ({}) is not routeable", address.getHostString())
|
||||
);
|
||||
} else {
|
||||
voterMap.put(voterId, new InetAddressSpec(address));
|
||||
voterMap.put(voterId, address);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -245,14 +203,12 @@ public class QuorumConfig {
|
|||
return voterConnectionsToNodes(parseVoterConnections(voters));
|
||||
}
|
||||
|
||||
public static List<Node> voterConnectionsToNodes(Map<Integer, QuorumConfig.AddressSpec> voterConnections) {
|
||||
return voterConnections.entrySet().stream()
|
||||
public static List<Node> voterConnectionsToNodes(Map<Integer, InetSocketAddress> 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.getHostName(), spec.address.getPort());
|
||||
})
|
||||
.map(entry -> new Node(entry.getKey(), entry.getValue().getHostString(), entry.getValue().getPort()))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
|
@ -267,7 +223,7 @@ public class QuorumConfig {
|
|||
List<String> voterStrings = (List<String>) value;
|
||||
|
||||
// Attempt to parse the connect strings
|
||||
parseVoterConnections(voterStrings);
|
||||
parseVoterConnections(voterStrings, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -140,9 +140,13 @@ public class QuorumState {
|
|||
+ " with a voted candidate, which indicates this node was previously "
|
||||
+ " a voter, but the local id " + localIdDescription);
|
||||
} else if (election.epoch < logEndOffsetAndEpoch.epoch()) {
|
||||
log.warn("Epoch from quorum-state file is {}, which is " +
|
||||
"smaller than last written epoch {} in the log",
|
||||
election.epoch, logEndOffsetAndEpoch.epoch());
|
||||
log.warn(
|
||||
"Epoch from quorum store file ({}) is {}, which is smaller than last written " +
|
||||
"epoch {} in the log",
|
||||
store.path(),
|
||||
election.epoch,
|
||||
logEndOffsetAndEpoch.epoch()
|
||||
);
|
||||
initialState = new UnattachedState(
|
||||
time,
|
||||
logEndOffsetAndEpoch.epoch(),
|
||||
|
|
|
@ -16,6 +16,8 @@
|
|||
*/
|
||||
package org.apache.kafka.raft;
|
||||
|
||||
import java.nio.file.Path;
|
||||
|
||||
/**
|
||||
* Maintain the save and retrieval of quorum state information, so far only supports
|
||||
* read and write of election states.
|
||||
|
@ -39,6 +41,11 @@ public interface QuorumStateStore {
|
|||
*/
|
||||
void writeElectionState(ElectionState latest);
|
||||
|
||||
/**
|
||||
* Path to the quorum state store
|
||||
*/
|
||||
Path path();
|
||||
|
||||
/**
|
||||
* Clear any state associated to the store for a fresh start
|
||||
*/
|
||||
|
|
|
@ -87,11 +87,6 @@ public interface RaftClient<T> extends AutoCloseable {
|
|||
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.
|
||||
*
|
||||
|
|
|
@ -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<Integer> {
|
||||
|
@ -139,9 +137,20 @@ public class ReplicatedCounter implements RaftClient.Listener<Integer> {
|
|||
public synchronized void handleLoadSnapshot(SnapshotReader<Integer> 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<Integer> batch = reader.next();
|
||||
if (batch.records().size() != 1) {
|
||||
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",
|
||||
|
@ -151,6 +160,9 @@ public class ReplicatedCounter implements RaftClient.Listener<Integer> {
|
|||
);
|
||||
}
|
||||
|
||||
foundDataRecord = true;
|
||||
}
|
||||
|
||||
for (Integer value : batch) {
|
||||
log.debug("Setting value: {}", value);
|
||||
committed = value;
|
||||
|
|
|
@ -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 {
|
||||
|
||||
/**
|
||||
|
@ -236,14 +235,14 @@ public interface ReplicatedLog extends AutoCloseable {
|
|||
* 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<RawSnapshotWriter> createNewSnapshot(OffsetAndEpoch snapshotId);
|
||||
|
||||
|
@ -254,14 +253,15 @@ public interface ReplicatedLog extends AutoCloseable {
|
|||
* 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
|
||||
*/
|
||||
Optional<RawSnapshotWriter> storeSnapshot(OffsetAndEpoch snapshotId);
|
||||
Optional<RawSnapshotWriter> createNewSnapshotUnchecked(OffsetAndEpoch snapshotId);
|
||||
|
||||
/**
|
||||
* Opens a readable snapshot for the given snapshot id.
|
||||
|
|
|
@ -21,6 +21,7 @@ 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.record.MemoryRecords;
|
||||
import org.apache.kafka.common.record.MutableRecordBatch;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
import org.apache.kafka.raft.errors.BufferAllocationException;
|
||||
import org.apache.kafka.raft.errors.NotLeaderException;
|
||||
|
@ -40,12 +41,17 @@ import java.util.Objects;
|
|||
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<T> 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;
|
||||
|
@ -213,25 +219,29 @@ public class BatchAccumulator<T> implements Closeable {
|
|||
*
|
||||
* @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 appendControlMessage(Function<ByteBuffer, MemoryRecords> valueCreator) {
|
||||
public void appendControlMessages(MemoryRecordsCreator valueCreator) {
|
||||
appendLock.lock();
|
||||
try {
|
||||
ByteBuffer buffer = memoryPool.tryAllocate(256);
|
||||
ByteBuffer buffer = memoryPool.tryAllocate(maxBatchSize);
|
||||
if (buffer != null) {
|
||||
try {
|
||||
forceDrain();
|
||||
MemoryRecords memoryRecords = valueCreator.create(nextOffset, epoch, buffer);
|
||||
|
||||
int numberOfRecords = validateMemoryRecordAndReturnCount(memoryRecords);
|
||||
|
||||
completed.add(
|
||||
new CompletedBatch<>(
|
||||
nextOffset,
|
||||
1,
|
||||
valueCreator.apply(buffer),
|
||||
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);
|
||||
|
@ -245,6 +255,42 @@ public class BatchAccumulator<T> implements Closeable {
|
|||
}
|
||||
}
|
||||
|
||||
private int validateMemoryRecordAndReturnCount(MemoryRecords memoryRecord) {
|
||||
// Confirm that it is at most one batch and it is a control record
|
||||
Iterator<MutableRecordBatch> batches = memoryRecord.batches().iterator();
|
||||
if (!batches.hasNext()) {
|
||||
throw new IllegalArgumentException("valueCreator didn't create a batch");
|
||||
}
|
||||
|
||||
MutableRecordBatch batch = batches.next();
|
||||
Integer numberOfRecords = batch.countOrNull();
|
||||
if (!batch.isControlBatch()) {
|
||||
throw new IllegalArgumentException("valueCreator didn't creatte a control batch");
|
||||
} 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");
|
||||
}
|
||||
|
||||
return numberOfRecords;
|
||||
}
|
||||
|
||||
/**
|
||||
* Append a {@link LeaderChangeMessage} record to the batch
|
||||
*
|
||||
|
@ -256,13 +302,15 @@ public class BatchAccumulator<T> implements Closeable {
|
|||
LeaderChangeMessage leaderChangeMessage,
|
||||
long currentTimestamp
|
||||
) {
|
||||
appendControlMessage(buffer -> MemoryRecords.withLeaderChangeMessage(
|
||||
this.nextOffset,
|
||||
appendControlMessages((baseOffset, epoch, buffer) ->
|
||||
MemoryRecords.withLeaderChangeMessage(
|
||||
baseOffset,
|
||||
currentTimestamp,
|
||||
this.epoch,
|
||||
epoch,
|
||||
buffer,
|
||||
leaderChangeMessage
|
||||
));
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
|
||||
|
@ -277,13 +325,15 @@ public class BatchAccumulator<T> implements Closeable {
|
|||
SnapshotHeaderRecord snapshotHeaderRecord,
|
||||
long currentTimestamp
|
||||
) {
|
||||
appendControlMessage(buffer -> MemoryRecords.withSnapshotHeaderRecord(
|
||||
this.nextOffset,
|
||||
appendControlMessages((baseOffset, epoch, buffer) ->
|
||||
MemoryRecords.withSnapshotHeaderRecord(
|
||||
baseOffset,
|
||||
currentTimestamp,
|
||||
this.epoch,
|
||||
epoch,
|
||||
buffer,
|
||||
snapshotHeaderRecord
|
||||
));
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -297,13 +347,15 @@ public class BatchAccumulator<T> implements Closeable {
|
|||
SnapshotFooterRecord snapshotFooterRecord,
|
||||
long currentTimestamp
|
||||
) {
|
||||
appendControlMessage(buffer -> MemoryRecords.withSnapshotFooterRecord(
|
||||
this.nextOffset,
|
||||
appendControlMessages((baseOffset, epoch, buffer) ->
|
||||
MemoryRecords.withSnapshotFooterRecord(
|
||||
baseOffset,
|
||||
currentTimestamp,
|
||||
this.epoch,
|
||||
epoch,
|
||||
buffer,
|
||||
snapshotFooterRecord
|
||||
));
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
public void forceDrain() {
|
||||
|
@ -530,5 +582,4 @@ public class BatchAccumulator<T> implements Closeable {
|
|||
return Math.max(0, deadlineMs.get() - currentTimeMs);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -16,8 +16,7 @@
|
|||
*/
|
||||
package org.apache.kafka.raft.internals;
|
||||
|
||||
@FunctionalInterface
|
||||
public interface CloseListener<T extends AutoCloseable> {
|
||||
|
||||
void onClose(T closeable);
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,261 @@
|
|||
/*
|
||||
* 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 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 org.apache.kafka.snapshot.RawSnapshotReader;
|
||||
import org.apache.kafka.snapshot.RecordsSnapshotReader;
|
||||
import org.apache.kafka.snapshot.SnapshotReader;
|
||||
import org.slf4j.Logger;
|
||||
|
||||
/**
|
||||
* 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 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 KRaftControlRecordStateMachine {
|
||||
private final ReplicatedLog log;
|
||||
private final RecordSerde<?> serde;
|
||||
private final BufferSupplier bufferSupplier;
|
||||
private final Logger logger;
|
||||
private final int maxBatchSizeBytes;
|
||||
|
||||
// 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 LogHistory<Short> kraftVersionHistory = new TreeMapLogHistory<>();
|
||||
|
||||
// This synchronization is enough because
|
||||
// 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
|
||||
// the nextOffset first before reading voterSetHistory or kraftVersionHistory
|
||||
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 KRaftControlRecordStateMachine(
|
||||
Optional<VoterSet> staticVoterSet,
|
||||
ReplicatedLog log,
|
||||
RecordSerde<?> serde,
|
||||
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());
|
||||
}
|
||||
|
||||
/**
|
||||
* Must be called whenever the {@code log} has changed.
|
||||
*/
|
||||
public void updateState() {
|
||||
maybeLoadSnapshot();
|
||||
maybeLoadLog();
|
||||
}
|
||||
|
||||
/**
|
||||
* Remove the head of the log until the given offset.
|
||||
*
|
||||
* @param endOffset the end offset (exclusive)
|
||||
*/
|
||||
public void truncateNewEntries(long endOffset) {
|
||||
synchronized (voterSetHistory) {
|
||||
voterSetHistory.truncateNewEntries(endOffset);
|
||||
}
|
||||
synchronized (kraftVersionHistory) {
|
||||
kraftVersionHistory.truncateNewEntries(endOffset);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Remove the tail of the log until the given offset.
|
||||
*
|
||||
* @param @startOffset the start offset (inclusive)
|
||||
*/
|
||||
public void truncateOldEntries(long startOffset) {
|
||||
synchronized (voterSetHistory) {
|
||||
voterSetHistory.truncateOldEntries(startOffset);
|
||||
}
|
||||
synchronized (kraftVersionHistory) {
|
||||
kraftVersionHistory.truncateOldEntries(startOffset);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the last voter set.
|
||||
*/
|
||||
public VoterSet lastVoterSet() {
|
||||
synchronized (voterSetHistory) {
|
||||
return voterSetHistory.lastValue();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* 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<VoterSet> voterSetAtOffset(long offset) {
|
||||
checkOffsetIsValid(offset);
|
||||
|
||||
synchronized (voterSetHistory) {
|
||||
return voterSetHistory.valueAtOrBefore(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) {
|
||||
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 a value at an offset (%d) which is greater than or " +
|
||||
"equal to the largest known offset (%d)",
|
||||
offset,
|
||||
fixedNextOffset - 1
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
private void maybeLoadLog() {
|
||||
while (log.endOffset().offset > nextOffset) {
|
||||
LogFetchInfo info = log.read(nextOffset, Isolation.UNCOMMITTED);
|
||||
try (RecordsIterator<?> iterator = new RecordsIterator<>(
|
||||
info.records,
|
||||
serde,
|
||||
bufferSupplier,
|
||||
maxBatchSizeBytes,
|
||||
true // Validate batch CRC
|
||||
)
|
||||
) {
|
||||
while (iterator.hasNext()) {
|
||||
Batch<?> batch = iterator.next();
|
||||
handleBatch(batch, OptionalLong.empty());
|
||||
nextOffset = batch.lastOffset() + 1;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void maybeLoadSnapshot() {
|
||||
if ((nextOffset == 0 || nextOffset < log.startOffset()) && log.latestSnapshot().isPresent()) {
|
||||
RawSnapshotReader rawSnapshot = log.latestSnapshot().get();
|
||||
// Clear the current state
|
||||
synchronized (kraftVersionHistory) {
|
||||
kraftVersionHistory.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(
|
||||
rawSnapshot,
|
||||
serde,
|
||||
bufferSupplier,
|
||||
maxBatchSizeBytes,
|
||||
true // Validate batch CRC
|
||||
)
|
||||
) {
|
||||
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();
|
||||
handleBatch(batch, currentOffset);
|
||||
}
|
||||
|
||||
nextOffset = reader.lastContainedLogOffset() + 1;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void handleBatch(Batch<?> batch, OptionalLong overrideOffset) {
|
||||
int offsetDelta = 0;
|
||||
for (ControlRecord record : batch.controlRecords()) {
|
||||
long currentOffset = overrideOffset.orElse(batch.baseOffset() + offsetDelta);
|
||||
switch (record.type()) {
|
||||
case KRAFT_VOTERS:
|
||||
synchronized (voterSetHistory) {
|
||||
voterSetHistory.addAt(currentOffset, VoterSet.fromVotersRecord((VotersRecord) record.message()));
|
||||
}
|
||||
break;
|
||||
|
||||
case KRAFT_VERSION:
|
||||
synchronized (kraftVersionHistory) {
|
||||
kraftVersionHistory.addAt(currentOffset, ((KRaftVersionRecord) record.message()).kRaftVersion());
|
||||
}
|
||||
break;
|
||||
|
||||
default:
|
||||
// Skip the rest of the control records
|
||||
break;
|
||||
}
|
||||
++offsetDelta;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,119 @@
|
|||
/*
|
||||
* 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.Objects;
|
||||
import java.util.Optional;
|
||||
|
||||
/**
|
||||
* An object that tracks values of {@code T} at different offsets.
|
||||
*/
|
||||
public interface LogHistory<T> {
|
||||
/**
|
||||
* 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
|
||||
*/
|
||||
void addAt(long offset, T value);
|
||||
|
||||
/**
|
||||
* 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 exists, otherwise {@code Optional.empty()}
|
||||
*/
|
||||
Optional<T> valueAtOrBefore(long offset);
|
||||
|
||||
/**
|
||||
* Returns the value with the largest offset.
|
||||
*
|
||||
* @return the value if it exists, otherwise {@code Optional.empty()}
|
||||
*/
|
||||
Optional<Entry<T>> lastEntry();
|
||||
|
||||
/**
|
||||
* Removes all entries with an offset greater than or equal to {@code endOffset}.
|
||||
*
|
||||
* @param endOffset the ending offset
|
||||
*/
|
||||
void truncateNewEntries(long endOffset);
|
||||
|
||||
/**
|
||||
* 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
|
||||
*/
|
||||
void truncateOldEntries(long startOffset);
|
||||
|
||||
/**
|
||||
* Removes all of the values from this object.
|
||||
*/
|
||||
void clear();
|
||||
|
||||
final static class Entry<T> {
|
||||
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;
|
||||
}
|
||||
|
||||
@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 (!Objects.equals(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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -76,6 +76,7 @@ public final class RecordsBatchReader<T> implements BatchReader<T> {
|
|||
return baseOffset;
|
||||
}
|
||||
|
||||
@Override
|
||||
public OptionalLong lastOffset() {
|
||||
if (isClosed) {
|
||||
return OptionalLong.of(lastReturnedOffset);
|
||||
|
|
|
@ -214,7 +214,11 @@ public final class RecordsIterator<T> implements Iterator<Batch<T>>, AutoCloseab
|
|||
if (batch.isControlBatch()) {
|
||||
List<ControlRecord> 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 @@ public final class RecordsIterator<T> implements Iterator<Batch<T>>, AutoCloseab
|
|||
case SNAPSHOT_FOOTER:
|
||||
message = ControlRecordUtils.deserializeSnapshotFooterRecord(value.get());
|
||||
break;
|
||||
case KRAFT_VERSION:
|
||||
message = ControlRecordUtils.deserializeKRaftVersionRecord(value.get());
|
||||
break;
|
||||
case KRAFT_VOTERS:
|
||||
message = ControlRecordUtils.deserializeVotersRecord(value.get());
|
||||
break;
|
||||
default:
|
||||
throw new IllegalArgumentException(String.format("Unknown control record type %s", type));
|
||||
}
|
||||
|
|
|
@ -0,0 +1,76 @@
|
|||
/*
|
||||
* 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;
|
||||
|
||||
/**
|
||||
* A implementation for {@code LogHistory} which uses a red-black tree to store values sorted by offset.
|
||||
*/
|
||||
final public class TreeMapLogHistory<T> implements LogHistory<T> {
|
||||
private final NavigableMap<Long, T> history = new TreeMap<>();
|
||||
|
||||
@Override
|
||||
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<Long, ?> 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.put(offset, value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<T> valueAtOrBefore(long offset) {
|
||||
return Optional.ofNullable(history.floorEntry(offset)).map(Map.Entry::getValue);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<Entry<T>> lastEntry() {
|
||||
return Optional.ofNullable(history.lastEntry()).map(entry -> new Entry<>(entry.getKey(), entry.getValue()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void truncateNewEntries(long endOffset) {
|
||||
history.tailMap(endOffset, true).clear();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void truncateOldEntries(long startOffset) {
|
||||
NavigableMap<Long, T> 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();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,377 @@
|
|||
/*
|
||||
* 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.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;
|
||||
|
||||
/**
|
||||
* 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<Integer, VoterNode> voters;
|
||||
|
||||
VoterSet(Map<Integer, VoterNode> voters) {
|
||||
if (voters.isEmpty()) {
|
||||
throw new IllegalArgumentException("Voters cannot be empty");
|
||||
}
|
||||
|
||||
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 exists, otherwise {@code Optional.empty()}
|
||||
*/
|
||||
public Optional<InetSocketAddress> voterAddress(int voter, String listener) {
|
||||
return Optional.ofNullable(voters.get(voter))
|
||||
.flatMap(voterNode -> voterNode.address(listener));
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns all of the voter ids.
|
||||
*/
|
||||
public Set<Integer> 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<VoterSet> addVoter(VoterNode voter) {
|
||||
if (voters.containsKey(voter.voterKey().id())) {
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
HashMap<Integer, VoterNode> newVoters = new HashMap<>(voters);
|
||||
newVoters.put(voter.voterKey().id(), voter);
|
||||
|
||||
return Optional.of(new VoterSet(newVoters));
|
||||
}
|
||||
|
||||
/**
|
||||
* Remove 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 directory id match.
|
||||
*
|
||||
* @param voterKey the voter key
|
||||
* @return a new voter set if the voter was removed, otherwise {@code Optional.empty()}
|
||||
*/
|
||||
public Optional<VoterSet> removeVoter(VoterKey voterKey) {
|
||||
VoterNode oldVoter = voters.get(voterKey.id());
|
||||
if (oldVoter != null && Objects.equals(oldVoter.voterKey(), voterKey)) {
|
||||
HashMap<Integer, VoterNode> newVoters = new HashMap<>(voters);
|
||||
newVoters.remove(voterKey.id());
|
||||
|
||||
return Optional.of(new VoterSet(newVoters));
|
||||
}
|
||||
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
/**
|
||||
* 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) {
|
||||
Function<VoterNode, VotersRecord.Voter> voterConvertor = voter -> {
|
||||
Iterator<VotersRecord.Endpoint> 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.voterKey().id())
|
||||
.setVoterDirectoryId(voter.voterKey().directoryId().orElse(Uuid.ZERO_UUID))
|
||||
.setEndpoints(new VotersRecord.EndpointCollection(endpoints))
|
||||
.setKRaftVersionFeature(kraftVersionFeature);
|
||||
};
|
||||
|
||||
List<VotersRecord.Voter> voterRecordVoters = voters
|
||||
.values()
|
||||
.stream()
|
||||
.map(voterConvertor)
|
||||
.collect(Collectors.toList());
|
||||
|
||||
return new VotersRecord()
|
||||
.setVersion(version)
|
||||
.setVoters(voterRecordVoters);
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines if two sets of voters have an overlapping majority.
|
||||
*
|
||||
* 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.
|
||||
*
|
||||
* 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) {
|
||||
Set<VoterKey> thisVoterKeys = voters
|
||||
.values()
|
||||
.stream()
|
||||
.map(VoterNode::voterKey)
|
||||
.collect(Collectors.toSet());
|
||||
|
||||
Set<VoterKey> 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;
|
||||
}
|
||||
|
||||
@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);
|
||||
}
|
||||
|
||||
public final static class VoterKey {
|
||||
private final int id;
|
||||
private final Optional<Uuid> directoryId;
|
||||
|
||||
private VoterKey(int id, Optional<Uuid> directoryId) {
|
||||
this.id = id;
|
||||
this.directoryId = directoryId;
|
||||
}
|
||||
|
||||
public int id() {
|
||||
return id;
|
||||
}
|
||||
|
||||
public Optional<Uuid> 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);
|
||||
}
|
||||
|
||||
public static VoterKey of(int id, Optional<Uuid> directoryId) {
|
||||
return new VoterKey(id, directoryId);
|
||||
}
|
||||
}
|
||||
|
||||
final static class VoterNode {
|
||||
private final VoterKey voterKey;
|
||||
private final Map<String, InetSocketAddress> listeners;
|
||||
private final SupportedVersionRange supportedKRaftVersion;
|
||||
|
||||
VoterNode(
|
||||
VoterKey voterKey,
|
||||
Map<String, InetSocketAddress> listeners,
|
||||
SupportedVersionRange supportedKRaftVersion
|
||||
) {
|
||||
this.voterKey = voterKey;
|
||||
this.listeners = listeners;
|
||||
this.supportedKRaftVersion = supportedKRaftVersion;
|
||||
}
|
||||
|
||||
VoterKey voterKey() {
|
||||
return voterKey;
|
||||
}
|
||||
|
||||
Map<String, InetSocketAddress> listeners() {
|
||||
return listeners;
|
||||
}
|
||||
|
||||
SupportedVersionRange supportedKRaftVersion() {
|
||||
return supportedKRaftVersion;
|
||||
}
|
||||
|
||||
|
||||
Optional<InetSocketAddress> 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 (!Objects.equals(voterKey, that.voterKey)) return false;
|
||||
if (!Objects.equals(supportedKRaftVersion, that.supportedKRaftVersion)) return false;
|
||||
if (!Objects.equals(listeners, that.listeners)) return false;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(voterKey, listeners, supportedKRaftVersion);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return String.format(
|
||||
"VoterNode(voterKey=%s, listeners=%s, supportedKRaftVersion=%s)",
|
||||
voterKey,
|
||||
listeners,
|
||||
supportedKRaftVersion
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* 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) {
|
||||
HashMap<Integer, VoterNode> voterNodes = new HashMap<>(voters.voters().size());
|
||||
for (VotersRecord.Voter voter: voters.voters()) {
|
||||
final Optional<Uuid> directoryId;
|
||||
if (!voter.voterDirectoryId().equals(Uuid.ZERO_UUID)) {
|
||||
directoryId = Optional.of(voter.voterDirectoryId());
|
||||
} else {
|
||||
directoryId = Optional.empty();
|
||||
}
|
||||
|
||||
Map<String, InetSocketAddress> 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(
|
||||
VoterKey.of(voter.voterId(), directoryId),
|
||||
listeners,
|
||||
new SupportedVersionRange(
|
||||
voter.kRaftVersionFeature().minSupportedVersion(),
|
||||
voter.kRaftVersionFeature().maxSupportedVersion()
|
||||
)
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
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 fromInetSocketAddresses(String listener, Map<Integer, InetSocketAddress> voters) {
|
||||
Map<Integer, VoterNode> voterNodes = voters
|
||||
.entrySet()
|
||||
.stream()
|
||||
.collect(
|
||||
Collectors.toMap(
|
||||
Map.Entry::getKey,
|
||||
entry -> new VoterNode(
|
||||
VoterKey.of(entry.getKey(), Optional.empty()),
|
||||
Collections.singletonMap(listener, entry.getValue()),
|
||||
new SupportedVersionRange((short) 0, (short) 0)
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
return new VoterSet(voterNodes);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,124 @@
|
|||
/*
|
||||
* 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;
|
||||
|
||||
/**
|
||||
* 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 generating a new snapshot at a given offset or when evaulating
|
||||
* the latest set of voters.
|
||||
*/
|
||||
final public class VoterSetHistory {
|
||||
private final Optional<VoterSet> staticVoterSet;
|
||||
private final LogHistory<VoterSet> votersHistory = new TreeMapLogHistory<>();
|
||||
|
||||
VoterSetHistory(Optional<VoterSet> staticVoterSet) {
|
||||
this.staticVoterSet = staticVoterSet;
|
||||
}
|
||||
|
||||
/**
|
||||
* 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<LogHistory.Entry<VoterSet>> 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().value();
|
||||
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.addAt(offset, voters);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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()}
|
||||
*/
|
||||
public Optional<VoterSet> valueAtOrBefore(long offset) {
|
||||
return votersHistory.valueAtOrBefore(offset);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the latest set of voters.
|
||||
*/
|
||||
public VoterSet lastValue() {
|
||||
Optional<LogHistory.Entry<VoterSet>> result = votersHistory.lastEntry();
|
||||
if (result.isPresent()) {
|
||||
return result.get().value();
|
||||
}
|
||||
|
||||
return staticVoterSet
|
||||
.orElseThrow(() -> new IllegalStateException("No voter set found"));
|
||||
}
|
||||
|
||||
/**
|
||||
* 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);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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);
|
||||
}
|
||||
|
||||
/**
|
||||
* Removes all of the values from this object.
|
||||
*/
|
||||
public void clear() {
|
||||
votersHistory.clear();
|
||||
}
|
||||
}
|
|
@ -20,7 +20,6 @@ import org.apache.kafka.common.record.MemoryRecords;
|
|||
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.channels.FileChannel;
|
|||
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> replicatedLog;
|
||||
private long frozenSize;
|
||||
|
||||
private FileRawSnapshotWriter(
|
||||
Path tempSnapshotPath,
|
||||
FileChannel channel,
|
||||
OffsetAndEpoch snapshotId,
|
||||
Optional<ReplicatedLog> replicatedLog
|
||||
OffsetAndEpoch snapshotId
|
||||
) {
|
||||
this.tempSnapshotPath = tempSnapshotPath;
|
||||
this.channel = channel;
|
||||
this.snapshotId = snapshotId;
|
||||
this.replicatedLog = replicatedLog;
|
||||
this.frozenSize = -1L;
|
||||
}
|
||||
|
||||
|
@ -123,8 +118,6 @@ public final class FileRawSnapshotWriter implements RawSnapshotWriter {
|
|||
|
||||
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 @@ public final class FileRawSnapshotWriter implements RawSnapshotWriter {
|
|||
* @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> 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(
|
||||
|
|
|
@ -0,0 +1,86 @@
|
|||
/*
|
||||
* 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;
|
||||
|
||||
/**
|
||||
* 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<OffsetAndEpoch> callback;
|
||||
|
||||
/**
|
||||
* Constructs a {@code RawSnapshotWriter}.
|
||||
*
|
||||
* @param writer the raw snapshot writer
|
||||
* @param callback the consumer to call when freeze succeeds
|
||||
*/
|
||||
public NotifyingRawSnapshotWriter(RawSnapshotWriter writer, Consumer<OffsetAndEpoch> 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();
|
||||
}
|
||||
|
||||
/**
|
||||
* Delegates the call to the internal writer and invokes the callback on success.
|
||||
*/
|
||||
@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();
|
||||
}
|
||||
}
|
|
@ -127,11 +127,13 @@ public final class RecordsSnapshotReader<T> implements SnapshotReader<T> {
|
|||
Batch<T> 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("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)",
|
||||
|
@ -145,10 +147,8 @@ public final class RecordsSnapshotReader<T> implements SnapshotReader<T> {
|
|||
);
|
||||
}
|
||||
|
||||
if (!batch.records().isEmpty()) {
|
||||
return Optional.of(batch);
|
||||
}
|
||||
}
|
||||
|
||||
return Optional.empty();
|
||||
}
|
||||
|
|
|
@ -18,39 +18,40 @@
|
|||
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.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.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.Optional;
|
||||
import java.util.OptionalLong;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
final public class RecordsSnapshotWriter<T> implements SnapshotWriter<T> {
|
||||
final private RawSnapshotWriter snapshot;
|
||||
final private BatchAccumulator<T> accumulator;
|
||||
final private Time time;
|
||||
final private long lastContainedLogTimestamp;
|
||||
|
||||
private RecordsSnapshotWriter(
|
||||
RawSnapshotWriter snapshot,
|
||||
int maxBatchSize,
|
||||
MemoryPool memoryPool,
|
||||
Time time,
|
||||
long lastContainedLogTimestamp,
|
||||
CompressionType compressionType,
|
||||
RecordSerde<T> serde
|
||||
) {
|
||||
this.snapshot = snapshot;
|
||||
this.time = time;
|
||||
this.lastContainedLogTimestamp = lastContainedLogTimestamp;
|
||||
|
||||
this.accumulator = new BatchAccumulator<>(
|
||||
snapshot.snapshotId().epoch(),
|
||||
|
@ -64,27 +65,6 @@ final public class RecordsSnapshotWriter<T> implements SnapshotWriter<T> {
|
|||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a {@link SnapshotHeaderRecord} to snapshot
|
||||
*
|
||||
* @throws IllegalStateException if the snapshot is not empty
|
||||
*/
|
||||
private void initializeSnapshotWithHeader() {
|
||||
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
|
||||
*
|
||||
|
@ -97,63 +77,6 @@ final public class RecordsSnapshotWriter<T> implements SnapshotWriter<T> {
|
|||
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 <T> Optional<SnapshotWriter<T>> createWithHeader(
|
||||
Supplier<Optional<RawSnapshotWriter>> supplier,
|
||||
int maxBatchSize,
|
||||
MemoryPool memoryPool,
|
||||
Time snapshotTime,
|
||||
long lastContainedLogTimestamp,
|
||||
CompressionType compressionType,
|
||||
RecordSerde<T> serde
|
||||
) {
|
||||
return supplier.get().map(writer ->
|
||||
createWithHeader(
|
||||
writer,
|
||||
maxBatchSize,
|
||||
memoryPool,
|
||||
snapshotTime,
|
||||
lastContainedLogTimestamp,
|
||||
compressionType,
|
||||
serde
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
public static <T> RecordsSnapshotWriter<T> createWithHeader(
|
||||
RawSnapshotWriter rawSnapshotWriter,
|
||||
int maxBatchSize,
|
||||
MemoryPool memoryPool,
|
||||
Time snapshotTime,
|
||||
long lastContainedLogTimestamp,
|
||||
CompressionType compressionType,
|
||||
RecordSerde<T> serde
|
||||
) {
|
||||
RecordsSnapshotWriter<T> writer = new RecordsSnapshotWriter<>(
|
||||
rawSnapshotWriter,
|
||||
maxBatchSize,
|
||||
memoryPool,
|
||||
snapshotTime,
|
||||
lastContainedLogTimestamp,
|
||||
compressionType,
|
||||
serde
|
||||
);
|
||||
writer.initializeSnapshotWithHeader();
|
||||
return writer;
|
||||
}
|
||||
|
||||
@Override
|
||||
public OffsetAndEpoch snapshotId() {
|
||||
return snapshot.snapshotId();
|
||||
|
@ -216,4 +139,125 @@ final public class RecordsSnapshotWriter<T> implements SnapshotWriter<T> {
|
|||
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;
|
||||
private MemoryPool memoryPool = MemoryPool.NONE;
|
||||
private short kraftVersion = 1;
|
||||
private Optional<VoterSet> voterSet = Optional.empty();
|
||||
private Optional<RawSnapshotWriter> 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.ofNullable(rawSnapshotWriter);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setKraftVersion(short kraftVersion) {
|
||||
this.kraftVersion = kraftVersion;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setVoterSet(Optional<VoterSet> voterSet) {
|
||||
this.voterSet = voterSet;
|
||||
return this;
|
||||
}
|
||||
|
||||
public <T> RecordsSnapshotWriter<T> build(RecordSerde<T> 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<T> writer = new RecordsSnapshotWriter<>(
|
||||
rawSnapshotWriter.get(),
|
||||
maxBatchSize,
|
||||
memoryPool,
|
||||
time,
|
||||
compressionType,
|
||||
serde
|
||||
);
|
||||
|
||||
writer.accumulator.appendControlMessages((baseOffset, epoch, buffer) -> {
|
||||
long now = time.milliseconds();
|
||||
try (MemoryRecordsBuilder builder = new MemoryRecordsBuilder(
|
||||
buffer,
|
||||
RecordBatch.CURRENT_MAGIC_VALUE,
|
||||
compressionType,
|
||||
TimestampType.CREATE_TIME,
|
||||
baseOffset,
|
||||
now,
|
||||
RecordBatch.NO_PRODUCER_ID,
|
||||
RecordBatch.NO_PRODUCER_EPOCH,
|
||||
RecordBatch.NO_SEQUENCE,
|
||||
false, // isTransactional
|
||||
true, // isControlBatch
|
||||
epoch,
|
||||
buffer.capacity()
|
||||
)
|
||||
) {
|
||||
builder.appendSnapshotHeaderMessage(
|
||||
now,
|
||||
new SnapshotHeaderRecord()
|
||||
.setVersion(ControlRecordUtils.SNAPSHOT_HEADER_CURRENT_VERSION)
|
||||
.setLastContainedLogTimestamp(lastContainedLogTimestamp)
|
||||
);
|
||||
|
||||
if (kraftVersion > 0) {
|
||||
builder.appendKRaftVersionMessage(
|
||||
now,
|
||||
new KRaftVersionRecord()
|
||||
.setVersion(ControlRecordUtils.KRAFT_VERSION_CURRENT_VERSION)
|
||||
.setKRaftVersion(kraftVersion)
|
||||
);
|
||||
|
||||
if (voterSet.isPresent()) {
|
||||
builder.appendVotersMessage(
|
||||
now,
|
||||
voterSet.get().toVotersRecord(ControlRecordUtils.KRAFT_VOTERS_CURRENT_VERSION)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
return builder.build();
|
||||
}
|
||||
});
|
||||
|
||||
return writer;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 @@ public final class ControlRecordTest {
|
|||
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.KRAFT_VOTERS, new VotersRecord());
|
||||
|
||||
// Invalid constructions
|
||||
assertThrows(
|
||||
|
@ -54,6 +58,6 @@ public final class ControlRecordTest {
|
|||
// 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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -107,8 +107,7 @@ public class KafkaNetworkChannelTest {
|
|||
public void testSendToBlackedOutDestination() throws ExecutionException, InterruptedException {
|
||||
int destinationId = 2;
|
||||
Node destinationNode = new Node(destinationId, "127.0.0.1", 9092);
|
||||
channel.updateEndpoint(destinationId, new QuorumConfig.InetAddressSpec(
|
||||
new InetSocketAddress(destinationNode.host(), destinationNode.port())));
|
||||
channel.updateEndpoint(destinationId, new InetSocketAddress(destinationNode.host(), destinationNode.port()));
|
||||
client.backoff(destinationNode, 500);
|
||||
assertBrokerNotAvailable(destinationId);
|
||||
}
|
||||
|
@ -117,8 +116,7 @@ public class KafkaNetworkChannelTest {
|
|||
public void testWakeupClientOnSend() throws InterruptedException, ExecutionException {
|
||||
int destinationId = 2;
|
||||
Node destinationNode = new Node(destinationId, "127.0.0.1", 9092);
|
||||
channel.updateEndpoint(destinationId, new QuorumConfig.InetAddressSpec(
|
||||
new InetSocketAddress(destinationNode.host(), destinationNode.port())));
|
||||
channel.updateEndpoint(destinationId, new InetSocketAddress(destinationNode.host(), destinationNode.port()));
|
||||
|
||||
client.enableBlockingUntilWakeup(1);
|
||||
|
||||
|
@ -144,8 +142,7 @@ public class KafkaNetworkChannelTest {
|
|||
public void testSendAndDisconnect() throws ExecutionException, InterruptedException {
|
||||
int destinationId = 2;
|
||||
Node destinationNode = new Node(destinationId, "127.0.0.1", 9092);
|
||||
channel.updateEndpoint(destinationId, new QuorumConfig.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));
|
||||
|
@ -158,8 +155,7 @@ public class KafkaNetworkChannelTest {
|
|||
public void testSendAndFailAuthentication() throws ExecutionException, InterruptedException {
|
||||
int destinationId = 2;
|
||||
Node destinationNode = new Node(destinationId, "127.0.0.1", 9092);
|
||||
channel.updateEndpoint(destinationId, new QuorumConfig.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);
|
||||
|
@ -180,8 +176,7 @@ public class KafkaNetworkChannelTest {
|
|||
public void testSendAndReceiveOutboundRequest() throws ExecutionException, InterruptedException {
|
||||
int destinationId = 2;
|
||||
Node destinationNode = new Node(destinationId, "127.0.0.1", 9092);
|
||||
channel.updateEndpoint(destinationId, new QuorumConfig.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;
|
||||
|
@ -196,8 +191,7 @@ public class KafkaNetworkChannelTest {
|
|||
public void testUnsupportedVersionError() throws ExecutionException, InterruptedException {
|
||||
int destinationId = 2;
|
||||
Node destinationNode = new Node(destinationId, "127.0.0.1", 9092);
|
||||
channel.updateEndpoint(destinationId, new QuorumConfig.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);
|
||||
|
@ -210,8 +204,7 @@ public class KafkaNetworkChannelTest {
|
|||
public void testFetchRequestDowngrade(short version) {
|
||||
int destinationId = 2;
|
||||
Node destinationNode = new Node(destinationId, "127.0.0.1", 9092);
|
||||
channel.updateEndpoint(destinationId, new QuorumConfig.InetAddressSpec(
|
||||
new InetSocketAddress(destinationNode.host(), destinationNode.port())));
|
||||
channel.updateEndpoint(destinationId, new InetSocketAddress(destinationNode.host(), destinationNode.port()));
|
||||
sendTestRequest(ApiKeys.FETCH, destinationId);
|
||||
channel.pollOnce();
|
||||
|
||||
|
|
|
@ -18,12 +18,10 @@ package org.apache.kafka.raft;
|
|||
|
||||
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;
|
||||
|
@ -1948,15 +1946,10 @@ final public class KafkaRaftClientSnapshotTest {
|
|||
}
|
||||
|
||||
private static SnapshotWriter<String> 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 {
|
||||
|
|
|
@ -303,12 +303,11 @@ public class KafkaRaftClientTest {
|
|||
Set<Integer> 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 class KafkaRaftClientTest {
|
|||
int epoch = context.currentEpoch();
|
||||
|
||||
assertThrows(BufferAllocationException.class, () -> context.client.scheduleAppend(epoch, singletonList("a")));
|
||||
Mockito.verify(memoryPool).release(buffer);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -881,11 +881,8 @@ public class KafkaRaftClientTest {
|
|||
|
||||
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 class KafkaRaftClientTest {
|
|||
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 class KafkaRaftClientTest {
|
|||
|
||||
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 class KafkaRaftClientTest {
|
|||
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 class KafkaRaftClientTest {
|
|||
|
||||
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 class KafkaRaftClientTest {
|
|||
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
|
||||
|
|
|
@ -489,11 +489,11 @@ public class MockLog implements ReplicatedLog {
|
|||
);
|
||||
}
|
||||
|
||||
return storeSnapshot(snapshotId);
|
||||
return createNewSnapshotUnchecked(snapshotId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<RawSnapshotWriter> storeSnapshot(OffsetAndEpoch snapshotId) {
|
||||
public Optional<RawSnapshotWriter> createNewSnapshotUnchecked(OffsetAndEpoch snapshotId) {
|
||||
if (snapshots.containsKey(snapshotId)) {
|
||||
return Optional.empty();
|
||||
} else {
|
||||
|
|
|
@ -221,7 +221,7 @@ public class MockLogTest {
|
|||
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();
|
||||
|
@ -383,7 +383,7 @@ public class MockLogTest {
|
|||
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();
|
||||
|
@ -639,7 +639,7 @@ public class MockLogTest {
|
|||
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();
|
||||
}
|
||||
|
||||
|
@ -657,7 +657,7 @@ public class MockLogTest {
|
|||
|
||||
appendBatch(numberOfRecords, epoch);
|
||||
|
||||
try (RawSnapshotWriter snapshot = log.storeSnapshot(sameEpochSnapshotId).get()) {
|
||||
try (RawSnapshotWriter snapshot = log.createNewSnapshotUnchecked(sameEpochSnapshotId).get()) {
|
||||
snapshot.freeze();
|
||||
}
|
||||
|
||||
|
@ -671,7 +671,7 @@ public class MockLogTest {
|
|||
|
||||
appendBatch(numberOfRecords, epoch);
|
||||
|
||||
try (RawSnapshotWriter snapshot = log.storeSnapshot(greaterEpochSnapshotId).get()) {
|
||||
try (RawSnapshotWriter snapshot = log.createNewSnapshotUnchecked(greaterEpochSnapshotId).get()) {
|
||||
snapshot.freeze();
|
||||
}
|
||||
|
||||
|
@ -690,7 +690,7 @@ public class MockLogTest {
|
|||
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();
|
||||
}
|
||||
|
||||
|
@ -699,7 +699,7 @@ public class MockLogTest {
|
|||
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();
|
||||
}
|
||||
|
||||
|
@ -722,7 +722,7 @@ public class MockLogTest {
|
|||
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();
|
||||
}
|
||||
|
||||
|
@ -773,7 +773,7 @@ public class MockLogTest {
|
|||
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();
|
||||
|
@ -788,7 +788,7 @@ public class MockLogTest {
|
|||
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();
|
||||
|
@ -803,7 +803,7 @@ public class MockLogTest {
|
|||
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();
|
||||
|
@ -818,7 +818,7 @@ public class MockLogTest {
|
|||
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();
|
||||
|
@ -838,7 +838,7 @@ public class MockLogTest {
|
|||
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();
|
||||
|
|
|
@ -18,6 +18,7 @@ package org.apache.kafka.raft;
|
|||
|
||||
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 class MockNetworkChannel implements NetworkChannel {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void updateEndpoint(int id, QuorumConfig.InetAddressSpec address) {
|
||||
public void updateEndpoint(int id, InetSocketAddress address) {
|
||||
// empty
|
||||
}
|
||||
|
||||
|
|
|
@ -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 class MockQuorumStateStore implements QuorumStateStore {
|
|||
this.current = update;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Path path() {
|
||||
return FileSystems.getDefault().getPath("mock-file");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clear() {
|
||||
current = null;
|
||||
|
|
|
@ -76,6 +76,7 @@ import java.util.OptionalInt;
|
|||
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;
|
||||
|
@ -202,7 +203,7 @@ public final class RaftClientTestContext {
|
|||
}
|
||||
|
||||
Builder withEmptySnapshot(OffsetAndEpoch snapshotId) {
|
||||
try (RawSnapshotWriter snapshot = log.storeSnapshot(snapshotId).get()) {
|
||||
try (RawSnapshotWriter snapshot = log.createNewSnapshotUnchecked(snapshotId).get()) {
|
||||
snapshot.freeze();
|
||||
}
|
||||
return this;
|
||||
|
@ -231,31 +232,42 @@ public final class RaftClientTestContext {
|
|||
Metrics metrics = new Metrics(time);
|
||||
MockNetworkChannel channel = new MockNetworkChannel(voters);
|
||||
MockListener listener = new MockListener(localId);
|
||||
Map<Integer, QuorumConfig.AddressSpec> voterAddressMap = voters.stream()
|
||||
.collect(Collectors.toMap(id -> id, RaftClientTestContext::mockAddress));
|
||||
QuorumConfig quorumConfig = new QuorumConfig(voterAddressMap, requestTimeoutMs, RETRY_BACKOFF_MS, electionTimeoutMs,
|
||||
ELECTION_BACKOFF_MAX_MS, FETCH_TIMEOUT_MS, appendLingerMs);
|
||||
Map<Integer, InetSocketAddress> voterAddressMap = voters
|
||||
.stream()
|
||||
.collect(Collectors.toMap(Function.identity(), RaftClientTestContext::mockAddress));
|
||||
|
||||
QuorumConfig quorumConfig = new QuorumConfig(
|
||||
requestTimeoutMs,
|
||||
RETRY_BACKOFF_MS,
|
||||
electionTimeoutMs,
|
||||
ELECTION_BACKOFF_MAX_MS,
|
||||
FETCH_TIMEOUT_MS,
|
||||
appendLingerMs
|
||||
);
|
||||
|
||||
KafkaRaftClient<String> client = new KafkaRaftClient<>(
|
||||
localId,
|
||||
SERDE,
|
||||
channel,
|
||||
messageQueue,
|
||||
log,
|
||||
quorumStateStore,
|
||||
memoryPool,
|
||||
time,
|
||||
metrics,
|
||||
new MockExpirationService(time),
|
||||
FETCH_MAX_WAIT_MS,
|
||||
clusterId.toString(),
|
||||
localId,
|
||||
logContext,
|
||||
random,
|
||||
quorumConfig
|
||||
);
|
||||
|
||||
client.register(listener);
|
||||
client.initialize();
|
||||
client.initialize(
|
||||
voterAddressMap,
|
||||
"CONTROLLER",
|
||||
quorumStateStore,
|
||||
metrics
|
||||
);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext(
|
||||
clusterId,
|
||||
|
@ -811,8 +823,8 @@ public final class RaftClientTestContext {
|
|||
return requests;
|
||||
}
|
||||
|
||||
private static QuorumConfig.AddressSpec mockAddress(int id) {
|
||||
return new QuorumConfig.InetAddressSpec(new InetSocketAddress("localhost", 9990 + id));
|
||||
private static InetSocketAddress mockAddress(int id) {
|
||||
return new InetSocketAddress("localhost", 9990 + id);
|
||||
}
|
||||
|
||||
EndQuorumEpochResponseData endEpochResponse(
|
||||
|
|
|
@ -59,6 +59,7 @@ import java.util.Set;
|
|||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.function.Consumer;
|
||||
import java.util.function.Function;
|
||||
import java.util.function.Supplier;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
|
@ -708,8 +709,8 @@ public class RaftEventSimulationTest {
|
|||
nodes.put(nodeId, new PersistentState(nodeId));
|
||||
}
|
||||
|
||||
private static QuorumConfig.AddressSpec nodeAddress(int id) {
|
||||
return new QuorumConfig.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 @@ public class RaftEventSimulationTest {
|
|||
PersistentState persistentState = nodes.get(nodeId);
|
||||
MockNetworkChannel channel = new MockNetworkChannel(correlationIdCounter, voters);
|
||||
MockMessageQueue messageQueue = new MockMessageQueue();
|
||||
Map<Integer, QuorumConfig.AddressSpec> voterAddressMap = voters.stream()
|
||||
.collect(Collectors.toMap(id -> id, Cluster::nodeAddress));
|
||||
QuorumConfig quorumConfig = new QuorumConfig(voterAddressMap, REQUEST_TIMEOUT_MS, RETRY_BACKOFF_MS, ELECTION_TIMEOUT_MS,
|
||||
ELECTION_JITTER_MS, FETCH_TIMEOUT_MS, LINGER_MS);
|
||||
Map<Integer, InetSocketAddress> voterAddressMap = voters
|
||||
.stream()
|
||||
.collect(Collectors.toMap(Function.identity(), Cluster::nodeAddress));
|
||||
|
||||
QuorumConfig quorumConfig = new QuorumConfig(
|
||||
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();
|
||||
|
@ -729,18 +738,16 @@ public class RaftEventSimulationTest {
|
|||
MemoryPool memoryPool = new BatchMemoryPool(2, KafkaRaftClient.MAX_BATCH_SIZE_BYTES);
|
||||
|
||||
KafkaRaftClient<Integer> client = new KafkaRaftClient<>(
|
||||
OptionalInt.of(nodeId),
|
||||
serde,
|
||||
channel,
|
||||
messageQueue,
|
||||
persistentState.log,
|
||||
persistentState.store,
|
||||
memoryPool,
|
||||
time,
|
||||
metrics,
|
||||
new MockExpirationService(time),
|
||||
FETCH_MAX_WAIT_MS,
|
||||
clusterId.toString(),
|
||||
OptionalInt.of(nodeId),
|
||||
logContext,
|
||||
random,
|
||||
quorumConfig
|
||||
|
@ -757,7 +764,8 @@ public class RaftEventSimulationTest {
|
|||
random,
|
||||
serde
|
||||
);
|
||||
node.initialize();
|
||||
node.initialize(voterAddressMap, metrics);
|
||||
|
||||
running.put(nodeId, node);
|
||||
}
|
||||
}
|
||||
|
@ -800,9 +808,14 @@ public class RaftEventSimulationTest {
|
|||
this.intSerde = intSerde;
|
||||
}
|
||||
|
||||
void initialize() {
|
||||
client.register(this.counter);
|
||||
client.initialize();
|
||||
void initialize(Map<Integer, InetSocketAddress> voterAddresses, Metrics metrics) {
|
||||
client.register(counter);
|
||||
client.initialize(
|
||||
voterAddresses,
|
||||
"CONTROLLER",
|
||||
store,
|
||||
metrics
|
||||
);
|
||||
}
|
||||
|
||||
void poll() {
|
||||
|
@ -1111,22 +1124,33 @@ public class RaftEventSimulationTest {
|
|||
assertTrue(snapshotId.offset() <= highWatermark.getAsLong());
|
||||
startOffset.set(snapshotId.offset());
|
||||
|
||||
try (SnapshotReader<Integer> snapshot =
|
||||
RecordsSnapshotReader.of(log.readSnapshot(snapshotId).get(), node.intSerde, BufferSupplier.create(), Integer.MAX_VALUE, true)) {
|
||||
try (SnapshotReader<Integer> 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());
|
||||
OptionalInt sequence = OptionalInt.empty();
|
||||
while (snapshot.hasNext()) {
|
||||
Batch<Integer> batch = snapshot.next();
|
||||
assertFalse(snapshot.hasNext());
|
||||
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)
|
||||
);
|
||||
}
|
||||
|
@ -1146,8 +1170,10 @@ public class RaftEventSimulationTest {
|
|||
|
||||
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)
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
@ -78,8 +83,8 @@ class BatchAccumulatorTest {
|
|||
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<String> acc = buildAccumulator(
|
||||
|
@ -491,6 +496,8 @@ class BatchAccumulatorTest {
|
|||
completedBatch.data.batches().forEach(recordBatch -> {
|
||||
assertEquals(leaderEpoch, recordBatch.partitionLeaderEpoch()); });
|
||||
});
|
||||
|
||||
acc.close();
|
||||
}
|
||||
|
||||
int recordSizeInBytes(String record, int numberOfRecords) {
|
||||
|
@ -552,4 +559,150 @@ class BatchAccumulatorTest {
|
|||
}
|
||||
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<String> 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<BatchAccumulator.CompletedBatch<String>> batches = acc.drain();
|
||||
assertEquals(1, batches.size());
|
||||
|
||||
BatchAccumulator.CompletedBatch<String> 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<String> 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<String> 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()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 KRaftControlRecordStateMachineTest {
|
||||
private static final RecordSerde<String> STRING_SERDE = new StringSerde();
|
||||
|
||||
private static MockLog buildLog() {
|
||||
return new MockLog(new TopicPartition("partition", 0), Uuid.randomUuid(), new LogContext());
|
||||
}
|
||||
|
||||
private static KRaftControlRecordStateMachine buildPartitionListener(MockLog log, Optional<VoterSet> staticVoterSet) {
|
||||
return new KRaftControlRecordStateMachine(
|
||||
staticVoterSet,
|
||||
log,
|
||||
STRING_SERDE,
|
||||
BufferSupplier.NO_CACHING,
|
||||
1024,
|
||||
new LogContext()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testEmptyPartition() {
|
||||
MockLog log = buildLog();
|
||||
VoterSet voterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(1, 2, 3)));
|
||||
|
||||
KRaftControlRecordStateMachine partitionState = buildPartitionListener(log, Optional.of(voterSet));
|
||||
|
||||
// This should be a no-op operation
|
||||
partitionState.updateState();
|
||||
|
||||
assertEquals(voterSet, partitionState.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;
|
||||
|
||||
KRaftControlRecordStateMachine partitionState = 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
|
||||
partitionState.updateState();
|
||||
|
||||
assertEquals(voterSet, partitionState.lastVoterSet());
|
||||
assertEquals(Optional.of(voterSet), partitionState.voterSetAtOffset(log.endOffset().offset - 1));
|
||||
assertEquals(kraftVersion, partitionState.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;
|
||||
|
||||
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()
|
||||
.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
|
||||
partitionState.updateState();
|
||||
|
||||
assertEquals(voterSet, partitionState.lastVoterSet());
|
||||
assertEquals(Optional.of(voterSet), partitionState.voterSetAtOffset(log.endOffset().offset - 1));
|
||||
assertEquals(kraftVersion, partitionState.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;
|
||||
|
||||
KRaftControlRecordStateMachine partitionState = 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
|
||||
partitionState.updateState();
|
||||
|
||||
assertEquals(voterSet, partitionState.lastVoterSet());
|
||||
assertEquals(Optional.of(voterSet), partitionState.voterSetAtOffset(log.endOffset().offset - 1));
|
||||
assertEquals(kraftVersion, partitionState.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;
|
||||
|
||||
KRaftControlRecordStateMachine partitionState = 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
|
||||
partitionState.updateState();
|
||||
|
||||
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), partitionState.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;
|
||||
|
||||
KRaftControlRecordStateMachine partitionState = 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
|
||||
partitionState.updateState();
|
||||
|
||||
assertEquals(voterSet, partitionState.lastVoterSet());
|
||||
|
||||
// Truncate log and listener
|
||||
log.truncateTo(voterSetOffset);
|
||||
partitionState.truncateNewEntries(voterSetOffset);
|
||||
|
||||
assertEquals(firstVoterSet, partitionState.lastVoterSet());
|
||||
|
||||
// Truncate the entire log
|
||||
log.truncateTo(0);
|
||||
partitionState.truncateNewEntries(0);
|
||||
|
||||
assertEquals(staticVoterSet, partitionState.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;
|
||||
|
||||
KRaftControlRecordStateMachine partitionState = 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
|
||||
partitionState.updateState();
|
||||
|
||||
assertEquals(voterSet, partitionState.lastVoterSet());
|
||||
assertEquals(kraftVersion, partitionState.kraftVersionAtOffset(kraftVersionOffset));
|
||||
|
||||
// Trim the prefix for the partition listener up to the kraft.version
|
||||
partitionState.truncateOldEntries(kraftVersionOffset);
|
||||
assertEquals(kraftVersion, partitionState.kraftVersionAtOffset(kraftVersionOffset));
|
||||
|
||||
// Trim the prefix for the partition listener up to the first voter set
|
||||
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
|
||||
partitionState.truncateOldEntries(voterSetOffset);
|
||||
assertEquals(kraftVersion, partitionState.kraftVersionAtOffset(kraftVersionOffset));
|
||||
assertEquals(Optional.empty(), partitionState.voterSetAtOffset(firstVoterSetOffset));
|
||||
assertEquals(Optional.of(voterSet), partitionState.voterSetAtOffset(voterSetOffset));
|
||||
}
|
||||
}
|
|
@ -21,10 +21,12 @@ import java.nio.ByteBuffer;
|
|||
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;
|
||||
|
@ -33,17 +35,23 @@ import java.util.stream.Stream;
|
|||
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;
|
||||
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;
|
||||
|
@ -145,18 +153,16 @@ public final class RecordsIteratorTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testControlRecordIteration() {
|
||||
public void testControlRecordIterationWithKraftVerion0() {
|
||||
AtomicReference<ByteBuffer> buffer = new AtomicReference<>(null);
|
||||
try (RecordsSnapshotWriter<String> 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())
|
||||
.setKraftVersion((short) 0)
|
||||
.setVoterSet(Optional.empty())
|
||||
.setRawSnapshotWriter(
|
||||
new MockRawSnapshotWriter(new OffsetAndEpoch(100, 10), snapshotBuf -> buffer.set(snapshotBuf))
|
||||
);
|
||||
try (RecordsSnapshotWriter<String> 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"));
|
||||
|
@ -169,10 +175,11 @@ public final class RecordsIteratorTest {
|
|||
true
|
||||
)
|
||||
) {
|
||||
// Check snapshot header control record
|
||||
// Consume the control record batch
|
||||
Batch<String> 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());
|
||||
|
||||
|
@ -192,24 +199,70 @@ public final class RecordsIteratorTest {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testControlRecordIterationWithKraftVerion1() {
|
||||
AtomicReference<ByteBuffer> 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<String> 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<String> iterator = createIterator(
|
||||
MemoryRecords.readableRecords(buffer.get()),
|
||||
BufferSupplier.NO_CACHING,
|
||||
true
|
||||
)
|
||||
) {
|
||||
// Consume the control record batch
|
||||
Batch<String> 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.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
|
||||
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());
|
||||
}
|
||||
}
|
||||
|
||||
@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", "KRAFT_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<String> iterator = createIterator(records, BufferSupplier.NO_CACHING, true)) {
|
||||
assertTrue(iterator.hasNext());
|
||||
|
@ -226,7 +279,7 @@ public final class RecordsIteratorTest {
|
|||
// 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 +361,33 @@ public final class RecordsIteratorTest {
|
|||
}
|
||||
|
||||
public static MemoryRecords buildControlRecords(ControlRecordType type) {
|
||||
final MemoryRecords records;
|
||||
switch (type) {
|
||||
case LEADER_CHANGE:
|
||||
records = MemoryRecords.withLeaderChangeMessage(
|
||||
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,
|
||||
0,
|
||||
1,
|
||||
ByteBuffer.allocate(128),
|
||||
new LeaderChangeMessage()
|
||||
type,
|
||||
MessageUtil.toByteBuffer(defaultControlRecord(type), defaultControlRecordVersion(type))
|
||||
);
|
||||
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));
|
||||
}
|
||||
|
||||
return records;
|
||||
buffer.flip();
|
||||
return MemoryRecords.readableRecords(buffer);
|
||||
}
|
||||
|
||||
public static MemoryRecords buildRecords(
|
||||
|
@ -415,4 +461,38 @@ public final class RecordsIteratorTest {
|
|||
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 KRAFT_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 KRAFT_VOTERS:
|
||||
return ControlRecordUtils.KRAFT_VOTERS_CURRENT_VERSION;
|
||||
default:
|
||||
throw new RuntimeException("Should not happen. Poorly configured test");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,104 @@
|
|||
/*
|
||||
* 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 TreeMapLogHistoryTest {
|
||||
@Test
|
||||
void testEmpty() {
|
||||
TreeMapLogHistory<String> history = new TreeMapLogHistory<>();
|
||||
assertEquals(Optional.empty(), history.valueAtOrBefore(100));
|
||||
assertEquals(Optional.empty(), history.lastEntry());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testAddAt() {
|
||||
TreeMapLogHistory<String> history = new TreeMapLogHistory<>();
|
||||
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, ""));
|
||||
assertEquals(Optional.of("100"), history.valueAtOrBefore(100));
|
||||
assertEquals(Optional.of("100"), history.valueAtOrBefore(201));
|
||||
|
||||
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 LogHistory.Entry<>(200, "200")), history.lastEntry());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testTruncateTo() {
|
||||
TreeMapLogHistory<String> history = new TreeMapLogHistory<>();
|
||||
history.addAt(100, "100");
|
||||
history.addAt(200, "200");
|
||||
|
||||
history.truncateNewEntries(201);
|
||||
assertEquals(Optional.of(new LogHistory.Entry<>(200, "200")), history.lastEntry());
|
||||
|
||||
history.truncateNewEntries(200);
|
||||
assertEquals(Optional.of(new LogHistory.Entry<>(100, "100")), history.lastEntry());
|
||||
|
||||
history.truncateNewEntries(101);
|
||||
assertEquals(Optional.of(new LogHistory.Entry<>(100, "100")), history.lastEntry());
|
||||
|
||||
history.truncateNewEntries(100);
|
||||
assertEquals(Optional.empty(), history.lastEntry());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testTrimPrefixTo() {
|
||||
TreeMapLogHistory<String> history = new TreeMapLogHistory<>();
|
||||
history.addAt(100, "100");
|
||||
history.addAt(200, "200");
|
||||
|
||||
history.truncateOldEntries(99);
|
||||
assertEquals(Optional.empty(), history.valueAtOrBefore(99));
|
||||
assertEquals(Optional.of("100"), history.valueAtOrBefore(100));
|
||||
|
||||
history.truncateOldEntries(100);
|
||||
assertEquals(Optional.empty(), history.valueAtOrBefore(99));
|
||||
assertEquals(Optional.of("100"), history.valueAtOrBefore(100));
|
||||
|
||||
history.truncateOldEntries(101);
|
||||
assertEquals(Optional.empty(), history.valueAtOrBefore(99));
|
||||
assertEquals(Optional.of("100"), history.valueAtOrBefore(100));
|
||||
|
||||
history.truncateOldEntries(200);
|
||||
assertEquals(Optional.empty(), history.valueAtOrBefore(199));
|
||||
assertEquals(Optional.of("200"), history.valueAtOrBefore(200));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testClear() {
|
||||
TreeMapLogHistory<String> history = new TreeMapLogHistory<>();
|
||||
history.addAt(100, "100");
|
||||
history.addAt(200, "200");
|
||||
history.clear();
|
||||
assertEquals(Optional.empty(), history.lastEntry());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,216 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.kafka.raft.internals;
|
||||
|
||||
import java.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
|
||||
void testStaicVoterSet() {
|
||||
VoterSet staticVoterSet = new VoterSet(VoterSetTest.voterMap(Arrays.asList(1, 2, 3)));
|
||||
VoterSetHistory votersHistory = new VoterSetHistory(Optional.of(staticVoterSet));
|
||||
|
||||
assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(0));
|
||||
assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(100));
|
||||
assertEquals(staticVoterSet, votersHistory.lastValue());
|
||||
|
||||
// Should be a no-op
|
||||
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.truncateOldEntries(100);
|
||||
assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(0));
|
||||
assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(100));
|
||||
assertEquals(staticVoterSet, votersHistory.lastValue());
|
||||
}
|
||||
|
||||
@Test
|
||||
void TestNoStaticVoterSet() {
|
||||
VoterSetHistory votersHistory = new VoterSetHistory(Optional.empty());
|
||||
|
||||
assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(0));
|
||||
assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(100));
|
||||
assertThrows(IllegalStateException.class, votersHistory::lastValue);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testAddAt() {
|
||||
Map<Integer, VoterSet.VoterNode> 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 testAddAtNonOverlapping() {
|
||||
VoterSetHistory votersHistory = new VoterSetHistory(Optional.empty());
|
||||
|
||||
Map<Integer, VoterSet.VoterNode> 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<Integer, VoterSet.VoterNode> 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<Integer, VoterSet.VoterNode> 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.truncateNewEntries(201);
|
||||
assertEquals(voterSet12345, votersHistory.lastValue());
|
||||
votersHistory.truncateNewEntries(200);
|
||||
assertEquals(voterSet1234, votersHistory.lastValue());
|
||||
votersHistory.truncateNewEntries(101);
|
||||
assertEquals(voterSet1234, votersHistory.lastValue());
|
||||
votersHistory.truncateNewEntries(100);
|
||||
assertEquals(staticVoterSet, votersHistory.lastValue());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testTrimPrefixTo() {
|
||||
Map<Integer, VoterSet.VoterNode> 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.truncateOldEntries(99);
|
||||
assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(99));
|
||||
assertEquals(Optional.of(voterSet1234), votersHistory.valueAtOrBefore(100));
|
||||
|
||||
votersHistory.truncateOldEntries(100);
|
||||
assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(99));
|
||||
assertEquals(Optional.of(voterSet1234), votersHistory.valueAtOrBefore(100));
|
||||
|
||||
votersHistory.truncateOldEntries(101);
|
||||
assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(99));
|
||||
assertEquals(Optional.of(voterSet1234), votersHistory.valueAtOrBefore(100));
|
||||
|
||||
votersHistory.truncateOldEntries(200);
|
||||
assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(199));
|
||||
assertEquals(Optional.of(voterSet12345), votersHistory.valueAtOrBefore(200));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testClear() {
|
||||
Map<Integer, VoterSet.VoterNode> 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(staticVoterSet, votersHistory.lastValue());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,184 @@
|
|||
/*
|
||||
* 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<Integer, VoterSet.VoterNode> 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.voterKey().id(), voter4);
|
||||
assertEquals(Optional.of(new VoterSet(new HashMap<>(aVoterMap))), voterSet.addVoter(voter4));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testRemoveVoter() {
|
||||
Map<Integer, VoterSet.VoterNode> aVoterMap = voterMap(Arrays.asList(1, 2, 3));
|
||||
VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap));
|
||||
|
||||
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.voterKey())
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testRecordRoundTrip() {
|
||||
VoterSet voterSet = new VoterSet(voterMap(Arrays.asList(1, 2, 3)));
|
||||
|
||||
assertEquals(voterSet, VoterSet.fromVotersRecord(voterSet.toVotersRecord((short) 0)));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testOverlappingMajority() {
|
||||
Map<Integer, VoterSet.VoterNode> 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(startingVoterMap.get(1).voterKey())
|
||||
.get();
|
||||
assertMajorities(true, startingVoterSet, smallerVoterSet);
|
||||
|
||||
VoterSet replacedVoterSet = startingVoterSet
|
||||
.removeVoter(startingVoterMap.get(1).voterKey())
|
||||
.get()
|
||||
.addVoter(voterNode(1))
|
||||
.get();
|
||||
assertMajorities(true, startingVoterSet, replacedVoterSet);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testNonoverlappingMajority() {
|
||||
Map<Integer, VoterSet.VoterNode> 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(startingVoterMap.get(1).voterKey())
|
||||
.get()
|
||||
.removeVoter(startingVoterMap.get(2).voterKey())
|
||||
.get();
|
||||
assertMajorities(false, startingVoterSet, smallerVoterSet);
|
||||
|
||||
// Two replacements don't have an overlapping majority
|
||||
VoterSet replacedVoterSet = startingVoterSet
|
||||
.removeVoter(startingVoterMap.get(1).voterKey())
|
||||
.get()
|
||||
.addVoter(voterNode(1))
|
||||
.get()
|
||||
.removeVoter(startingVoterMap.get(2).voterKey())
|
||||
.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<Integer, VoterSet.VoterNode> voterMap(List<Integer> replicas) {
|
||||
return replicas
|
||||
.stream()
|
||||
.collect(
|
||||
Collectors.toMap(
|
||||
Function.identity(),
|
||||
VoterSetTest::voterNode
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
static VoterSet.VoterNode voterNode(int id) {
|
||||
return new VoterSet.VoterNode(
|
||||
VoterSet.VoterKey.of(id, Optional.of(Uuid.randomUuid())),
|
||||
Collections.singletonMap(
|
||||
"LISTENER",
|
||||
InetSocketAddress.createUnresolved(String.format("replica-%d", id), 1234)
|
||||
),
|
||||
new SupportedVersionRange((short) 0, (short) 0)
|
||||
);
|
||||
}
|
||||
|
||||
public static VoterSet voterSet(Map<Integer, VoterSet.VoterNode> voters) {
|
||||
return new VoterSet(voters);
|
||||
}
|
||||
}
|
|
@ -37,7 +37,6 @@ import java.nio.file.Files;
|
|||
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 @@ public final class FileRawSnapshotTest {
|
|||
Path dir,
|
||||
OffsetAndEpoch snapshotId
|
||||
) {
|
||||
return FileRawSnapshotWriter.create(dir, snapshotId, Optional.empty());
|
||||
return FileRawSnapshotWriter.create(dir, snapshotId);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<OffsetAndEpoch> 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<OffsetAndEpoch> 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<OffsetAndEpoch> 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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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> STRING_SERDE = new StringSerde();
|
||||
|
||||
@Test
|
||||
void testBuilderKRaftVersion0() {
|
||||
OffsetAndEpoch snapshotId = new OffsetAndEpoch(100, 10);
|
||||
int maxBatchSize = 1024;
|
||||
AtomicReference<ByteBuffer> 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<String> snapshot = builder.build(STRING_SERDE)) {
|
||||
snapshot.freeze();
|
||||
}
|
||||
|
||||
try (RecordsSnapshotReader<String> reader = RecordsSnapshotReader.of(
|
||||
new MockRawSnapshotReader(snapshotId, buffer.get()),
|
||||
STRING_SERDE,
|
||||
BufferSupplier.NO_CACHING,
|
||||
maxBatchSize,
|
||||
true
|
||||
)
|
||||
) {
|
||||
// Consume the control record batch
|
||||
Batch<String> 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<ByteBuffer> 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<ByteBuffer> 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<String> snapshot = builder.build(STRING_SERDE)) {
|
||||
snapshot.freeze();
|
||||
}
|
||||
|
||||
try (RecordsSnapshotReader<String> reader = RecordsSnapshotReader.of(
|
||||
new MockRawSnapshotReader(snapshotId, buffer.get()),
|
||||
STRING_SERDE,
|
||||
BufferSupplier.NO_CACHING,
|
||||
maxBatchSize,
|
||||
true
|
||||
)
|
||||
) {
|
||||
// Consume the control record batch
|
||||
Batch<String> 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.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
|
||||
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<ByteBuffer> 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<String> snapshot = builder.build(STRING_SERDE)) {
|
||||
snapshot.freeze();
|
||||
}
|
||||
|
||||
try (RecordsSnapshotReader<String> reader = RecordsSnapshotReader.of(
|
||||
new MockRawSnapshotReader(snapshotId, buffer.get()),
|
||||
STRING_SERDE,
|
||||
BufferSupplier.NO_CACHING,
|
||||
maxBatchSize,
|
||||
true
|
||||
)
|
||||
) {
|
||||
// Consume the control record batch
|
||||
Batch<String> 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());
|
||||
}
|
||||
}
|
||||
}
|
|
@ -108,7 +108,7 @@ final public class SnapshotsTest {
|
|||
);
|
||||
|
||||
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);
|
||||
|
|
Loading…
Reference in New Issue