showuon commented on code in PR #14127: URL: https://github.com/apache/kafka/pull/14127#discussion_r1299460688
########## storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataCache.java: ########## @@ -104,6 +106,22 @@ public class RemoteLogMetadataCache { // https://issues.apache.org/jira/browse/KAFKA-12641 protected final ConcurrentMap<Integer, RemoteLogLeaderEpochState> leaderEpochEntries = new ConcurrentHashMap<>(); + private final CountDownLatch initializedLatch = new CountDownLatch(1); + + public void markInitialized() { + initializedLatch.countDown(); + } + + public void ensureInitialized() throws InterruptedException { + if (!initializedLatch.await(2, TimeUnit.MINUTES)) { Review Comment: Thanks for the explanation, @kamalcph ! One correction: `UNKNOWN_SERVER_ERROR` is **unretriable**. So if we throw `RemoteResourceNotFoundException`, the client won't be retried. I think we don't have to consider if client will auto retry or not when facing retriable error. IMO, sending a request and returned a request timeout error without any explanation (suppose the 2 min timeout is not expired here, so no error thrown in the broker side), which is not good. I think reduce to a small number (I think 10 secs is still too long, maybe 1 sec, or return error immediately if it's not initialized), and wrap the `RemoteResourceNotFoundException` with a retriable error, ex: `ReplicaNotAvailableException`, and add clear messages there. I think as long as we returned clear error reason with a retriable exception, this is already a good API. We don't want to block the request for a long time because we're waiting for some state change. Like `COORDINATOR_LOAD_IN_PROGRESS` error, it's clear that we returned this error and want the client retry later, instead of waiting in the broker for coordinator load completion. Does that make sense? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org