kamalcph commented on code in PR #15820:
URL: https://github.com/apache/kafka/pull/15820#discussion_r1617105411
##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -738,6 +750,23 @@ public void copyLogSegmentsToRemote(UnifiedLog log) throws
InterruptedException
isCancelled(), isLeader());
return;
}
+
+ copyQuotaManagerLock.lock();
+ try {
+ while (rlmCopyQuotaManager.isQuotaExceeded()) {
Review Comment:
If the copy quota gets breached, all the `kafka-rlm-thread-pool` threads
will wait for the quota to be available which might delay the deletion of
remote log segments since the same thread does both copy and delete.
Do you plan to split the copy/delete operations in a separate thread pool?
##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -237,6 +242,13 @@ private void removeMetrics() {
remoteStorageReaderThreadPool.removeMetrics();
}
+ /**
+ * Returns the timeout for the RLM Tasks to wait for the quota to be
available
+ */
+ Duration quotaTimeout() {
Review Comment:
shall we rename `quotaTimeout` to `throttleTimeMs`?
##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -153,6 +156,8 @@ public class RemoteLogManager implements Closeable {
private final RemoteLogMetadataManager remoteLogMetadataManager;
+ private final ReentrantLock copyQuotaManagerLock = new ReentrantLock(true);
Review Comment:
Why `fairness` is turned on to true?
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]