[
https://issues.apache.org/jira/browse/KAFKA-17980?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Kamal Chandraprakash updated KAFKA-17980:
-----------------------------------------
Description:
When a broker comes up online, the RemoteLogMetadataManager (RLMM) gets
initialized in the background using the async fashion. The server starts to
copy/delete the expired remote log segments, then the RLMM throws exception
until it gets initialized. By adding the {{isReady}} method, the server can
delay the copying (or) deleting the remote log segments until the RLMM gets
initialized. This is a graceful handling to avoid exception/error logs while
starting the server.
(eg) Exception during init
{noformat}
[2024-11-06 09:33:16,867] WARN [RemoteLogManager=0
partition=xp3zuxhTQ9uUwk-uQhTx5g:topicA-0] Current task for topic-partition
xp3zuxhTQ9uUwk-uQhTx5g:topicA-0 received error but it will be scheduled
(kafka.log.remote.RemoteLogManager$RLMTask:793)
java.lang.IllegalStateException: This instance is in invalid state,
initialized: false close: false
at
org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManager.ensureInitializedAndNotClosed(TopicBasedRemoteLogMetadataManager.java:557)
at
org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManager.highestOffsetForEpoch(TopicBasedRemoteLogMetadataManager.java:236)
at
kafka.log.remote.RemoteLogManager.findHighestRemoteOffset(RemoteLogManager.java:1827)
at
kafka.log.remote.RemoteLogManager$RLMFollowerTask.execute(RemoteLogManager.java:1434)
at kafka.log.remote.RemoteLogManager$RLMTask.run(RemoteLogManager.java:784)
{noformat}
Proposed to add a new `isReady` API in RLMM:
{code:java}
/**
* Denotes whether the partition metadata is ready to serve.
*
* @param topicIdPartition topic partition
* @return True if the partition is ready to serve for remote storage
operations.
*/
default boolean isReady(TopicIdPartition topicIdPartition) {
return true;
}
{code}
This task is related to
[KIP-1105|https://cwiki.apache.org/confluence/display/KAFKA/KIP-1105%3A+Make+remote+log+manager+thread-pool+configs+dynamic]
was:This task is related to
[KIP-1105|https://cwiki.apache.org/confluence/display/KAFKA/KIP-1105%3A+Make+remote+log+manager+thread-pool+configs+dynamic]
> Add isReady API to RemoteLogMetadataManager
> -------------------------------------------
>
> Key: KAFKA-17980
> URL: https://issues.apache.org/jira/browse/KAFKA-17980
> Project: Kafka
> Issue Type: Task
> Reporter: Kamal Chandraprakash
> Assignee: Kamal Chandraprakash
> Priority: Major
>
> When a broker comes up online, the RemoteLogMetadataManager (RLMM) gets
> initialized in the background using the async fashion. The server starts to
> copy/delete the expired remote log segments, then the RLMM throws exception
> until it gets initialized. By adding the {{isReady}} method, the server can
> delay the copying (or) deleting the remote log segments until the RLMM gets
> initialized. This is a graceful handling to avoid exception/error logs while
> starting the server.
> (eg) Exception during init
> {noformat}
> [2024-11-06 09:33:16,867] WARN [RemoteLogManager=0
> partition=xp3zuxhTQ9uUwk-uQhTx5g:topicA-0] Current task for topic-partition
> xp3zuxhTQ9uUwk-uQhTx5g:topicA-0 received error but it will be scheduled
> (kafka.log.remote.RemoteLogManager$RLMTask:793)
> java.lang.IllegalStateException: This instance is in invalid state,
> initialized: false close: false
> at
> org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManager.ensureInitializedAndNotClosed(TopicBasedRemoteLogMetadataManager.java:557)
> at
> org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManager.highestOffsetForEpoch(TopicBasedRemoteLogMetadataManager.java:236)
> at
> kafka.log.remote.RemoteLogManager.findHighestRemoteOffset(RemoteLogManager.java:1827)
> at
> kafka.log.remote.RemoteLogManager$RLMFollowerTask.execute(RemoteLogManager.java:1434)
> at
> kafka.log.remote.RemoteLogManager$RLMTask.run(RemoteLogManager.java:784)
> {noformat}
> Proposed to add a new `isReady` API in RLMM:
> {code:java}
> /**
> * Denotes whether the partition metadata is ready to serve.
> *
> * @param topicIdPartition topic partition
> * @return True if the partition is ready to serve for remote storage
> operations.
> */
> default boolean isReady(TopicIdPartition topicIdPartition) {
> return true;
> }
> {code}
> This task is related to
> [KIP-1105|https://cwiki.apache.org/confluence/display/KAFKA/KIP-1105%3A+Make+remote+log+manager+thread-pool+configs+dynamic]
--
This message was sent by Atlassian Jira
(v8.20.10#820010)