...
Code Block |
---|
MetadataResponse#TopicMetadata. Add new field tiered_epoch. TopicMetadata => topic_error_code topic is_internal partition_metadata topic_error_code => int16 topic => str is_internal => boolean partition_metadata => [PartitionMetadata] + tiered_epoch => int32 + tiered_state => [TieredState] |
Public Interfaces
Client API Changes
The AlterConfigs
API will be updated to accommodate the following changes:
- Updating the topic-level configuration
remote.storage.enable
with a new configuration value offalse
will disable tiered storage. - The
AlterConfigs
API will also be updated to support an optional tiered storage disablement policy,remote.log.disable.policy=retain|delete
. If the disablement policy is not specified, it will default toRetain
the remote logs. - Re-enablement after disablement will not introduce any changes to the public interface.
- Re-enablement will not provide an option to specify a policy.
Exceptions
There are certain exceptional scenarios to consider:
- If a new disablement request is encountered while the topic is still in the
DISABLING
state, an exception namedTIERED_STORAGET_DISABLEMENT_IN_PROGRESS
will be thrown. This exception will have the following details: - Error: TIERED_STORAGET_DISABLEMENT_IN_PROGRESS
- Code: > 71 (To be determined during implementation time)
- For other invalid formatted requests, such as an unsupported disablement policy option, the existing error
INVALID_REQUEST (42)
will be returned with an appropriate error message.
Configuration
A new configuration to support the disablement policy will be introduced.
Configuration: remote.log.disable.policy
Description: Determines whether tiered data for a topic should be retained or deleted after tiered storage disablement on a topic.
Type: String
Default: retain
Valid values: retain, delete
Scope: topic wide
The configuration setting remote.log.disable.policy
can also be introduced as a broker-level configuration, with the default value set to retain
. For the initial implementation, we are suggesting this to be a topic level configuration, but we have the flexibility to make it to a broker-level configuration in future iterations.
Code Block |
---|
# Defaults to remote.log.disable.policy=retain
bin/kafka-configs.sh --bootstrap-server {bootstrap-string} \
--alter --entity-type topics --entity-name {topic-name} \
--add-config 'remote.storage.enable=false'
#Disable with remote.log.disable.policy=retain
bin/kafka-configs.sh --bootstrap-server {bootstrap-string} \
--alter --entity-type topics --entity-name {topic-name} \
--add-config 'remote.log.disable.policy=retain, remote.storage.enable=false'
#Disable with remote.log.disable.policy=delete
bin/kafka-configs.sh --bootstrap-server {bootstrap-string} \
--alter --entity-type topics --entity-name {topic-name} \
--add-config 'remote.log.disable.policy=delete, remote.storage.enable=false' |