THIS IS A TEST INSTANCE. ALL YOUR CHANGES WILL BE LOST!!!!
...
- Users modify the topic configuration:
- Users set the configuration "remote.storage.enable=false" for the desired topic, indicating the disablement of tiered storage.
- Optionally, users specify a disablement policy to either retain or delete the remote data associated with the topic:
remote.log.disable.policy=retain|delete
. - Controller persists configuration change:
- The controller creates a ConfigRecord and persists it in the metadata topic.
- The controller also updates the Topic metadata to increment the
tiered_epoch
and update thetiered_state
toDISABLING
state. - Broker detects the configuration change:
- Broker replicate the metadata topic and fetch the latest records of ConfigRecord changes.
- The broker calls the ConfigHandler to process the ConfigRecord change with tiered storage disablement.
- The broker ConfigHandler calls
RemoteLogManager#stopPartitions()
. - Execution of
RemoteLogManager#stopPartitions()
: - Removes the scheduled tasks for the topic-partition from the thread pool responsible for archiving logs to remote storage.
- If the disablement policy is set to d
elete
, the Log start offset (LSO) is updated to match the Local Log Start Offset and the remote log is deleted by calling theRemoteStorageManager#deleteLogSegmentData()
. - If the disablement policy is set to
retain
, there is no extra call required to delete remote data as the RemoteLogManager leader partition task will periodically expire segments that are earlier than the log start offset. - Controller interaction:
- The controller enqueues the disablement intent for tiered storage to a local queue.
- It sends an API call, called
DisableRemoteTopic
, to each broker with a callback to disable the remote topic. DisableRemoteTopic
is a new controller-to-broker API that will be introduced in this KIP.- Note that the current approach for communication between the Controller and brokers involves a familiar pattern. Tasks are enqueued in a local queue within the Controller Context, and a request is sent to the brokers along with a callback mechanism. This method is currently employed by the controller for operations, such as deleting topics. For instance, when the controller initiates the deletion of topics, it maintains a local queue called
topicsToBeDeleted
. This queue serves the purpose of monitoring the progress of topic deletions. - Completion of disablement:
- Once the controller receives responses from all brokers for the
DisableRemoteTopic
request, it updates thetiered_state
in the TopicMetadata to "DISABLED". - This update marks the completion of the disablement process, indicating that tiered storage has been successfully disabled for the Kraft-based clusters.
Failure Modes
- Controller Failover in DISABLING State: Following a new controller election, the controller context will be reconstructed for all topic-partitions that are in the DISABLING state. Additionally, the controller will initiate a new request for DisableRemoteTopic to all brokers.
- Broker Dies in DISABLING State or Fails to Complete the DisableRemoteTopic Request: The controller maintains an internal queue to track completed DisableRemoteTopic calls. In the event of a leader failover during this state, the controller will retry the operation until it receives a successful response. This behavior mirrors how we currently handle topic deletion.
Internal Interface Changes
1. DisableRemoteTopic
is a new controller-to-broker API with the following schema. Note that this will bump the inter.broker.protocol.version
.
...