Skip to content

Conversation

jiafu1115
Copy link
Contributor

@jiafu1115 jiafu1115 commented Jul 20, 2025

refer to #20008. I submit another proposal to solve the issue which can also help to solve the following issue:
the DEFAULT_REMOTE_LOG_METADATA_INITIALIZATION_RETRY_MAX_TIMEOUT_MS is hard to be defined due to that it relay on the startup process.

If it the value is not well defined. the initial will fail and cause local disk never get deleted. (propose #20007 to prevent it.)

Different env's kafka process need different value. It is caused by followed reason:
When restarting broker. The connection to query/create topic in "TopicBasedRemoteLogMetadataManager#initializeResources"will fail until the broker's self get ready. The detail reason can refer to :#20007 (comment)

So propose this change to improve it. After the improve it and default value is easy to be defined. I don't need to set a very big value to all our kafkas.

cc @kamalcph @FrankYang0529 @showuon

stroller.fu added 4 commits June 4, 2025 21:43
…he __remote_log_metadata topic if it already exists.
� Conflicts:
�	storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManager.java
@github-actions github-actions bot added triage PRs from the community core Kafka Broker storage Pull requests that target the storage module tiered-storage Related to the Tiered Storage feature small Small PRs labels Jul 20, 2025
Copy link
Contributor

@kamalcph kamalcph left a comment

Choose a reason for hiding this comment

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

Thanks @jiafu1115 for the patch!

The current approach require a KIP to proceed.

Another approach is not to call RLMM#configure() method while instantiating the RemoteLogManager#L422 and define a new method in RemoteLogManager#configureRLMM and this can be called from the BrokerServer.

public void configureRLMM(CompletableFuture<Void> brokerReadyFuture) {
    brokerReadyFuture.whenComplete((r, t) -> {
        if (t != null) {
            remoteLogMetadataManagerPlugin.get().configure(rlmmProps);
        }
    });
}

@kamalcph
Copy link
Contributor

kamalcph commented Jul 20, 2025

When we wait for the SocketServer to be ready before initializing the TBRLMM, then the user FETCH requests (that want to read from remote) might be served before the initialization, and ReplicaNotAvailableException gets thrown back to the caller which is a client retriable error so it is fine.

If you're interested to know why ReplicaNotAvaibleException gets thrown, then refer to KIP-1007.

@kamalcph
Copy link
Contributor

Opened #20204 to refactor the TBRLMM implementation.

@github-actions github-actions bot removed the triage PRs from the community label Jul 21, 2025
@jiafu1115
Copy link
Contributor Author

jiafu1115 commented Jul 21, 2025

When we wait for the SocketServer to be ready before initializing the TBRLMM, then the user FETCH requests (that want to read from remote) might be served before the initialization, and ReplicaNotAvailableException gets thrown back to the caller which is a client retriable error so it is fine.

If you're interested to know why ReplicaNotAvaibleException gets thrown, then refer to KIP-1007.

@kamalcph thanks for so detail information. That is the key information which I worried about. Thanks a lot. I will write one KIP. thanks!

stroller.fu added 4 commits July 24, 2025 06:15
@jiafu1115
Copy link
Contributor Author

jiafu1115 commented Jul 23, 2025

@kamalcph Hi , I compare my original propose and your approach . I think your approach is much better on follow items:
(1) no any public interface change (No need KIP)
(2) less code (I also remove the brokerReadyFuture and call it directly in BrokerServer after server is ready for accepting request).
(3) no change for TopicBasedRemoteLogMetadataManager

The possible shortcoming‌ found when comparing the my propose.
(1) new propose postpone the configure/initial after Server Socket is ready for all the possible RemoteLogMetadataManager's implements. is it suitable for all the case (not only TopicBasedRemoteLogMetadataManager)?

I update the code for your compare (you can compare it with https://github.com/apache/kafka/pull/20231/files which I create for your compare).

Can you help to review it again for compare the two proposes.

Thanks a lot.

@jiafu1115 jiafu1115 requested a review from kamalcph July 23, 2025 22:46
@kamalcph
Copy link
Contributor

kamalcph commented Jul 24, 2025

With this patch, RLMM#configure is called post the SocketServer initialization, this is required only for TBRLMM implementation as it has to create/read/write to the __remote_log_metadata topic.

For other RLMM plugin impl, the RLMM#configure can be called before the SocketServer initialization if they don't keep the metadata in Kafka. Since, TBRLMM is packaged as default / out-of-the-box implementation, this approach looks good to me.

Compared with and without this patch.

Without patch, the Admin tries continuously until the server is ready:

% grep -c "Rebootstrapping with Cluster" nohup.out
6555

With this patch, the Admin succeeds on the first try:

% grep -c "Rebootstrapping with Cluster" nohup.out.1
0

The other approach #20231 is generic and allows any RLMM implementor to invoke broker based on the status. But, it requires KIP.

@showuon / @FrankYang0529 Could you take a second pass? Thanks!

Sample logs:

[2025-07-24 12:59:05,004] INFO [AdminClient clientId=adminclient-1] Node -1 disconnected. (org.apache.kafka.clients.NetworkClient)
[2025-07-24 12:59:05,004] WARN [AdminClient clientId=adminclient-1] Connection to node -1 (localhost/127.0.0.1:9092) could not be established. Node may not be available. (org.apache.kafka.clients.NetworkClient)
[2025-07-24 12:59:05,004] INFO [AdminClient clientId=adminclient-1] Rebootstrapping with Cluster(id = null, nodes = [localhost:9092 (id: -1 rack: null isFenced: false)], partitions = [], controller = null) (org.apache.kafka.clients.admin.internals.AdminMetadataManager)

@jiafu1115
Copy link
Contributor Author

@kamalcph ok. thanks for your comments and test. the test result look well as expected . With this patch, we don't need to worry the max retry time for different Kafka clusters which take different time to complete startup.
Thanks!

Copy link
Collaborator

@Yunyung Yunyung 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 PR. Accroding to KIP-877
"If a plugin implements this interface, the withPluginMetrics() method will be called when the plugin is instantiated (after configure() if the plugin also implements Configurable). "

So I think the current change breaks that contract.

@jiafu1115
Copy link
Contributor Author

jiafu1115 commented Jul 24, 2025

Thanks for the PR. Accroding to KIP-877 "If a plugin implements this interface, the withPluginMetrics() method will be called when the plugin is instantiated (after configure() if the plugin also implements Configurable). "

So I think the current change breaks that contract.

@Yunyung thanks for your review. I will also take some time to understand the contract. cc @kamalcph

BTW. Can you also help to take a look at the original propose:c8c2280

WDTY for this one?

@jiafu1115
Copy link
Contributor Author

jiafu1115 commented Jul 24, 2025

@kamalcph I read the current code implement and the KIP provided by @Yunyung .
I think even current code is ok but it break the contract as he mentioned:
If one user write one RemoteLogMetadataManager's implement which also implement Monitorable. And he will no longer be able to use the configure items to setup metric on Monitorable#withPluginMetrics due to we postpone the configure process after wrap it as plugin.
So maybe the original propose is better now though it will involve writing one KIP and more codes.

WDTY? Or do we have some better idea? Thank you !

@jiafu1115 jiafu1115 requested a review from Yunyung July 24, 2025 14:44
@kamalcph
Copy link
Contributor

I didn't thought about the KIP-877. Then, the other proposal looks clean: c8c2280

@jiafu1115
Copy link
Contributor Author

jiafu1115 commented Jul 25, 2025

@kamalcph thanks.
I had already rollbacked the code to original propose with some improvements to keep the code more simple/clean.
You can help to review the latest code! also cc @Yunyung for review.
BTW. for the method "default void onBrokerReadyForRequests()" which is not must need for all implement. I still need one KIP. right?

@jiafu1115
Copy link
Contributor Author

@kamalcph create KIP-1197 cc @Yunyung

@jiafu1115
Copy link
Contributor Author

jiafu1115 commented Jul 31, 2025

Hi, @kamalcph @Yunyung @showuon @FrankYang0529
Sorry to trouble you. I'd like to initiate a vote on https://lists.apache.org/thread/twxbflptq6xsxt97t1qx8pkvsjpbtlvo. It seems that not many people are familiar with the related feature, so I mentioned you in the thread to help provide suggestions or cast your vote.
Thanks in advance!

@Yunyung
Copy link
Collaborator

Yunyung commented Aug 8, 2025

Thanks for the update. This looks reasonable. Let's see if anyone else has any feedback

@jiafu1115
Copy link
Contributor Author

jiafu1115 commented Aug 11, 2025

Thanks for the update. This looks reasonable. Let's see if anyone else has any feedback

@Yunyung thanks for your feedback!

@kamalcph @Yunyung @showuon @FrankYang0529
Sorry to trouble you. The KIP requires 3 binding +1 votes. If the KIP looks good to you. Can you help to reply the mail (https://lists.apache.org/thread/twxbflptq6xsxt97t1qx8pkvsjpbtlvo) with +1 so that we can continue for this PR. Thanks a lot!

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
core Kafka Broker small Small PRs storage Pull requests that target the storage module tiered-storage Related to the Tiered Storage feature
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants