Skip to content

Conversation

@mannoopj
Copy link
Contributor

KIP-1170
Change formatter code to ensure that when the directories are formatted through the kafka storage command, metadata records normally written to the bootstrap.checkpoint file are instead located in the 00000000000000000000-0000000000.checkpoint file. Then if any kraft records generated during this format they will be located after the metadata records in the 00000000000000000000-0000000000.checkpoint file. This PR is currently a draft and opened to allow for reviews to code changes to formatter for this kip.

@github-actions github-actions bot added triage PRs from the community kraft small Small PRs labels Oct 15, 2025
@mannoopj mannoopj changed the title KIP-1170: Formatter changes KAFKA-19648: Formatter refactoring Oct 15, 2025
Copy link
Contributor

@kevin-wu24 kevin-wu24 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the changes @mannoopj. Some high level comments:

Comment on lines 522 to 540
static void writeDynamicQuorumSnapshot(
String writeLogDir,
DynamicVoters initialControllers,
short kraftVersion,
String controllerListenerName
) {
File parentDir = new File(writeLogDir);
File clusterMetadataDirectory = new File(parentDir, String.format("%s-%d",
CLUSTER_METADATA_TOPIC_PARTITION.topic(),
CLUSTER_METADATA_TOPIC_PARTITION.partition()));
VoterSet voterSet = initialControllers.toVoterSet(controllerListenerName);
RecordsSnapshotWriter.Builder builder = new RecordsSnapshotWriter.Builder().
setLastContainedLogTimestamp(Time.SYSTEM.milliseconds()).
setMaxBatchSizeBytes(KafkaRaftClient.MAX_BATCH_SIZE_BYTES).
setRawSnapshotWriter(FileRawSnapshotWriter.create(
clusterMetadataDirectory.toPath(),
Paths.get(writeLogDir),
Snapshots.BOOTSTRAP_SNAPSHOT_ID)).
setKraftVersion(KRaftVersion.fromFeatureLevel(kraftVersion)).
setVoterSet(Optional.of(voterSet));
try (RecordsSnapshotWriter<ApiMessageAndVersion> writer = builder.build(new MetadataRecordSerde())) {
writer.freeze();
}
}
Copy link
Contributor

@kevin-wu24 kevin-wu24 Oct 15, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There is a clearer way to make these changes. This method is what writes the 0-0.checkpoint currently. We should pass the bootstrap metadata object here and append the metadata records using writer.append.append(bootstrapMetadata.records()) before calling writer.freeze().

Comment on lines 448 to 449
if (directoryTypes.get(writeLogDir).isDynamicMetadataDirectory()) {
writeDynamicQuorumSnapshot(writeLogDir,
writeDynamicQuorumSnapshot(clusterMetadataDirectory.getPath(),
Copy link
Contributor

@kevin-wu24 kevin-wu24 Oct 15, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should rename writeDynamicQuorumSnapshot to writeZeroSnapshot, and write it when formatting metadata directories. The semantics that change here are that we should not write the KRaft control records (KRaft version and voter set) when !isDynamicMetadataDirectory().

Comment on lines 453 to 472
File createdBoostrapCheckpoint = new File(clusterMetadataDirectory.getPath() + "/" + BootstrapDirectory.BINARY_BOOTSTRAP_FILENAME);
File created000Checkpoint = new File(clusterMetadataDirectory.getPath() + "/" + BootstrapDirectory.BINARY_CHECKPOINT_FILENAME);
Files.write(
createdBoostrapCheckpoint.toPath(),
Files.readAllBytes(created000Checkpoint.toPath()),
StandardOpenOption.APPEND);
try {
created000Checkpoint.delete();
createdBoostrapCheckpoint.renameTo(created000Checkpoint);
} catch (Exception ex) {
throw new RuntimeException("Failed operation to combine metadata and kraft records: ", ex);
}
} else {
File createdBoostrapCheckpoint = new File(clusterMetadataDirectory.getPath() + "/" + BootstrapDirectory.BINARY_BOOTSTRAP_FILENAME);
File created000Checkpoint = new File(clusterMetadataDirectory.getPath() + "/" + BootstrapDirectory.BINARY_CHECKPOINT_FILENAME);
try {
createdBoostrapCheckpoint.renameTo(created000Checkpoint);
} catch (Exception ex) {
throw new RuntimeException("Failed to rename file: ", ex);
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This code is confusing. Instead of doing this renaming and deleting. We should instead remove the call to write the bootstrap metadata to disk on Line 447, since we're no longer writing bootstrap.checkpoint anymore, and follow the other comments for writing metadata records to 0-0.checkpoint.

We can check if an old bootstrap.checkpoint exists and delete it, since IIRC that was part of the KIP.

@github-actions github-actions bot removed the triage PRs from the community label Oct 16, 2025
@github-actions github-actions bot added the core Kafka Broker label Oct 20, 2025
Copy link
Contributor

@kevin-wu24 kevin-wu24 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the changes @mannoopj. Left some more comments

Comment on lines 513 to 514
try (RecordsSnapshotWriter<ApiMessageAndVersion> writer = builder.build(new MetadataRecordSerde(), Optional.of(bootstrapMetadata.records()))) {
writer.freeze();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
try (RecordsSnapshotWriter<ApiMessageAndVersion> writer = builder.build(new MetadataRecordSerde(), Optional.of(bootstrapMetadata.records()))) {
writer.freeze();
try (RecordsSnapshotWriter<ApiMessageAndVersion> writer = builder.build(new MetadataRecordSerde()))) {
writer.append(bootstrapMetadata.records());
writer.freeze();

Copy link
Contributor Author

@mannoopj mannoopj Oct 20, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Wouldn't this write the bootstrap records after the control records, since in RecordsSnapshotWriter.build() is where we append the kraft records and we would be calling that ahead of writer.append in this scenario? we want the bootstrap records ahead correct?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It doesn't matter what order we write them in because KRaft only reads the control records, and the metadata module will only read the "data" records. When we read the 0-0.checkpoint back into memory, we only deal with either its control records or its data records, not both in the same code.

.setKraftVersion(KRaftVersion.KRAFT_VERSION_1)
.setVoterSet(Optional.of(VoterSetTest.voterSet(VoterSetTest.voterMap(IntStream.of(1, 2, 3), true))))
.build(MetadataRecordSerde.INSTANCE)
.build(MetadataRecordSerde.INSTANCE, emptyOptional)
Copy link
Contributor

@kevin-wu24 kevin-wu24 Oct 20, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should be able to remove this. This applies to other instances where we build the RecordsSnapshotWriter.

}
writeBoostrapSnapshot(writeLogDir,
bootstrapMetadata,
initialControllers.get(),
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We need to pass the optional for initialControllers here. We can only do a .get() if initialControllers.isPresent().

File clusterMetadataDirectory = new File(parentDir, String.format("%s-%d",
CLUSTER_METADATA_TOPIC_PARTITION.topic(),
CLUSTER_METADATA_TOPIC_PARTITION.partition()));
VoterSet voterSet = initialControllers.toVoterSet(controllerListenerName);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should only construct the VoterSet object if initialControllers.isPresent().

@github-actions github-actions bot added clients and removed small Small PRs labels Oct 24, 2025
Comment on lines +1030 to +1033
// For bootstrap snapshots, extract feature levels from all data records
if (batch.controlRecords().isEmpty()) {
bootstrapMetadata = BootstrapMetadata.fromRecords(messages, "bootstrap");
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The correct logic here is:
If the batch has records, read them into bootstrapMetadata (this means 0-0.checkpoint has bootstrap metadata records).
If the batch doesn't have records, try to read the bootstrapMetadata from bootstrap.checkpoint.

Copy link
Contributor

@kevin-wu24 kevin-wu24 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the changes @mannoopj. Review of the metadata layer implementation:

Comment on lines -63 to +66
if (level > 0) {
records.add(new ApiMessageAndVersion(new FeatureLevelRecord().
setName(featureName).
setFeatureLevel(level), (short) 0));
}
// Include all feature levels, including level 0 which may disable features
records.add(new ApiMessageAndVersion(new FeatureLevelRecord().
setName(featureName).
setFeatureLevel(level), (short) 0));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This should not change. The default level of features is 0, and that is why we don't add a record for them when the level is 0.

CLUSTER_METADATA_TOPIC_PARTITION.partition()),
BINARY_BOOTSTRAP_CHECKPOINT_FILENAME);
if (!Files.exists(binaryBootstrapPath)) {
return readFromConfiguration();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If we are at L76, that means we do 0-0.checkpoint doesn't exist. This is where we should read from bootstrap.checkpoint. If that doesn't exist too, then we call readFromConfiguration().

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actually, if you look at how we're reading stuff in, we probably don't even need to change this file. We don't need to call BootstrapDirectory#read for the 0-0.checkpoint because we are using handleLoadSnapshot, which already puts the checkpoint in memory for us.

(metaPropsEnsemble, bootstrapMetadata)
// val bootstrapDirectory = new BootstrapDirectory(config.metadataLogDir)
// val bootstrapMetadata = bootstrapDirectory.read()
(metaPropsEnsemble, null)
Copy link
Contributor

@kevin-wu24 kevin-wu24 Oct 24, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should remove bootstrapMetadata here and in ControllerServer since it is just being passed down to QuorumController eventually. We initialize it in QuorumController.

Comment on lines +486 to +497

// Copy feature levels from TestKitNodes bootstrap metadata to ensure test annotations are respected
for (var record : nodes.bootstrapMetadata().records()) {
if (record.message() instanceof FeatureLevelRecord featureLevelRecord) {
String featureName = featureLevelRecord.name();
short level = featureLevelRecord.featureLevel();
// Don't override MetadataVersion as it's handled by setReleaseVersion()
if (!featureName.equals("metadata.version")) {
formatter.setFeatureLevel(featureName, level);
}
}
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Some background on how the KafkaClusterTestKit works:

Basically, tests that use this class are "integration tests" in the sense that we're trying to replicate a real cluster, just all within the same JVM. That means multiple brokerServers and controllerServers. This file shouldn't change outside of removing nodes.bootstrapMetadata() from the ControllerServer constructor.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants