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 1048
// 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.

@github-actions github-actions bot added streams build Gradle build or GitHub Actions labels Nov 3, 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.

Left a high level comment @mannoopj

Comment on lines 184 to 186
// val bootstrapDirectory = new BootstrapDirectory(config.metadataLogDir)
// val bootstrapMetadata = bootstrapDirectory.read()
(metaPropsEnsemble, null)
Copy link
Contributor

@kevin-wu24 kevin-wu24 Nov 3, 2025

Choose a reason for hiding this comment

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

To make this change more compatible for the existing test framework, we should instead pass down a BootstrapCheckpointFactory/Builder or something like that. Then have two separate implementations:

One for tests that specifies a BootstrapMetadata object all in-memory based on the factory.
In the actual implementation, we can point that factory to the actual files on disk we would be reading.

Either way, in QuorumController#handleLoadSnapshot, that is when we actually "resolve" this bootstrap metadata stuff by calling a method on the factory/builder object.\

EDIT: after looking at QuorumTestHarness and KafkaClusterTestKit, we shouldn't need to do this.

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.

Left some more comments regarding the metadata layer implementation @mannoopj

Comment on lines 143 to 181
// Write bootstrap records to the log so brokers can read them, but only if not handling a partial transaction
// Brokers can't read snapshots, only log entries
boolean shouldWriteBootstrapRecords = (transactionStartOffset == -1L);
if (shouldWriteBootstrapRecords) {
logMessageBuilder
.append("Writing bootstrap records to log for broker consumption. ")
.append("Appending ")
.append(bootstrapMetadata.records().size())
.append(" bootstrap record(s) ");

if (curMetadataVersion.isMetadataTransactionSupported()) {
records.add(new ApiMessageAndVersion(
new BeginTransactionRecord().setName("Bootstrap records"), (short) 0));
logMessageBuilder.append("in metadata transaction ");
}
logMessageBuilder
.append("at metadata.version ")
.append(curMetadataVersion)
.append(" from bootstrap source '")
.append(bootstrapMetadata.source())
.append("'. ");

// Add bootstrap records
records.addAll(bootstrapMetadata.records());

// If ELR is enabled, we need to set a cluster-level min.insync.replicas.
if (bootstrapMetadata.featureLevel(EligibleLeaderReplicasVersion.FEATURE_NAME) > 0) {
records.add(new ApiMessageAndVersion(new ConfigRecord().
setResourceType(BROKER.id()).
setResourceName("").
setName(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG).
setValue(Integer.toString(defaultMinInSyncReplicas)), (short) 0));
}

if (curMetadataVersion.isMetadataTransactionSupported()) {
records.add(new ApiMessageAndVersion(new EndTransactionRecord(), (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.

Why are we changing this code?

We shouldn't change this code though, because when the log is non-empty, it means the bootstrap metadata records have already been written in the log before.

Comment on lines 1027 to 1031
if (batch.controlRecords().isEmpty()) {
System.out.println("DEBUG: Extracting bootstrap metadata from " + messages.size() + " records");
bootstrapMetadata = BootstrapMetadata.fromRecords(messages, "bootstrap");
System.out.println("DEBUG: Bootstrap metadata extracted: " + bootstrapMetadata);
}
Copy link
Contributor

@kevin-wu24 kevin-wu24 Nov 3, 2025

Choose a reason for hiding this comment

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

This is the wrong if condition. We should check !batch.records.isEmpty.

If the 0-0.checkpoint does not have metadata records, AND bootstrapMetadata == null at this point, we should throw an IllegalStateException, because we cannot construct bootstrapMetadata.

Comment on lines 1032 to 1037
} else {
Map<String, Short> featureVersions = new HashMap<>();
MetadataVersion metadataVersion = MetadataVersion.latestProduction();
featureVersions.put(MetadataVersion.FEATURE_NAME, metadataVersion.featureLevel());
featureVersions.put(KRaftVersion.FEATURE_NAME, raftClient.kraftVersion().featureLevel());
bootstrapMetadata = BootstrapMetadata.fromVersions(metadataVersion, featureVersions, "generated default");
Copy link
Contributor

@kevin-wu24 kevin-wu24 Nov 3, 2025

Choose a reason for hiding this comment

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

If we're not reading the 0-0.checkpoint, bootstrapMetadata is either:

  1. read from bootstrap.checkpoint and passed down here, so it is non-null.
  2. null, because it should have already been written to the log as part of the 0-0.checkpoint.

Comment on lines 1156 to 1159
if (bootstrapMetadata == null) {
throw new IllegalStateException("Bootstrap metadata not available during activation. " +
"This should not happen if a bootstrap snapshot was processed.");
}
Copy link
Contributor

@kevin-wu24 kevin-wu24 Nov 3, 2025

Choose a reason for hiding this comment

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

We should allow bootstrapMetadata to be null here because we can be in the case where we bootstrapped using 0-0.checkpoint, but that file no longer exists because it was cleaned up by KRaft. However, bootstrapMetadata cannot be null when we call recordsForEmptyLog. It can be null when we call recordsForNonEmptyLog

List<ApiMessageAndVersion> messages = batch.records();

if (bootstrapMetadata == null) {
if (reader.snapshotId().equals(Snapshots.BOOTSTRAP_SNAPSHOT_ID)) {
Copy link
Contributor

@kevin-wu24 kevin-wu24 Nov 3, 2025

Choose a reason for hiding this comment

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

When we are reading in the 0-0.checkpoint, the ONLY thing we should be doing if !messages.isEmpty() in this method is using messages to construct a bootstrapMetadata object. It should not append an event even I think...

0-0.checkpoint is special because its records are uncommitted, unlike all other checkpoints this method handles, and need to be written to the log when a leader is determined.

This changed because previously, 0-0.checkpoint did not contain any metadata records, just KRaft control records potentially.

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.

One thing to consider is that before KIP-1170, 0-0.checkpoint did not contain any metadata records (i.e. batch.records() is empty). Because there are no metadata records to apply to state, the invariant that RaftClient.Listener implementations only update state based on committed metadata is upheld in the handleLoadSnapshot(0-0.checkpoint) case.

However, now that the 0-0.checkpoint can contain metadata records, the RaftClient.Listener implementations need to make sure they do not update any state based on these uncommitted bootstrap records, so that the above invariant is maintained.

Comment on lines 439 to 443
writeBoostrapSnapshot(writeLogDir,
bootstrapMetadata,
initialControllers,
featureLevels.get(KRaftVersion.FEATURE_NAME),
controllerListenerName);
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 make sense to write the 0-0.checkpoint on a broker-only node. It looks like bootstrap.checkpoint does get written on brokers, but IMO that is incorrect.

You can add like a setRole method and have an enum to represent the possible roles for a node.

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 updates @mannoopj . Left a few comments on the metadata layer code:


if (reader.snapshotId().equals(Snapshots.BOOTSTRAP_SNAPSHOT_ID)) {
// For bootstrap snapshots, extract feature levels from all data records
if (batch.controlRecords().isEmpty()) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Please fix this if condition. We should be checking if (!messages.isEmpty()) on L1025. We should not be doing anything else besides setting bootstrapMetadata when loading a 0-0.checkpoint that contains metadata records. In the current code, you are still iterating through the messages after setting bootstrapMetadata. This is why you need forceBootstrapWrite, but that boolean should be removed.

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
if (batch.controlRecords().isEmpty()) {
// check if 0-0.checkpoint has data records
if (!messages.isEmpty()) {
bootstrapMetadata = BootstrapMetadata.fromRecords(messages, "bootstrap");
return;
}

Comment on lines 1145 to 1147
boolean forceBootstrapWrite = featureControl.metadataVersion().isPresent() &&
offsetControl.lastCommittedOffset() == Snapshots.BOOTSTRAP_SNAPSHOT_ID.offset() &&
offsetControl.lastCommittedEpoch() == Snapshots.BOOTSTRAP_SNAPSHOT_ID.epoch();
Copy link
Contributor

Choose a reason for hiding this comment

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

Please remove forceBootstrapWrite. Instead, we should not be iterating through the data records for the 0-0.checkpoint in handleLoadSnapshot.

int defaultMinInSyncReplicas
) {
if (curMetadataVersion.isEmpty()) {
if (forceBootstrapWrite || curMetadataVersion.isEmpty()) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Please remove this boolean. See my other comments.

* A read-only class that holds the controller bootstrap metadata. A file named "bootstrap.checkpoint" is used and the
* format is the same as a KRaft snapshot.
*/
public class BootstrapDirectory {
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 not need to materially change this file at all (at least for now), since this file only deals with the old bootstrap.checkpoint file that is deprecated going forward.

It looks like it handles the bootstrap.checkpoint file not existing by defaulting to a bootstrap record set of just the latest MV. This means bootstrapMetadata is never null.

Maybe we want to rename the file to LegacyBootstrapDirectory, mark this as deprecated, and log a message when we successfully read bootstrap.checkpoint.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

These changes were mainly made to address failing tests. (Only used in two test files). I agree that we can probably deprecate this.

Copy link
Contributor

Choose a reason for hiding this comment

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

Hmm, looked at StorageToolTest and FormatterTest again, and I am okay with your solution of having two separate methods. In that case, we should deprecate maybeReadLegacyBootstrapCheckpoint() and BINARY_BOOTSTRAP_FILENAME.

Comment on lines -387 to -388
} else if (bootstrapMetadata == null) {
throw new IllegalStateException("You must specify an initial metadata.version using the kafka-storage tool.");
Copy link
Contributor

Choose a reason for hiding this comment

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

We can keep this assertion because bootstrapMetadata is never null.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The idea here is that for when we try to read bootstrap.checkpoint from KafkaRaftServer, in the scenarios where this file doesn't exist we would send a null value. Currently BootstrapDirectory.read() defaults to a non null value. I was thinking we change this to null as a way of determining if the bootstrap.checkpoint exists for WARN messages. Unless we do want to keep the behavior of defaulting to non null records?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

but also we could just check if both files exist. actually yeah I'll just revert this

Copy link
Contributor

@kevin-wu24 kevin-wu24 Nov 7, 2025

Choose a reason for hiding this comment

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

Unless we do want to keep the behavior of defaulting to non null records?

I think we should keep this behavior unless there is a good reason to change it. This behavior address the case where the bootstrap.checkpoint is accidentally deleted before bringing up a cluster.

I was thinking we change this to null as a way of determining if the bootstrap.checkpoint exists for WARN messages

If we preserve the existing behavior, there's no way to print the WARN message at the QuorumController level, but that case should be impossible from a formatting perspective. For now, let's keep the existing behavior around the default value of bootstrapMetadata, and we can revisit this discussion later.

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.

Left some more comments @mannoopj.

Comment on lines +1023 to +1029
if (reader.snapshotId().equals(Snapshots.BOOTSTRAP_SNAPSHOT_ID)) {
// For bootstrap snapshots, extract feature levels from all data records
if (!messages.isEmpty()) {
bootstrapMetadata = BootstrapMetadata.fromRecords(messages, "bootstrap");
return;
}
}
Copy link
Contributor

@kevin-wu24 kevin-wu24 Nov 10, 2025

Choose a reason for hiding this comment

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

Let's add an in-line comment describing what we're doing here for KIP-1170, since it is not super obvious to the reader. I also think we should move this logic below to when we are about to iterate through messages.

Suggested change
if (reader.snapshotId().equals(Snapshots.BOOTSTRAP_SNAPSHOT_ID)) {
// For bootstrap snapshots, extract feature levels from all data records
if (!messages.isEmpty()) {
bootstrapMetadata = BootstrapMetadata.fromRecords(messages, "bootstrap");
return;
}
}
// KIP-1170: The 0-0.checkpoint can contain metadata records. If it does, they should be considered the bootstrap metadata for the cluster.
if (reader.snapshotId().equals(Snapshots.BOOTSTRAP_SNAPSHOT_ID) && !messages.isEmpty()) {
bootstrapMetadata = ...
} else {
int i = 1;
for(ApiMessageAndVersion message : messages) {
...
}
offsetControl.endLoadSnapshot(...

MetadataDelta delta = new MetadataDelta.Builder().
setImage(image).
build();
SnapshotManifest manifest = loadSnapshot(delta, reader);
Copy link
Contributor

Choose a reason for hiding this comment

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

I think the better way to "skip loading metadata records for 0-0.checkpoint" is to modify MetadataLoader#loadSnapshot(), since this method is also doing other things like updating metrics that we want to keep doing for the 0-0.checkpoint.

In loadSnapshot(), we can skip over iterating the batch.records() if the snapshot is the 0-0.checkpoint, similar to what we're doing in QuorumController#handleLoadSnapshot().

Comment on lines +127 to +137
case KRAFT_VERSION: {
KRaftVersionRecord message = new KRaftVersionRecord();
message.read(new ByteBufferAccessor(record.value()), (short) 0);
messages.add(new ApiMessageAndVersion(message, (short) 0));
break;
}
case KRAFT_VOTERS:
VotersRecord message = new VotersRecord();
message.read(new ByteBufferAccessor(record.value()), (short) 0);
messages.add(new ApiMessageAndVersion(message, (short) 0));
break;
Copy link
Contributor

Choose a reason for hiding this comment

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

Why did we add this? I think this should be removed. If it is causing a test failure, that indicates that we are reading KRaft control records in the metadata module, which is incorrect.

logMsg -> { },
0L,
BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_6_IV1, "test"),
false,
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 remove these booleans in this test file.

Comment on lines -42 to +41
setFeatureLevel((short) 7), (short) 0),
new ApiMessageAndVersion(new NoOpRecord(), (short) 0),
new ApiMessageAndVersion(new NoOpRecord(), (short) 0));
setFeatureLevel((short) 7), (short) 0));
Copy link
Contributor

@kevin-wu24 kevin-wu24 Nov 10, 2025

Choose a reason for hiding this comment

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

We should revert the changes to this file

FormatterContext context = testEnv.newFormatter();
context.formatter.setWriteBootstrapSnapshot(false);
context.formatter.run();
File clusterMetadataDir = new File(testEnv.directory(0), String.format("%s-%d",
Copy link
Contributor

@kevin-wu24 kevin-wu24 Nov 10, 2025

Choose a reason for hiding this comment

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

testEnv.directory(0) contains the cluster metadata directory in these tests. Look at testEnv.newFormatter(). When you create the file here, you are looking for tempDirectoryPath/__cluster-metadata-0, which is a directory that SHOULD exist on the broker after formatting. However, the filetempDirectoryPath/__cluster-metadata-0/0-0.checkpoint SHOULD NOT exist on the broker.

EDIT: Apologies, we don't write /__cluster_metadata-0/ at format time unless we are also writing 0-0.checkpoint. We can keep this test as is. However, for the tests we format successfully on a controller, we should add this check:

File clusterMetadataDir = new File(testEnv.directory(0), String.format("%s-%d",
     CLUSTER_METADATA_TOPIC_PARTITION.topic(),
     CLUSTER_METADATA_TOPIC_PARTITION.partition()));
assertTrue(clusterMetadataDir.exists());
File bootstrapCheckpoint = new File(clusterMetadataDir,
     "00000000000000000000-0000000000.checkpoint");
assertTrue(bootstrapCheckpoint.exists());

@@ -0,0 +1,56 @@
{
Copy link
Contributor

Choose a reason for hiding this comment

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

Please remove this and the other gradle files. I assume that cursor probably created them to run things locally? Look at the README.md for the ./gradlew commands to run for testing/verifying locally.

Comment on lines +79 to +100
public BootstrapMetadata read() throws Exception {
Path path = Paths.get(directoryPath);
if (!Files.isDirectory(path)) {
if (Files.exists(path)) {
throw new RuntimeException("Path " + directoryPath + " exists, but is not " +
"a directory.");
} else {
throw new RuntimeException("No such directory as " + directoryPath);
}
}
Path binaryBootstrapPath = Paths.get(directoryPath, String.format("%s-%d",
CLUSTER_METADATA_TOPIC_PARTITION.topic(),
CLUSTER_METADATA_TOPIC_PARTITION.partition()),
BINARY_BOOTSTRAP_CHECKPOINT_FILENAME);
if (!Files.exists(binaryBootstrapPath)) {
return readFromConfiguration();
} else {
return readFromBinaryFile(binaryBootstrapPath.toString());
}

}

Copy link
Contributor

Choose a reason for hiding this comment

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

It looks like a lot of tests call BotostrapDirectory#read(), but we want to avoid explicitly reading in 0-0.checkpoint twice in the actual implementation, since we'll load it from KRaft in QuorumController.

I think I'm okay with having 2 separate methods, since we should only call read() from tests to assert things about the 0-0.checkpoint (which would otherwise be unreachable until KRaft loads it into memory) going forward.

}

public BootstrapMetadata read() throws Exception {
public BootstrapMetadata readBootstrapCheckpoint() throws Exception {
Copy link
Contributor

@kevin-wu24 kevin-wu24 Nov 10, 2025

Choose a reason for hiding this comment

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

Let's rename this to maybeReadLegacyBootstrapCheckpoint(), and log a message if we successfully read bootstrap.checkpoint.

* A read-only class that holds the controller bootstrap metadata. A file named "bootstrap.checkpoint" is used and the
* format is the same as a KRaft snapshot.
*/
public class BootstrapDirectory {
Copy link
Contributor

Choose a reason for hiding this comment

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

Hmm, looked at StorageToolTest and FormatterTest again, and I am okay with your solution of having two separate methods. In that case, we should deprecate maybeReadLegacyBootstrapCheckpoint() and BINARY_BOOTSTRAP_FILENAME.

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

Labels

build Gradle build or GitHub Actions clients core Kafka Broker kraft streams

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants