DUE TO SPAM, SIGN-UP IS DISABLED. Goto Selfserve wiki signup and request an account.
Status
Current state: Accept
Discussion thread: here
Vote thread: here
JIRA: KAFKA-19858
PR: https://github.com/apache/kafka/pull/20811
Motivation
The __remote_log_metadata topic is a critical component of Kafka's Tiered Storage feature (KIP-405) as the default implemetation, storing essential metadata including:
- RemoteLogSegmentMetadata: Remote log segment lifecycle state transitions along with segment identifiers, offset ranges
(startOffset, endOffset), leader epoch mappings, timestamps, segment sizes,
and custom RSM-specific metadata (e.g., remote storage paths) - RemoteLogSegmentMetadataUpdate: State transition events for existing remote log segments
- RemotePartitionDeleteMetadata: Partition-level deletion state tracking
Currently, this topic lacks an explicit min.insync.replicas (min.isr) configuration and relies on the broker-level default, which is typically 1.
With the default configuration of replication.factor=3 and min.isr=1, the system is vulnerable to metadata loss in the following scenario:
This data loss is particularly severe because remote data can becomes unreachable even though it still exists in remote storage.
As we knonw, .the 3 factor + 2 min sync in the stardard best practice for kafka topic. Besides this. it is inconsistency with other critical internal metadata topics:
__transaction_state: Explicitly sets min.isr via the transaction.state.log.min.isr broker configuration (default: 2)
Given that __remote_log_metadata stores equally critical metadata, it should have equivalent durability guarantees.
Public Interfaces
This KIP introduces one new broker configuration property: remote.log.metadata.topic.min.isr
Type: short Default: 2 Valid Values: atLeast(1) Importance: LOW Documentation: The minimum number of replicas that must acknowledge a write to remote log metadata topic.
When the __remote_log_metadata topic is created, the topic-level configuration (min.insync.replicas) will be set to the value of the configure.
Proposed Changes
You can refer to https://github.com/apache/kafka/pull/20811 for the detailed changes.
Configuration Loading:
TopicBasedRemoteLogMetadataManagerConfigpublic static final String REMOTE_LOG_METADATA_TOPIC_MIN_ISR_PROP = "remote.log.metadata.topic.min.isr"; public static final String REMOTE_LOG_METADATA_TOPIC_MIN_ISR_DOC = "The minimum number of replicas that must acknowledge a write to remote log metadata topic"; public static final short DEFAULT_REMOTE_LOG_METADATA_TOPIC_MIN_ISR = 2;
The new configuration will be added to TopicBasedRemoteLogMetadataManagerConfig, the configuration will be validated to ensure it does not exceed the replication factor.
Topic Creation:
TopicBasedRemoteLogMetadataManagertopicConfigs.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_DELETE); topicConfigs.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "false"); topicConfigs.put(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, Short.toString(rlmmConfig.metadataTopicMinIsr())); //add the configure return new NewTopic(rlmmConfig.remoteLogMetadataTopicName(), rlmmConfig.metadataTopicPartitionsCount(), rlmmConfig.metadataTopicReplicationFactor()).configs(topicConfigs);
When TopicBasedRemoteLogMetadataManager creates the __remote_log_metadata topic, it will include the min.isr configuration:
Compatibility, Deprecation, and Migration Plan
Backward Compatibility
This change is fully backward compatible:
1. Existing Deployments: Clusters with an existing __remote_log_metadata topic will continue to operate unchanged. The topic's current min.insync.replicas setting (typically inherited from the broker default) will remain in effect.
2. New Deployments: Only clusters enabling Tiered Storage for the first time, after upgrading to the release containing this change will automatically receive min.isr=2 for the topic.
3. Protocol Compatibility: No changes
4. Rolling Upgrade: The cluster can be upgraded using standard rolling upgrade procedures without special considerations due to the topic only be created once.- Deprecation
N/A - Migration for Existing Deployments
Users with existing __remote_log_metadata topics can evaluate their current configuration and consider updating it. This operation is not mandatory.
Step 1: Check Current Configuration
kafka-configs.sh --bootstrap-server localhost:9092 --describe --topic __remote_log_metadata
Step 2: Update Configuration if need. The change takes effect immediately and requires no restart.
If the current min.isr is 1 and replication.factor is 3 or higher:
kafka-configs.sh --bootstrap-server localhost:9092 --alter --topic __remote_log_metadata --add-config min.insync.replicas=2
Test Plan
We can use follow tests to cover the change:
Unit Tests:
* Verify that NewTopic includes min.insync.replicas configuration
Integration Tests:
* Create __remote_log_metadata topic with default configuration
* Verify topic can be created with custom replication.factor and min.isr
Rejected Alternatives
This another approach is maintaining the current default of min.isr=1, It was rejected for the following reasons:
1. Security by Default Principle
Kafka should provide secure defaults for critical internal topics. The current default of min.isr=1 creates an unacceptable data loss risk that most users will not proactively address.
2. Inconsistency with other critical metadata topics
The __transaction_state topic explicitly sets min.isr=2 via transaction.state.log.min.isr, There is no justifiable reason for treating remote log metadata differently.
