From cb6f22f1d0e31a0b4e44cf22319ac9b74bb66f13 Mon Sep 17 00:00:00 2001 From: Jian Date: Tue, 18 Nov 2025 11:18:31 +0800 Subject: [PATCH 001/110] Reduce remote storage by checking local retention Signed-off-by: stroller --- .../kafka/common/config/TopicConfig.java | 5 ++ .../log/remote/storage/RemoteLogManager.java | 75 +++++++++++++++++++ .../storage/internals/log/LogConfig.java | 9 +++ 3 files changed, 89 insertions(+) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index e97c39bc61911..4543373d27f82 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -101,6 +101,11 @@ public class TopicConfig { "(i.e. local.retention.ms/bytes) becomes irrelevant, and all data expiration follows the topic-wide retention configuration" + "(i.e. retention.ms/bytes)."; + public static final String REMOTE_LOG_KEEP_LATEST_CONFIG = "remote.log.keep.latest"; + public static final String REMOTE_LOG_KEEP_LATEST_DOC = "Determines whether to upload all segments to remote storage including the latest ones within local retention. " + + "When set to true (default), all committed segments will be uploaded without checking local retention constraints. " + + "When set to false, only segments beyond local retention period will be uploaded to remote storage."; + public static final String REMOTE_LOG_DELETE_ON_DISABLE_CONFIG = "remote.log.delete.on.disable"; public static final String REMOTE_LOG_DELETE_ON_DISABLE_DOC = "Determines whether tiered data for a topic should be " + "deleted after tiered storage is disabled on a topic. This configuration should be enabled when trying to " + diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index a9848633effa3..472cfed118d52 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -902,11 +902,65 @@ private void maybeUpdateCopiedOffset(UnifiedLog log) throws RemoteStorageExcepti } } + /** + * Check if segment is still within local retention time. + * @param segment The log segment to check + * @param localRetentionMs Local retention time in milliseconds; + * @param currentTimeMs Current time in milliseconds + * @return true if segment is still within local retention time, false otherwise + */ + private boolean isWithinLocalRetentionTime(LogSegment segment, long localRetentionMs, long currentTimeMs) { + if (localRetentionMs <= 0) { + return false; + } + + try { + long segmentLargestTimestamp = segment.largestTimestamp(); + if (segmentLargestTimestamp >= 0) { + long segmentAge = currentTimeMs - segmentLargestTimestamp; + if (segmentAge < localRetentionMs) { + logger.info("Segment {} is still within local retention time. Segment age: {} ms, local retention: {} ms, skipping upload", + segment, segmentAge, localRetentionMs); + return true; + } + } + } catch (IOException e) { + logger.warn("Failed to get largest timestamp for segment {}, will not skip based on time", segment, e); + } + return false; + } + + /** + * Check if local log size is still within local retention size after uploading the segment to remote. + * @param localLogSize Current local log size + * @param segment The segment to check + * @param accumulatedUploadedSize Total size of segments already decided to upload to remote + * @param localRetentionBytes Local retention size in bytes + * @return true if local log size after uploading this segment to remote would still be within local retention size, false otherwise + */ + private boolean isWithinLocalRetentionSize(long localLogSize, LogSegment segment, long accumulatedUploadedSize, long localRetentionBytes) { + if (localRetentionBytes <= 0) { + return false; + } + + int segmentSize = segment.size(); + long sizeAfterUploading = localLogSize - accumulatedUploadedSize - segmentSize; + + if (sizeAfterUploading < localRetentionBytes) { + logger.info("Segment {}: size after planing to upload to remote {} bytes < retention {} bytes, skipping upload", + segment, sizeAfterUploading, localRetentionBytes); + return true; + } + return false; + } + /** * Segments which match the following criteria are eligible for copying to remote storage: * 1) Segment is not the active segment and * 2) Segment end-offset is less than the last-stable-offset as remote storage should contain only * committed/acked messages + * 3) When remote.log.keep.latest is true (default): All segments are eligible including the latest ones within local retention + * When remote.log.keep.latest is false: Segment is not within local retention time or size (close to expiration or already expired) * @param log The log from which the segments are to be copied * @param fromOffset The offset from which the segments are to be copied * @param lastStableOffset The last stable offset of the log @@ -916,11 +970,32 @@ List candidateLogSegments(UnifiedLog log, Long fromOffset, L List candidateLogSegments = new ArrayList<>(); List segments = log.logSegments(fromOffset, Long.MAX_VALUE); if (!segments.isEmpty()) { + long localLogSize = log.size(); + long accumulatedUploadedSize = 0; + long currentTimeMs = time.milliseconds(); + boolean remoteLogKeepLatest = log.config().remoteLogKeepLatest(); + for (int idx = 1; idx < segments.size(); idx++) { LogSegment previousSeg = segments.get(idx - 1); LogSegment currentSeg = segments.get(idx); if (currentSeg.baseOffset() <= lastStableOffset) { + if (!remoteLogKeepLatest) { + // Check time-based retention + long localRetentionMs = log.config().localRetentionMs(); + if (isWithinLocalRetentionTime(previousSeg, localRetentionMs, currentTimeMs)) { + break; + } + + // Check size-based retention + long localRetentionBytes = log.config().localRetentionBytes(); + if (isWithinLocalRetentionSize(localLogSize, previousSeg, accumulatedUploadedSize, localRetentionBytes)) { + break; + } + } + + // Segment is eligible for upload candidateLogSegments.add(new EnrichedLogSegment(previousSeg, currentSeg.baseOffset())); + accumulatedUploadedSize += previousSeg.size(); } } // Discard the last active segment diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index a687f3c529e32..9808f08cf02ef 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -67,12 +67,14 @@ private static class RemoteLogConfig { private final boolean remoteStorageEnable; private final boolean remoteLogDeleteOnDisable; private final boolean remoteLogCopyDisable; + private final boolean remoteLogKeepLatest; private final long localRetentionMs; private final long localRetentionBytes; private RemoteLogConfig(LogConfig config) { this.remoteStorageEnable = config.getBoolean(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG); this.remoteLogCopyDisable = config.getBoolean(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG); + this.remoteLogKeepLatest = config.getBoolean(TopicConfig.REMOTE_LOG_KEEP_LATEST_CONFIG); this.remoteLogDeleteOnDisable = config.getBoolean(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG); this.localRetentionMs = config.getLong(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG); this.localRetentionBytes = config.getLong(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG); @@ -83,6 +85,7 @@ public String toString() { return "RemoteLogConfig{" + "remoteStorageEnable=" + remoteStorageEnable + ", remoteLogCopyDisable=" + remoteLogCopyDisable + + ", remoteLogKeepLatest=" + remoteLogKeepLatest + ", remoteLogDeleteOnDisable=" + remoteLogDeleteOnDisable + ", localRetentionMs=" + localRetentionMs + ", localRetentionBytes=" + localRetentionBytes + @@ -135,6 +138,7 @@ public Optional serverConfigName(String configName) { public static final boolean DEFAULT_REMOTE_STORAGE_ENABLE = false; public static final boolean DEFAULT_REMOTE_LOG_COPY_DISABLE_CONFIG = false; + public static final boolean DEFAULT_REMOTE_LOG_KEEP_LATEST_CONFIG = true; public static final boolean DEFAULT_REMOTE_LOG_DELETE_ON_DISABLE_CONFIG = false; public static final long DEFAULT_LOCAL_RETENTION_BYTES = -2; // It indicates the value to be derived from RetentionBytes public static final long DEFAULT_LOCAL_RETENTION_MS = -2; // It indicates the value to be derived from RetentionMs @@ -246,6 +250,7 @@ public Optional serverConfigName(String configName) { .define(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, LONG, DEFAULT_LOCAL_RETENTION_BYTES, atLeast(-2), MEDIUM, TopicConfig.LOCAL_LOG_RETENTION_BYTES_DOC) .define(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_COPY_DISABLE_DOC) + .define(TopicConfig.REMOTE_LOG_KEEP_LATEST_CONFIG, BOOLEAN, DEFAULT_REMOTE_LOG_KEEP_LATEST_CONFIG, MEDIUM, TopicConfig.REMOTE_LOG_KEEP_LATEST_DOC) .define(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_DOC) .defineInternal(INTERNAL_SEGMENT_BYTES_CONFIG, INT, null, null, MEDIUM, INTERNAL_SEGMENT_BYTES_DOC); } @@ -398,6 +403,10 @@ public Boolean remoteLogCopyDisable() { return remoteLogConfig.remoteLogCopyDisable; } + public Boolean remoteLogKeepLatest() { + return remoteLogConfig.remoteLogKeepLatest; + } + public long localRetentionMs() { return remoteLogConfig.localRetentionMs == LogConfig.DEFAULT_LOCAL_RETENTION_MS ? retentionMs : remoteLogConfig.localRetentionMs; } From 4f962750785c9c9a2f37b7183cd29504ae940f02 Mon Sep 17 00:00:00 2001 From: Jian Date: Wed, 19 Nov 2025 20:45:59 +0800 Subject: [PATCH 002/110] correct the configure name to keep same style with existed configures for remote storage. Signed-off-by: Jian --- .../apache/kafka/common/config/TopicConfig.java | 4 ++-- .../log/remote/storage/RemoteLogManager.java | 8 ++++---- .../kafka/storage/internals/log/LogConfig.java | 14 +++++++------- 3 files changed, 13 insertions(+), 13 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index 4543373d27f82..e0eec6c752a95 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -101,8 +101,8 @@ public class TopicConfig { "(i.e. local.retention.ms/bytes) becomes irrelevant, and all data expiration follows the topic-wide retention configuration" + "(i.e. retention.ms/bytes)."; - public static final String REMOTE_LOG_KEEP_LATEST_CONFIG = "remote.log.keep.latest"; - public static final String REMOTE_LOG_KEEP_LATEST_DOC = "Determines whether to upload all segments to remote storage including the latest ones within local retention. " + + public static final String REMOTE_LOG_LATEST_ENABLE_CONFIG = "remote.log.latest.enable"; + public static final String REMOTE_LOG_LATEST_ENABLE_DOC = "Determines whether to upload all segments to remote storage including the latest ones within local retention. " + "When set to true (default), all committed segments will be uploaded without checking local retention constraints. " + "When set to false, only segments beyond local retention period will be uploaded to remote storage."; diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index 472cfed118d52..adf23d43f214e 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -959,8 +959,8 @@ private boolean isWithinLocalRetentionSize(long localLogSize, LogSegment segment * 1) Segment is not the active segment and * 2) Segment end-offset is less than the last-stable-offset as remote storage should contain only * committed/acked messages - * 3) When remote.log.keep.latest is true (default): All segments are eligible including the latest ones within local retention - * When remote.log.keep.latest is false: Segment is not within local retention time or size (close to expiration or already expired) + * 3) When remote.log.latest.enable is true (default): All segments are eligible including the latest ones within local retention + * When remote.log.latest.enable is false: Segment is not within local retention time or size (close to expiration or already expired) * @param log The log from which the segments are to be copied * @param fromOffset The offset from which the segments are to be copied * @param lastStableOffset The last stable offset of the log @@ -973,13 +973,13 @@ List candidateLogSegments(UnifiedLog log, Long fromOffset, L long localLogSize = log.size(); long accumulatedUploadedSize = 0; long currentTimeMs = time.milliseconds(); - boolean remoteLogKeepLatest = log.config().remoteLogKeepLatest(); + boolean remoteLogLatestEnable = log.config().remoteLogLatestEnable(); for (int idx = 1; idx < segments.size(); idx++) { LogSegment previousSeg = segments.get(idx - 1); LogSegment currentSeg = segments.get(idx); if (currentSeg.baseOffset() <= lastStableOffset) { - if (!remoteLogKeepLatest) { + if (!remoteLogLatestEnable) { // Check time-based retention long localRetentionMs = log.config().localRetentionMs(); if (isWithinLocalRetentionTime(previousSeg, localRetentionMs, currentTimeMs)) { diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 9808f08cf02ef..b99663afbd42f 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -67,14 +67,14 @@ private static class RemoteLogConfig { private final boolean remoteStorageEnable; private final boolean remoteLogDeleteOnDisable; private final boolean remoteLogCopyDisable; - private final boolean remoteLogKeepLatest; + private final boolean remoteLogLatestEnable; private final long localRetentionMs; private final long localRetentionBytes; private RemoteLogConfig(LogConfig config) { this.remoteStorageEnable = config.getBoolean(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG); this.remoteLogCopyDisable = config.getBoolean(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG); - this.remoteLogKeepLatest = config.getBoolean(TopicConfig.REMOTE_LOG_KEEP_LATEST_CONFIG); + this.remoteLogLatestEnable = config.getBoolean(TopicConfig.REMOTE_LOG_LATEST_ENABLE_CONFIG); this.remoteLogDeleteOnDisable = config.getBoolean(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG); this.localRetentionMs = config.getLong(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG); this.localRetentionBytes = config.getLong(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG); @@ -85,7 +85,7 @@ public String toString() { return "RemoteLogConfig{" + "remoteStorageEnable=" + remoteStorageEnable + ", remoteLogCopyDisable=" + remoteLogCopyDisable + - ", remoteLogKeepLatest=" + remoteLogKeepLatest + + ", remoteLogLatestEnable=" + remoteLogLatestEnable + ", remoteLogDeleteOnDisable=" + remoteLogDeleteOnDisable + ", localRetentionMs=" + localRetentionMs + ", localRetentionBytes=" + localRetentionBytes + @@ -138,7 +138,7 @@ public Optional serverConfigName(String configName) { public static final boolean DEFAULT_REMOTE_STORAGE_ENABLE = false; public static final boolean DEFAULT_REMOTE_LOG_COPY_DISABLE_CONFIG = false; - public static final boolean DEFAULT_REMOTE_LOG_KEEP_LATEST_CONFIG = true; + public static final boolean DEFAULT_REMOTE_LOG_LATEST_ENABLE_CONFIG = true; public static final boolean DEFAULT_REMOTE_LOG_DELETE_ON_DISABLE_CONFIG = false; public static final long DEFAULT_LOCAL_RETENTION_BYTES = -2; // It indicates the value to be derived from RetentionBytes public static final long DEFAULT_LOCAL_RETENTION_MS = -2; // It indicates the value to be derived from RetentionMs @@ -250,7 +250,7 @@ public Optional serverConfigName(String configName) { .define(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, LONG, DEFAULT_LOCAL_RETENTION_BYTES, atLeast(-2), MEDIUM, TopicConfig.LOCAL_LOG_RETENTION_BYTES_DOC) .define(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_COPY_DISABLE_DOC) - .define(TopicConfig.REMOTE_LOG_KEEP_LATEST_CONFIG, BOOLEAN, DEFAULT_REMOTE_LOG_KEEP_LATEST_CONFIG, MEDIUM, TopicConfig.REMOTE_LOG_KEEP_LATEST_DOC) + .define(TopicConfig.REMOTE_LOG_LATEST_ENABLE_CONFIG, BOOLEAN, DEFAULT_REMOTE_LOG_LATEST_ENABLE_CONFIG, MEDIUM, TopicConfig.REMOTE_LOG_LATEST_ENABLE_DOC) .define(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_DOC) .defineInternal(INTERNAL_SEGMENT_BYTES_CONFIG, INT, null, null, MEDIUM, INTERNAL_SEGMENT_BYTES_DOC); } @@ -403,8 +403,8 @@ public Boolean remoteLogCopyDisable() { return remoteLogConfig.remoteLogCopyDisable; } - public Boolean remoteLogKeepLatest() { - return remoteLogConfig.remoteLogKeepLatest; + public Boolean remoteLogLatestEnable() { + return remoteLogConfig.remoteLogLatestEnable; } public long localRetentionMs() { From c410ae3c0679b5aa3f274a2c2202e2ac141f3cbb Mon Sep 17 00:00:00 2001 From: Jian Date: Wed, 19 Nov 2025 21:05:51 +0800 Subject: [PATCH 003/110] fix the unit test for null point protect Signed-off-by: Jian --- .../kafka/server/log/remote/storage/RemoteLogManager.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index adf23d43f214e..b01cf01de8a0e 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -973,13 +973,14 @@ List candidateLogSegments(UnifiedLog log, Long fromOffset, L long localLogSize = log.size(); long accumulatedUploadedSize = 0; long currentTimeMs = time.milliseconds(); - boolean remoteLogLatestEnable = log.config().remoteLogLatestEnable(); + // Use default value true (original behavior) if config is null + boolean remoteLogLatestEnable = log.config() != null ? log.config().remoteLogLatestEnable() : true; for (int idx = 1; idx < segments.size(); idx++) { LogSegment previousSeg = segments.get(idx - 1); LogSegment currentSeg = segments.get(idx); if (currentSeg.baseOffset() <= lastStableOffset) { - if (!remoteLogLatestEnable) { + if (!remoteLogLatestEnable && log.config() != null) { // Check time-based retention long localRetentionMs = log.config().localRetentionMs(); if (isWithinLocalRetentionTime(previousSeg, localRetentionMs, currentTimeMs)) { From c4a77b966ae8c4dcbbd68c1cc6c5e89a33faf1fa Mon Sep 17 00:00:00 2001 From: Jian Date: Wed, 7 Jan 2026 20:09:06 +0800 Subject: [PATCH 004/110] correct the desc according to kip's review --- .../main/java/org/apache/kafka/common/config/TopicConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index e0eec6c752a95..9cedb92fd1d7b 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -102,7 +102,7 @@ public class TopicConfig { "(i.e. retention.ms/bytes)."; public static final String REMOTE_LOG_LATEST_ENABLE_CONFIG = "remote.log.latest.enable"; - public static final String REMOTE_LOG_LATEST_ENABLE_DOC = "Determines whether to upload all segments to remote storage including the latest ones within local retention. " + + public static final String REMOTE_LOG_LATEST_ENABLE_DOC = "Determines whether to upload all non-active segments to remote storage, including those still within local retention. " + "When set to true (default), all committed segments will be uploaded without checking local retention constraints. " + "When set to false, only segments beyond local retention period will be uploaded to remote storage."; From 1b0705e7a5d101ca7745409e4f947bc974f2f5c0 Mon Sep 17 00:00:00 2001 From: Jian Date: Wed, 7 Jan 2026 20:37:39 +0800 Subject: [PATCH 005/110] Clarify remote log latest enable documentation Updated documentation for REMOTE_LOG_LATEST_ENABLE_DOC to clarify that non-active segments are uploaded. --- .../main/java/org/apache/kafka/common/config/TopicConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index 9cedb92fd1d7b..19176370179b9 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -103,7 +103,7 @@ public class TopicConfig { public static final String REMOTE_LOG_LATEST_ENABLE_CONFIG = "remote.log.latest.enable"; public static final String REMOTE_LOG_LATEST_ENABLE_DOC = "Determines whether to upload all non-active segments to remote storage, including those still within local retention. " + - "When set to true (default), all committed segments will be uploaded without checking local retention constraints. " + + "When set to true (default), all non-active segments will be uploaded without checking local retention constraints. " + "When set to false, only segments beyond local retention period will be uploaded to remote storage."; public static final String REMOTE_LOG_DELETE_ON_DISABLE_CONFIG = "remote.log.delete.on.disable"; From 2cb87c7017abed4cf540252b8f8cca8aa5b032bb Mon Sep 17 00:00:00 2001 From: stroller Date: Thu, 29 Jan 2026 10:38:40 +0800 Subject: [PATCH 006/110] change the naming according to KIP review Signed-off-by: stroller --- .../apache/kafka/common/config/TopicConfig.java | 8 ++++---- .../log/remote/storage/RemoteLogManager.java | 10 +++++----- .../kafka/storage/internals/log/LogConfig.java | 14 +++++++------- 3 files changed, 16 insertions(+), 16 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index 19176370179b9..9cb150ee11bfe 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -101,10 +101,10 @@ public class TopicConfig { "(i.e. local.retention.ms/bytes) becomes irrelevant, and all data expiration follows the topic-wide retention configuration" + "(i.e. retention.ms/bytes)."; - public static final String REMOTE_LOG_LATEST_ENABLE_CONFIG = "remote.log.latest.enable"; - public static final String REMOTE_LOG_LATEST_ENABLE_DOC = "Determines whether to upload all non-active segments to remote storage, including those still within local retention. " + - "When set to true (default), all non-active segments will be uploaded without checking local retention constraints. " + - "When set to false, only segments beyond local retention period will be uploaded to remote storage."; + public static final String REMOTE_COPY_LAZY_ENABLE_CONFIG = "remote.copy.lazy.enable"; + public static final String REMOTE_COPY_LAZY_ENABLE_DOC = "Determines whether to delay uploading segments to remote storage. " + + "When set to false (default), all non-active segments will be uploaded immediately without checking local retention constraints. " + + "When set to true, segments within local retention will not be uploaded, the upload is delayed until they expire."; public static final String REMOTE_LOG_DELETE_ON_DISABLE_CONFIG = "remote.log.delete.on.disable"; public static final String REMOTE_LOG_DELETE_ON_DISABLE_DOC = "Determines whether tiered data for a topic should be " + diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index c5e37e1926e23..baa064084a31b 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -964,8 +964,8 @@ private boolean isWithinLocalRetentionSize(long localLogSize, LogSegment segment * 1) Segment is not the active segment and * 2) Segment end-offset is less than the last-stable-offset as remote storage should contain only * committed/acked messages - * 3) When remote.log.latest.enable is true (default): All segments are eligible including the latest ones within local retention - * When remote.log.latest.enable is false: Segment is not within local retention time or size (close to expiration or already expired) + * 3) When remote.copy.lazy.enable is false (default): All segments are eligible including the latest ones within local retention + * When remote.copy.lazy.enable is true: Segments within local retention will not be uploaded, upload is delayed until they expire * @param log The log from which the segments are to be copied * @param fromOffset The offset from which the segments are to be copied * @param lastStableOffset The last stable offset of the log @@ -978,14 +978,14 @@ List candidateLogSegments(UnifiedLog log, Long fromOffset, L long localLogSize = log.size(); long accumulatedUploadedSize = 0; long currentTimeMs = time.milliseconds(); - // Use default value true (original behavior) if config is null - boolean remoteLogLatestEnable = log.config() != null ? log.config().remoteLogLatestEnable() : true; + // Use default value false if config is null + boolean remoteCopyLazyEnable = log.config() != null ? log.config().remoteCopyLazyEnable() : false; for (int idx = 1; idx < segments.size(); idx++) { LogSegment previousSeg = segments.get(idx - 1); LogSegment currentSeg = segments.get(idx); if (currentSeg.baseOffset() <= lastStableOffset) { - if (!remoteLogLatestEnable && log.config() != null) { + if (remoteCopyLazyEnable && log.config() != null) { // Check time-based retention long localRetentionMs = log.config().localRetentionMs(); if (isWithinLocalRetentionTime(previousSeg, localRetentionMs, currentTimeMs)) { diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index b99663afbd42f..3e29598bab2e6 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -67,14 +67,14 @@ private static class RemoteLogConfig { private final boolean remoteStorageEnable; private final boolean remoteLogDeleteOnDisable; private final boolean remoteLogCopyDisable; - private final boolean remoteLogLatestEnable; + private final boolean remoteCopyLazyEnable; private final long localRetentionMs; private final long localRetentionBytes; private RemoteLogConfig(LogConfig config) { this.remoteStorageEnable = config.getBoolean(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG); this.remoteLogCopyDisable = config.getBoolean(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG); - this.remoteLogLatestEnable = config.getBoolean(TopicConfig.REMOTE_LOG_LATEST_ENABLE_CONFIG); + this.remoteCopyLazyEnable = config.getBoolean(TopicConfig.REMOTE_COPY_LAZY_ENABLE_CONFIG); this.remoteLogDeleteOnDisable = config.getBoolean(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG); this.localRetentionMs = config.getLong(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG); this.localRetentionBytes = config.getLong(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG); @@ -85,7 +85,7 @@ public String toString() { return "RemoteLogConfig{" + "remoteStorageEnable=" + remoteStorageEnable + ", remoteLogCopyDisable=" + remoteLogCopyDisable + - ", remoteLogLatestEnable=" + remoteLogLatestEnable + + ", remoteCopyLazyEnable=" + remoteCopyLazyEnable + ", remoteLogDeleteOnDisable=" + remoteLogDeleteOnDisable + ", localRetentionMs=" + localRetentionMs + ", localRetentionBytes=" + localRetentionBytes + @@ -138,7 +138,7 @@ public Optional serverConfigName(String configName) { public static final boolean DEFAULT_REMOTE_STORAGE_ENABLE = false; public static final boolean DEFAULT_REMOTE_LOG_COPY_DISABLE_CONFIG = false; - public static final boolean DEFAULT_REMOTE_LOG_LATEST_ENABLE_CONFIG = true; + public static final boolean DEFAULT_REMOTE_COPY_LAZY_ENABLE_CONFIG = false; public static final boolean DEFAULT_REMOTE_LOG_DELETE_ON_DISABLE_CONFIG = false; public static final long DEFAULT_LOCAL_RETENTION_BYTES = -2; // It indicates the value to be derived from RetentionBytes public static final long DEFAULT_LOCAL_RETENTION_MS = -2; // It indicates the value to be derived from RetentionMs @@ -250,7 +250,7 @@ public Optional serverConfigName(String configName) { .define(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, LONG, DEFAULT_LOCAL_RETENTION_BYTES, atLeast(-2), MEDIUM, TopicConfig.LOCAL_LOG_RETENTION_BYTES_DOC) .define(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_COPY_DISABLE_DOC) - .define(TopicConfig.REMOTE_LOG_LATEST_ENABLE_CONFIG, BOOLEAN, DEFAULT_REMOTE_LOG_LATEST_ENABLE_CONFIG, MEDIUM, TopicConfig.REMOTE_LOG_LATEST_ENABLE_DOC) + .define(TopicConfig.REMOTE_COPY_LAZY_ENABLE_CONFIG, BOOLEAN, DEFAULT_REMOTE_COPY_LAZY_ENABLE_CONFIG, MEDIUM, TopicConfig.REMOTE_COPY_LAZY_ENABLE_DOC) .define(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_DOC) .defineInternal(INTERNAL_SEGMENT_BYTES_CONFIG, INT, null, null, MEDIUM, INTERNAL_SEGMENT_BYTES_DOC); } @@ -403,8 +403,8 @@ public Boolean remoteLogCopyDisable() { return remoteLogConfig.remoteLogCopyDisable; } - public Boolean remoteLogLatestEnable() { - return remoteLogConfig.remoteLogLatestEnable; + public Boolean remoteCopyLazyEnable() { + return remoteLogConfig.remoteCopyLazyEnable; } public long localRetentionMs() { From aefdac14149bf7a67c9a6d3e2ffffdfce07a5053 Mon Sep 17 00:00:00 2001 From: stroller Date: Thu, 29 Jan 2026 10:45:58 +0800 Subject: [PATCH 007/110] add broker level configure according to KIP review Signed-off-by: stroller --- .../java/org/apache/kafka/server/config/ServerLogConfigs.java | 3 +++ .../apache/kafka/server/config/ServerTopicConfigSynonyms.java | 3 ++- .../java/org/apache/kafka/storage/internals/log/LogConfig.java | 1 + 3 files changed, 6 insertions(+), 1 deletion(-) diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java index 5438a0a59cb6f..f1102b33d8caf 100644 --- a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java @@ -160,4 +160,7 @@ public class ServerLogConfigs { public static final int MAX_MESSAGE_BYTES_DEFAULT = 1024 * 1024 + Records.LOG_OVERHEAD; public static final String COMPRESSION_TYPE_DEFAULT = BrokerCompressionType.PRODUCER.name; + + public static final String LOG_REMOTE_COPY_LAZY_ENABLE_CONFIG = LOG_PREFIX + TopicConfig.REMOTE_COPY_LAZY_ENABLE_CONFIG; + public static final String LOG_REMOTE_COPY_LAZY_ENABLE_DOC = TopicConfig.REMOTE_COPY_LAZY_ENABLE_DOC; } diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java index c05f9f2816ae0..b1d350cda3c3e 100644 --- a/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java @@ -84,7 +84,8 @@ public final class ServerTopicConfigSynonyms { sameNameWithLogPrefix(TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG), sameNameWithLogPrefix(TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG), sameNameWithLogPrefix(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG), - sameNameWithLogPrefix(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG) + sameNameWithLogPrefix(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG), + sameNameWithLogPrefix(TopicConfig.REMOTE_COPY_LAZY_ENABLE_CONFIG) ); /** diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 3e29598bab2e6..1d8d1eff05fac 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -183,6 +183,7 @@ public Optional serverConfigName(String configName) { .define(ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_DOC) .define(ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_DOC) .define(ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG, LONG, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DEFAULT, atLeast(1), LOW, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DOC) + .define(ServerLogConfigs.LOG_REMOTE_COPY_LAZY_ENABLE_CONFIG, BOOLEAN, DEFAULT_REMOTE_COPY_LAZY_ENABLE_CONFIG, LOW, ServerLogConfigs.LOG_REMOTE_COPY_LAZY_ENABLE_DOC) .defineInternal(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT, atLeast(0), LOW, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DOC); private static final LogConfigDef CONFIG = new LogConfigDef(); From 3c5131b1ce38761c69c0e003e449328b0077a931 Mon Sep 17 00:00:00 2001 From: stroller Date: Thu, 29 Jan 2026 13:16:33 +0800 Subject: [PATCH 008/110] fix unit test Signed-off-by: stroller --- core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 2e366a971171a..cfe1bddebba7d 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -1166,6 +1166,7 @@ class KafkaConfigTest { case TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG => assertDynamic(kafkaConfigProp, 10016L, () => config.remoteLogManagerConfig.logLocalRetentionBytes) // not dynamically updatable + case TopicConfig.REMOTE_COPY_LAZY_ENABLE_CONFIG => case QuotaConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG => // topic only config case QuotaConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG => From 44d74ac4e496c8fea84f4c8eece9d11482df40fd Mon Sep 17 00:00:00 2001 From: stroller Date: Thu, 29 Jan 2026 17:50:46 +0800 Subject: [PATCH 009/110] improve the code Signed-off-by: stroller --- .../kafka/server/log/remote/storage/RemoteLogManager.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index baa064084a31b..f877b9098e151 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -924,7 +924,7 @@ private boolean isWithinLocalRetentionTime(LogSegment segment, long localRetenti if (segmentLargestTimestamp >= 0) { long segmentAge = currentTimeMs - segmentLargestTimestamp; if (segmentAge < localRetentionMs) { - logger.info("Segment {} is still within local retention time. Segment age: {} ms, local retention: {} ms, skipping upload", + logger.debug("Segment {} is still within local retention time. Segment age: {} ms, local retention: {} ms, skipping upload", segment, segmentAge, localRetentionMs); return true; } @@ -952,7 +952,7 @@ private boolean isWithinLocalRetentionSize(long localLogSize, LogSegment segment long sizeAfterUploading = localLogSize - accumulatedUploadedSize - segmentSize; if (sizeAfterUploading < localRetentionBytes) { - logger.info("Segment {}: size after planing to upload to remote {} bytes < retention {} bytes, skipping upload", + logger.debug("Segment {}: size after planning to upload to remote {} bytes < retention {} bytes, skipping upload", segment, sizeAfterUploading, localRetentionBytes); return true; } From 791881e0272e4b5a8256f0921c7c651a2e11c281 Mon Sep 17 00:00:00 2001 From: stroller Date: Fri, 30 Jan 2026 08:51:32 +0800 Subject: [PATCH 010/110] remove duplicated code for check Signed-off-by: stroller --- .../kafka/server/log/remote/storage/RemoteLogManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index f877b9098e151..44509d4ba2e81 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -985,7 +985,7 @@ List candidateLogSegments(UnifiedLog log, Long fromOffset, L LogSegment previousSeg = segments.get(idx - 1); LogSegment currentSeg = segments.get(idx); if (currentSeg.baseOffset() <= lastStableOffset) { - if (remoteCopyLazyEnable && log.config() != null) { + if (remoteCopyLazyEnable) { // Check time-based retention long localRetentionMs = log.config().localRetentionMs(); if (isWithinLocalRetentionTime(previousSeg, localRetentionMs, currentTimeMs)) { From 31ecadba162724076334588f0111d06f0aa1ca77 Mon Sep 17 00:00:00 2001 From: Jian Date: Thu, 5 Feb 2026 21:13:04 +0800 Subject: [PATCH 011/110] KAFKA-19893: change according to the KIP review Signed-off-by: Jian --- .../kafka/common/config/TopicConfig.java | 12 +++ .../log/remote/storage/RemoteLogManager.java | 82 +++++-------------- .../storage/internals/log/LogConfig.java | 30 +++++++ 3 files changed, 61 insertions(+), 63 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index 19176370179b9..a0dd538492dca 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -106,6 +106,18 @@ public class TopicConfig { "When set to true (default), all non-active segments will be uploaded without checking local retention constraints. " + "When set to false, only segments beyond local retention period will be uploaded to remote storage."; + public static final String REMOTE_LOG_COPY_LAG_MS_CONFIG = "remote.log.copy.lag.ms"; + public static final String REMOTE_LOG_COPY_LAG_MS_DOC = "Controls how long to delay uploading segments to remote storage. " + + "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay). " + + "When set to -1, segments are uploaded only when close to or beyond local retention (maximum delay). " + + "When set to a positive value (ms), a segment is eligible for upload only after it has been closed for at least this many milliseconds."; + + public static final String REMOTE_LOG_COPY_LAG_BYTES_CONFIG = "remote.log.copy.lag.bytes"; + public static final String REMOTE_LOG_COPY_LAG_BYTES_DOC = "Controls size-based delay for uploading segments to remote storage. " + + "When set to 0 (default), no size-based constraint. " + + "When set to -1, resolves to local retention bytes. " + + "When set to a positive value (bytes), a segment is eligible for upload only when remaining local size after uploading would be at or below (local.retention.bytes - this value)."; + public static final String REMOTE_LOG_DELETE_ON_DISABLE_CONFIG = "remote.log.delete.on.disable"; public static final String REMOTE_LOG_DELETE_ON_DISABLE_DOC = "Determines whether tiered data for a topic should be " + "deleted after tiered storage is disabled on a topic. This configuration should be enabled when trying to " + diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index c5e37e1926e23..66a4def4daf47 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -907,28 +907,11 @@ private void maybeUpdateCopiedOffset(UnifiedLog log) throws RemoteStorageExcepti } } - /** - * Check if segment is still within local retention time. - * @param segment The log segment to check - * @param localRetentionMs Local retention time in milliseconds; - * @param currentTimeMs Current time in milliseconds - * @return true if segment is still within local retention time, false otherwise - */ - private boolean isWithinLocalRetentionTime(LogSegment segment, long localRetentionMs, long currentTimeMs) { - if (localRetentionMs <= 0) { - return false; - } - + private boolean hasExceededCopyLagTime(LogSegment segment, long currentTimeMs, long copyLagMs) { + if (copyLagMs <= 0) return true; try { - long segmentLargestTimestamp = segment.largestTimestamp(); - if (segmentLargestTimestamp >= 0) { - long segmentAge = currentTimeMs - segmentLargestTimestamp; - if (segmentAge < localRetentionMs) { - logger.info("Segment {} is still within local retention time. Segment age: {} ms, local retention: {} ms, skipping upload", - segment, segmentAge, localRetentionMs); - return true; - } - } + long ts = segment.largestTimestamp(); + if (ts >= 0) return (currentTimeMs - ts) >= copyLagMs; } catch (IOException e) { logger.warn("Failed to get largest timestamp for segment {}, will not skip based on time", segment, e); } @@ -936,27 +919,13 @@ private boolean isWithinLocalRetentionTime(LogSegment segment, long localRetenti } /** - * Check if local log size is still within local retention size after uploading the segment to remote. - * @param localLogSize Current local log size - * @param segment The segment to check - * @param accumulatedUploadedSize Total size of segments already decided to upload to remote - * @param localRetentionBytes Local retention size in bytes - * @return true if local log size after uploading this segment to remote would still be within local retention size, false otherwise + * Check if segment has exceeded the copy lag by size. + * @param cumulativeLogSize Cumulative size in bytes of segments from the start of the copy range up to and including the current segment + * @param copyLagBytes Copy lag in bytes (segment is eligible when cumulative log size >= this value) + * @return true if cumulativeLogSize >= copyLagBytes (exceeded, eligible for upload), false otherwise */ - private boolean isWithinLocalRetentionSize(long localLogSize, LogSegment segment, long accumulatedUploadedSize, long localRetentionBytes) { - if (localRetentionBytes <= 0) { - return false; - } - - int segmentSize = segment.size(); - long sizeAfterUploading = localLogSize - accumulatedUploadedSize - segmentSize; - - if (sizeAfterUploading < localRetentionBytes) { - logger.info("Segment {}: size after planing to upload to remote {} bytes < retention {} bytes, skipping upload", - segment, sizeAfterUploading, localRetentionBytes); - return true; - } - return false; + private boolean hasExceededCopyLagSize(long cumulativeLogSize, long copyLagBytes) { + return copyLagBytes <= 0 || cumulativeLogSize >= copyLagBytes; } /** @@ -964,8 +933,7 @@ private boolean isWithinLocalRetentionSize(long localLogSize, LogSegment segment * 1) Segment is not the active segment and * 2) Segment end-offset is less than the last-stable-offset as remote storage should contain only * committed/acked messages - * 3) When remote.log.latest.enable is true (default): All segments are eligible including the latest ones within local retention - * When remote.log.latest.enable is false: Segment is not within local retention time or size (close to expiration or already expired) + * 3) Segment has exceeded copy lag by time and size when configured (remote.log.copy.lag.ms, remote.log.copy.lag.bytes) * @param log The log from which the segments are to be copied * @param fromOffset The offset from which the segments are to be copied * @param lastStableOffset The last stable offset of the log @@ -975,33 +943,21 @@ List candidateLogSegments(UnifiedLog log, Long fromOffset, L List candidateLogSegments = new ArrayList<>(); List segments = log.logSegments(fromOffset, Long.MAX_VALUE); if (!segments.isEmpty()) { - long localLogSize = log.size(); - long accumulatedUploadedSize = 0; long currentTimeMs = time.milliseconds(); - // Use default value true (original behavior) if config is null - boolean remoteLogLatestEnable = log.config() != null ? log.config().remoteLogLatestEnable() : true; + long copyLagMs = log.config() != null ? log.config().remoteLogCopyLagMs() : 0L; + long copyLagBytes = log.config() != null ? log.config().remoteLogCopyLagBytes() : 0L; + long cumulativeLogSize = 0; for (int idx = 1; idx < segments.size(); idx++) { LogSegment previousSeg = segments.get(idx - 1); LogSegment currentSeg = segments.get(idx); if (currentSeg.baseOffset() <= lastStableOffset) { - if (!remoteLogLatestEnable && log.config() != null) { - // Check time-based retention - long localRetentionMs = log.config().localRetentionMs(); - if (isWithinLocalRetentionTime(previousSeg, localRetentionMs, currentTimeMs)) { - break; - } - - // Check size-based retention - long localRetentionBytes = log.config().localRetentionBytes(); - if (isWithinLocalRetentionSize(localLogSize, previousSeg, accumulatedUploadedSize, localRetentionBytes)) { - break; - } - } - - // Segment is eligible for upload + cumulativeLogSize += previousSeg.size(); + if (copyLagMs != 0 && !hasExceededCopyLagTime(previousSeg, currentTimeMs, copyLagMs)) + break; + if (copyLagBytes != 0 && !hasExceededCopyLagSize(cumulativeLogSize, copyLagBytes)) + break; candidateLogSegments.add(new EnrichedLogSegment(previousSeg, currentSeg.baseOffset())); - accumulatedUploadedSize += previousSeg.size(); } } // Discard the last active segment diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index b99663afbd42f..743d5cb6dc5a6 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -68,6 +68,9 @@ private static class RemoteLogConfig { private final boolean remoteLogDeleteOnDisable; private final boolean remoteLogCopyDisable; private final boolean remoteLogLatestEnable; + /** Effective copy lag: 0 = upload immediately, -1 = delay until local retention boundary, >0 = min segment age in ms. */ + private final long remoteLogCopyLagMs; + private final long remoteLogCopyLagBytes; private final long localRetentionMs; private final long localRetentionBytes; @@ -78,6 +81,11 @@ private RemoteLogConfig(LogConfig config) { this.remoteLogDeleteOnDisable = config.getBoolean(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG); this.localRetentionMs = config.getLong(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG); this.localRetentionBytes = config.getLong(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG); + // Use remote.log.copy.lag.ms if explicitly set, else derive from remote.log.latest.enable for backward compatibility + this.remoteLogCopyLagMs = config.overriddenConfigs.contains(TopicConfig.REMOTE_LOG_COPY_LAG_MS_CONFIG) + ? config.getLong(TopicConfig.REMOTE_LOG_COPY_LAG_MS_CONFIG) + : (config.getBoolean(TopicConfig.REMOTE_LOG_LATEST_ENABLE_CONFIG) ? 0L : -1L); + this.remoteLogCopyLagBytes = config.getLong(TopicConfig.REMOTE_LOG_COPY_LAG_BYTES_CONFIG); } @Override @@ -86,6 +94,8 @@ public String toString() { "remoteStorageEnable=" + remoteStorageEnable + ", remoteLogCopyDisable=" + remoteLogCopyDisable + ", remoteLogLatestEnable=" + remoteLogLatestEnable + + ", remoteLogCopyLagMs=" + remoteLogCopyLagMs + + ", remoteLogCopyLagBytes=" + remoteLogCopyLagBytes + ", remoteLogDeleteOnDisable=" + remoteLogDeleteOnDisable + ", localRetentionMs=" + localRetentionMs + ", localRetentionBytes=" + localRetentionBytes + @@ -142,6 +152,8 @@ public Optional serverConfigName(String configName) { public static final boolean DEFAULT_REMOTE_LOG_DELETE_ON_DISABLE_CONFIG = false; public static final long DEFAULT_LOCAL_RETENTION_BYTES = -2; // It indicates the value to be derived from RetentionBytes public static final long DEFAULT_LOCAL_RETENTION_MS = -2; // It indicates the value to be derived from RetentionMs + public static final long DEFAULT_REMOTE_LOG_COPY_LAG_MS = 0L; + public static final long DEFAULT_REMOTE_LOG_COPY_LAG_BYTES = 0L; public static final String INTERNAL_SEGMENT_BYTES_CONFIG = "internal.segment.bytes"; public static final String INTERNAL_SEGMENT_BYTES_DOC = "The maximum size of a single log file. This should be used for testing only."; @@ -251,6 +263,8 @@ public Optional serverConfigName(String configName) { TopicConfig.LOCAL_LOG_RETENTION_BYTES_DOC) .define(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_COPY_DISABLE_DOC) .define(TopicConfig.REMOTE_LOG_LATEST_ENABLE_CONFIG, BOOLEAN, DEFAULT_REMOTE_LOG_LATEST_ENABLE_CONFIG, MEDIUM, TopicConfig.REMOTE_LOG_LATEST_ENABLE_DOC) + .define(TopicConfig.REMOTE_LOG_COPY_LAG_MS_CONFIG, LONG, DEFAULT_REMOTE_LOG_COPY_LAG_MS, atLeast(-1), MEDIUM, TopicConfig.REMOTE_LOG_COPY_LAG_MS_DOC) + .define(TopicConfig.REMOTE_LOG_COPY_LAG_BYTES_CONFIG, LONG, DEFAULT_REMOTE_LOG_COPY_LAG_BYTES, atLeast(-1), MEDIUM, TopicConfig.REMOTE_LOG_COPY_LAG_BYTES_DOC) .define(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_DOC) .defineInternal(INTERNAL_SEGMENT_BYTES_CONFIG, INT, null, null, MEDIUM, INTERNAL_SEGMENT_BYTES_DOC); } @@ -407,6 +421,22 @@ public Boolean remoteLogLatestEnable() { return remoteLogConfig.remoteLogLatestEnable; } + /** + * Effective remote copy lag in ms: 0 = upload immediately, -1 resolves to local retention time, >0 = segment must be at least this old (ms) to be eligible. + */ + public long remoteLogCopyLagMs() { + long raw = remoteLogConfig.remoteLogCopyLagMs; + return raw == -1 ? localRetentionMs() : raw; + } + + /** + * Effective remote copy lag in bytes: 0 = no size constraint, -1 resolves to local retention bytes, >0 = size-based eligibility. + */ + public long remoteLogCopyLagBytes() { + long raw = remoteLogConfig.remoteLogCopyLagBytes; + return raw == -1 ? localRetentionBytes() : raw; + } + public long localRetentionMs() { return remoteLogConfig.localRetentionMs == LogConfig.DEFAULT_LOCAL_RETENTION_MS ? retentionMs : remoteLogConfig.localRetentionMs; } From 5b059e15b1572f744fa83617b7ca6b1418a5f001 Mon Sep 17 00:00:00 2001 From: Jian Date: Thu, 5 Feb 2026 21:24:49 +0800 Subject: [PATCH 012/110] KAFKA-19893: change according to the KIP review Signed-off-by: Jian --- .../kafka/common/config/TopicConfig.java | 5 ---- .../main/scala/kafka/server/KafkaConfig.scala | 2 ++ .../unit/kafka/server/KafkaConfigTest.scala | 5 +++- .../kafka/server/config/ServerLogConfigs.java | 3 -- .../config/ServerTopicConfigSynonyms.java | 3 +- .../storage/RemoteLogManagerConfig.java | 29 +++++++++++++++++++ 6 files changed, 37 insertions(+), 10 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index b2357e1f43103..1458f2cb65eac 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -101,11 +101,6 @@ public class TopicConfig { "(i.e. local.retention.ms/bytes) becomes irrelevant, and all data expiration follows the topic-wide retention configuration" + "(i.e. retention.ms/bytes)."; - public static final String REMOTE_COPY_LAZY_ENABLE_CONFIG = "remote.copy.lazy.enable"; - public static final String REMOTE_COPY_LAZY_ENABLE_DOC = "Determines whether to delay uploading segments to remote storage. " + - "When set to false (default), all non-active segments will be uploaded immediately without checking local retention constraints. " + - "When set to true, segments within local retention will not be uploaded, the upload is delayed until they expire."; - public static final String REMOTE_LOG_COPY_LAG_MS_CONFIG = "remote.log.copy.lag.ms"; public static final String REMOTE_LOG_COPY_LAG_MS_DOC = "Controls how long to delay uploading segments to remote storage. " + "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay). " + diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index c469f53bdda23..2aad50820f29d 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -691,6 +691,8 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) logProps.put(TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG, logMessageTimestampAfterMaxMs: java.lang.Long) logProps.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, remoteLogManagerConfig.logLocalRetentionMs: java.lang.Long) logProps.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, remoteLogManagerConfig.logLocalRetentionBytes: java.lang.Long) + logProps.put(TopicConfig.REMOTE_LOG_COPY_LAG_MS_CONFIG, remoteLogManagerConfig.logRemoteLogCopyLagMs: java.lang.Long) + logProps.put(TopicConfig.REMOTE_LOG_COPY_LAG_BYTES_CONFIG, remoteLogManagerConfig.logRemoteLogCopyLagBytes: java.lang.Long) logProps } } diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index cfe1bddebba7d..dba9173b8486e 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -1165,8 +1165,11 @@ class KafkaConfigTest { assertDynamic(kafkaConfigProp, 10015L, () => config.remoteLogManagerConfig.logLocalRetentionMs) case TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG => assertDynamic(kafkaConfigProp, 10016L, () => config.remoteLogManagerConfig.logLocalRetentionBytes) + case TopicConfig.REMOTE_LOG_COPY_LAG_MS_CONFIG => + assertDynamic(kafkaConfigProp, 10017L, () => config.remoteLogManagerConfig.logRemoteLogCopyLagMs) + case TopicConfig.REMOTE_LOG_COPY_LAG_BYTES_CONFIG => + assertDynamic(kafkaConfigProp, 10018L, () => config.remoteLogManagerConfig.logRemoteLogCopyLagBytes) // not dynamically updatable - case TopicConfig.REMOTE_COPY_LAZY_ENABLE_CONFIG => case QuotaConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG => // topic only config case QuotaConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG => diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java index f1102b33d8caf..5438a0a59cb6f 100644 --- a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java @@ -160,7 +160,4 @@ public class ServerLogConfigs { public static final int MAX_MESSAGE_BYTES_DEFAULT = 1024 * 1024 + Records.LOG_OVERHEAD; public static final String COMPRESSION_TYPE_DEFAULT = BrokerCompressionType.PRODUCER.name; - - public static final String LOG_REMOTE_COPY_LAZY_ENABLE_CONFIG = LOG_PREFIX + TopicConfig.REMOTE_COPY_LAZY_ENABLE_CONFIG; - public static final String LOG_REMOTE_COPY_LAZY_ENABLE_DOC = TopicConfig.REMOTE_COPY_LAZY_ENABLE_DOC; } diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java index b1d350cda3c3e..456e4f7c0af4b 100644 --- a/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java @@ -85,7 +85,8 @@ public final class ServerTopicConfigSynonyms { sameNameWithLogPrefix(TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG), sameNameWithLogPrefix(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG), sameNameWithLogPrefix(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG), - sameNameWithLogPrefix(TopicConfig.REMOTE_COPY_LAZY_ENABLE_CONFIG) + sameNameWithLogPrefix(TopicConfig.REMOTE_LOG_COPY_LAG_MS_CONFIG), + sameNameWithLogPrefix(TopicConfig.REMOTE_LOG_COPY_LAG_BYTES_CONFIG) ); /** diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java index dcdec1d68f7d9..ce73ad728647e 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.TopicConfig; import java.util.Collections; import java.util.Map; @@ -168,6 +169,14 @@ public final class RemoteLogManagerConfig { "less than or equal to log.retention.bytes value."; public static final Long DEFAULT_LOG_LOCAL_RETENTION_BYTES = -2L; + public static final String LOG_REMOTE_LOG_COPY_LAG_MS_PROP = "log.remote.log.copy.lag.ms"; + public static final String LOG_REMOTE_LOG_COPY_LAG_MS_DOC = "Broker default for " + TopicConfig.REMOTE_LOG_COPY_LAG_MS_CONFIG + ". " + TopicConfig.REMOTE_LOG_COPY_LAG_MS_DOC; + public static final Long DEFAULT_LOG_REMOTE_LOG_COPY_LAG_MS = 0L; + + public static final String LOG_REMOTE_LOG_COPY_LAG_BYTES_PROP = "log.remote.log.copy.lag.bytes"; + public static final String LOG_REMOTE_LOG_COPY_LAG_BYTES_DOC = "Broker default for " + TopicConfig.REMOTE_LOG_COPY_LAG_BYTES_CONFIG + ". " + TopicConfig.REMOTE_LOG_COPY_LAG_BYTES_DOC; + public static final Long DEFAULT_LOG_REMOTE_LOG_COPY_LAG_BYTES = 0L; + public static final String REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP = "remote.log.manager.copy.max.bytes.per.second"; public static final String REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_DOC = "The maximum number of bytes that can be copied from local storage to remote storage per second. " + "This is a global limit for all the partitions that are being copied from local storage to remote storage. " + @@ -347,6 +356,18 @@ public static ConfigDef configDef() { atLeast(DEFAULT_LOG_LOCAL_RETENTION_BYTES), MEDIUM, LOG_LOCAL_RETENTION_BYTES_DOC) + .define(LOG_REMOTE_LOG_COPY_LAG_MS_PROP, + LONG, + DEFAULT_LOG_REMOTE_LOG_COPY_LAG_MS, + atLeast(-1), + MEDIUM, + LOG_REMOTE_LOG_COPY_LAG_MS_DOC) + .define(LOG_REMOTE_LOG_COPY_LAG_BYTES_PROP, + LONG, + DEFAULT_LOG_REMOTE_LOG_COPY_LAG_BYTES, + atLeast(-1), + MEDIUM, + LOG_REMOTE_LOG_COPY_LAG_BYTES_DOC) .define(REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP, LONG, DEFAULT_REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND, @@ -564,6 +585,14 @@ public long logLocalRetentionMs() { return config.getLong(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP); } + public long logRemoteLogCopyLagMs() { + return config.getLong(LOG_REMOTE_LOG_COPY_LAG_MS_PROP); + } + + public long logRemoteLogCopyLagBytes() { + return config.getLong(LOG_REMOTE_LOG_COPY_LAG_BYTES_PROP); + } + public long remoteListOffsetsRequestTimeoutMs() { return config.getLong(REMOTE_LIST_OFFSETS_REQUEST_TIMEOUT_MS_PROP); } From 4c108d3ee9af41cb0a2248b1f590344cebc9eb22 Mon Sep 17 00:00:00 2001 From: Jian Date: Thu, 5 Feb 2026 21:29:30 +0800 Subject: [PATCH 013/110] KAFKA-19893: change according to the KIP review Signed-off-by: Jian --- .../server/log/remote/storage/RemoteLogManager.java | 3 +-- .../apache/kafka/storage/internals/log/LogConfig.java | 10 +++------- 2 files changed, 4 insertions(+), 9 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index 793e875d14d0e..55fb5f116faff 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -928,8 +928,7 @@ private boolean hasExceededCopyLagSize(long cumulativeLogSize, long copyLagBytes * 1) Segment is not the active segment and * 2) Segment end-offset is less than the last-stable-offset as remote storage should contain only * committed/acked messages - * 3) When remote.log.latest.enable is true (default): All segments are eligible including the latest ones within local retention - * When remote.log.latest.enable is false: Segment is not within local retention time or size (close to expiration or already expired) + * 3) Segment has exceeded copy lag by time and size when configured (remote.log.copy.lag.ms, remote.log.copy.lag.bytes) * @param log The log from which the segments are to be copied * @param fromOffset The offset from which the segments are to be copied * @param lastStableOffset The last stable offset of the log diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index fa89cd12989c7..a15f340a859ef 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -67,7 +67,6 @@ private static class RemoteLogConfig { private final boolean remoteStorageEnable; private final boolean remoteLogDeleteOnDisable; private final boolean remoteLogCopyDisable; - private final boolean remoteLogLatestEnable; /** Effective copy lag: 0 = upload immediately, -1 = delay until local retention boundary, >0 = min segment age in ms. */ private final long remoteLogCopyLagMs; private final long remoteLogCopyLagBytes; @@ -77,14 +76,10 @@ private static class RemoteLogConfig { private RemoteLogConfig(LogConfig config) { this.remoteStorageEnable = config.getBoolean(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG); this.remoteLogCopyDisable = config.getBoolean(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG); - this.remoteLogLatestEnable = config.getBoolean(TopicConfig.REMOTE_LOG_LATEST_ENABLE_CONFIG); this.remoteLogDeleteOnDisable = config.getBoolean(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG); this.localRetentionMs = config.getLong(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG); this.localRetentionBytes = config.getLong(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG); - // Use remote.log.copy.lag.ms if explicitly set, else derive from remote.log.latest.enable for backward compatibility - this.remoteLogCopyLagMs = config.overriddenConfigs.contains(TopicConfig.REMOTE_LOG_COPY_LAG_MS_CONFIG) - ? config.getLong(TopicConfig.REMOTE_LOG_COPY_LAG_MS_CONFIG) - : (config.getBoolean(TopicConfig.REMOTE_LOG_LATEST_ENABLE_CONFIG) ? 0L : -1L); + this.remoteLogCopyLagMs = config.getLong(TopicConfig.REMOTE_LOG_COPY_LAG_MS_CONFIG); this.remoteLogCopyLagBytes = config.getLong(TopicConfig.REMOTE_LOG_COPY_LAG_BYTES_CONFIG); } @@ -93,8 +88,9 @@ public String toString() { return "RemoteLogConfig{" + "remoteStorageEnable=" + remoteStorageEnable + ", remoteLogCopyDisable=" + remoteLogCopyDisable + - ", remoteLogLatestEnable=" + remoteLogLatestEnable + ", remoteLogDeleteOnDisable=" + remoteLogDeleteOnDisable + + ", remoteLogCopyLagMs=" + remoteLogCopyLagMs + + ", remoteLogCopyLagBytes=" + remoteLogCopyLagBytes + ", localRetentionMs=" + localRetentionMs + ", localRetentionBytes=" + localRetentionBytes + '}'; From 05392d5062e054c682da519b942333ecc126beac Mon Sep 17 00:00:00 2001 From: Jian Date: Thu, 5 Feb 2026 21:32:52 +0800 Subject: [PATCH 014/110] KAFKA-19893: change according to the KIP review Signed-off-by: Jian --- .../java/org/apache/kafka/common/config/TopicConfig.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index 1458f2cb65eac..55e22ffb5f358 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -105,13 +105,15 @@ public class TopicConfig { public static final String REMOTE_LOG_COPY_LAG_MS_DOC = "Controls how long to delay uploading segments to remote storage. " + "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay). " + "When set to -1, segments are uploaded only when close to or beyond local retention (maximum delay). " + - "When set to a positive value (ms), a segment is eligible for upload only after it has been closed for at least this many milliseconds."; + "When set to a positive value (ms), a segment is eligible for upload only after it has been closed for at least this many milliseconds. " + + "The value should not exceed local.retention.ms (local retention time)."; public static final String REMOTE_LOG_COPY_LAG_BYTES_CONFIG = "remote.log.copy.lag.bytes"; public static final String REMOTE_LOG_COPY_LAG_BYTES_DOC = "Controls size-based delay for uploading segments to remote storage. " + "When set to 0 (default), no size-based constraint. " + "When set to -1, resolves to local retention bytes. " + - "When set to a positive value (bytes), a segment is eligible for upload only when remaining local size after uploading would be at or below (local.retention.bytes - this value)."; + "When set to a positive value (bytes), a segment is eligible for upload only when cumulative log size from the copy range has reached this value. " + + "The value should not exceed local.retention.bytes (local retention size)."; public static final String REMOTE_LOG_DELETE_ON_DISABLE_CONFIG = "remote.log.delete.on.disable"; public static final String REMOTE_LOG_DELETE_ON_DISABLE_DOC = "Determines whether tiered data for a topic should be " + From 60060e2beeef0f5ad95d79d6f478a99957e3e681 Mon Sep 17 00:00:00 2001 From: Jian Date: Thu, 5 Feb 2026 21:41:05 +0800 Subject: [PATCH 015/110] KAFKA-19893: change according to the KIP review Signed-off-by: Jian --- .../log/remote/storage/RemoteLogManagerConfig.java | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java index ce73ad728647e..aba430d449b9c 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java @@ -18,7 +18,6 @@ import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; -import org.apache.kafka.common.config.TopicConfig; import java.util.Collections; import java.util.Map; @@ -170,11 +169,15 @@ public final class RemoteLogManagerConfig { public static final Long DEFAULT_LOG_LOCAL_RETENTION_BYTES = -2L; public static final String LOG_REMOTE_LOG_COPY_LAG_MS_PROP = "log.remote.log.copy.lag.ms"; - public static final String LOG_REMOTE_LOG_COPY_LAG_MS_DOC = "Broker default for " + TopicConfig.REMOTE_LOG_COPY_LAG_MS_CONFIG + ". " + TopicConfig.REMOTE_LOG_COPY_LAG_MS_DOC; + public static final String LOG_REMOTE_LOG_COPY_LAG_MS_DOC = "The minimum age in milliseconds a segment must reach before being eligible for upload to remote storage. " + + "When set to 0 (default), segments are uploaded as soon as they are eligible. When set to -1, resolves to local retention time. " + + "The value should not exceed log.local.retention.ms."; public static final Long DEFAULT_LOG_REMOTE_LOG_COPY_LAG_MS = 0L; public static final String LOG_REMOTE_LOG_COPY_LAG_BYTES_PROP = "log.remote.log.copy.lag.bytes"; - public static final String LOG_REMOTE_LOG_COPY_LAG_BYTES_DOC = "Broker default for " + TopicConfig.REMOTE_LOG_COPY_LAG_BYTES_CONFIG + ". " + TopicConfig.REMOTE_LOG_COPY_LAG_BYTES_DOC; + public static final String LOG_REMOTE_LOG_COPY_LAG_BYTES_DOC = "The cumulative log size in bytes from the copy range that must be reached before a segment is eligible for upload to remote storage. " + + "When set to 0 (default), no size-based constraint. When set to -1, resolves to local retention bytes. " + + "The value should not exceed log.local.retention.bytes."; public static final Long DEFAULT_LOG_REMOTE_LOG_COPY_LAG_BYTES = 0L; public static final String REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP = "remote.log.manager.copy.max.bytes.per.second"; From 997ecbdedb020ff779f41981e4ce4acf7e55192a Mon Sep 17 00:00:00 2001 From: Jian Date: Thu, 5 Feb 2026 22:02:46 +0800 Subject: [PATCH 016/110] KAFKA-19893: change according to the KIP review Signed-off-by: Jian --- .../kafka/storage/internals/log/LogConfig.java | 13 ++----------- 1 file changed, 2 insertions(+), 11 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index a15f340a859ef..88cb6420e1202 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -67,7 +67,6 @@ private static class RemoteLogConfig { private final boolean remoteStorageEnable; private final boolean remoteLogDeleteOnDisable; private final boolean remoteLogCopyDisable; - /** Effective copy lag: 0 = upload immediately, -1 = delay until local retention boundary, >0 = min segment age in ms. */ private final long remoteLogCopyLagMs; private final long remoteLogCopyLagBytes; private final long localRetentionMs; @@ -410,20 +409,12 @@ public Boolean remoteLogCopyDisable() { } - /** - * Effective remote copy lag in ms: 0 = upload immediately, -1 resolves to local retention time, >0 = segment must be at least this old (ms) to be eligible. - */ public long remoteLogCopyLagMs() { - long raw = remoteLogConfig.remoteLogCopyLagMs; - return raw == -1 ? localRetentionMs() : raw; + return remoteLogConfig.remoteLogCopyLagMs == -1 ? localRetentionMs() : remoteLogConfig.remoteLogCopyLagMs; } - /** - * Effective remote copy lag in bytes: 0 = no size constraint, -1 resolves to local retention bytes, >0 = size-based eligibility. - */ public long remoteLogCopyLagBytes() { - long raw = remoteLogConfig.remoteLogCopyLagBytes; - return raw == -1 ? localRetentionBytes() : raw; + return remoteLogConfig.remoteLogCopyLagBytes == -1 ? localRetentionBytes() : remoteLogConfig.remoteLogCopyLagBytes; } public long localRetentionMs() { From 3398d49544a98c145d2dc622930bafea4c89a7c3 Mon Sep 17 00:00:00 2001 From: Jian Date: Thu, 5 Feb 2026 23:56:17 +0800 Subject: [PATCH 017/110] KAFKA-19893: change according to the KIP review Signed-off-by: Jian --- .../server/log/remote/storage/RemoteLogManager.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index 55fb5f116faff..0ba4edab70ec6 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -908,7 +908,6 @@ private void maybeUpdateCopiedOffset(UnifiedLog log) throws RemoteStorageExcepti } private boolean hasExceededCopyLagTime(LogSegment segment, long currentTimeMs, long copyLagMs) { - if (copyLagMs <= 0) return true; try { long ts = segment.largestTimestamp(); if (ts >= 0) return (currentTimeMs - ts) >= copyLagMs; @@ -919,8 +918,8 @@ private boolean hasExceededCopyLagTime(LogSegment segment, long currentTimeMs, l } - private boolean hasExceededCopyLagSize(long cumulativeLogSize, long copyLagBytes) { - return copyLagBytes <= 0 || cumulativeLogSize >= copyLagBytes; + private boolean hasExceededCopyLagSize(long totalLogSize, long cumulativeSize, long copyLagBytes) { + return totalLogSize - cumulativeSize >= copyLagBytes; } /** @@ -941,16 +940,17 @@ List candidateLogSegments(UnifiedLog log, Long fromOffset, L long currentTimeMs = time.milliseconds(); long copyLagMs = log.config() != null ? log.config().remoteLogCopyLagMs() : 0L; long copyLagBytes = log.config() != null ? log.config().remoteLogCopyLagBytes() : 0L; - long cumulativeLogSize = 0; + long totalLogSize = log.size(); + long cumulativeSize = 0; for (int idx = 1; idx < segments.size(); idx++) { LogSegment previousSeg = segments.get(idx - 1); LogSegment currentSeg = segments.get(idx); if (currentSeg.baseOffset() <= lastStableOffset) { - cumulativeLogSize += previousSeg.size(); + cumulativeSize += previousSeg.size(); if (copyLagMs != 0 && !hasExceededCopyLagTime(previousSeg, currentTimeMs, copyLagMs)) break; - if (copyLagBytes != 0 && !hasExceededCopyLagSize(cumulativeLogSize, copyLagBytes)) + if (copyLagBytes != 0 && !hasExceededCopyLagSize(totalLogSize, cumulativeSize, copyLagBytes)) break; candidateLogSegments.add(new EnrichedLogSegment(previousSeg, currentSeg.baseOffset())); } From 2ae53dda98fa338ad32a27d2ebf2176e35d9fc7b Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 6 Feb 2026 00:08:35 +0800 Subject: [PATCH 018/110] KAFKA-19893: change according to the KIP review Signed-off-by: Jian --- .../kafka/common/config/TopicConfig.java | 2 +- .../log/remote/storage/RemoteLogManager.java | 19 +++++++++++++++++-- .../storage/RemoteLogManagerConfig.java | 2 +- 3 files changed, 19 insertions(+), 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index 55e22ffb5f358..eda5a3feeb681 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -112,7 +112,7 @@ public class TopicConfig { public static final String REMOTE_LOG_COPY_LAG_BYTES_DOC = "Controls size-based delay for uploading segments to remote storage. " + "When set to 0 (default), no size-based constraint. " + "When set to -1, resolves to local retention bytes. " + - "When set to a positive value (bytes), a segment is eligible for upload only when cumulative log size from the copy range has reached this value. " + + "When set to a positive value (bytes), a segment is eligible for upload only when at least this many bytes of log data exist after the segment (i.e. the segment has \"lagged\" by this much in size). " + "The value should not exceed local.retention.bytes (local retention size)."; public static final String REMOTE_LOG_DELETE_ON_DISABLE_CONFIG = "remote.log.delete.on.disable"; diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index 0ba4edab70ec6..443ba617c7544 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -910,16 +910,31 @@ private void maybeUpdateCopiedOffset(UnifiedLog log) throws RemoteStorageExcepti private boolean hasExceededCopyLagTime(LogSegment segment, long currentTimeMs, long copyLagMs) { try { long ts = segment.largestTimestamp(); - if (ts >= 0) return (currentTimeMs - ts) >= copyLagMs; + if (ts >= 0) { + long segmentAgeMs = currentTimeMs - ts; + boolean exceeded = segmentAgeMs >= copyLagMs; + if (!exceeded) { + logger.debug("Segment {} not eligible for upload: segment age {} ms < copy lag {} ms", + segment.baseOffset(), segmentAgeMs, copyLagMs); + } + return exceeded; + } } catch (IOException e) { logger.warn("Failed to get largest timestamp for segment {}, will not skip based on time", segment, e); } + logger.debug("Segment {} not eligible for upload: cannot determine segment age (invalid timestamp)", segment.baseOffset()); return false; } private boolean hasExceededCopyLagSize(long totalLogSize, long cumulativeSize, long copyLagBytes) { - return totalLogSize - cumulativeSize >= copyLagBytes; + long sizeLagBytes = totalLogSize - cumulativeSize; + boolean exceeded = sizeLagBytes >= copyLagBytes; + if (!exceeded) { + logger.debug("Segment not eligible for upload: size lag {} bytes < copy lag {} bytes (totalLogSize={}, cumulativeSize={})", + sizeLagBytes, copyLagBytes, totalLogSize, cumulativeSize); + } + return exceeded; } /** diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java index aba430d449b9c..dc6df80d29c79 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java @@ -175,7 +175,7 @@ public final class RemoteLogManagerConfig { public static final Long DEFAULT_LOG_REMOTE_LOG_COPY_LAG_MS = 0L; public static final String LOG_REMOTE_LOG_COPY_LAG_BYTES_PROP = "log.remote.log.copy.lag.bytes"; - public static final String LOG_REMOTE_LOG_COPY_LAG_BYTES_DOC = "The cumulative log size in bytes from the copy range that must be reached before a segment is eligible for upload to remote storage. " + + public static final String LOG_REMOTE_LOG_COPY_LAG_BYTES_DOC = "The minimum number of bytes of log data that must exist after a segment (newer data) before the segment is eligible for upload to remote storage. " + "When set to 0 (default), no size-based constraint. When set to -1, resolves to local retention bytes. " + "The value should not exceed log.local.retention.bytes."; public static final Long DEFAULT_LOG_REMOTE_LOG_COPY_LAG_BYTES = 0L; From 7ef0a9e5f11904b58dbcff9a0abfdb49cded007b Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 6 Feb 2026 08:09:50 +0800 Subject: [PATCH 019/110] KAFKA-19893: refactor the name and fix unit test Signed-off-by: Jian --- .../kafka/common/config/TopicConfig.java | 8 ++--- .../main/scala/kafka/server/KafkaConfig.scala | 4 +-- .../scala/unit/kafka/log/LogConfigTest.scala | 2 ++ .../unit/kafka/server/KafkaConfigTest.scala | 10 +++--- .../config/ServerTopicConfigSynonyms.java | 4 +-- .../log/remote/storage/RemoteLogManager.java | 6 ++-- .../storage/RemoteLogManagerConfig.java | 32 +++++++++---------- .../storage/internals/log/LogConfig.java | 28 ++++++++-------- 8 files changed, 49 insertions(+), 45 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index eda5a3feeb681..a02f78cf13b67 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -101,15 +101,15 @@ public class TopicConfig { "(i.e. local.retention.ms/bytes) becomes irrelevant, and all data expiration follows the topic-wide retention configuration" + "(i.e. retention.ms/bytes)."; - public static final String REMOTE_LOG_COPY_LAG_MS_CONFIG = "remote.log.copy.lag.ms"; - public static final String REMOTE_LOG_COPY_LAG_MS_DOC = "Controls how long to delay uploading segments to remote storage. " + + public static final String REMOTE_COPY_LAG_MS_CONFIG = "remote.copy.lag.ms"; + public static final String REMOTE_COPY_LAG_MS_DOC = "Controls how long to delay uploading segments to remote storage. " + "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay). " + "When set to -1, segments are uploaded only when close to or beyond local retention (maximum delay). " + "When set to a positive value (ms), a segment is eligible for upload only after it has been closed for at least this many milliseconds. " + "The value should not exceed local.retention.ms (local retention time)."; - public static final String REMOTE_LOG_COPY_LAG_BYTES_CONFIG = "remote.log.copy.lag.bytes"; - public static final String REMOTE_LOG_COPY_LAG_BYTES_DOC = "Controls size-based delay for uploading segments to remote storage. " + + public static final String REMOTE_COPY_LAG_BYTES_CONFIG = "remote.copy.lag.bytes"; + public static final String REMOTE_COPY_LAG_BYTES_DOC = "Controls size-based delay for uploading segments to remote storage. " + "When set to 0 (default), no size-based constraint. " + "When set to -1, resolves to local retention bytes. " + "When set to a positive value (bytes), a segment is eligible for upload only when at least this many bytes of log data exist after the segment (i.e. the segment has \"lagged\" by this much in size). " + diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 2aad50820f29d..425d411c406d9 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -691,8 +691,8 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) logProps.put(TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG, logMessageTimestampAfterMaxMs: java.lang.Long) logProps.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, remoteLogManagerConfig.logLocalRetentionMs: java.lang.Long) logProps.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, remoteLogManagerConfig.logLocalRetentionBytes: java.lang.Long) - logProps.put(TopicConfig.REMOTE_LOG_COPY_LAG_MS_CONFIG, remoteLogManagerConfig.logRemoteLogCopyLagMs: java.lang.Long) - logProps.put(TopicConfig.REMOTE_LOG_COPY_LAG_BYTES_CONFIG, remoteLogManagerConfig.logRemoteLogCopyLagBytes: java.lang.Long) + logProps.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, remoteLogManagerConfig.logRemoteCopyLagMs: java.lang.Long) + logProps.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, remoteLogManagerConfig.logRemoteCopyLagBytes: java.lang.Long) logProps } } diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index 98a49070913b1..b15e865f5d4e8 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -73,6 +73,8 @@ class LogConfigTest { case TopicConfig.COMPRESSION_ZSTD_LEVEL_CONFIG => assertPropertyInvalid(name, "not_a_number", "-0.1") case TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG => assertPropertyInvalid(name, "not_a_number", "remove", "0") case TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG => assertPropertyInvalid(name, "not_a_number", "remove", "0") + case TopicConfig.REMOTE_COPY_LAG_MS_CONFIG => assertPropertyInvalid(name, "not_a_number", "-2") + case TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG => assertPropertyInvalid(name, "not_a_number", "-2") case LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG => // no op case _ => assertPropertyInvalid(name, "not_a_number", "-1") diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index dba9173b8486e..cc7556d39c4e3 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -1029,6 +1029,8 @@ class KafkaConfigTest { case RemoteLogManagerConfig.REMOTE_LOG_READER_MAX_PENDING_TASKS_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", 0, -1) case RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", -3) case RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", -3) + case RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_MS_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", "-2") + case RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_BYTES_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", "-2") /** New group coordinator configs */ case GroupCoordinatorConfig.GROUP_COORDINATOR_NUM_THREADS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", 0, -1) @@ -1165,10 +1167,10 @@ class KafkaConfigTest { assertDynamic(kafkaConfigProp, 10015L, () => config.remoteLogManagerConfig.logLocalRetentionMs) case TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG => assertDynamic(kafkaConfigProp, 10016L, () => config.remoteLogManagerConfig.logLocalRetentionBytes) - case TopicConfig.REMOTE_LOG_COPY_LAG_MS_CONFIG => - assertDynamic(kafkaConfigProp, 10017L, () => config.remoteLogManagerConfig.logRemoteLogCopyLagMs) - case TopicConfig.REMOTE_LOG_COPY_LAG_BYTES_CONFIG => - assertDynamic(kafkaConfigProp, 10018L, () => config.remoteLogManagerConfig.logRemoteLogCopyLagBytes) + case TopicConfig.REMOTE_COPY_LAG_MS_CONFIG => + assertDynamic(kafkaConfigProp, 10017L, () => config.remoteLogManagerConfig.logRemoteCopyLagMs) + case TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG => + assertDynamic(kafkaConfigProp, 10018L, () => config.remoteLogManagerConfig.logRemoteCopyLagBytes) // not dynamically updatable case QuotaConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG => // topic only config diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java index 456e4f7c0af4b..e195e7626dea2 100644 --- a/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java @@ -85,8 +85,8 @@ public final class ServerTopicConfigSynonyms { sameNameWithLogPrefix(TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG), sameNameWithLogPrefix(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG), sameNameWithLogPrefix(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG), - sameNameWithLogPrefix(TopicConfig.REMOTE_LOG_COPY_LAG_MS_CONFIG), - sameNameWithLogPrefix(TopicConfig.REMOTE_LOG_COPY_LAG_BYTES_CONFIG) + sameNameWithLogPrefix(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG), + sameNameWithLogPrefix(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG) ); /** diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index 443ba617c7544..a64c036857e4b 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -942,7 +942,7 @@ private boolean hasExceededCopyLagSize(long totalLogSize, long cumulativeSize, l * 1) Segment is not the active segment and * 2) Segment end-offset is less than the last-stable-offset as remote storage should contain only * committed/acked messages - * 3) Segment has exceeded copy lag by time and size when configured (remote.log.copy.lag.ms, remote.log.copy.lag.bytes) + * 3) Segment has exceeded copy lag by time and size when configured (remote.copy.lag.ms, remote.copy.lag.bytes) * @param log The log from which the segments are to be copied * @param fromOffset The offset from which the segments are to be copied * @param lastStableOffset The last stable offset of the log @@ -953,8 +953,8 @@ List candidateLogSegments(UnifiedLog log, Long fromOffset, L List segments = log.logSegments(fromOffset, Long.MAX_VALUE); if (!segments.isEmpty()) { long currentTimeMs = time.milliseconds(); - long copyLagMs = log.config() != null ? log.config().remoteLogCopyLagMs() : 0L; - long copyLagBytes = log.config() != null ? log.config().remoteLogCopyLagBytes() : 0L; + long copyLagMs = log.config() != null ? log.config().remoteCopyLagMs() : 0L; + long copyLagBytes = log.config() != null ? log.config().remoteCopyLagBytes() : 0L; long totalLogSize = log.size(); long cumulativeSize = 0; diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java index dc6df80d29c79..0937928954ff6 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java @@ -168,17 +168,17 @@ public final class RemoteLogManagerConfig { "less than or equal to log.retention.bytes value."; public static final Long DEFAULT_LOG_LOCAL_RETENTION_BYTES = -2L; - public static final String LOG_REMOTE_LOG_COPY_LAG_MS_PROP = "log.remote.log.copy.lag.ms"; - public static final String LOG_REMOTE_LOG_COPY_LAG_MS_DOC = "The minimum age in milliseconds a segment must reach before being eligible for upload to remote storage. " + + public static final String LOG_REMOTE_COPY_LAG_MS_PROP = "log.remote.copy.lag.ms"; + public static final String LOG_REMOTE_COPY_LAG_MS_DOC = "The minimum age in milliseconds a segment must reach before being eligible for upload to remote storage. " + "When set to 0 (default), segments are uploaded as soon as they are eligible. When set to -1, resolves to local retention time. " + "The value should not exceed log.local.retention.ms."; - public static final Long DEFAULT_LOG_REMOTE_LOG_COPY_LAG_MS = 0L; + public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_MS = 0L; - public static final String LOG_REMOTE_LOG_COPY_LAG_BYTES_PROP = "log.remote.log.copy.lag.bytes"; - public static final String LOG_REMOTE_LOG_COPY_LAG_BYTES_DOC = "The minimum number of bytes of log data that must exist after a segment (newer data) before the segment is eligible for upload to remote storage. " + + public static final String LOG_REMOTE_COPY_LAG_BYTES_PROP = "log.remote.copy.lag.bytes"; + public static final String LOG_REMOTE_COPY_LAG_BYTES_DOC = "The minimum number of bytes of log data that must exist after a segment (newer data) before the segment is eligible for upload to remote storage. " + "When set to 0 (default), no size-based constraint. When set to -1, resolves to local retention bytes. " + "The value should not exceed log.local.retention.bytes."; - public static final Long DEFAULT_LOG_REMOTE_LOG_COPY_LAG_BYTES = 0L; + public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_BYTES = 0L; public static final String REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP = "remote.log.manager.copy.max.bytes.per.second"; public static final String REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_DOC = "The maximum number of bytes that can be copied from local storage to remote storage per second. " + @@ -359,18 +359,18 @@ public static ConfigDef configDef() { atLeast(DEFAULT_LOG_LOCAL_RETENTION_BYTES), MEDIUM, LOG_LOCAL_RETENTION_BYTES_DOC) - .define(LOG_REMOTE_LOG_COPY_LAG_MS_PROP, + .define(LOG_REMOTE_COPY_LAG_MS_PROP, LONG, - DEFAULT_LOG_REMOTE_LOG_COPY_LAG_MS, + DEFAULT_LOG_REMOTE_COPY_LAG_MS, atLeast(-1), MEDIUM, - LOG_REMOTE_LOG_COPY_LAG_MS_DOC) - .define(LOG_REMOTE_LOG_COPY_LAG_BYTES_PROP, + LOG_REMOTE_COPY_LAG_MS_DOC) + .define(LOG_REMOTE_COPY_LAG_BYTES_PROP, LONG, - DEFAULT_LOG_REMOTE_LOG_COPY_LAG_BYTES, + DEFAULT_LOG_REMOTE_COPY_LAG_BYTES, atLeast(-1), MEDIUM, - LOG_REMOTE_LOG_COPY_LAG_BYTES_DOC) + LOG_REMOTE_COPY_LAG_BYTES_DOC) .define(REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP, LONG, DEFAULT_REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND, @@ -588,12 +588,12 @@ public long logLocalRetentionMs() { return config.getLong(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP); } - public long logRemoteLogCopyLagMs() { - return config.getLong(LOG_REMOTE_LOG_COPY_LAG_MS_PROP); + public long logRemoteCopyLagMs() { + return config.getLong(LOG_REMOTE_COPY_LAG_MS_PROP); } - public long logRemoteLogCopyLagBytes() { - return config.getLong(LOG_REMOTE_LOG_COPY_LAG_BYTES_PROP); + public long logRemoteCopyLagBytes() { + return config.getLong(LOG_REMOTE_COPY_LAG_BYTES_PROP); } public long remoteListOffsetsRequestTimeoutMs() { diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 88cb6420e1202..c8a92ccb00aa9 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -67,8 +67,8 @@ private static class RemoteLogConfig { private final boolean remoteStorageEnable; private final boolean remoteLogDeleteOnDisable; private final boolean remoteLogCopyDisable; - private final long remoteLogCopyLagMs; - private final long remoteLogCopyLagBytes; + private final long remoteCopyLagMs; + private final long remoteCopyLagBytes; private final long localRetentionMs; private final long localRetentionBytes; @@ -78,8 +78,8 @@ private RemoteLogConfig(LogConfig config) { this.remoteLogDeleteOnDisable = config.getBoolean(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG); this.localRetentionMs = config.getLong(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG); this.localRetentionBytes = config.getLong(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG); - this.remoteLogCopyLagMs = config.getLong(TopicConfig.REMOTE_LOG_COPY_LAG_MS_CONFIG); - this.remoteLogCopyLagBytes = config.getLong(TopicConfig.REMOTE_LOG_COPY_LAG_BYTES_CONFIG); + this.remoteCopyLagMs = config.getLong(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG); + this.remoteCopyLagBytes = config.getLong(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG); } @Override @@ -88,8 +88,8 @@ public String toString() { "remoteStorageEnable=" + remoteStorageEnable + ", remoteLogCopyDisable=" + remoteLogCopyDisable + ", remoteLogDeleteOnDisable=" + remoteLogDeleteOnDisable + - ", remoteLogCopyLagMs=" + remoteLogCopyLagMs + - ", remoteLogCopyLagBytes=" + remoteLogCopyLagBytes + + ", remoteCopyLagMs=" + remoteCopyLagMs + + ", remoteCopyLagBytes=" + remoteCopyLagBytes + ", localRetentionMs=" + localRetentionMs + ", localRetentionBytes=" + localRetentionBytes + '}'; @@ -144,8 +144,8 @@ public Optional serverConfigName(String configName) { public static final boolean DEFAULT_REMOTE_LOG_DELETE_ON_DISABLE_CONFIG = false; public static final long DEFAULT_LOCAL_RETENTION_BYTES = -2; // It indicates the value to be derived from RetentionBytes public static final long DEFAULT_LOCAL_RETENTION_MS = -2; // It indicates the value to be derived from RetentionMs - public static final long DEFAULT_REMOTE_LOG_COPY_LAG_MS = 0L; - public static final long DEFAULT_REMOTE_LOG_COPY_LAG_BYTES = 0L; + public static final long DEFAULT_REMOTE_COPY_LAG_MS = 0L; + public static final long DEFAULT_REMOTE_COPY_LAG_BYTES = 0L; public static final String INTERNAL_SEGMENT_BYTES_CONFIG = "internal.segment.bytes"; public static final String INTERNAL_SEGMENT_BYTES_DOC = "The maximum size of a single log file. This should be used for testing only."; @@ -254,8 +254,8 @@ public Optional serverConfigName(String configName) { .define(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, LONG, DEFAULT_LOCAL_RETENTION_BYTES, atLeast(-2), MEDIUM, TopicConfig.LOCAL_LOG_RETENTION_BYTES_DOC) .define(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_COPY_DISABLE_DOC) - .define(TopicConfig.REMOTE_LOG_COPY_LAG_MS_CONFIG, LONG, DEFAULT_REMOTE_LOG_COPY_LAG_MS, atLeast(-1), MEDIUM, TopicConfig.REMOTE_LOG_COPY_LAG_MS_DOC) - .define(TopicConfig.REMOTE_LOG_COPY_LAG_BYTES_CONFIG, LONG, DEFAULT_REMOTE_LOG_COPY_LAG_BYTES, atLeast(-1), MEDIUM, TopicConfig.REMOTE_LOG_COPY_LAG_BYTES_DOC) + .define(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, LONG, DEFAULT_REMOTE_COPY_LAG_MS, atLeast(-1), MEDIUM, TopicConfig.REMOTE_COPY_LAG_MS_DOC) + .define(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, LONG, DEFAULT_REMOTE_COPY_LAG_BYTES, atLeast(-1), MEDIUM, TopicConfig.REMOTE_COPY_LAG_BYTES_DOC) .define(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_DOC) .defineInternal(INTERNAL_SEGMENT_BYTES_CONFIG, INT, null, null, MEDIUM, INTERNAL_SEGMENT_BYTES_DOC); } @@ -409,12 +409,12 @@ public Boolean remoteLogCopyDisable() { } - public long remoteLogCopyLagMs() { - return remoteLogConfig.remoteLogCopyLagMs == -1 ? localRetentionMs() : remoteLogConfig.remoteLogCopyLagMs; + public long remoteCopyLagMs() { + return remoteLogConfig.remoteCopyLagMs == -1 ? localRetentionMs() : remoteLogConfig.remoteCopyLagMs; } - public long remoteLogCopyLagBytes() { - return remoteLogConfig.remoteLogCopyLagBytes == -1 ? localRetentionBytes() : remoteLogConfig.remoteLogCopyLagBytes; + public long remoteCopyLagBytes() { + return remoteLogConfig.remoteCopyLagBytes == -1 ? localRetentionBytes() : remoteLogConfig.remoteCopyLagBytes; } public long localRetentionMs() { From b87f0af059500359f50dc43de2e7b3eb31e9847d Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 6 Feb 2026 08:17:05 +0800 Subject: [PATCH 020/110] KAFKA-19893: correct the java doc Signed-off-by: Jian --- .../java/org/apache/kafka/common/config/TopicConfig.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index a02f78cf13b67..b44d0a27bed6a 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -104,15 +104,15 @@ public class TopicConfig { public static final String REMOTE_COPY_LAG_MS_CONFIG = "remote.copy.lag.ms"; public static final String REMOTE_COPY_LAG_MS_DOC = "Controls how long to delay uploading segments to remote storage. " + "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay). " + - "When set to -1, segments are uploaded only when close to or beyond local retention (maximum delay). " + + "When set to -1, resolves to local.retention.ms (maximum delay). " + "When set to a positive value (ms), a segment is eligible for upload only after it has been closed for at least this many milliseconds. " + "The value should not exceed local.retention.ms (local retention time)."; public static final String REMOTE_COPY_LAG_BYTES_CONFIG = "remote.copy.lag.bytes"; public static final String REMOTE_COPY_LAG_BYTES_DOC = "Controls size-based delay for uploading segments to remote storage. " + - "When set to 0 (default), no size-based constraint. " + - "When set to -1, resolves to local retention bytes. " + - "When set to a positive value (bytes), a segment is eligible for upload only when at least this many bytes of log data exist after the segment (i.e. the segment has \"lagged\" by this much in size). " + + "When set to 0 (default), segments are uploaded as soon as they are eligible (no size-based constraint). " + + "When set to -1, resolves to local.retention.bytes (maximum delay). " + + "When set to a positive value (bytes), a segment is eligible for upload only when at least this many bytes of log data exist after the segment. " + "The value should not exceed local.retention.bytes (local retention size)."; public static final String REMOTE_LOG_DELETE_ON_DISABLE_CONFIG = "remote.log.delete.on.disable"; From f54c033d45229dd7b1ee58921cbc62737ddd9abe Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 6 Feb 2026 08:28:19 +0800 Subject: [PATCH 021/110] KAFKA-19893: add protected for the configures Signed-off-by: Jian --- .../kafka/server/DynamicBrokerConfig.scala | 28 +++++++++++++++ .../storage/internals/log/LogConfig.java | 34 +++++++++++++++++++ 2 files changed, 62 insertions(+) diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index f0b077b17f1c8..8fc4453294147 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -698,8 +698,36 @@ class DynamicLogConfig(logManager: LogManager) extends BrokerReconfigurable with } } + def validateRemoteCopyLagNotExceedingLocalRetention(): Unit = { + validateRemoteCopyLagMsNotExceedingLocalRetention() + validateRemoteCopyLagBytesNotExceedingLocalRetention() + } + + def validateRemoteCopyLagMsNotExceedingLocalRetention(): Unit = { + val logRetentionMs = newConfig.logRetentionTimeMillis + val logLocalRetentionMs = newConfig.remoteLogManagerConfig.logLocalRetentionMs + val effectiveLocalRetentionMs = if (logLocalRetentionMs == -2L) logRetentionMs else logLocalRetentionMs + val logRemoteCopyLagMs = newConfig.remoteLogManagerConfig.logRemoteCopyLagMs + if (logRemoteCopyLagMs > 0L && effectiveLocalRetentionMs > 0L && logRemoteCopyLagMs > effectiveLocalRetentionMs) { + throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_MS_PROP, logRemoteCopyLagMs, + s"Value must not exceed ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP} (effective value: $effectiveLocalRetentionMs)") + } + } + + def validateRemoteCopyLagBytesNotExceedingLocalRetention(): Unit = { + val logRetentionBytes = newConfig.logRetentionBytes + val logLocalRetentionBytes = newConfig.remoteLogManagerConfig.logLocalRetentionBytes + val effectiveLocalRetentionBytes = if (logLocalRetentionBytes == -2L) logRetentionBytes else logLocalRetentionBytes + val logRemoteCopyLagBytes = newConfig.remoteLogManagerConfig.logRemoteCopyLagBytes + if (logRemoteCopyLagBytes > 0L && effectiveLocalRetentionBytes > 0L && logRemoteCopyLagBytes > effectiveLocalRetentionBytes) { + throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_BYTES_PROP, logRemoteCopyLagBytes, + s"Value must not exceed ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP} (effective value: $effectiveLocalRetentionBytes)") + } + } + validateLogLocalRetentionMs() validateLogLocalRetentionBytes() + validateRemoteCopyLagNotExceedingLocalRetention() } private def updateLogsConfig(newBrokerDefaults: Map[String, Object]): Unit = { diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index c8a92ccb00aa9..21fb9c70a015e 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -530,6 +530,7 @@ private static void validateTopicLogConfigValues(Map existingCon validateRemoteStorageRequiresDeleteCleanupPolicy(newConfigs); validateRemoteStorageRetentionSize(newConfigs); validateRemoteStorageRetentionTime(newConfigs); + validateRemoteCopyLagNotExceedingLocalRetention(newConfigs); validateRetentionConfigsWhenRemoteCopyDisabled(newConfigs, isRemoteLogStorageEnabled); } else { // The new config "remote.storage.enable" is false, validate if it's turning from true to false @@ -619,6 +620,39 @@ private static void validateRemoteStorageRetentionTime(Map props) { } } + private static void validateRemoteCopyLagNotExceedingLocalRetention(Map props) { + validateRemoteCopyLagMsNotExceedingLocalRetention(props); + validateRemoteCopyLagBytesNotExceedingLocalRetention(props); + } + + private static void validateRemoteCopyLagMsNotExceedingLocalRetention(Map props) { + Long retentionMs = (Long) props.get(TopicConfig.RETENTION_MS_CONFIG); + Long localRetentionMs = (Long) props.get(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG); + Long remoteCopyLagMs = (Long) props.get(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG); + long effectiveLocalRetentionMs = (localRetentionMs != null && localRetentionMs == -2 && retentionMs != null) + ? retentionMs : (localRetentionMs != null ? localRetentionMs : -2); + if (remoteCopyLagMs != null && remoteCopyLagMs > 0 && effectiveLocalRetentionMs > 0 + && remoteCopyLagMs > effectiveLocalRetentionMs) { + String message = String.format("Value must not exceed %s (effective value: %d)", + TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, effectiveLocalRetentionMs); + throw new ConfigException(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, remoteCopyLagMs, message); + } + } + + private static void validateRemoteCopyLagBytesNotExceedingLocalRetention(Map props) { + Long retentionBytes = (Long) props.get(TopicConfig.RETENTION_BYTES_CONFIG); + Long localRetentionBytes = (Long) props.get(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG); + Long remoteCopyLagBytes = (Long) props.get(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG); + long effectiveLocalRetentionBytes = (localRetentionBytes != null && localRetentionBytes == -2 && retentionBytes != null) + ? retentionBytes : (localRetentionBytes != null ? localRetentionBytes : -2); + if (remoteCopyLagBytes != null && remoteCopyLagBytes > 0 && effectiveLocalRetentionBytes > 0 + && remoteCopyLagBytes > effectiveLocalRetentionBytes) { + String message = String.format("Value must not exceed %s (effective value: %d)", + TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, effectiveLocalRetentionBytes); + throw new ConfigException(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, remoteCopyLagBytes, message); + } + } + /** * Check that the given properties contain only valid log config names and that all values can be parsed and are valid */ From ef1ee288ad75e387a89c57e723185cdfb2c47d63 Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 6 Feb 2026 08:32:27 +0800 Subject: [PATCH 022/110] KAFKA-19893: correct the java doc Signed-off-by: Jian --- .../server/log/remote/storage/RemoteLogManagerConfig.java | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java index 0937928954ff6..3b3a954ef67b8 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java @@ -169,15 +169,11 @@ public final class RemoteLogManagerConfig { public static final Long DEFAULT_LOG_LOCAL_RETENTION_BYTES = -2L; public static final String LOG_REMOTE_COPY_LAG_MS_PROP = "log.remote.copy.lag.ms"; - public static final String LOG_REMOTE_COPY_LAG_MS_DOC = "The minimum age in milliseconds a segment must reach before being eligible for upload to remote storage. " + - "When set to 0 (default), segments are uploaded as soon as they are eligible. When set to -1, resolves to local retention time. " + - "The value should not exceed log.local.retention.ms."; + public static final String LOG_REMOTE_COPY_LAG_MS_DOC = "Time-based delay before uploading segments. 0 = no delay, -1 = log.local.retention.ms, positive = min segment age (ms). Must not exceed log.local.retention.ms."; public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_MS = 0L; public static final String LOG_REMOTE_COPY_LAG_BYTES_PROP = "log.remote.copy.lag.bytes"; - public static final String LOG_REMOTE_COPY_LAG_BYTES_DOC = "The minimum number of bytes of log data that must exist after a segment (newer data) before the segment is eligible for upload to remote storage. " + - "When set to 0 (default), no size-based constraint. When set to -1, resolves to local retention bytes. " + - "The value should not exceed log.local.retention.bytes."; + public static final String LOG_REMOTE_COPY_LAG_BYTES_DOC = "Size-based delay before uploading segments. 0 = no delay, -1 = log.local.retention.bytes, positive = min bytes after segment. Must not exceed log.local.retention.bytes."; public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_BYTES = 0L; public static final String REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP = "remote.log.manager.copy.max.bytes.per.second"; From 1658a7069c74202cdc37a1dc7efaa06beaadb13d Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 6 Feb 2026 08:33:55 +0800 Subject: [PATCH 023/110] KAFKA-19893: correct the java doc Signed-off-by: Jian --- .../server/log/remote/storage/RemoteLogManagerConfig.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java index 3b3a954ef67b8..4a84191a813d9 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java @@ -169,11 +169,11 @@ public final class RemoteLogManagerConfig { public static final Long DEFAULT_LOG_LOCAL_RETENTION_BYTES = -2L; public static final String LOG_REMOTE_COPY_LAG_MS_PROP = "log.remote.copy.lag.ms"; - public static final String LOG_REMOTE_COPY_LAG_MS_DOC = "Time-based delay before uploading segments. 0 = no delay, -1 = log.local.retention.ms, positive = min segment age (ms). Must not exceed log.local.retention.ms."; + public static final String LOG_REMOTE_COPY_LAG_MS_DOC = "Time-based delay before uploading segments. 0 = no delay, -1 = log.local.retention.ms (max delay), positive = min segment age (ms). Must not exceed log.local.retention.ms."; public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_MS = 0L; public static final String LOG_REMOTE_COPY_LAG_BYTES_PROP = "log.remote.copy.lag.bytes"; - public static final String LOG_REMOTE_COPY_LAG_BYTES_DOC = "Size-based delay before uploading segments. 0 = no delay, -1 = log.local.retention.bytes, positive = min bytes after segment. Must not exceed log.local.retention.bytes."; + public static final String LOG_REMOTE_COPY_LAG_BYTES_DOC = "Size-based delay before uploading segments. 0 = no delay, -1 = log.local.retention.bytes (max delay), positive = min bytes after segment. Must not exceed log.local.retention.bytes."; public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_BYTES = 0L; public static final String REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP = "remote.log.manager.copy.max.bytes.per.second"; From d99d7def3585d927704df6bc69869b60aff80b14 Mon Sep 17 00:00:00 2001 From: stroller Date: Fri, 6 Feb 2026 12:38:05 +0800 Subject: [PATCH 024/110] fix compile issue Signed-off-by: stroller --- .../scala/kafka/server/DynamicBrokerConfig.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index 8fc4453294147..650bc3a18937d 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -669,7 +669,7 @@ class DynamicLogConfig(logManager: LogManager) extends BrokerReconfigurable with // validation, no additional validation is performed. def validateLogLocalRetentionMs(): Unit = { - val logRetentionMs = newConfig.logRetentionTimeMillis + val logRetentionMs: Long = newConfig.logRetentionTimeMillis val logLocalRetentionMs: java.lang.Long = newConfig.remoteLogManagerConfig.logLocalRetentionMs if (logRetentionMs != -1L && logLocalRetentionMs != -2L) { if (logLocalRetentionMs == -1L) { @@ -684,10 +684,10 @@ class DynamicLogConfig(logManager: LogManager) extends BrokerReconfigurable with } def validateLogLocalRetentionBytes(): Unit = { - val logRetentionBytes = newConfig.logRetentionBytes + val logRetentionBytes: Long = newConfig.logRetentionBytes val logLocalRetentionBytes: java.lang.Long = newConfig.remoteLogManagerConfig.logLocalRetentionBytes - if (logRetentionBytes > -1 && logLocalRetentionBytes != -2) { - if (logLocalRetentionBytes == -1) { + if (logRetentionBytes > -1L && logLocalRetentionBytes != -2L) { + if (logLocalRetentionBytes == -1L) { throw new ConfigException(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP, logLocalRetentionBytes, s"Value must not be -1 as ${ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG} value is set as $logRetentionBytes.") } @@ -704,7 +704,7 @@ class DynamicLogConfig(logManager: LogManager) extends BrokerReconfigurable with } def validateRemoteCopyLagMsNotExceedingLocalRetention(): Unit = { - val logRetentionMs = newConfig.logRetentionTimeMillis + val logRetentionMs: Long = newConfig.logRetentionTimeMillis val logLocalRetentionMs = newConfig.remoteLogManagerConfig.logLocalRetentionMs val effectiveLocalRetentionMs = if (logLocalRetentionMs == -2L) logRetentionMs else logLocalRetentionMs val logRemoteCopyLagMs = newConfig.remoteLogManagerConfig.logRemoteCopyLagMs @@ -715,7 +715,7 @@ class DynamicLogConfig(logManager: LogManager) extends BrokerReconfigurable with } def validateRemoteCopyLagBytesNotExceedingLocalRetention(): Unit = { - val logRetentionBytes = newConfig.logRetentionBytes + val logRetentionBytes: Long = newConfig.logRetentionBytes val logLocalRetentionBytes = newConfig.remoteLogManagerConfig.logLocalRetentionBytes val effectiveLocalRetentionBytes = if (logLocalRetentionBytes == -2L) logRetentionBytes else logLocalRetentionBytes val logRemoteCopyLagBytes = newConfig.remoteLogManagerConfig.logRemoteCopyLagBytes From a5b6733dbba7b12ddf35050b0052f6393d993450 Mon Sep 17 00:00:00 2001 From: stroller Date: Fri, 6 Feb 2026 13:09:32 +0800 Subject: [PATCH 025/110] simple the code Signed-off-by: stroller --- .../kafka/server/DynamicBrokerConfig.scala | 26 +++++++++---------- 1 file changed, 12 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index 650bc3a18937d..ef416fbb35d96 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -704,24 +704,22 @@ class DynamicLogConfig(logManager: LogManager) extends BrokerReconfigurable with } def validateRemoteCopyLagMsNotExceedingLocalRetention(): Unit = { - val logRetentionMs: Long = newConfig.logRetentionTimeMillis - val logLocalRetentionMs = newConfig.remoteLogManagerConfig.logLocalRetentionMs - val effectiveLocalRetentionMs = if (logLocalRetentionMs == -2L) logRetentionMs else logLocalRetentionMs - val logRemoteCopyLagMs = newConfig.remoteLogManagerConfig.logRemoteCopyLagMs - if (logRemoteCopyLagMs > 0L && effectiveLocalRetentionMs > 0L && logRemoteCopyLagMs > effectiveLocalRetentionMs) { - throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_MS_PROP, logRemoteCopyLagMs, - s"Value must not exceed ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP} (effective value: $effectiveLocalRetentionMs)") + val copyLagMs = newConfig.remoteLogManagerConfig.logRemoteCopyLagMs + val localRetentionMs = newConfig.remoteLogManagerConfig.logLocalRetentionMs + val effectiveRetentionMs = if (localRetentionMs == -2L) newConfig.logRetentionTimeMillis else localRetentionMs + if (copyLagMs > 0L && effectiveRetentionMs > 0L && copyLagMs > effectiveRetentionMs) { + throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_MS_PROP, copyLagMs, + s"Value must not exceed ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP} (effective value: $effectiveRetentionMs)") } } def validateRemoteCopyLagBytesNotExceedingLocalRetention(): Unit = { - val logRetentionBytes: Long = newConfig.logRetentionBytes - val logLocalRetentionBytes = newConfig.remoteLogManagerConfig.logLocalRetentionBytes - val effectiveLocalRetentionBytes = if (logLocalRetentionBytes == -2L) logRetentionBytes else logLocalRetentionBytes - val logRemoteCopyLagBytes = newConfig.remoteLogManagerConfig.logRemoteCopyLagBytes - if (logRemoteCopyLagBytes > 0L && effectiveLocalRetentionBytes > 0L && logRemoteCopyLagBytes > effectiveLocalRetentionBytes) { - throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_BYTES_PROP, logRemoteCopyLagBytes, - s"Value must not exceed ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP} (effective value: $effectiveLocalRetentionBytes)") + val copyLagBytes = newConfig.remoteLogManagerConfig.logRemoteCopyLagBytes + val localRetentionBytes = newConfig.remoteLogManagerConfig.logLocalRetentionBytes + val effectiveRetentionBytes = if (localRetentionBytes == -2L) newConfig.logRetentionBytes else localRetentionBytes + if (copyLagBytes > 0L && effectiveRetentionBytes > 0L && copyLagBytes > effectiveRetentionBytes) { + throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_BYTES_PROP, copyLagBytes, + s"Value must not exceed ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP} (effective value: $effectiveRetentionBytes)") } } From 8b35773e84fc41758ec058050cbfb850eca305be Mon Sep 17 00:00:00 2001 From: stroller Date: Fri, 6 Feb 2026 13:10:35 +0800 Subject: [PATCH 026/110] Revert "simple the code" This reverts commit a5b6733dbba7b12ddf35050b0052f6393d993450. --- .../kafka/server/DynamicBrokerConfig.scala | 26 ++++++++++--------- 1 file changed, 14 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index ef416fbb35d96..650bc3a18937d 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -704,22 +704,24 @@ class DynamicLogConfig(logManager: LogManager) extends BrokerReconfigurable with } def validateRemoteCopyLagMsNotExceedingLocalRetention(): Unit = { - val copyLagMs = newConfig.remoteLogManagerConfig.logRemoteCopyLagMs - val localRetentionMs = newConfig.remoteLogManagerConfig.logLocalRetentionMs - val effectiveRetentionMs = if (localRetentionMs == -2L) newConfig.logRetentionTimeMillis else localRetentionMs - if (copyLagMs > 0L && effectiveRetentionMs > 0L && copyLagMs > effectiveRetentionMs) { - throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_MS_PROP, copyLagMs, - s"Value must not exceed ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP} (effective value: $effectiveRetentionMs)") + val logRetentionMs: Long = newConfig.logRetentionTimeMillis + val logLocalRetentionMs = newConfig.remoteLogManagerConfig.logLocalRetentionMs + val effectiveLocalRetentionMs = if (logLocalRetentionMs == -2L) logRetentionMs else logLocalRetentionMs + val logRemoteCopyLagMs = newConfig.remoteLogManagerConfig.logRemoteCopyLagMs + if (logRemoteCopyLagMs > 0L && effectiveLocalRetentionMs > 0L && logRemoteCopyLagMs > effectiveLocalRetentionMs) { + throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_MS_PROP, logRemoteCopyLagMs, + s"Value must not exceed ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP} (effective value: $effectiveLocalRetentionMs)") } } def validateRemoteCopyLagBytesNotExceedingLocalRetention(): Unit = { - val copyLagBytes = newConfig.remoteLogManagerConfig.logRemoteCopyLagBytes - val localRetentionBytes = newConfig.remoteLogManagerConfig.logLocalRetentionBytes - val effectiveRetentionBytes = if (localRetentionBytes == -2L) newConfig.logRetentionBytes else localRetentionBytes - if (copyLagBytes > 0L && effectiveRetentionBytes > 0L && copyLagBytes > effectiveRetentionBytes) { - throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_BYTES_PROP, copyLagBytes, - s"Value must not exceed ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP} (effective value: $effectiveRetentionBytes)") + val logRetentionBytes: Long = newConfig.logRetentionBytes + val logLocalRetentionBytes = newConfig.remoteLogManagerConfig.logLocalRetentionBytes + val effectiveLocalRetentionBytes = if (logLocalRetentionBytes == -2L) logRetentionBytes else logLocalRetentionBytes + val logRemoteCopyLagBytes = newConfig.remoteLogManagerConfig.logRemoteCopyLagBytes + if (logRemoteCopyLagBytes > 0L && effectiveLocalRetentionBytes > 0L && logRemoteCopyLagBytes > effectiveLocalRetentionBytes) { + throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_BYTES_PROP, logRemoteCopyLagBytes, + s"Value must not exceed ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP} (effective value: $effectiveLocalRetentionBytes)") } } From 9ea9ac2491f0c973436fa9851fb430b554724855 Mon Sep 17 00:00:00 2001 From: Jian Date: Sun, 8 Feb 2026 05:19:44 +0800 Subject: [PATCH 027/110] Don't change other codes --- .../src/main/scala/kafka/server/DynamicBrokerConfig.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index 650bc3a18937d..39c1ba64f6736 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -669,7 +669,7 @@ class DynamicLogConfig(logManager: LogManager) extends BrokerReconfigurable with // validation, no additional validation is performed. def validateLogLocalRetentionMs(): Unit = { - val logRetentionMs: Long = newConfig.logRetentionTimeMillis + val logRetentionMs = newConfig.logRetentionTimeMillis val logLocalRetentionMs: java.lang.Long = newConfig.remoteLogManagerConfig.logLocalRetentionMs if (logRetentionMs != -1L && logLocalRetentionMs != -2L) { if (logLocalRetentionMs == -1L) { @@ -684,10 +684,10 @@ class DynamicLogConfig(logManager: LogManager) extends BrokerReconfigurable with } def validateLogLocalRetentionBytes(): Unit = { - val logRetentionBytes: Long = newConfig.logRetentionBytes + val logRetentionBytes = newConfig.logRetentionBytes val logLocalRetentionBytes: java.lang.Long = newConfig.remoteLogManagerConfig.logLocalRetentionBytes - if (logRetentionBytes > -1L && logLocalRetentionBytes != -2L) { - if (logLocalRetentionBytes == -1L) { + if (logRetentionBytes > -1 && logLocalRetentionBytes != -2) { + if (logLocalRetentionBytes == -1) { throw new ConfigException(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP, logLocalRetentionBytes, s"Value must not be -1 as ${ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG} value is set as $logRetentionBytes.") } From 7eefc4c29dd1311e54b45fe69bee419a8349106a Mon Sep 17 00:00:00 2001 From: Jian Date: Sun, 8 Feb 2026 05:22:31 +0800 Subject: [PATCH 028/110] Keep the same style --- .../scala/kafka/server/DynamicBrokerConfig.scala | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index 39c1ba64f6736..29bae21a8734e 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -698,12 +698,7 @@ class DynamicLogConfig(logManager: LogManager) extends BrokerReconfigurable with } } - def validateRemoteCopyLagNotExceedingLocalRetention(): Unit = { - validateRemoteCopyLagMsNotExceedingLocalRetention() - validateRemoteCopyLagBytesNotExceedingLocalRetention() - } - - def validateRemoteCopyLagMsNotExceedingLocalRetention(): Unit = { + def validateLogRemoteCopyLagMs(): Unit = { val logRetentionMs: Long = newConfig.logRetentionTimeMillis val logLocalRetentionMs = newConfig.remoteLogManagerConfig.logLocalRetentionMs val effectiveLocalRetentionMs = if (logLocalRetentionMs == -2L) logRetentionMs else logLocalRetentionMs @@ -714,7 +709,7 @@ class DynamicLogConfig(logManager: LogManager) extends BrokerReconfigurable with } } - def validateRemoteCopyLagBytesNotExceedingLocalRetention(): Unit = { + def validateLogRemoteCopyLagBytes(): Unit = { val logRetentionBytes: Long = newConfig.logRetentionBytes val logLocalRetentionBytes = newConfig.remoteLogManagerConfig.logLocalRetentionBytes val effectiveLocalRetentionBytes = if (logLocalRetentionBytes == -2L) logRetentionBytes else logLocalRetentionBytes @@ -727,7 +722,8 @@ class DynamicLogConfig(logManager: LogManager) extends BrokerReconfigurable with validateLogLocalRetentionMs() validateLogLocalRetentionBytes() - validateRemoteCopyLagNotExceedingLocalRetention() + validateLogRemoteCopyLagMs() + validateLogRemoteCopyLagBytes() } private def updateLogsConfig(newBrokerDefaults: Map[String, Object]): Unit = { From f0adf67ded0cd333f4dfec908b7303d1d91e54cc Mon Sep 17 00:00:00 2001 From: Jian Date: Sun, 8 Feb 2026 05:28:55 +0800 Subject: [PATCH 029/110] code refactor Signed-off-by: Jian --- .../kafka/server/log/remote/storage/RemoteLogManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index 610fe2e9552c4..95e0a78fe1945 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -962,9 +962,9 @@ List candidateLogSegments(UnifiedLog log, Long fromOffset, L LogSegment previousSeg = segments.get(idx - 1); LogSegment currentSeg = segments.get(idx); if (currentSeg.baseOffset() <= lastStableOffset) { - cumulativeSize += previousSeg.size(); if (copyLagMs != 0 && !hasExceededCopyLagTime(previousSeg, currentTimeMs, copyLagMs)) break; + cumulativeSize += previousSeg.size(); if (copyLagBytes != 0 && !hasExceededCopyLagSize(totalLogSize, cumulativeSize, copyLagBytes)) break; candidateLogSegments.add(new EnrichedLogSegment(previousSeg, currentSeg.baseOffset())); From 3d148560c746e5b53575c301b4fee9b548849512 Mon Sep 17 00:00:00 2001 From: Jian Date: Sun, 8 Feb 2026 05:33:25 +0800 Subject: [PATCH 030/110] Refactor remote copy lag validation methods --- .../kafka/storage/internals/log/LogConfig.java | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 21fb9c70a015e..cccad6727f200 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -530,7 +530,8 @@ private static void validateTopicLogConfigValues(Map existingCon validateRemoteStorageRequiresDeleteCleanupPolicy(newConfigs); validateRemoteStorageRetentionSize(newConfigs); validateRemoteStorageRetentionTime(newConfigs); - validateRemoteCopyLagNotExceedingLocalRetention(newConfigs); + validateRemoteCopyLagSize(newConfigs); + validateRemoteCopyLagTime(newConfigs); validateRetentionConfigsWhenRemoteCopyDisabled(newConfigs, isRemoteLogStorageEnabled); } else { // The new config "remote.storage.enable" is false, validate if it's turning from true to false @@ -620,12 +621,7 @@ private static void validateRemoteStorageRetentionTime(Map props) { } } - private static void validateRemoteCopyLagNotExceedingLocalRetention(Map props) { - validateRemoteCopyLagMsNotExceedingLocalRetention(props); - validateRemoteCopyLagBytesNotExceedingLocalRetention(props); - } - - private static void validateRemoteCopyLagMsNotExceedingLocalRetention(Map props) { + private static void validateRemoteCopyLagTime(Map props) { Long retentionMs = (Long) props.get(TopicConfig.RETENTION_MS_CONFIG); Long localRetentionMs = (Long) props.get(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG); Long remoteCopyLagMs = (Long) props.get(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG); @@ -639,7 +635,7 @@ private static void validateRemoteCopyLagMsNotExceedingLocalRetention(Map } } - private static void validateRemoteCopyLagBytesNotExceedingLocalRetention(Map props) { + private static void validateRemoteCopyLagSize(Map props) { Long retentionBytes = (Long) props.get(TopicConfig.RETENTION_BYTES_CONFIG); Long localRetentionBytes = (Long) props.get(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG); Long remoteCopyLagBytes = (Long) props.get(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG); From b6c53b6f1d42aee30fbeb802ec205f1cad1af332 Mon Sep 17 00:00:00 2001 From: Jian Date: Sun, 8 Feb 2026 10:32:13 +0800 Subject: [PATCH 031/110] define the constants value Signed-off-by: Jian --- .../apache/kafka/storage/internals/log/LogConfig.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index cccad6727f200..c429ae168c172 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -144,8 +144,10 @@ public Optional serverConfigName(String configName) { public static final boolean DEFAULT_REMOTE_LOG_DELETE_ON_DISABLE_CONFIG = false; public static final long DEFAULT_LOCAL_RETENTION_BYTES = -2; // It indicates the value to be derived from RetentionBytes public static final long DEFAULT_LOCAL_RETENTION_MS = -2; // It indicates the value to be derived from RetentionMs - public static final long DEFAULT_REMOTE_COPY_LAG_MS = 0L; - public static final long DEFAULT_REMOTE_COPY_LAG_BYTES = 0L; + public static final long DEFAULT_REMOTE_COPY_LAG_MS = 0; + public static final long DEFAULT_REMOTE_COPY_LAG_BYTES = 0; + public static final long MAX_REMOTE_COPY_LAG_MS = -1; // It indicates the value to be derived from localRetentionBytes + public static final long MAX_REMOTE_COPY_LAG_BYTES = -1; // It indicates the value to be derived from localRetentionMs public static final String INTERNAL_SEGMENT_BYTES_CONFIG = "internal.segment.bytes"; public static final String INTERNAL_SEGMENT_BYTES_DOC = "The maximum size of a single log file. This should be used for testing only."; @@ -410,11 +412,11 @@ public Boolean remoteLogCopyDisable() { public long remoteCopyLagMs() { - return remoteLogConfig.remoteCopyLagMs == -1 ? localRetentionMs() : remoteLogConfig.remoteCopyLagMs; + return remoteLogConfig.remoteCopyLagMs == MAX_REMOTE_COPY_LAG_MS ? localRetentionMs() : remoteLogConfig.remoteCopyLagMs; } public long remoteCopyLagBytes() { - return remoteLogConfig.remoteCopyLagBytes == -1 ? localRetentionBytes() : remoteLogConfig.remoteCopyLagBytes; + return remoteLogConfig.remoteCopyLagBytes == MAX_REMOTE_COPY_LAG_BYTES ? localRetentionBytes() : remoteLogConfig.remoteCopyLagBytes; } public long localRetentionMs() { From e19efef3a588eb476bc01e5511c71e6dc4adca6d Mon Sep 17 00:00:00 2001 From: Jian Date: Sun, 8 Feb 2026 10:33:24 +0800 Subject: [PATCH 032/110] Update comments for MAX_REMOTE_COPY_LAG constants --- .../org/apache/kafka/storage/internals/log/LogConfig.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index c429ae168c172..b73b6cda6b2e6 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -146,8 +146,8 @@ public Optional serverConfigName(String configName) { public static final long DEFAULT_LOCAL_RETENTION_MS = -2; // It indicates the value to be derived from RetentionMs public static final long DEFAULT_REMOTE_COPY_LAG_MS = 0; public static final long DEFAULT_REMOTE_COPY_LAG_BYTES = 0; - public static final long MAX_REMOTE_COPY_LAG_MS = -1; // It indicates the value to be derived from localRetentionBytes - public static final long MAX_REMOTE_COPY_LAG_BYTES = -1; // It indicates the value to be derived from localRetentionMs + public static final long MAX_REMOTE_COPY_LAG_MS = -1; // It indicates the value to be derived from localRetentionMs + public static final long MAX_REMOTE_COPY_LAG_BYTES = -1; // It indicates the value to be derived from localRetentionBytes public static final String INTERNAL_SEGMENT_BYTES_CONFIG = "internal.segment.bytes"; public static final String INTERNAL_SEGMENT_BYTES_DOC = "The maximum size of a single log file. This should be used for testing only."; From dea59a38a3e2615b040fbbc3e3b8b7da67c11b14 Mon Sep 17 00:00:00 2001 From: Jian Date: Sun, 8 Feb 2026 10:59:40 +0800 Subject: [PATCH 033/110] refactor Signed-off-by: Jian --- .../log/remote/storage/RemoteLogManager.java | 29 ++++++++----------- 1 file changed, 12 insertions(+), 17 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index 95e0a78fe1945..2312119a468d5 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -909,30 +909,25 @@ private void maybeUpdateCopiedOffset(UnifiedLog log) throws RemoteStorageExcepti private boolean hasExceededCopyLagTime(LogSegment segment, long currentTimeMs, long copyLagMs) { try { - long ts = segment.largestTimestamp(); - if (ts >= 0) { - long segmentAgeMs = currentTimeMs - ts; - boolean exceeded = segmentAgeMs >= copyLagMs; - if (!exceeded) { - logger.debug("Segment {} not eligible for upload: segment age {} ms < copy lag {} ms", - segment.baseOffset(), segmentAgeMs, copyLagMs); - } - return exceeded; + long segmentAgeMs = currentTimeMs - segment.largestTimestamp(); + boolean exceeded = segmentAgeMs >= copyLagMs; + if (!exceeded) { + logger.debug("Segment {} not eligible for upload: segment age {} ms < copy lag {} ms", + segment.baseOffset(), segmentAgeMs, copyLagMs); } + return exceeded; } catch (IOException e) { - logger.warn("Failed to get largest timestamp for segment {}, will not skip based on time", segment, e); + logger.warn("Failed to get largest timestamp for segment {}, take it as eligible for upload based on time", segment, e); + return true; } - logger.debug("Segment {} not eligible for upload: cannot determine segment age (invalid timestamp)", segment.baseOffset()); - return false; } - - private boolean hasExceededCopyLagSize(long totalLogSize, long cumulativeSize, long copyLagBytes) { + private boolean hasExceededCopyLagSize(LogSegment segment, long totalLogSize, long cumulativeSize, long copyLagBytes) { long sizeLagBytes = totalLogSize - cumulativeSize; boolean exceeded = sizeLagBytes >= copyLagBytes; if (!exceeded) { - logger.debug("Segment not eligible for upload: size lag {} bytes < copy lag {} bytes (totalLogSize={}, cumulativeSize={})", - sizeLagBytes, copyLagBytes, totalLogSize, cumulativeSize); + logger.debug("Segment {} not eligible for upload: size lag {} bytes < copy lag {} bytes (totalLogSize={}, cumulativeSize={})", + segment, sizeLagBytes, copyLagBytes, totalLogSize, cumulativeSize); } return exceeded; } @@ -965,7 +960,7 @@ List candidateLogSegments(UnifiedLog log, Long fromOffset, L if (copyLagMs != 0 && !hasExceededCopyLagTime(previousSeg, currentTimeMs, copyLagMs)) break; cumulativeSize += previousSeg.size(); - if (copyLagBytes != 0 && !hasExceededCopyLagSize(totalLogSize, cumulativeSize, copyLagBytes)) + if (copyLagBytes != 0 && !hasExceededCopyLagSize(previousSeg, totalLogSize, cumulativeSize, copyLagBytes)) break; candidateLogSegments.add(new EnrichedLogSegment(previousSeg, currentSeg.baseOffset())); } From b2d2079d7af19553b33d101334666341035c6fe0 Mon Sep 17 00:00:00 2001 From: Jian Date: Sun, 8 Feb 2026 11:06:17 +0800 Subject: [PATCH 034/110] Code format --- .../kafka/server/log/remote/storage/RemoteLogManager.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index 2312119a468d5..58a2391d15bf0 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -947,9 +947,10 @@ List candidateLogSegments(UnifiedLog log, Long fromOffset, L List candidateLogSegments = new ArrayList<>(); List segments = log.logSegments(fromOffset, Long.MAX_VALUE); if (!segments.isEmpty()) { - long currentTimeMs = time.milliseconds(); long copyLagMs = log.config() != null ? log.config().remoteCopyLagMs() : 0L; long copyLagBytes = log.config() != null ? log.config().remoteCopyLagBytes() : 0L; + + long currentTimeMs = time.milliseconds(); long totalLogSize = log.size(); long cumulativeSize = 0; @@ -959,9 +960,11 @@ List candidateLogSegments(UnifiedLog log, Long fromOffset, L if (currentSeg.baseOffset() <= lastStableOffset) { if (copyLagMs != 0 && !hasExceededCopyLagTime(previousSeg, currentTimeMs, copyLagMs)) break; + cumulativeSize += previousSeg.size(); if (copyLagBytes != 0 && !hasExceededCopyLagSize(previousSeg, totalLogSize, cumulativeSize, copyLagBytes)) break; + candidateLogSegments.add(new EnrichedLogSegment(previousSeg, currentSeg.baseOffset())); } } From cf80f85e672f73e4e474de880532b3cb02b15d4f Mon Sep 17 00:00:00 2001 From: Jian Date: Sun, 8 Feb 2026 11:17:01 +0800 Subject: [PATCH 035/110] correct the total size calculate Signed-off-by: Jian --- .../kafka/server/log/remote/storage/RemoteLogManager.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index 58a2391d15bf0..b1a80902d2f80 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -912,8 +912,8 @@ private boolean hasExceededCopyLagTime(LogSegment segment, long currentTimeMs, l long segmentAgeMs = currentTimeMs - segment.largestTimestamp(); boolean exceeded = segmentAgeMs >= copyLagMs; if (!exceeded) { - logger.debug("Segment {} not eligible for upload: segment age {} ms < copy lag {} ms", - segment.baseOffset(), segmentAgeMs, copyLagMs); + logger.debug("{} not eligible for upload: segment age {} ms < copy lag {} ms", + segment, segmentAgeMs, copyLagMs); } return exceeded; } catch (IOException e) { @@ -926,7 +926,7 @@ private boolean hasExceededCopyLagSize(LogSegment segment, long totalLogSize, lo long sizeLagBytes = totalLogSize - cumulativeSize; boolean exceeded = sizeLagBytes >= copyLagBytes; if (!exceeded) { - logger.debug("Segment {} not eligible for upload: size lag {} bytes < copy lag {} bytes (totalLogSize={}, cumulativeSize={})", + logger.debug("{} not eligible for upload: size lag {} bytes < copy lag {} bytes (totalLogSize={}, cumulativeSize={})", segment, sizeLagBytes, copyLagBytes, totalLogSize, cumulativeSize); } return exceeded; @@ -951,7 +951,7 @@ List candidateLogSegments(UnifiedLog log, Long fromOffset, L long copyLagBytes = log.config() != null ? log.config().remoteCopyLagBytes() : 0L; long currentTimeMs = time.milliseconds(); - long totalLogSize = log.size(); + long totalLogSize = UnifiedLog.sizeInBytes(segments); long cumulativeSize = 0; for (int idx = 1; idx < segments.size(); idx++) { From a747a1acb81d45b7b3200702a9b8e6df0d86de02 Mon Sep 17 00:00:00 2001 From: Jian Date: Sun, 8 Feb 2026 11:24:53 +0800 Subject: [PATCH 036/110] refactor Signed-off-by: Jian --- .../server/log/remote/storage/RemoteLogManager.java | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index b1a80902d2f80..6acf78d404775 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -911,10 +911,8 @@ private boolean hasExceededCopyLagTime(LogSegment segment, long currentTimeMs, l try { long segmentAgeMs = currentTimeMs - segment.largestTimestamp(); boolean exceeded = segmentAgeMs >= copyLagMs; - if (!exceeded) { - logger.debug("{} not eligible for upload: segment age {} ms < copy lag {} ms", - segment, segmentAgeMs, copyLagMs); - } + logger.debug("{} eligible for upload by time? {} (segment age {} ms, copy lag {} ms)", + segment, exceeded, segmentAgeMs, copyLagMs); return exceeded; } catch (IOException e) { logger.warn("Failed to get largest timestamp for segment {}, take it as eligible for upload based on time", segment, e); @@ -925,10 +923,8 @@ private boolean hasExceededCopyLagTime(LogSegment segment, long currentTimeMs, l private boolean hasExceededCopyLagSize(LogSegment segment, long totalLogSize, long cumulativeSize, long copyLagBytes) { long sizeLagBytes = totalLogSize - cumulativeSize; boolean exceeded = sizeLagBytes >= copyLagBytes; - if (!exceeded) { - logger.debug("{} not eligible for upload: size lag {} bytes < copy lag {} bytes (totalLogSize={}, cumulativeSize={})", - segment, sizeLagBytes, copyLagBytes, totalLogSize, cumulativeSize); - } + logger.debug("{} eligible for upload by size? {} (size lag {} bytes, copy lag {} bytes, totalLogSize={}, cumulativeSize={})", + segment, exceeded, sizeLagBytes, copyLagBytes, totalLogSize, cumulativeSize); return exceeded; } From 71cdaf91f25a5ffc00139174d46988d6f26f7011 Mon Sep 17 00:00:00 2001 From: Jian Date: Tue, 10 Feb 2026 16:22:30 +0800 Subject: [PATCH 037/110] Clarify segment upload eligibility in documentation Updated documentation for REMOTE_COPY_LAG_MS_DOC to clarify segment eligibility for upload. --- .../main/java/org/apache/kafka/common/config/TopicConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index b44d0a27bed6a..2425a5912c2ba 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -105,7 +105,7 @@ public class TopicConfig { public static final String REMOTE_COPY_LAG_MS_DOC = "Controls how long to delay uploading segments to remote storage. " + "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay). " + "When set to -1, resolves to local.retention.ms (maximum delay). " + - "When set to a positive value (ms), a segment is eligible for upload only after it has been closed for at least this many milliseconds. " + + "When set to a positive value (ms), a segment is eligible for upload only after it has lived for at least this many milliseconds. " + "The value should not exceed local.retention.ms (local retention time)."; public static final String REMOTE_COPY_LAG_BYTES_CONFIG = "remote.copy.lag.bytes"; From b5ad7986fe9a93b5ee33537928168dfbfb7411ab Mon Sep 17 00:00:00 2001 From: Jian Date: Tue, 10 Feb 2026 17:57:33 +0800 Subject: [PATCH 038/110] Update documentation for REMOTE_COPY_LAG_MS_CONFIG --- .../main/java/org/apache/kafka/common/config/TopicConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index 2425a5912c2ba..b44d0a27bed6a 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -105,7 +105,7 @@ public class TopicConfig { public static final String REMOTE_COPY_LAG_MS_DOC = "Controls how long to delay uploading segments to remote storage. " + "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay). " + "When set to -1, resolves to local.retention.ms (maximum delay). " + - "When set to a positive value (ms), a segment is eligible for upload only after it has lived for at least this many milliseconds. " + + "When set to a positive value (ms), a segment is eligible for upload only after it has been closed for at least this many milliseconds. " + "The value should not exceed local.retention.ms (local retention time)."; public static final String REMOTE_COPY_LAG_BYTES_CONFIG = "remote.copy.lag.bytes"; From eaa2e40a6644711d235a7648781c9fa1f6c0b949 Mon Sep 17 00:00:00 2001 From: Jian Date: Tue, 10 Feb 2026 21:09:50 +0800 Subject: [PATCH 039/110] Update documentation for remote copy lag configurations --- .../java/org/apache/kafka/common/config/TopicConfig.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index b44d0a27bed6a..588648a1b283a 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -105,14 +105,14 @@ public class TopicConfig { public static final String REMOTE_COPY_LAG_MS_DOC = "Controls how long to delay uploading segments to remote storage. " + "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay). " + "When set to -1, resolves to local.retention.ms (maximum delay). " + - "When set to a positive value (ms), a segment is eligible for upload only after it has been closed for at least this many milliseconds. " + + "When set to a positive value (ms), a segment becomes eligible for upload only after the time since the latest record in the segment reaches the value. " + "The value should not exceed local.retention.ms (local retention time)."; public static final String REMOTE_COPY_LAG_BYTES_CONFIG = "remote.copy.lag.bytes"; public static final String REMOTE_COPY_LAG_BYTES_DOC = "Controls size-based delay for uploading segments to remote storage. " + - "When set to 0 (default), segments are uploaded as soon as they are eligible (no size-based constraint). " + + "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay). " + "When set to -1, resolves to local.retention.bytes (maximum delay). " + - "When set to a positive value (bytes), a segment is eligible for upload only when at least this many bytes of log data exist after the segment. " + + "When set to a positive value (bytes), a segment becomes eligible for upload only when the bytes of log data after the segment reach the value. " + "The value should not exceed local.retention.bytes (local retention size)."; public static final String REMOTE_LOG_DELETE_ON_DISABLE_CONFIG = "remote.log.delete.on.disable"; From 4ffe31f63b961aec4b463d8c059dbbdb134f33fb Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 13 Feb 2026 08:19:47 +0800 Subject: [PATCH 040/110] Improve the log --- .../main/java/org/apache/kafka/common/config/TopicConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index 588648a1b283a..7eb552b97489c 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -112,7 +112,7 @@ public class TopicConfig { public static final String REMOTE_COPY_LAG_BYTES_DOC = "Controls size-based delay for uploading segments to remote storage. " + "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay). " + "When set to -1, resolves to local.retention.bytes (maximum delay). " + - "When set to a positive value (bytes), a segment becomes eligible for upload only when the bytes of log data after the segment reach the value. " + + "When set to a positive value (bytes), a segment becomes eligible for upload only when the total bytes of log data after the segment reach the value. " + "The value should not exceed local.retention.bytes (local retention size)."; public static final String REMOTE_LOG_DELETE_ON_DISABLE_CONFIG = "remote.log.delete.on.disable"; From 7ee6b12c4efad65193d8300314cb25c0101c1ea1 Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 13 Feb 2026 08:52:00 +0800 Subject: [PATCH 041/110] Update retention checks to allow zero values --- core/src/main/scala/kafka/server/DynamicBrokerConfig.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index 63c57aded3ce1..87630cc9c07d3 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -582,7 +582,7 @@ class DynamicLogConfig(logManager: LogManager) extends BrokerReconfigurable with val logLocalRetentionMs = newConfig.remoteLogManagerConfig.logLocalRetentionMs val effectiveLocalRetentionMs = if (logLocalRetentionMs == -2L) logRetentionMs else logLocalRetentionMs val logRemoteCopyLagMs = newConfig.remoteLogManagerConfig.logRemoteCopyLagMs - if (logRemoteCopyLagMs > 0L && effectiveLocalRetentionMs > 0L && logRemoteCopyLagMs > effectiveLocalRetentionMs) { + if (logRemoteCopyLagMs > 0L && effectiveLocalRetentionMs >= 0L && logRemoteCopyLagMs > effectiveLocalRetentionMs) { throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_MS_PROP, logRemoteCopyLagMs, s"Value must not exceed ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP} (effective value: $effectiveLocalRetentionMs)") } @@ -593,7 +593,7 @@ class DynamicLogConfig(logManager: LogManager) extends BrokerReconfigurable with val logLocalRetentionBytes = newConfig.remoteLogManagerConfig.logLocalRetentionBytes val effectiveLocalRetentionBytes = if (logLocalRetentionBytes == -2L) logRetentionBytes else logLocalRetentionBytes val logRemoteCopyLagBytes = newConfig.remoteLogManagerConfig.logRemoteCopyLagBytes - if (logRemoteCopyLagBytes > 0L && effectiveLocalRetentionBytes > 0L && logRemoteCopyLagBytes > effectiveLocalRetentionBytes) { + if (logRemoteCopyLagBytes > 0L && effectiveLocalRetentionBytes >= 0L && logRemoteCopyLagBytes > effectiveLocalRetentionBytes) { throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_BYTES_PROP, logRemoteCopyLagBytes, s"Value must not exceed ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP} (effective value: $effectiveLocalRetentionBytes)") } From b3b1d130f09379a073a6e1b00574d7cf0c30f4ac Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 13 Feb 2026 08:58:38 +0800 Subject: [PATCH 042/110] Fix comparison for remote copy lag checks --- .../org/apache/kafka/storage/internals/log/LogConfig.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 1680e081c1582..e4813db3c8c94 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -629,7 +629,7 @@ private static void validateRemoteCopyLagTime(Map props) { Long remoteCopyLagMs = (Long) props.get(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG); long effectiveLocalRetentionMs = (localRetentionMs != null && localRetentionMs == -2 && retentionMs != null) ? retentionMs : (localRetentionMs != null ? localRetentionMs : -2); - if (remoteCopyLagMs != null && remoteCopyLagMs > 0 && effectiveLocalRetentionMs > 0 + if (remoteCopyLagMs != null && remoteCopyLagMs > 0 && effectiveLocalRetentionMs >= 0 && remoteCopyLagMs > effectiveLocalRetentionMs) { String message = String.format("Value must not exceed %s (effective value: %d)", TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, effectiveLocalRetentionMs); @@ -643,7 +643,7 @@ private static void validateRemoteCopyLagSize(Map props) { Long remoteCopyLagBytes = (Long) props.get(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG); long effectiveLocalRetentionBytes = (localRetentionBytes != null && localRetentionBytes == -2 && retentionBytes != null) ? retentionBytes : (localRetentionBytes != null ? localRetentionBytes : -2); - if (remoteCopyLagBytes != null && remoteCopyLagBytes > 0 && effectiveLocalRetentionBytes > 0 + if (remoteCopyLagBytes != null && remoteCopyLagBytes > 0 && effectiveLocalRetentionBytes >= 0 && remoteCopyLagBytes > effectiveLocalRetentionBytes) { String message = String.format("Value must not exceed %s (effective value: %d)", TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, effectiveLocalRetentionBytes); From a6bc5526c9b451ed94993b4bf54fcdbed57a49a6 Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 13 Feb 2026 09:31:18 +0800 Subject: [PATCH 043/110] Fix conditions for copy lag checks in RemoteLogManager --- .../kafka/server/log/remote/storage/RemoteLogManager.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index b0f3c28ce83e7..47ebb85725c4b 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -954,11 +954,11 @@ List candidateLogSegments(UnifiedLog log, Long fromOffset, L LogSegment previousSeg = segments.get(idx - 1); LogSegment currentSeg = segments.get(idx); if (currentSeg.baseOffset() <= lastStableOffset) { - if (copyLagMs != 0 && !hasExceededCopyLagTime(previousSeg, currentTimeMs, copyLagMs)) + if (copyLagMs > 0 && !hasExceededCopyLagTime(previousSeg, currentTimeMs, copyLagMs)) break; cumulativeSize += previousSeg.size(); - if (copyLagBytes != 0 && !hasExceededCopyLagSize(previousSeg, totalLogSize, cumulativeSize, copyLagBytes)) + if (copyLagBytes > 0 && !hasExceededCopyLagSize(previousSeg, totalLogSize, cumulativeSize, copyLagBytes)) break; candidateLogSegments.add(new EnrichedLogSegment(previousSeg, currentSeg.baseOffset())); From 1761b0a9ced34b0c90658a585a0b57f1eca9f4a8 Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 13 Feb 2026 09:59:22 +0800 Subject: [PATCH 044/110] refactor document Signed-off-by: Jian --- .../java/org/apache/kafka/common/config/TopicConfig.java | 8 ++++---- .../server/log/remote/storage/RemoteLogManagerConfig.java | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index 7eb552b97489c..255cd38ddecef 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -104,16 +104,16 @@ public class TopicConfig { public static final String REMOTE_COPY_LAG_MS_CONFIG = "remote.copy.lag.ms"; public static final String REMOTE_COPY_LAG_MS_DOC = "Controls how long to delay uploading segments to remote storage. " + "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay). " + - "When set to -1, resolves to local.retention.ms (maximum delay). " + + "When set to -1, the effective value is local.retention.ms (maximum delay, or upload immediately when local.retention.ms is -1). " + "When set to a positive value (ms), a segment becomes eligible for upload only after the time since the latest record in the segment reaches the value. " + - "The value should not exceed local.retention.ms (local retention time)."; + "The value should not exceed the effective local.retention.ms (local retention time)."; public static final String REMOTE_COPY_LAG_BYTES_CONFIG = "remote.copy.lag.bytes"; public static final String REMOTE_COPY_LAG_BYTES_DOC = "Controls size-based delay for uploading segments to remote storage. " + "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay). " + - "When set to -1, resolves to local.retention.bytes (maximum delay). " + + "When set to -1, the effective value is local.retention.bytes (maximum delay, or upload immediately when local.retention.bytes is -1). " + "When set to a positive value (bytes), a segment becomes eligible for upload only when the total bytes of log data after the segment reach the value. " + - "The value should not exceed local.retention.bytes (local retention size)."; + "The value should not exceed the effective local.retention.bytes (local retention size)."; public static final String REMOTE_LOG_DELETE_ON_DISABLE_CONFIG = "remote.log.delete.on.disable"; public static final String REMOTE_LOG_DELETE_ON_DISABLE_DOC = "Determines whether tiered data for a topic should be " + diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java index 4a84191a813d9..0d5286e21d701 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java @@ -169,11 +169,11 @@ public final class RemoteLogManagerConfig { public static final Long DEFAULT_LOG_LOCAL_RETENTION_BYTES = -2L; public static final String LOG_REMOTE_COPY_LAG_MS_PROP = "log.remote.copy.lag.ms"; - public static final String LOG_REMOTE_COPY_LAG_MS_DOC = "Time-based delay before uploading segments. 0 = no delay, -1 = log.local.retention.ms (max delay), positive = min segment age (ms). Must not exceed log.local.retention.ms."; + public static final String LOG_REMOTE_COPY_LAG_MS_DOC = "Time-based delay before uploading segments. 0 = no delay. -1 = effective value is log.local.retention.ms (max delay, or upload immediately when log.local.retention.ms is -1). Positive = segment becomes eligible only after the time since the latest record in the segment reaches the value (ms). Must not exceed the effective log.local.retention.ms."; public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_MS = 0L; public static final String LOG_REMOTE_COPY_LAG_BYTES_PROP = "log.remote.copy.lag.bytes"; - public static final String LOG_REMOTE_COPY_LAG_BYTES_DOC = "Size-based delay before uploading segments. 0 = no delay, -1 = log.local.retention.bytes (max delay), positive = min bytes after segment. Must not exceed log.local.retention.bytes."; + public static final String LOG_REMOTE_COPY_LAG_BYTES_DOC = "Size-based delay before uploading segments. 0 = no delay. -1 = effective value is log.local.retention.bytes (max delay, or upload immediately when log.local.retention.bytes is -1). Positive = segment becomes eligible only when the bytes of log data after the segment reach the value. Must not exceed the effective log.local.retention.bytes."; public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_BYTES = 0L; public static final String REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP = "remote.log.manager.copy.max.bytes.per.second"; From 31478c5c8d9ec46b9f00576c389ae6f448958f6b Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 13 Feb 2026 10:08:44 +0800 Subject: [PATCH 045/110] refactor document Signed-off-by: Jian --- .../java/org/apache/kafka/common/config/TopicConfig.java | 8 ++++---- .../server/log/remote/storage/RemoteLogManagerConfig.java | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index 255cd38ddecef..7788785915de1 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -104,16 +104,16 @@ public class TopicConfig { public static final String REMOTE_COPY_LAG_MS_CONFIG = "remote.copy.lag.ms"; public static final String REMOTE_COPY_LAG_MS_DOC = "Controls how long to delay uploading segments to remote storage. " + "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay). " + - "When set to -1, the effective value is local.retention.ms (maximum delay, or upload immediately when local.retention.ms is -1). " + + "When set to -1, resolves to the effective local.retention.ms (maximum delay, or upload immediately when local.retention.ms is -1). " + "When set to a positive value (ms), a segment becomes eligible for upload only after the time since the latest record in the segment reaches the value. " + - "The value should not exceed the effective local.retention.ms (local retention time)."; + "The value should not exceed the effective local.retention.ms when that value is non-negative."; public static final String REMOTE_COPY_LAG_BYTES_CONFIG = "remote.copy.lag.bytes"; public static final String REMOTE_COPY_LAG_BYTES_DOC = "Controls size-based delay for uploading segments to remote storage. " + "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay). " + - "When set to -1, the effective value is local.retention.bytes (maximum delay, or upload immediately when local.retention.bytes is -1). " + + "When set to -1, resolves to the effective local.retention.bytes (maximum delay, or upload immediately when local.retention.bytes is -1). " + "When set to a positive value (bytes), a segment becomes eligible for upload only when the total bytes of log data after the segment reach the value. " + - "The value should not exceed the effective local.retention.bytes (local retention size)."; + "The value should not exceed the effective local.retention.bytes when that value is non-negative."; public static final String REMOTE_LOG_DELETE_ON_DISABLE_CONFIG = "remote.log.delete.on.disable"; public static final String REMOTE_LOG_DELETE_ON_DISABLE_DOC = "Determines whether tiered data for a topic should be " + diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java index 0d5286e21d701..bb52c20d84339 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java @@ -169,11 +169,11 @@ public final class RemoteLogManagerConfig { public static final Long DEFAULT_LOG_LOCAL_RETENTION_BYTES = -2L; public static final String LOG_REMOTE_COPY_LAG_MS_PROP = "log.remote.copy.lag.ms"; - public static final String LOG_REMOTE_COPY_LAG_MS_DOC = "Time-based delay before uploading segments. 0 = no delay. -1 = effective value is log.local.retention.ms (max delay, or upload immediately when log.local.retention.ms is -1). Positive = segment becomes eligible only after the time since the latest record in the segment reaches the value (ms). Must not exceed the effective log.local.retention.ms."; + public static final String LOG_REMOTE_COPY_LAG_MS_DOC = "Time-based delay before uploading segments. 0 = no delay. -1 = resolves to the effective log.local.retention.ms (max delay, or upload immediately when log.local.retention.ms is -1). Positive = segment becomes eligible only after the time since the latest record in the segment reaches the value (ms). Must not exceed the effective log.local.retention.ms when that value is non-negative."; public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_MS = 0L; public static final String LOG_REMOTE_COPY_LAG_BYTES_PROP = "log.remote.copy.lag.bytes"; - public static final String LOG_REMOTE_COPY_LAG_BYTES_DOC = "Size-based delay before uploading segments. 0 = no delay. -1 = effective value is log.local.retention.bytes (max delay, or upload immediately when log.local.retention.bytes is -1). Positive = segment becomes eligible only when the bytes of log data after the segment reach the value. Must not exceed the effective log.local.retention.bytes."; + public static final String LOG_REMOTE_COPY_LAG_BYTES_DOC = "Size-based delay before uploading segments. 0 = no delay. -1 = resolves to the effective log.local.retention.bytes (max delay, or upload immediately when log.local.retention.bytes is -1). Positive = segment becomes eligible only when the bytes of log data after the segment reach the value. Must not exceed the effective log.local.retention.bytes when that value is non-negative."; public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_BYTES = 0L; public static final String REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP = "remote.log.manager.copy.max.bytes.per.second"; From d493ba07bf6186c7f3b3c37a77ec958c7de0c144 Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 13 Feb 2026 10:11:31 +0800 Subject: [PATCH 046/110] refactor document Signed-off-by: Jian --- .../main/java/org/apache/kafka/common/config/TopicConfig.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index 7788785915de1..2e37a150d9c37 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -106,14 +106,14 @@ public class TopicConfig { "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay). " + "When set to -1, resolves to the effective local.retention.ms (maximum delay, or upload immediately when local.retention.ms is -1). " + "When set to a positive value (ms), a segment becomes eligible for upload only after the time since the latest record in the segment reaches the value. " + - "The value should not exceed the effective local.retention.ms when that value is non-negative."; + "The value should not exceed the effective local.retention.ms when the latter is non-negative."; public static final String REMOTE_COPY_LAG_BYTES_CONFIG = "remote.copy.lag.bytes"; public static final String REMOTE_COPY_LAG_BYTES_DOC = "Controls size-based delay for uploading segments to remote storage. " + "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay). " + "When set to -1, resolves to the effective local.retention.bytes (maximum delay, or upload immediately when local.retention.bytes is -1). " + "When set to a positive value (bytes), a segment becomes eligible for upload only when the total bytes of log data after the segment reach the value. " + - "The value should not exceed the effective local.retention.bytes when that value is non-negative."; + "The value should not exceed the effective local.retention.bytes when the latter is non-negative."; public static final String REMOTE_LOG_DELETE_ON_DISABLE_CONFIG = "remote.log.delete.on.disable"; public static final String REMOTE_LOG_DELETE_ON_DISABLE_DOC = "Determines whether tiered data for a topic should be " + From f994e152d35adb624ca088ef20ec6013dc4fd455 Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 13 Feb 2026 11:45:39 +0800 Subject: [PATCH 047/110] correct the document Signed-off-by: Jian --- .../apache/kafka/common/config/TopicConfig.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index 2e37a150d9c37..84611f000f540 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -103,17 +103,17 @@ public class TopicConfig { public static final String REMOTE_COPY_LAG_MS_CONFIG = "remote.copy.lag.ms"; public static final String REMOTE_COPY_LAG_MS_DOC = "Controls how long to delay uploading segments to remote storage. " + - "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay). " + - "When set to -1, resolves to the effective local.retention.ms (maximum delay, or upload immediately when local.retention.ms is -1). " + - "When set to a positive value (ms), a segment becomes eligible for upload only after the time since the latest record in the segment reaches the value. " + - "The value should not exceed the effective local.retention.ms when the latter is non-negative."; + "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay check). " + + "When set to -1, resolves to the effective local retention ms (maximum delay, or no delay check when local retention ms is -1). " + + "When set to a positive value (ms), a segment can become eligible for upload only after the time since the latest record in the segment reaches the value. " + + "The value should not exceed the effective local retention ms when the latter is non-negative."; public static final String REMOTE_COPY_LAG_BYTES_CONFIG = "remote.copy.lag.bytes"; public static final String REMOTE_COPY_LAG_BYTES_DOC = "Controls size-based delay for uploading segments to remote storage. " + - "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay). " + - "When set to -1, resolves to the effective local.retention.bytes (maximum delay, or upload immediately when local.retention.bytes is -1). " + - "When set to a positive value (bytes), a segment becomes eligible for upload only when the total bytes of log data after the segment reach the value. " + - "The value should not exceed the effective local.retention.bytes when the latter is non-negative."; + "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay check). " + + "When set to -1, resolves to the effective local retention bytes (maximum delay, or no delay check when local retention bytes is -1). " + + "When set to a positive value (bytes), a segment can become eligible for upload only when the total bytes of log data after the segment reach the value. " + + "The value should not exceed the effective local retention bytes when the latter is non-negative."; public static final String REMOTE_LOG_DELETE_ON_DISABLE_CONFIG = "remote.log.delete.on.disable"; public static final String REMOTE_LOG_DELETE_ON_DISABLE_DOC = "Determines whether tiered data for a topic should be " + From 21200a0eecf50bdcf7f1f2b256e9f2bfc9d08202 Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 13 Feb 2026 11:56:46 +0800 Subject: [PATCH 048/110] correct the document Signed-off-by: Jian --- .../java/org/apache/kafka/common/config/TopicConfig.java | 8 ++++---- .../server/log/remote/storage/RemoteLogManagerConfig.java | 5 +++-- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index 84611f000f540..60da73195d602 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -104,16 +104,16 @@ public class TopicConfig { public static final String REMOTE_COPY_LAG_MS_CONFIG = "remote.copy.lag.ms"; public static final String REMOTE_COPY_LAG_MS_DOC = "Controls how long to delay uploading segments to remote storage. " + "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay check). " + - "When set to -1, resolves to the effective local retention ms (maximum delay, or no delay check when local retention ms is -1). " + "When set to a positive value (ms), a segment can become eligible for upload only after the time since the latest record in the segment reaches the value. " + - "The value should not exceed the effective local retention ms when the latter is non-negative."; + "The value should not exceed the effective local retention ms when the latter is non-negative. " + + "When set to -1, resolves to the effective local retention ms (maximum delay, or no delay check when local retention ms is -1)."; public static final String REMOTE_COPY_LAG_BYTES_CONFIG = "remote.copy.lag.bytes"; public static final String REMOTE_COPY_LAG_BYTES_DOC = "Controls size-based delay for uploading segments to remote storage. " + "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay check). " + - "When set to -1, resolves to the effective local retention bytes (maximum delay, or no delay check when local retention bytes is -1). " + "When set to a positive value (bytes), a segment can become eligible for upload only when the total bytes of log data after the segment reach the value. " + - "The value should not exceed the effective local retention bytes when the latter is non-negative."; + "The value should not exceed the effective local retention bytes when the latter is non-negative. " + + "When set to -1, resolves to the effective local retention bytes (maximum delay, or no delay check when local retention bytes is -1)."; public static final String REMOTE_LOG_DELETE_ON_DISABLE_CONFIG = "remote.log.delete.on.disable"; public static final String REMOTE_LOG_DELETE_ON_DISABLE_DOC = "Determines whether tiered data for a topic should be " + diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java index bb52c20d84339..472731bf79e9e 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.TopicConfig; import java.util.Collections; import java.util.Map; @@ -169,11 +170,11 @@ public final class RemoteLogManagerConfig { public static final Long DEFAULT_LOG_LOCAL_RETENTION_BYTES = -2L; public static final String LOG_REMOTE_COPY_LAG_MS_PROP = "log.remote.copy.lag.ms"; - public static final String LOG_REMOTE_COPY_LAG_MS_DOC = "Time-based delay before uploading segments. 0 = no delay. -1 = resolves to the effective log.local.retention.ms (max delay, or upload immediately when log.local.retention.ms is -1). Positive = segment becomes eligible only after the time since the latest record in the segment reaches the value (ms). Must not exceed the effective log.local.retention.ms when that value is non-negative."; + public static final String LOG_REMOTE_COPY_LAG_MS_DOC = TopicConfig.REMOTE_COPY_LAG_MS_DOC; public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_MS = 0L; public static final String LOG_REMOTE_COPY_LAG_BYTES_PROP = "log.remote.copy.lag.bytes"; - public static final String LOG_REMOTE_COPY_LAG_BYTES_DOC = "Size-based delay before uploading segments. 0 = no delay. -1 = resolves to the effective log.local.retention.bytes (max delay, or upload immediately when log.local.retention.bytes is -1). Positive = segment becomes eligible only when the bytes of log data after the segment reach the value. Must not exceed the effective log.local.retention.bytes when that value is non-negative."; + public static final String LOG_REMOTE_COPY_LAG_BYTES_DOC = TopicConfig.REMOTE_COPY_LAG_BYTES_DOC; public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_BYTES = 0L; public static final String REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP = "remote.log.manager.copy.max.bytes.per.second"; From 7aa8e4462d8a3f093df2958ba925d58dd39f5adb Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 13 Feb 2026 12:54:32 +0800 Subject: [PATCH 049/110] correct the document again Signed-off-by: Jian --- .../apache/kafka/common/config/TopicConfig.java | 10 ++++++---- .../remote/storage/RemoteLogManagerConfig.java | 15 ++++++++++++--- 2 files changed, 18 insertions(+), 7 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index 60da73195d602..ea842d07fbc5c 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -105,15 +105,17 @@ public class TopicConfig { public static final String REMOTE_COPY_LAG_MS_DOC = "Controls how long to delay uploading segments to remote storage. " + "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay check). " + "When set to a positive value (ms), a segment can become eligible for upload only after the time since the latest record in the segment reaches the value. " + - "The value should not exceed the effective local retention ms when the latter is non-negative. " + - "When set to -1, resolves to the effective local retention ms (maximum delay, or no delay check when local retention ms is -1)."; + "The value should not exceed the real local retention ms when the latter is non-negative. " + + "When set to -1, resolves to the real local retention ms (maximum delay, or no delay check when local retention ms is -1). " + + "For how the real local retention time is computed, see local.retention.ms."; public static final String REMOTE_COPY_LAG_BYTES_CONFIG = "remote.copy.lag.bytes"; public static final String REMOTE_COPY_LAG_BYTES_DOC = "Controls size-based delay for uploading segments to remote storage. " + "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay check). " + "When set to a positive value (bytes), a segment can become eligible for upload only when the total bytes of log data after the segment reach the value. " + - "The value should not exceed the effective local retention bytes when the latter is non-negative. " + - "When set to -1, resolves to the effective local retention bytes (maximum delay, or no delay check when local retention bytes is -1)."; + "The value should not exceed the real local retention bytes when the latter is non-negative. " + + "When set to -1, resolves to the real local retention bytes (maximum delay, or no delay check when local retention bytes is -1). " + + "For how the real local retention size is computed, see local.retention.bytes."; public static final String REMOTE_LOG_DELETE_ON_DISABLE_CONFIG = "remote.log.delete.on.disable"; public static final String REMOTE_LOG_DELETE_ON_DISABLE_DOC = "Determines whether tiered data for a topic should be " + diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java index 472731bf79e9e..da2f7cf81a9b6 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java @@ -18,7 +18,6 @@ import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; -import org.apache.kafka.common.config.TopicConfig; import java.util.Collections; import java.util.Map; @@ -170,11 +169,21 @@ public final class RemoteLogManagerConfig { public static final Long DEFAULT_LOG_LOCAL_RETENTION_BYTES = -2L; public static final String LOG_REMOTE_COPY_LAG_MS_PROP = "log.remote.copy.lag.ms"; - public static final String LOG_REMOTE_COPY_LAG_MS_DOC = TopicConfig.REMOTE_COPY_LAG_MS_DOC; + public static final String LOG_REMOTE_COPY_LAG_MS_DOC = "Controls how long to delay uploading segments to remote storage. " + + "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay check). " + + "When set to a positive value (ms), a segment can become eligible for upload only after the time since the latest record in the segment reaches the value. " + + "The value should not exceed the real local retention ms when the latter is non-negative. " + + "When set to -1, resolves to the real local retention ms (maximum delay, or no delay check when local retention ms is -1). " + + "For how the real local retention time is computed, see log.local.retention.ms."; public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_MS = 0L; public static final String LOG_REMOTE_COPY_LAG_BYTES_PROP = "log.remote.copy.lag.bytes"; - public static final String LOG_REMOTE_COPY_LAG_BYTES_DOC = TopicConfig.REMOTE_COPY_LAG_BYTES_DOC; + public static final String LOG_REMOTE_COPY_LAG_BYTES_DOC = "Controls size-based delay for uploading segments to remote storage. " + + "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay check). " + + "When set to a positive value (bytes), a segment can become eligible for upload only when the total bytes of log data after the segment reach the value. " + + "The value should not exceed the real local retention bytes when the latter is non-negative. " + + "When set to -1, resolves to the real local retention bytes (maximum delay, or no delay check when local retention bytes is -1). " + + "For how the real local retention size is computed, see log.local.retention.bytes."; public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_BYTES = 0L; public static final String REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP = "remote.log.manager.copy.max.bytes.per.second"; From 32ca1385b38c631a5cc84d7547248fb4e59b7ce9 Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 13 Feb 2026 13:13:39 +0800 Subject: [PATCH 050/110] Fix documentation for remote copy lag configs --- .../java/org/apache/kafka/common/config/TopicConfig.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index ea842d07fbc5c..6c8f5973a8a36 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -105,16 +105,16 @@ public class TopicConfig { public static final String REMOTE_COPY_LAG_MS_DOC = "Controls how long to delay uploading segments to remote storage. " + "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay check). " + "When set to a positive value (ms), a segment can become eligible for upload only after the time since the latest record in the segment reaches the value. " + - "The value should not exceed the real local retention ms when the latter is non-negative. " + - "When set to -1, resolves to the real local retention ms (maximum delay, or no delay check when local retention ms is -1). " + + "The value should not exceed the real local retention ms when the latter is not save forever. " + + "When set to -1, resolves to the real local retention ms (maximum delay, or no delay check when the retention is save forever). " + "For how the real local retention time is computed, see local.retention.ms."; public static final String REMOTE_COPY_LAG_BYTES_CONFIG = "remote.copy.lag.bytes"; public static final String REMOTE_COPY_LAG_BYTES_DOC = "Controls size-based delay for uploading segments to remote storage. " + "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay check). " + "When set to a positive value (bytes), a segment can become eligible for upload only when the total bytes of log data after the segment reach the value. " + - "The value should not exceed the real local retention bytes when the latter is non-negative. " + - "When set to -1, resolves to the real local retention bytes (maximum delay, or no delay check when local retention bytes is -1). " + + "The value should not exceed the real local retention bytes when the latter is not save forever. " + + "When set to -1, resolves to the real local retention bytes (maximum delay, or no delay check when the retention is save forever). " + "For how the real local retention size is computed, see local.retention.bytes."; public static final String REMOTE_LOG_DELETE_ON_DISABLE_CONFIG = "remote.log.delete.on.disable"; From ee9e6cab74ae38ed0bedc83c9cfc12d46228459f Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 13 Feb 2026 15:20:31 +0800 Subject: [PATCH 051/110] Update comments for MAX_REMOTE_COPY_LAG constants --- .../org/apache/kafka/storage/internals/log/LogConfig.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index e4813db3c8c94..408bb651140b9 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -146,8 +146,8 @@ public Optional serverConfigName(String configName) { public static final long DEFAULT_LOCAL_RETENTION_MS = -2; // It indicates the value to be derived from RetentionMs public static final long DEFAULT_REMOTE_COPY_LAG_MS = 0; public static final long DEFAULT_REMOTE_COPY_LAG_BYTES = 0; - public static final long MAX_REMOTE_COPY_LAG_MS = -1; // It indicates the value to be derived from localRetentionMs - public static final long MAX_REMOTE_COPY_LAG_BYTES = -1; // It indicates the value to be derived from localRetentionBytes + public static final long MAX_REMOTE_COPY_LAG_MS = -1; // It indicates the value to be derived from localRetentionMs()'s result + public static final long MAX_REMOTE_COPY_LAG_BYTES = -1; // It indicates the value to be derived from localRetentionBytes()'s result public static final String INTERNAL_SEGMENT_BYTES_CONFIG = "internal.segment.bytes"; public static final String INTERNAL_SEGMENT_BYTES_DOC = "The maximum size of a single log file. This should be used for testing only."; From 95039d40e32c886174384566e87aeea969878f70 Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 13 Feb 2026 15:30:36 +0800 Subject: [PATCH 052/110] Fix typos in TopicConfig documentation --- .../java/org/apache/kafka/common/config/TopicConfig.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index 6c8f5973a8a36..82328fe2f7065 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -104,16 +104,16 @@ public class TopicConfig { public static final String REMOTE_COPY_LAG_MS_CONFIG = "remote.copy.lag.ms"; public static final String REMOTE_COPY_LAG_MS_DOC = "Controls how long to delay uploading segments to remote storage. " + "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay check). " + - "When set to a positive value (ms), a segment can become eligible for upload only after the time since the latest record in the segment reaches the value. " + - "The value should not exceed the real local retention ms when the latter is not save forever. " + + "When set to a positive value (ms), a segment can't become eligible for upload util the time since the latest record in the segment reaches the value. " + + "The value should not exceed the real local retention ms except the latter is save forever. " + "When set to -1, resolves to the real local retention ms (maximum delay, or no delay check when the retention is save forever). " + "For how the real local retention time is computed, see local.retention.ms."; public static final String REMOTE_COPY_LAG_BYTES_CONFIG = "remote.copy.lag.bytes"; public static final String REMOTE_COPY_LAG_BYTES_DOC = "Controls size-based delay for uploading segments to remote storage. " + "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay check). " + - "When set to a positive value (bytes), a segment can become eligible for upload only when the total bytes of log data after the segment reach the value. " + - "The value should not exceed the real local retention bytes when the latter is not save forever. " + + "When set to a positive value (bytes), a segment can't become eligible for upload util the total bytes of log data after the segment reach the value. " + + "The value should not exceed the real local retention bytes except the latter is save forever. " + "When set to -1, resolves to the real local retention bytes (maximum delay, or no delay check when the retention is save forever). " + "For how the real local retention size is computed, see local.retention.bytes."; From fc71a83b97ea297eb6c132aff9160505287c6141 Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 13 Feb 2026 15:36:30 +0800 Subject: [PATCH 053/110] Fix documentation for remote log copy lag properties --- .../log/remote/storage/RemoteLogManagerConfig.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java index da2f7cf81a9b6..cc4716ff859df 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java @@ -171,18 +171,18 @@ public final class RemoteLogManagerConfig { public static final String LOG_REMOTE_COPY_LAG_MS_PROP = "log.remote.copy.lag.ms"; public static final String LOG_REMOTE_COPY_LAG_MS_DOC = "Controls how long to delay uploading segments to remote storage. " + "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay check). " + - "When set to a positive value (ms), a segment can become eligible for upload only after the time since the latest record in the segment reaches the value. " + - "The value should not exceed the real local retention ms when the latter is non-negative. " + - "When set to -1, resolves to the real local retention ms (maximum delay, or no delay check when local retention ms is -1). " + + "When set to a positive value (ms), a segment can't become eligible for upload util the time since the latest record in the segment reaches the value. " + + "The value should not exceed the real local retention ms except the latter is save forever. " + + "When set to -1, resolves to the real local retention ms (maximum delay, or no delay check when the retention is save forever). " + "For how the real local retention time is computed, see log.local.retention.ms."; public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_MS = 0L; public static final String LOG_REMOTE_COPY_LAG_BYTES_PROP = "log.remote.copy.lag.bytes"; public static final String LOG_REMOTE_COPY_LAG_BYTES_DOC = "Controls size-based delay for uploading segments to remote storage. " + "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay check). " + - "When set to a positive value (bytes), a segment can become eligible for upload only when the total bytes of log data after the segment reach the value. " + - "The value should not exceed the real local retention bytes when the latter is non-negative. " + - "When set to -1, resolves to the real local retention bytes (maximum delay, or no delay check when local retention bytes is -1). " + + "When set to a positive value (bytes), a segment can't become eligible for upload util the total bytes of log data after the segment reach the value. " + + "The value should not exceed the real local retention bytes except the latter is save forever. " + + "When set to -1, resolves to the real local retention bytes (maximum delay, or no delay check when the retention is save forever). " + "For how the real local retention size is computed, see log.local.retention.bytes."; public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_BYTES = 0L; From 017deec606096229f37f992eda8b550fdbf07707 Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 13 Feb 2026 15:51:31 +0800 Subject: [PATCH 054/110] improve the document and implement Signed-off-by: Jian --- .../kafka/storage/internals/log/LogConfig.java | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 408bb651140b9..f48ea7be21070 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -412,11 +412,21 @@ public Boolean remoteLogCopyDisable() { public long remoteCopyLagMs() { - return remoteLogConfig.remoteCopyLagMs == MAX_REMOTE_COPY_LAG_MS ? localRetentionMs() : remoteLogConfig.remoteCopyLagMs; + if (remoteLogConfig.remoteCopyLagMs != MAX_REMOTE_COPY_LAG_MS) { + return remoteLogConfig.remoteCopyLagMs; + } + // MAX_REMOTE_COPY_LAG_MS (-1): use local retention; if local is to save forever (-1), return 0 (no delay). + long localRetentionMs = localRetentionMs(); + return localRetentionMs == -1 ? 0 : localRetentionMs; } public long remoteCopyLagBytes() { - return remoteLogConfig.remoteCopyLagBytes == MAX_REMOTE_COPY_LAG_BYTES ? localRetentionBytes() : remoteLogConfig.remoteCopyLagBytes; + if (remoteLogConfig.remoteCopyLagBytes != MAX_REMOTE_COPY_LAG_BYTES) { + return remoteLogConfig.remoteCopyLagBytes; + } + // MAX_REMOTE_COPY_LAG_BYTES (-1): use local retention; if local is to save forever (-1), return 0 (no delay). + long localRetentionBytes = localRetentionBytes(); + return localRetentionBytes == -1 ? 0 : localRetentionBytes; } public long localRetentionMs() { From 5b223b1fe6394d05c4dc64f10ebf25a2bd759c33 Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 13 Feb 2026 15:56:59 +0800 Subject: [PATCH 055/110] Update LogConfig.java --- .../org/apache/kafka/storage/internals/log/LogConfig.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index f48ea7be21070..18f93b7a4a1ec 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -146,8 +146,8 @@ public Optional serverConfigName(String configName) { public static final long DEFAULT_LOCAL_RETENTION_MS = -2; // It indicates the value to be derived from RetentionMs public static final long DEFAULT_REMOTE_COPY_LAG_MS = 0; public static final long DEFAULT_REMOTE_COPY_LAG_BYTES = 0; - public static final long MAX_REMOTE_COPY_LAG_MS = -1; // It indicates the value to be derived from localRetentionMs()'s result - public static final long MAX_REMOTE_COPY_LAG_BYTES = -1; // It indicates the value to be derived from localRetentionBytes()'s result + public static final long MAX_REMOTE_COPY_LAG_MS = -1; // It indicates the value depends on localRetentionMs() + public static final long MAX_REMOTE_COPY_LAG_BYTES = -1; // It indicates the value depends on localRetentionBytes() public static final String INTERNAL_SEGMENT_BYTES_CONFIG = "internal.segment.bytes"; public static final String INTERNAL_SEGMENT_BYTES_DOC = "The maximum size of a single log file. This should be used for testing only."; From f458edd8f15d1ecf3067502bc790eb601c8e2d55 Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 13 Feb 2026 15:58:09 +0800 Subject: [PATCH 056/110] Update comments for MAX_REMOTE_COPY_LAG constants --- .../org/apache/kafka/storage/internals/log/LogConfig.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 18f93b7a4a1ec..ce6067174338b 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -146,8 +146,8 @@ public Optional serverConfigName(String configName) { public static final long DEFAULT_LOCAL_RETENTION_MS = -2; // It indicates the value to be derived from RetentionMs public static final long DEFAULT_REMOTE_COPY_LAG_MS = 0; public static final long DEFAULT_REMOTE_COPY_LAG_BYTES = 0; - public static final long MAX_REMOTE_COPY_LAG_MS = -1; // It indicates the value depends on localRetentionMs() - public static final long MAX_REMOTE_COPY_LAG_BYTES = -1; // It indicates the value depends on localRetentionBytes() + public static final long MAX_REMOTE_COPY_LAG_MS = -1; // It indicates the value depends on local retention ms + public static final long MAX_REMOTE_COPY_LAG_BYTES = -1; // It indicates the value depends on local retention bytes public static final String INTERNAL_SEGMENT_BYTES_CONFIG = "internal.segment.bytes"; public static final String INTERNAL_SEGMENT_BYTES_DOC = "The maximum size of a single log file. This should be used for testing only."; From 2087e063164473adcb264b1cbf0a5de11162dc35 Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 13 Feb 2026 16:09:20 +0800 Subject: [PATCH 057/110] remove useless null protected judgement Signed-off-by: Jian --- .../apache/kafka/storage/internals/log/LogConfig.java | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index ce6067174338b..5793c2696a7c1 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -637,9 +637,8 @@ private static void validateRemoteCopyLagTime(Map props) { Long retentionMs = (Long) props.get(TopicConfig.RETENTION_MS_CONFIG); Long localRetentionMs = (Long) props.get(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG); Long remoteCopyLagMs = (Long) props.get(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG); - long effectiveLocalRetentionMs = (localRetentionMs != null && localRetentionMs == -2 && retentionMs != null) - ? retentionMs : (localRetentionMs != null ? localRetentionMs : -2); - if (remoteCopyLagMs != null && remoteCopyLagMs > 0 && effectiveLocalRetentionMs >= 0 + long effectiveLocalRetentionMs = localRetentionMs == -2 ? retentionMs : localRetentionMs; + if (remoteCopyLagMs > 0 && effectiveLocalRetentionMs >= 0 && remoteCopyLagMs > effectiveLocalRetentionMs) { String message = String.format("Value must not exceed %s (effective value: %d)", TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, effectiveLocalRetentionMs); @@ -651,9 +650,8 @@ private static void validateRemoteCopyLagSize(Map props) { Long retentionBytes = (Long) props.get(TopicConfig.RETENTION_BYTES_CONFIG); Long localRetentionBytes = (Long) props.get(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG); Long remoteCopyLagBytes = (Long) props.get(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG); - long effectiveLocalRetentionBytes = (localRetentionBytes != null && localRetentionBytes == -2 && retentionBytes != null) - ? retentionBytes : (localRetentionBytes != null ? localRetentionBytes : -2); - if (remoteCopyLagBytes != null && remoteCopyLagBytes > 0 && effectiveLocalRetentionBytes >= 0 + long effectiveLocalRetentionBytes = localRetentionBytes == -2 ? retentionBytes : localRetentionBytes; + if (remoteCopyLagBytes > 0 && effectiveLocalRetentionBytes >= 0 && remoteCopyLagBytes > effectiveLocalRetentionBytes) { String message = String.format("Value must not exceed %s (effective value: %d)", TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, effectiveLocalRetentionBytes); From 757423589a0ab210e710b12640c86a0a8066c2cf Mon Sep 17 00:00:00 2001 From: Jian Date: Mon, 23 Feb 2026 09:45:48 +0800 Subject: [PATCH 058/110] correct the judgement for upload Signed-off-by: Jian --- .../log/remote/storage/RemoteLogManager.java | 21 ++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index 47ebb85725c4b..9bcc468d33fbd 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -933,7 +933,7 @@ private boolean hasExceededCopyLagSize(LogSegment segment, long totalLogSize, lo * 1) Segment is not the active segment and * 2) Segment end-offset is less than the last-stable-offset as remote storage should contain only * committed/acked messages - * 3) Segment has exceeded copy lag by time and size when configured (remote.copy.lag.ms, remote.copy.lag.bytes) + * 3) Segment has exceeded copy lag by time or size when configured (remote.copy.lag.ms, remote.copy.lag.bytes) * @param log The log from which the segments are to be copied * @param fromOffset The offset from which the segments are to be copied * @param lastStableOffset The last stable offset of the log @@ -954,13 +954,20 @@ List candidateLogSegments(UnifiedLog log, Long fromOffset, L LogSegment previousSeg = segments.get(idx - 1); LogSegment currentSeg = segments.get(idx); if (currentSeg.baseOffset() <= lastStableOffset) { - if (copyLagMs > 0 && !hasExceededCopyLagTime(previousSeg, currentTimeMs, copyLagMs)) - break; - cumulativeSize += previousSeg.size(); - if (copyLagBytes > 0 && !hasExceededCopyLagSize(previousSeg, totalLogSize, cumulativeSize, copyLagBytes)) - break; - + if (copyLagMs > 0){ + if (!hasExceededCopyLagTime(previousSeg, currentTimeMs, copyLagMs)) { + if (copyLagBytes == 0) { + break; + } + if (copyLagBytes > 0 && !hasExceededCopyLagSize(previousSeg, totalLogSize, cumulativeSize, copyLagBytes)) + break; + } + } else { + if (copyLagBytes > 0 && !hasExceededCopyLagSize(previousSeg, totalLogSize, cumulativeSize, copyLagBytes)) + break; + } + candidateLogSegments.add(new EnrichedLogSegment(previousSeg, currentSeg.baseOffset())); } } From f2ffe8637a64185fcb2c08fe0bf42a0785742867 Mon Sep 17 00:00:00 2001 From: Jian Date: Mon, 23 Feb 2026 09:46:33 +0800 Subject: [PATCH 059/110] correct the format Signed-off-by: Jian --- .../kafka/server/log/remote/storage/RemoteLogManager.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index 9bcc468d33fbd..1a3b7dae91784 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -964,8 +964,9 @@ List candidateLogSegments(UnifiedLog log, Long fromOffset, L break; } } else { - if (copyLagBytes > 0 && !hasExceededCopyLagSize(previousSeg, totalLogSize, cumulativeSize, copyLagBytes)) + if (copyLagBytes > 0 && !hasExceededCopyLagSize(previousSeg, totalLogSize, cumulativeSize, copyLagBytes)){ break; + } } candidateLogSegments.add(new EnrichedLogSegment(previousSeg, currentSeg.baseOffset())); From 95d096bb0e12ee94d5fe1d9df740e796e29a8d54 Mon Sep 17 00:00:00 2001 From: Jian Date: Mon, 23 Feb 2026 09:48:01 +0800 Subject: [PATCH 060/110] correct the format Signed-off-by: Jian --- .../kafka/server/log/remote/storage/RemoteLogManager.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index 1a3b7dae91784..9b1c859aa9bd5 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -960,11 +960,12 @@ List candidateLogSegments(UnifiedLog log, Long fromOffset, L if (copyLagBytes == 0) { break; } - if (copyLagBytes > 0 && !hasExceededCopyLagSize(previousSeg, totalLogSize, cumulativeSize, copyLagBytes)) + if (copyLagBytes > 0 && !hasExceededCopyLagSize(previousSeg, totalLogSize, cumulativeSize, copyLagBytes)) { break; + } } } else { - if (copyLagBytes > 0 && !hasExceededCopyLagSize(previousSeg, totalLogSize, cumulativeSize, copyLagBytes)){ + if (copyLagBytes > 0 && !hasExceededCopyLagSize(previousSeg, totalLogSize, cumulativeSize, copyLagBytes)) { break; } } From b5aa4a52be5056df161fa3dcc19ec99818523613 Mon Sep 17 00:00:00 2001 From: Jian Date: Mon, 23 Feb 2026 09:55:47 +0800 Subject: [PATCH 061/110] correct the format Signed-off-by: Jian --- core/src/main/scala/kafka/server/DynamicBrokerConfig.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index 256c79f3ce2bb..4f5f66baefe3d 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -597,7 +597,8 @@ class DynamicLogConfig(logManager: LogManager, directoryEventHandler: DirectoryE throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_BYTES_PROP, logRemoteCopyLagBytes, s"Value must not exceed ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP} (effective value: $effectiveLocalRetentionBytes)") } - + } + def validateCordonedLogDirs(): Unit = { val logDirs = newConfig.logDirs() val cordonedLogDirs = newConfig.cordonedLogDirs() From 0c4c7c35019094af9e481a5dacca630143f9ba76 Mon Sep 17 00:00:00 2001 From: Jian Date: Mon, 23 Feb 2026 10:14:28 +0800 Subject: [PATCH 062/110] code refactor Signed-off-by: Jian --- .../log/remote/storage/RemoteLogManager.java | 30 ++++++++++--------- 1 file changed, 16 insertions(+), 14 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index 9b1c859aa9bd5..c7e615f382f36 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -955,20 +955,8 @@ List candidateLogSegments(UnifiedLog log, Long fromOffset, L LogSegment currentSeg = segments.get(idx); if (currentSeg.baseOffset() <= lastStableOffset) { cumulativeSize += previousSeg.size(); - if (copyLagMs > 0){ - if (!hasExceededCopyLagTime(previousSeg, currentTimeMs, copyLagMs)) { - if (copyLagBytes == 0) { - break; - } - if (copyLagBytes > 0 && !hasExceededCopyLagSize(previousSeg, totalLogSize, cumulativeSize, copyLagBytes)) { - break; - } - } - } else { - if (copyLagBytes > 0 && !hasExceededCopyLagSize(previousSeg, totalLogSize, cumulativeSize, copyLagBytes)) { - break; - } - } + if (delayCopy(previousSeg, copyLagMs, currentTimeMs, copyLagBytes, totalLogSize, cumulativeSize)) + break; candidateLogSegments.add(new EnrichedLogSegment(previousSeg, currentSeg.baseOffset())); } @@ -978,6 +966,20 @@ List candidateLogSegments(UnifiedLog log, Long fromOffset, L return candidateLogSegments; } + private boolean delayCopy(LogSegment segment, long copyLagMs, long currentTimeMs, long copyLagBytes, long totalLogSize, long cumulativeSize) { + if (copyLagMs > 0) { + if (hasExceededCopyLagTime(segment, currentTimeMs, copyLagMs)) { + return false; + } + + if (copyLagBytes == 0) { + return true; + } + } + + return copyLagBytes > 0 && !hasExceededCopyLagSize(segment, totalLogSize, cumulativeSize, copyLagBytes); + } + public void copyLogSegmentsToRemote(UnifiedLog log) throws InterruptedException, RetriableRemoteStorageException { if (isCancelled()) return; From f9afae2859812383e997843eee9653f3729fbcfe Mon Sep 17 00:00:00 2001 From: Jian Date: Mon, 23 Feb 2026 10:17:35 +0800 Subject: [PATCH 063/110] code refactor Signed-off-by: Jian --- .../kafka/server/log/remote/storage/RemoteLogManager.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index c7e615f382f36..01cdec93407fa 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -971,10 +971,6 @@ private boolean delayCopy(LogSegment segment, long copyLagMs, long currentTimeMs if (hasExceededCopyLagTime(segment, currentTimeMs, copyLagMs)) { return false; } - - if (copyLagBytes == 0) { - return true; - } } return copyLagBytes > 0 && !hasExceededCopyLagSize(segment, totalLogSize, cumulativeSize, copyLagBytes); From 1c251f0a3af2228abceb2cf962e2a00c2456bf34 Mon Sep 17 00:00:00 2001 From: Jian Date: Mon, 23 Feb 2026 10:24:47 +0800 Subject: [PATCH 064/110] correct the code Signed-off-by: Jian --- .../kafka/server/log/remote/storage/RemoteLogManager.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index 01cdec93407fa..c2bce992e6e9f 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -971,6 +971,9 @@ private boolean delayCopy(LogSegment segment, long copyLagMs, long currentTimeMs if (hasExceededCopyLagTime(segment, currentTimeMs, copyLagMs)) { return false; } + if (copyLagBytes == 0) { + return true; + } } return copyLagBytes > 0 && !hasExceededCopyLagSize(segment, totalLogSize, cumulativeSize, copyLagBytes); From 7148ac5f314f0a5975a0d543a568a2d0125376f8 Mon Sep 17 00:00:00 2001 From: Jian Date: Mon, 23 Feb 2026 10:29:56 +0800 Subject: [PATCH 065/110] improve the document Signed-off-by: Jian --- .../java/org/apache/kafka/storage/internals/log/LogConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 73d801107328d..84f0a9ab17090 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -416,7 +416,7 @@ public long remoteCopyLagMs() { if (remoteLogConfig.remoteCopyLagMs != MAX_REMOTE_COPY_LAG_MS) { return remoteLogConfig.remoteCopyLagMs; } - // MAX_REMOTE_COPY_LAG_MS (-1): use local retention; if local is to save forever (-1), return 0 (no delay). + // MAX_REMOTE_COPY_LAG_MS (-1): use local retention; if local is to save forever (-1), return 0 (no delay check). long localRetentionMs = localRetentionMs(); return localRetentionMs == -1 ? 0 : localRetentionMs; } From cf52590b702cc811b3bb8bcdcb51d311e3fb3bf4 Mon Sep 17 00:00:00 2001 From: Jian Date: Wed, 1 Apr 2026 06:49:46 +0800 Subject: [PATCH 066/110] refactor the code using new definition for delay configure Signed-off-by: Jian --- .../log/remote/storage/RemoteLogManager.java | 49 +++++++------------ .../storage/internals/log/LogConfig.java | 22 +++------ 2 files changed, 25 insertions(+), 46 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index 03f73ec5dfd8a..02b3511826dd4 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -61,22 +61,7 @@ import org.apache.kafka.server.storage.log.FetchIsolation; import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpointFile; import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache; -import org.apache.kafka.storage.internals.log.AsyncOffsetReadFutureHolder; -import org.apache.kafka.storage.internals.log.AsyncOffsetReader; -import org.apache.kafka.storage.internals.log.EpochEntry; -import org.apache.kafka.storage.internals.log.FetchDataInfo; -import org.apache.kafka.storage.internals.log.LogOffsetMetadata; -import org.apache.kafka.storage.internals.log.LogSegment; -import org.apache.kafka.storage.internals.log.OffsetIndex; -import org.apache.kafka.storage.internals.log.OffsetPosition; -import org.apache.kafka.storage.internals.log.OffsetResultHolder; -import org.apache.kafka.storage.internals.log.RemoteIndexCache; -import org.apache.kafka.storage.internals.log.RemoteLogReadResult; -import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo; -import org.apache.kafka.storage.internals.log.RemoteStorageThreadPool; -import org.apache.kafka.storage.internals.log.TransactionIndex; -import org.apache.kafka.storage.internals.log.TxnIndexSearchResult; -import org.apache.kafka.storage.internals.log.UnifiedLog; +import org.apache.kafka.storage.internals.log.*; import org.apache.kafka.storage.log.metrics.BrokerTopicStats; import com.yammer.metrics.core.Timer; @@ -946,9 +931,6 @@ List candidateLogSegments(UnifiedLog log, Long fromOffset, L List candidateLogSegments = new ArrayList<>(); List segments = log.logSegments(fromOffset, Long.MAX_VALUE); if (!segments.isEmpty()) { - long copyLagMs = log.config() != null ? log.config().remoteCopyLagMs() : 0L; - long copyLagBytes = log.config() != null ? log.config().remoteCopyLagBytes() : 0L; - long currentTimeMs = time.milliseconds(); long totalLogSize = UnifiedLog.sizeInBytes(segments); long cumulativeSize = 0; @@ -958,9 +940,9 @@ List candidateLogSegments(UnifiedLog log, Long fromOffset, L LogSegment currentSeg = segments.get(idx); if (currentSeg.baseOffset() <= lastStableOffset) { cumulativeSize += previousSeg.size(); - if (delayCopy(previousSeg, copyLagMs, currentTimeMs, copyLagBytes, totalLogSize, cumulativeSize)) + if (delayCopy(log.config(), previousSeg, currentTimeMs, totalLogSize, cumulativeSize)) { break; - + } candidateLogSegments.add(new EnrichedLogSegment(previousSeg, currentSeg.baseOffset())); } } @@ -969,19 +951,26 @@ List candidateLogSegments(UnifiedLog log, Long fromOffset, L return candidateLogSegments; } - private boolean delayCopy(LogSegment segment, long copyLagMs, long currentTimeMs, long copyLagBytes, long totalLogSize, long cumulativeSize) { - if (copyLagMs > 0) { - if (hasExceededCopyLagTime(segment, currentTimeMs, copyLagMs)) { - return false; - } - if (copyLagBytes == 0) { - return true; - } + private boolean delayCopy(LogConfig logConfig, LogSegment previousSeg, long currentTimeMs, long totalLogSize, long cumulativeSize){ + if (logConfig == null) { + return false; + } + + long copyLagMs = logConfig.remoteCopyLagMs(); + long copyLagBytes = logConfig.remoteCopyLagBytes(); + + if (copyLagMs == 0 || copyLagBytes == 0) { + return false; } - return copyLagBytes > 0 && !hasExceededCopyLagSize(segment, totalLogSize, cumulativeSize, copyLagBytes); + if (copyLagMs != -2 && ! hasExceededCopyLagTime(previousSeg, currentTimeMs, copyLagMs)) { + return false; + } + + return copyLagBytes != -2 && ! hasExceededCopyLagSize(previousSeg, totalLogSize, cumulativeSize, copyLagBytes); } + public void copyLogSegmentsToRemote(UnifiedLog log) throws InterruptedException, RetriableRemoteStorageException { if (isCancelled()) return; diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 44e90c728bcfa..267b93d097997 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -144,8 +144,8 @@ public Optional serverConfigName(String configName) { public static final boolean DEFAULT_REMOTE_LOG_DELETE_ON_DISABLE_CONFIG = false; public static final long DEFAULT_LOCAL_RETENTION_BYTES = -2; // It indicates the value to be derived from RetentionBytes public static final long DEFAULT_LOCAL_RETENTION_MS = -2; // It indicates the value to be derived from RetentionMs - public static final long DEFAULT_REMOTE_COPY_LAG_MS = 0; - public static final long DEFAULT_REMOTE_COPY_LAG_BYTES = 0; + public static final long DEFAULT_REMOTE_COPY_LAG_MS = -2; // It indicates no delay check based on local retention ms + public static final long DEFAULT_REMOTE_COPY_LAG_BYTES = -2; // It indicates no delay check based on local retention bytes public static final long MAX_REMOTE_COPY_LAG_MS = -1; // It indicates the value depends on local retention ms public static final long MAX_REMOTE_COPY_LAG_BYTES = -1; // It indicates the value depends on local retention bytes @@ -257,8 +257,8 @@ public Optional serverConfigName(String configName) { .define(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, LONG, DEFAULT_LOCAL_RETENTION_BYTES, atLeast(-2), MEDIUM, TopicConfig.LOCAL_LOG_RETENTION_BYTES_DOC) .define(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_COPY_DISABLE_DOC) - .define(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, LONG, DEFAULT_REMOTE_COPY_LAG_MS, atLeast(-1), MEDIUM, TopicConfig.REMOTE_COPY_LAG_MS_DOC) - .define(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, LONG, DEFAULT_REMOTE_COPY_LAG_BYTES, atLeast(-1), MEDIUM, TopicConfig.REMOTE_COPY_LAG_BYTES_DOC) + .define(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, LONG, DEFAULT_REMOTE_COPY_LAG_MS, atLeast(-2), MEDIUM, TopicConfig.REMOTE_COPY_LAG_MS_DOC) + .define(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, LONG, DEFAULT_REMOTE_COPY_LAG_BYTES, atLeast(-2), MEDIUM, TopicConfig.REMOTE_COPY_LAG_BYTES_DOC) .define(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_DOC) .defineInternal(INTERNAL_SEGMENT_BYTES_CONFIG, INT, null, null, MEDIUM, INTERNAL_SEGMENT_BYTES_DOC); } @@ -413,21 +413,11 @@ public Boolean remoteLogCopyDisable() { public long remoteCopyLagMs() { - if (remoteLogConfig.remoteCopyLagMs != MAX_REMOTE_COPY_LAG_MS) { - return remoteLogConfig.remoteCopyLagMs; - } - // MAX_REMOTE_COPY_LAG_MS (-1): use local retention; if local is to save forever (-1), return 0 (no delay check). - long localRetentionMs = localRetentionMs(); - return localRetentionMs == -1 ? 0 : localRetentionMs; + return remoteLogConfig.remoteCopyLagMs == MAX_REMOTE_COPY_LAG_MS ? remoteLogConfig.remoteCopyLagMs : localRetentionMs(); } public long remoteCopyLagBytes() { - if (remoteLogConfig.remoteCopyLagBytes != MAX_REMOTE_COPY_LAG_BYTES) { - return remoteLogConfig.remoteCopyLagBytes; - } - // MAX_REMOTE_COPY_LAG_BYTES (-1): use local retention; if local is to save forever (-1), return 0 (no delay). - long localRetentionBytes = localRetentionBytes(); - return localRetentionBytes == -1 ? 0 : localRetentionBytes; + return remoteLogConfig.remoteCopyLagBytes == MAX_REMOTE_COPY_LAG_BYTES ? remoteLogConfig.remoteCopyLagBytes : localRetentionBytes(); } public long localRetentionMs() { From 7bbed2e3f5239ae8eee9897deca0f1bb4ac4449a Mon Sep 17 00:00:00 2001 From: Jian Date: Wed, 1 Apr 2026 06:58:09 +0800 Subject: [PATCH 067/110] Correct multiple log-related imports to RemoteLogManager --- .../log/remote/storage/RemoteLogManager.java | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index 02b3511826dd4..93a84deba1e25 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -61,7 +61,23 @@ import org.apache.kafka.server.storage.log.FetchIsolation; import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpointFile; import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache; -import org.apache.kafka.storage.internals.log.*; +import org.apache.kafka.storage.internals.log.AsyncOffsetReadFutureHolder; +import org.apache.kafka.storage.internals.log.AsyncOffsetReader; +import org.apache.kafka.storage.internals.log.EpochEntry; +import org.apache.kafka.storage.internals.log.FetchDataInfo; +import org.apache.kafka.storage.internals.log.LogConfig; +import org.apache.kafka.storage.internals.log.LogOffsetMetadata; +import org.apache.kafka.storage.internals.log.LogSegment; +import org.apache.kafka.storage.internals.log.OffsetIndex; +import org.apache.kafka.storage.internals.log.OffsetPosition; +import org.apache.kafka.storage.internals.log.OffsetResultHolder; +import org.apache.kafka.storage.internals.log.RemoteIndexCache; +import org.apache.kafka.storage.internals.log.RemoteLogReadResult; +import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo; +import org.apache.kafka.storage.internals.log.RemoteStorageThreadPool; +import org.apache.kafka.storage.internals.log.TransactionIndex; +import org.apache.kafka.storage.internals.log.TxnIndexSearchResult; +import org.apache.kafka.storage.internals.log.UnifiedLog; import org.apache.kafka.storage.log.metrics.BrokerTopicStats; import com.yammer.metrics.core.Timer; From 7a4ef68c946e06ae4b1e90bb54c42330c3591af4 Mon Sep 17 00:00:00 2001 From: Jian Date: Wed, 1 Apr 2026 07:09:04 +0800 Subject: [PATCH 068/110] Update copy lag condition checks in RemoteLogManager Refactor copy lag checks to ensure positive values are used. --- .../kafka/server/log/remote/storage/RemoteLogManager.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index 93a84deba1e25..1b66df8c432e3 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -979,11 +979,15 @@ private boolean delayCopy(LogConfig logConfig, LogSegment previousSeg, long curr return false; } - if (copyLagMs != -2 && ! hasExceededCopyLagTime(previousSeg, currentTimeMs, copyLagMs)) { + if (copyLagMs > 0 && hasExceededCopyLagTime(previousSeg, currentTimeMs, copyLagMs)) { return false; } - return copyLagBytes != -2 && ! hasExceededCopyLagSize(previousSeg, totalLogSize, cumulativeSize, copyLagBytes); + if (copyLagBytes > 0 && hasExceededCopyLagSize(previousSeg, totalLogSize, cumulativeSize, copyLagBytes)) { + return false; + } + + return false; } From fa34452612d3c0c922619cf140576e309a59dc4a Mon Sep 17 00:00:00 2001 From: Jian Date: Wed, 1 Apr 2026 07:36:34 +0800 Subject: [PATCH 069/110] Refactor copy lag checks in RemoteLogManager --- .../server/log/remote/storage/RemoteLogManager.java | 13 ++++--------- 1 file changed, 4 insertions(+), 9 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index 1b66df8c432e3..83b4eb4bfce59 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -978,19 +978,14 @@ private boolean delayCopy(LogConfig logConfig, LogSegment previousSeg, long curr if (copyLagMs == 0 || copyLagBytes == 0) { return false; } - + if (copyLagMs > 0 && hasExceededCopyLagTime(previousSeg, currentTimeMs, copyLagMs)) { - return false; - } - - if (copyLagBytes > 0 && hasExceededCopyLagSize(previousSeg, totalLogSize, cumulativeSize, copyLagBytes)) { - return false; + return true; } - return false; + return copyLagBytes > 0 && !hasExceededCopyLagSize(previousSeg, totalLogSize, cumulativeSize, copyLagBytes); } - - + public void copyLogSegmentsToRemote(UnifiedLog log) throws InterruptedException, RetriableRemoteStorageException { if (isCancelled()) return; From 8f2c42781013cd6f4840809700eb8c1c5ca219a9 Mon Sep 17 00:00:00 2001 From: Jian Date: Wed, 1 Apr 2026 07:40:01 +0800 Subject: [PATCH 070/110] Fix the check style issue --- .../kafka/server/log/remote/storage/RemoteLogManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index 83b4eb4bfce59..0e0448c5bb4ab 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -967,7 +967,7 @@ List candidateLogSegments(UnifiedLog log, Long fromOffset, L return candidateLogSegments; } - private boolean delayCopy(LogConfig logConfig, LogSegment previousSeg, long currentTimeMs, long totalLogSize, long cumulativeSize){ + private boolean delayCopy(LogConfig logConfig, LogSegment previousSeg, long currentTimeMs, long totalLogSize, long cumulativeSize) { if (logConfig == null) { return false; } From ab81e291786b5556c923d7a03d35457e0de6f08b Mon Sep 17 00:00:00 2001 From: Jian Date: Wed, 1 Apr 2026 21:45:38 +0800 Subject: [PATCH 071/110] correct the configure item's default value for delay upload Signed-off-by: Jian --- .../apache/kafka/common/config/TopicConfig.java | 6 ++++-- .../test/scala/unit/kafka/log/LogConfigTest.scala | 4 ++-- .../scala/unit/kafka/server/KafkaConfigTest.scala | 4 ++-- .../log/remote/storage/RemoteLogManagerConfig.java | 14 ++++++++------ 4 files changed, 16 insertions(+), 12 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index 82328fe2f7065..fe99a504d47d7 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -103,7 +103,8 @@ public class TopicConfig { public static final String REMOTE_COPY_LAG_MS_CONFIG = "remote.copy.lag.ms"; public static final String REMOTE_COPY_LAG_MS_DOC = "Controls how long to delay uploading segments to remote storage. " + - "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay check). " + + "When set to -2 (default), no delay check based on local retention ms. " + + "When set to 0, segments are uploaded as soon as they are eligible without delay. " + "When set to a positive value (ms), a segment can't become eligible for upload util the time since the latest record in the segment reaches the value. " + "The value should not exceed the real local retention ms except the latter is save forever. " + "When set to -1, resolves to the real local retention ms (maximum delay, or no delay check when the retention is save forever). " + @@ -111,7 +112,8 @@ public class TopicConfig { public static final String REMOTE_COPY_LAG_BYTES_CONFIG = "remote.copy.lag.bytes"; public static final String REMOTE_COPY_LAG_BYTES_DOC = "Controls size-based delay for uploading segments to remote storage. " + - "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay check). " + + "When set to -2 (default), no delay check based on local retention bytes. " + + "When set to 0, segments are uploaded as soon as they are eligible without delay. " + "When set to a positive value (bytes), a segment can't become eligible for upload util the total bytes of log data after the segment reach the value. " + "The value should not exceed the real local retention bytes except the latter is save forever. " + "When set to -1, resolves to the real local retention bytes (maximum delay, or no delay check when the retention is save forever). " + diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index 4e0b834f5cd6b..7431131eaffa0 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -73,8 +73,8 @@ class LogConfigTest { case TopicConfig.COMPRESSION_ZSTD_LEVEL_CONFIG => assertPropertyInvalid(name, "not_a_number", "-0.1") case TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG => assertPropertyInvalid(name, "not_a_number", "remove", "0") case TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG => assertPropertyInvalid(name, "not_a_number", "remove", "0") - case TopicConfig.REMOTE_COPY_LAG_MS_CONFIG => assertPropertyInvalid(name, "not_a_number", "-2") - case TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG => assertPropertyInvalid(name, "not_a_number", "-2") + case TopicConfig.REMOTE_COPY_LAG_MS_CONFIG => assertPropertyInvalid(name, "not_a_number", "-3") + case TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG => assertPropertyInvalid(name, "not_a_number", "-3") case LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG => // no op case _ => assertPropertyInvalid(name, "not_a_number", "-1") diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 12827b165f2b8..8b770ea9a0051 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -1032,8 +1032,8 @@ class KafkaConfigTest { case RemoteLogManagerConfig.REMOTE_LOG_READER_MAX_PENDING_TASKS_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", 0, -1) case RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", -3) case RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", -3) - case RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_MS_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", "-2") - case RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_BYTES_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", "-2") + case RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_MS_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", "-3") + case RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_BYTES_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", "-3") /** New group coordinator configs */ case GroupCoordinatorConfig.GROUP_COORDINATOR_NUM_THREADS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", 0, -1) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java index cc4716ff859df..40b31cbcd25df 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java @@ -170,21 +170,23 @@ public final class RemoteLogManagerConfig { public static final String LOG_REMOTE_COPY_LAG_MS_PROP = "log.remote.copy.lag.ms"; public static final String LOG_REMOTE_COPY_LAG_MS_DOC = "Controls how long to delay uploading segments to remote storage. " + - "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay check). " + + "When set to -2 (default), no delay check based on local retention ms. " + + "When set to 0, segments are uploaded as soon as they are eligible without delay. " + "When set to a positive value (ms), a segment can't become eligible for upload util the time since the latest record in the segment reaches the value. " + "The value should not exceed the real local retention ms except the latter is save forever. " + "When set to -1, resolves to the real local retention ms (maximum delay, or no delay check when the retention is save forever). " + "For how the real local retention time is computed, see log.local.retention.ms."; - public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_MS = 0L; + public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_MS = -2L; public static final String LOG_REMOTE_COPY_LAG_BYTES_PROP = "log.remote.copy.lag.bytes"; public static final String LOG_REMOTE_COPY_LAG_BYTES_DOC = "Controls size-based delay for uploading segments to remote storage. " + - "When set to 0 (default), segments are uploaded as soon as they are eligible (no delay check). " + + "When set to -2 (default), no delay check based on local retention bytes. " + + "When set to 0, segments are uploaded as soon as they are eligible without delay. " + "When set to a positive value (bytes), a segment can't become eligible for upload util the total bytes of log data after the segment reach the value. " + "The value should not exceed the real local retention bytes except the latter is save forever. " + "When set to -1, resolves to the real local retention bytes (maximum delay, or no delay check when the retention is save forever). " + "For how the real local retention size is computed, see log.local.retention.bytes."; - public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_BYTES = 0L; + public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_BYTES = -2L; public static final String REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP = "remote.log.manager.copy.max.bytes.per.second"; public static final String REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_DOC = "The maximum number of bytes that can be copied from local storage to remote storage per second. " + @@ -368,13 +370,13 @@ public static ConfigDef configDef() { .define(LOG_REMOTE_COPY_LAG_MS_PROP, LONG, DEFAULT_LOG_REMOTE_COPY_LAG_MS, - atLeast(-1), + atLeast(-2), MEDIUM, LOG_REMOTE_COPY_LAG_MS_DOC) .define(LOG_REMOTE_COPY_LAG_BYTES_PROP, LONG, DEFAULT_LOG_REMOTE_COPY_LAG_BYTES, - atLeast(-1), + atLeast(-2), MEDIUM, LOG_REMOTE_COPY_LAG_BYTES_DOC) .define(REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP, From 2b5fede91d6ca076dc974d7f74c3bc743b7650df Mon Sep 17 00:00:00 2001 From: Jian Date: Wed, 1 Apr 2026 21:49:10 +0800 Subject: [PATCH 072/110] refactor the method position Signed-off-by: Jian --- .../log/remote/storage/RemoteLogManager.java | 42 +++++++++---------- 1 file changed, 21 insertions(+), 21 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index 0e0448c5bb4ab..432dc2450f662 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -911,27 +911,6 @@ private void maybeUpdateCopiedOffset(UnifiedLog log) throws RemoteStorageExcepti } } - private boolean hasExceededCopyLagTime(LogSegment segment, long currentTimeMs, long copyLagMs) { - try { - long segmentAgeMs = currentTimeMs - segment.largestTimestamp(); - boolean exceeded = segmentAgeMs >= copyLagMs; - logger.debug("{} eligible for upload by time? {} (segment age {} ms, copy lag {} ms)", - segment, exceeded, segmentAgeMs, copyLagMs); - return exceeded; - } catch (IOException e) { - logger.warn("Failed to get largest timestamp for segment {}, take it as eligible for upload based on time", segment, e); - return true; - } - } - - private boolean hasExceededCopyLagSize(LogSegment segment, long totalLogSize, long cumulativeSize, long copyLagBytes) { - long sizeLagBytes = totalLogSize - cumulativeSize; - boolean exceeded = sizeLagBytes >= copyLagBytes; - logger.debug("{} eligible for upload by size? {} (size lag {} bytes, copy lag {} bytes, totalLogSize={}, cumulativeSize={})", - segment, exceeded, sizeLagBytes, copyLagBytes, totalLogSize, cumulativeSize); - return exceeded; - } - /** * Segments which match the following criteria are eligible for copying to remote storage: * 1) Segment is not the active segment and @@ -985,6 +964,27 @@ private boolean delayCopy(LogConfig logConfig, LogSegment previousSeg, long curr return copyLagBytes > 0 && !hasExceededCopyLagSize(previousSeg, totalLogSize, cumulativeSize, copyLagBytes); } + + private boolean hasExceededCopyLagTime(LogSegment segment, long currentTimeMs, long copyLagMs) { + try { + long segmentAgeMs = currentTimeMs - segment.largestTimestamp(); + boolean exceeded = segmentAgeMs >= copyLagMs; + logger.debug("{} eligible for upload by time? {} (segment age {} ms, copy lag {} ms)", + segment, exceeded, segmentAgeMs, copyLagMs); + return exceeded; + } catch (IOException e) { + logger.warn("Failed to get largest timestamp for segment {}, take it as eligible for upload based on time", segment, e); + return true; + } + } + + private boolean hasExceededCopyLagSize(LogSegment segment, long totalLogSize, long cumulativeSize, long copyLagBytes) { + long sizeLagBytes = totalLogSize - cumulativeSize; + boolean exceeded = sizeLagBytes >= copyLagBytes; + logger.debug("{} eligible for upload by size? {} (size lag {} bytes, copy lag {} bytes, totalLogSize={}, cumulativeSize={})", + segment, exceeded, sizeLagBytes, copyLagBytes, totalLogSize, cumulativeSize); + return exceeded; + } public void copyLogSegmentsToRemote(UnifiedLog log) throws InterruptedException, RetriableRemoteStorageException { if (isCancelled()) From 4050a5bc2faced08920607747edb31f922743ede Mon Sep 17 00:00:00 2001 From: Jian Date: Wed, 1 Apr 2026 21:57:32 +0800 Subject: [PATCH 073/110] Refine documentation for retention configuration options --- .../main/java/org/apache/kafka/common/config/TopicConfig.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index fe99a504d47d7..c8c98f8580c3e 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -107,7 +107,7 @@ public class TopicConfig { "When set to 0, segments are uploaded as soon as they are eligible without delay. " + "When set to a positive value (ms), a segment can't become eligible for upload util the time since the latest record in the segment reaches the value. " + "The value should not exceed the real local retention ms except the latter is save forever. " + - "When set to -1, resolves to the real local retention ms (maximum delay, or no delay check when the retention is save forever). " + + "When set to -1, resolves to the real local retention ms as maximum delay. " + "For how the real local retention time is computed, see local.retention.ms."; public static final String REMOTE_COPY_LAG_BYTES_CONFIG = "remote.copy.lag.bytes"; @@ -116,7 +116,7 @@ public class TopicConfig { "When set to 0, segments are uploaded as soon as they are eligible without delay. " + "When set to a positive value (bytes), a segment can't become eligible for upload util the total bytes of log data after the segment reach the value. " + "The value should not exceed the real local retention bytes except the latter is save forever. " + - "When set to -1, resolves to the real local retention bytes (maximum delay, or no delay check when the retention is save forever). " + + "When set to -1, resolves to the real local retention bytes as maximum delay. " + "For how the real local retention size is computed, see local.retention.bytes."; public static final String REMOTE_LOG_DELETE_ON_DISABLE_CONFIG = "remote.log.delete.on.disable"; From a1adb06887b9309651b05d327aff8979a97d07c3 Mon Sep 17 00:00:00 2001 From: Jian Date: Wed, 1 Apr 2026 21:58:49 +0800 Subject: [PATCH 074/110] Update RemoteLogManagerConfig.java --- .../server/log/remote/storage/RemoteLogManagerConfig.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java index 40b31cbcd25df..e8dda1ae38797 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java @@ -174,7 +174,7 @@ public final class RemoteLogManagerConfig { "When set to 0, segments are uploaded as soon as they are eligible without delay. " + "When set to a positive value (ms), a segment can't become eligible for upload util the time since the latest record in the segment reaches the value. " + "The value should not exceed the real local retention ms except the latter is save forever. " + - "When set to -1, resolves to the real local retention ms (maximum delay, or no delay check when the retention is save forever). " + + "When set to -1, resolves to the real local retention ms as maximum delay. " + "For how the real local retention time is computed, see log.local.retention.ms."; public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_MS = -2L; @@ -184,7 +184,7 @@ public final class RemoteLogManagerConfig { "When set to 0, segments are uploaded as soon as they are eligible without delay. " + "When set to a positive value (bytes), a segment can't become eligible for upload util the total bytes of log data after the segment reach the value. " + "The value should not exceed the real local retention bytes except the latter is save forever. " + - "When set to -1, resolves to the real local retention bytes (maximum delay, or no delay check when the retention is save forever). " + + "When set to -1, resolves to the real local retention bytes as maximum delay. " + "For how the real local retention size is computed, see log.local.retention.bytes."; public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_BYTES = -2L; From f2a1949c930625008b4a2216cc271ad727ae288c Mon Sep 17 00:00:00 2001 From: Jian Date: Wed, 1 Apr 2026 22:17:54 +0800 Subject: [PATCH 075/110] add more validation Signed-off-by: Jian --- .../main/scala/kafka/server/DynamicBrokerConfig.scala | 8 ++++++++ .../apache/kafka/storage/internals/log/LogConfig.java | 10 ++++++++++ 2 files changed, 18 insertions(+) diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index 645182c1d6641..6bc1f72ed1370 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -583,6 +583,10 @@ class DynamicLogConfig(logManager: LogManager, directoryEventHandler: DirectoryE val logLocalRetentionMs = newConfig.remoteLogManagerConfig.logLocalRetentionMs val effectiveLocalRetentionMs = if (logLocalRetentionMs == -2L) logRetentionMs else logLocalRetentionMs val logRemoteCopyLagMs = newConfig.remoteLogManagerConfig.logRemoteCopyLagMs + if (effectiveLocalRetentionMs == -1L && logRemoteCopyLagMs == -1L) { + throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_MS_PROP, logRemoteCopyLagMs, + s"Value must not be -1 when effective ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP} is -1") + } if (logRemoteCopyLagMs > 0L && effectiveLocalRetentionMs >= 0L && logRemoteCopyLagMs > effectiveLocalRetentionMs) { throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_MS_PROP, logRemoteCopyLagMs, s"Value must not exceed ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP} (effective value: $effectiveLocalRetentionMs)") @@ -594,6 +598,10 @@ class DynamicLogConfig(logManager: LogManager, directoryEventHandler: DirectoryE val logLocalRetentionBytes = newConfig.remoteLogManagerConfig.logLocalRetentionBytes val effectiveLocalRetentionBytes = if (logLocalRetentionBytes == -2L) logRetentionBytes else logLocalRetentionBytes val logRemoteCopyLagBytes = newConfig.remoteLogManagerConfig.logRemoteCopyLagBytes + if (effectiveLocalRetentionBytes == -1L && logRemoteCopyLagBytes == -1L) { + throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_BYTES_PROP, logRemoteCopyLagBytes, + s"Value must not be -1 when effective ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP} is -1") + } if (logRemoteCopyLagBytes > 0L && effectiveLocalRetentionBytes >= 0L && logRemoteCopyLagBytes > effectiveLocalRetentionBytes) { throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_BYTES_PROP, logRemoteCopyLagBytes, s"Value must not exceed ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP} (effective value: $effectiveLocalRetentionBytes)") diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 267b93d097997..8d5d2519069d5 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -629,6 +629,11 @@ private static void validateRemoteCopyLagTime(Map props) { Long localRetentionMs = (Long) props.get(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG); Long remoteCopyLagMs = (Long) props.get(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG); long effectiveLocalRetentionMs = localRetentionMs == -2 ? retentionMs : localRetentionMs; + if (effectiveLocalRetentionMs == -1 && remoteCopyLagMs == -1) { + String message = String.format("Value must not be -1 when effective %s is -1", + TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG); + throw new ConfigException(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, remoteCopyLagMs, message); + } if (remoteCopyLagMs > 0 && effectiveLocalRetentionMs >= 0 && remoteCopyLagMs > effectiveLocalRetentionMs) { String message = String.format("Value must not exceed %s (effective value: %d)", @@ -642,6 +647,11 @@ private static void validateRemoteCopyLagSize(Map props) { Long localRetentionBytes = (Long) props.get(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG); Long remoteCopyLagBytes = (Long) props.get(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG); long effectiveLocalRetentionBytes = localRetentionBytes == -2 ? retentionBytes : localRetentionBytes; + if (effectiveLocalRetentionBytes == -1 && remoteCopyLagBytes == -1) { + String message = String.format("Value must not be -1 when effective %s is -1", + TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG); + throw new ConfigException(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, remoteCopyLagBytes, message); + } if (remoteCopyLagBytes > 0 && effectiveLocalRetentionBytes >= 0 && remoteCopyLagBytes > effectiveLocalRetentionBytes) { String message = String.format("Value must not exceed %s (effective value: %d)", From 4dbd1deab8238191cc52f563c633d05f2e9cb032 Mon Sep 17 00:00:00 2001 From: Jian Date: Thu, 2 Apr 2026 06:12:45 +0800 Subject: [PATCH 076/110] Fix return value in copy lag condition --- .../kafka/server/log/remote/storage/RemoteLogManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index 432dc2450f662..60e5b4748d16d 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -959,7 +959,7 @@ private boolean delayCopy(LogConfig logConfig, LogSegment previousSeg, long curr } if (copyLagMs > 0 && hasExceededCopyLagTime(previousSeg, currentTimeMs, copyLagMs)) { - return true; + return false; } return copyLagBytes > 0 && !hasExceededCopyLagSize(previousSeg, totalLogSize, cumulativeSize, copyLagBytes); From 55734628bb50ec1b6cb2b4b5772a8ae1ac828cfa Mon Sep 17 00:00:00 2001 From: Jian Date: Thu, 2 Apr 2026 06:18:17 +0800 Subject: [PATCH 077/110] Fix documentation typos in RemoteLogManagerConfig --- .../server/log/remote/storage/RemoteLogManagerConfig.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java index e8dda1ae38797..445c5c2c4312a 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java @@ -171,7 +171,7 @@ public final class RemoteLogManagerConfig { public static final String LOG_REMOTE_COPY_LAG_MS_PROP = "log.remote.copy.lag.ms"; public static final String LOG_REMOTE_COPY_LAG_MS_DOC = "Controls how long to delay uploading segments to remote storage. " + "When set to -2 (default), no delay check based on local retention ms. " + - "When set to 0, segments are uploaded as soon as they are eligible without delay. " + + "When set to 0, segments are uploaded as soon as they are eligible without any delay. " + "When set to a positive value (ms), a segment can't become eligible for upload util the time since the latest record in the segment reaches the value. " + "The value should not exceed the real local retention ms except the latter is save forever. " + "When set to -1, resolves to the real local retention ms as maximum delay. " + @@ -181,7 +181,7 @@ public final class RemoteLogManagerConfig { public static final String LOG_REMOTE_COPY_LAG_BYTES_PROP = "log.remote.copy.lag.bytes"; public static final String LOG_REMOTE_COPY_LAG_BYTES_DOC = "Controls size-based delay for uploading segments to remote storage. " + "When set to -2 (default), no delay check based on local retention bytes. " + - "When set to 0, segments are uploaded as soon as they are eligible without delay. " + + "When set to 0, segments are uploaded as soon as they are eligible without any delay check. " + "When set to a positive value (bytes), a segment can't become eligible for upload util the total bytes of log data after the segment reach the value. " + "The value should not exceed the real local retention bytes except the latter is save forever. " + "When set to -1, resolves to the real local retention bytes as maximum delay. " + From a5dbf91a09b9b79a1ccfadf7ad40efc9fde71c0b Mon Sep 17 00:00:00 2001 From: Jian Date: Thu, 2 Apr 2026 06:22:18 +0800 Subject: [PATCH 078/110] Fix documentation for remote copy lag configuration --- .../main/java/org/apache/kafka/common/config/TopicConfig.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index c8c98f8580c3e..0d555f754e48e 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -104,7 +104,7 @@ public class TopicConfig { public static final String REMOTE_COPY_LAG_MS_CONFIG = "remote.copy.lag.ms"; public static final String REMOTE_COPY_LAG_MS_DOC = "Controls how long to delay uploading segments to remote storage. " + "When set to -2 (default), no delay check based on local retention ms. " + - "When set to 0, segments are uploaded as soon as they are eligible without delay. " + + "When set to 0, segments are uploaded as soon as they are eligible without any delay check. " + "When set to a positive value (ms), a segment can't become eligible for upload util the time since the latest record in the segment reaches the value. " + "The value should not exceed the real local retention ms except the latter is save forever. " + "When set to -1, resolves to the real local retention ms as maximum delay. " + @@ -113,7 +113,7 @@ public class TopicConfig { public static final String REMOTE_COPY_LAG_BYTES_CONFIG = "remote.copy.lag.bytes"; public static final String REMOTE_COPY_LAG_BYTES_DOC = "Controls size-based delay for uploading segments to remote storage. " + "When set to -2 (default), no delay check based on local retention bytes. " + - "When set to 0, segments are uploaded as soon as they are eligible without delay. " + + "When set to 0, segments are uploaded as soon as they are eligible without any delay check. " + "When set to a positive value (bytes), a segment can't become eligible for upload util the total bytes of log data after the segment reach the value. " + "The value should not exceed the real local retention bytes except the latter is save forever. " + "When set to -1, resolves to the real local retention bytes as maximum delay. " + From 3ac973d05c36869846b0381c3363593e95bbaadd Mon Sep 17 00:00:00 2001 From: Jian Date: Thu, 2 Apr 2026 06:24:07 +0800 Subject: [PATCH 079/110] Fix documentation for LOG_REMOTE_COPY_LAG_MS_PROP --- .../kafka/server/log/remote/storage/RemoteLogManagerConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java index 445c5c2c4312a..b1305bcfe8722 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java @@ -171,7 +171,7 @@ public final class RemoteLogManagerConfig { public static final String LOG_REMOTE_COPY_LAG_MS_PROP = "log.remote.copy.lag.ms"; public static final String LOG_REMOTE_COPY_LAG_MS_DOC = "Controls how long to delay uploading segments to remote storage. " + "When set to -2 (default), no delay check based on local retention ms. " + - "When set to 0, segments are uploaded as soon as they are eligible without any delay. " + + "When set to 0, segments are uploaded as soon as they are eligible without any delay check. " + "When set to a positive value (ms), a segment can't become eligible for upload util the time since the latest record in the segment reaches the value. " + "The value should not exceed the real local retention ms except the latter is save forever. " + "When set to -1, resolves to the real local retention ms as maximum delay. " + From 075ea7319ef1bd1b77330cc23323673db88c06d5 Mon Sep 17 00:00:00 2001 From: Jian Date: Thu, 2 Apr 2026 06:30:11 +0800 Subject: [PATCH 080/110] add more description Signed-off-by: Jian --- .../main/java/org/apache/kafka/common/config/TopicConfig.java | 2 ++ .../kafka/server/log/remote/storage/RemoteLogManagerConfig.java | 2 ++ 2 files changed, 4 insertions(+) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index 0d555f754e48e..0a4612683a5d3 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -108,6 +108,7 @@ public class TopicConfig { "When set to a positive value (ms), a segment can't become eligible for upload util the time since the latest record in the segment reaches the value. " + "The value should not exceed the real local retention ms except the latter is save forever. " + "When set to -1, resolves to the real local retention ms as maximum delay. " + + "If the real local retention ms is configured as infinite, -1 is treated as an invalid configuration. " + "For how the real local retention time is computed, see local.retention.ms."; public static final String REMOTE_COPY_LAG_BYTES_CONFIG = "remote.copy.lag.bytes"; @@ -117,6 +118,7 @@ public class TopicConfig { "When set to a positive value (bytes), a segment can't become eligible for upload util the total bytes of log data after the segment reach the value. " + "The value should not exceed the real local retention bytes except the latter is save forever. " + "When set to -1, resolves to the real local retention bytes as maximum delay. " + + "If the real local retention bytes is configured as infinite, -1 is treated as an invalid configuration. " + "For how the real local retention size is computed, see local.retention.bytes."; public static final String REMOTE_LOG_DELETE_ON_DISABLE_CONFIG = "remote.log.delete.on.disable"; diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java index b1305bcfe8722..6278bed0621d9 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java @@ -175,6 +175,7 @@ public final class RemoteLogManagerConfig { "When set to a positive value (ms), a segment can't become eligible for upload util the time since the latest record in the segment reaches the value. " + "The value should not exceed the real local retention ms except the latter is save forever. " + "When set to -1, resolves to the real local retention ms as maximum delay. " + + "If the real local retention ms is configured as infinite, -1 is treated as an invalid configuration. " + "For how the real local retention time is computed, see log.local.retention.ms."; public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_MS = -2L; @@ -185,6 +186,7 @@ public final class RemoteLogManagerConfig { "When set to a positive value (bytes), a segment can't become eligible for upload util the total bytes of log data after the segment reach the value. " + "The value should not exceed the real local retention bytes except the latter is save forever. " + "When set to -1, resolves to the real local retention bytes as maximum delay. " + + "If the real local retention bytes is configured as infinite, -1 is treated as an invalid configuration. " + "For how the real local retention size is computed, see log.local.retention.bytes."; public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_BYTES = -2L; From 6aa8c91de581a46d98e65c7c0a0ebac105aef681 Mon Sep 17 00:00:00 2001 From: Jian Date: Thu, 2 Apr 2026 06:42:38 +0800 Subject: [PATCH 081/110] improve the documents Signed-off-by: Jian --- .../main/java/org/apache/kafka/common/config/TopicConfig.java | 4 ++-- .../server/log/remote/storage/RemoteLogManagerConfig.java | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index 0a4612683a5d3..0dfa68004407d 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -106,7 +106,7 @@ public class TopicConfig { "When set to -2 (default), no delay check based on local retention ms. " + "When set to 0, segments are uploaded as soon as they are eligible without any delay check. " + "When set to a positive value (ms), a segment can't become eligible for upload util the time since the latest record in the segment reaches the value. " + - "The value should not exceed the real local retention ms except the latter is save forever. " + + "The value should not exceed the real local retention ms except the latter is retained indefinitely (-1). " + "When set to -1, resolves to the real local retention ms as maximum delay. " + "If the real local retention ms is configured as infinite, -1 is treated as an invalid configuration. " + "For how the real local retention time is computed, see local.retention.ms."; @@ -116,7 +116,7 @@ public class TopicConfig { "When set to -2 (default), no delay check based on local retention bytes. " + "When set to 0, segments are uploaded as soon as they are eligible without any delay check. " + "When set to a positive value (bytes), a segment can't become eligible for upload util the total bytes of log data after the segment reach the value. " + - "The value should not exceed the real local retention bytes except the latter is save forever. " + + "The value should not exceed the real local retention bytes except the latter is retained indefinitely (-1). " + "When set to -1, resolves to the real local retention bytes as maximum delay. " + "If the real local retention bytes is configured as infinite, -1 is treated as an invalid configuration. " + "For how the real local retention size is computed, see local.retention.bytes."; diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java index 6278bed0621d9..7c5924cc8524a 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java @@ -173,7 +173,7 @@ public final class RemoteLogManagerConfig { "When set to -2 (default), no delay check based on local retention ms. " + "When set to 0, segments are uploaded as soon as they are eligible without any delay check. " + "When set to a positive value (ms), a segment can't become eligible for upload util the time since the latest record in the segment reaches the value. " + - "The value should not exceed the real local retention ms except the latter is save forever. " + + "The value should not exceed the real local retention ms except the latter is retained indefinitely (-1). " + "When set to -1, resolves to the real local retention ms as maximum delay. " + "If the real local retention ms is configured as infinite, -1 is treated as an invalid configuration. " + "For how the real local retention time is computed, see log.local.retention.ms."; @@ -184,7 +184,7 @@ public final class RemoteLogManagerConfig { "When set to -2 (default), no delay check based on local retention bytes. " + "When set to 0, segments are uploaded as soon as they are eligible without any delay check. " + "When set to a positive value (bytes), a segment can't become eligible for upload util the total bytes of log data after the segment reach the value. " + - "The value should not exceed the real local retention bytes except the latter is save forever. " + + "The value should not exceed the real local retention bytes except the latter is retained indefinitely (-1). " + "When set to -1, resolves to the real local retention bytes as maximum delay. " + "If the real local retention bytes is configured as infinite, -1 is treated as an invalid configuration. " + "For how the real local retention size is computed, see log.local.retention.bytes."; From 7fe96a47b96e43268f3c87d8fb6cb9447492a637 Mon Sep 17 00:00:00 2001 From: Jian Date: Thu, 2 Apr 2026 13:58:58 +0800 Subject: [PATCH 082/110] fix code issue which cause unit test failed Signed-off-by: Jian --- .../org/apache/kafka/storage/internals/log/LogConfig.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 8d5d2519069d5..1e2155cec1316 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -413,11 +413,11 @@ public Boolean remoteLogCopyDisable() { public long remoteCopyLagMs() { - return remoteLogConfig.remoteCopyLagMs == MAX_REMOTE_COPY_LAG_MS ? remoteLogConfig.remoteCopyLagMs : localRetentionMs(); + return remoteLogConfig.remoteCopyLagMs == MAX_REMOTE_COPY_LAG_MS ? localRetentionMs() : remoteLogConfig.remoteCopyLagMs; } public long remoteCopyLagBytes() { - return remoteLogConfig.remoteCopyLagBytes == MAX_REMOTE_COPY_LAG_BYTES ? remoteLogConfig.remoteCopyLagBytes : localRetentionBytes(); + return remoteLogConfig.remoteCopyLagBytes == MAX_REMOTE_COPY_LAG_BYTES ? localRetentionBytes() : remoteLogConfig.remoteCopyLagBytes; } public long localRetentionMs() { From bfc753074b45652b77855d5bad17d765be2843f4 Mon Sep 17 00:00:00 2001 From: Jian Date: Thu, 2 Apr 2026 17:31:30 +0800 Subject: [PATCH 083/110] remove example line --- .../apache/kafka/server/log/remote/storage/RemoteLogManager.java | 1 - 1 file changed, 1 deletion(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index 60e5b4748d16d..35f08505fe0de 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -929,7 +929,6 @@ List candidateLogSegments(UnifiedLog log, Long fromOffset, L long currentTimeMs = time.milliseconds(); long totalLogSize = UnifiedLog.sizeInBytes(segments); long cumulativeSize = 0; - for (int idx = 1; idx < segments.size(); idx++) { LogSegment previousSeg = segments.get(idx - 1); LogSegment currentSeg = segments.get(idx); From a06de2b334aea0c4d60091e3247f25633cc0549b Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 3 Apr 2026 11:01:54 +0800 Subject: [PATCH 084/110] add more debug log Signed-off-by: Jian --- .../kafka/server/log/remote/storage/RemoteLogManager.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index 35f08505fe0de..e575066a23eaf 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -952,6 +952,8 @@ private boolean delayCopy(LogConfig logConfig, LogSegment previousSeg, long curr long copyLagMs = logConfig.remoteCopyLagMs(); long copyLagBytes = logConfig.remoteCopyLagBytes(); + logger.debug("delayCopy check for segment {}: copyLagMs={}, copyLagBytes={}, currentTimeMs={}, totalLogSize={}, cumulativeSize={}, sizeLagBytes={}", + previousSeg, copyLagMs, copyLagBytes, currentTimeMs, totalLogSize, cumulativeSize, totalLogSize - cumulativeSize); if (copyLagMs == 0 || copyLagBytes == 0) { return false; From 9115afd22ade5393f4bba7e55abd7226523e3194 Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 3 Apr 2026 12:33:15 +0800 Subject: [PATCH 085/110] refactor the basic log Signed-off-by: Jian --- .../log/remote/storage/RemoteLogManager.java | 25 +++++++++++++------ 1 file changed, 18 insertions(+), 7 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index e575066a23eaf..9c0e374c3828e 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -958,33 +958,44 @@ private boolean delayCopy(LogConfig logConfig, LogSegment previousSeg, long curr if (copyLagMs == 0 || copyLagBytes == 0) { return false; } - - if (copyLagMs > 0 && hasExceededCopyLagTime(previousSeg, currentTimeMs, copyLagMs)) { + + if (copyLagMs == LogConfig.DEFAULT_REMOTE_COPY_LAG_MS + && copyLagBytes == LogConfig.DEFAULT_REMOTE_COPY_LAG_BYTES) { return false; } - return copyLagBytes > 0 && !hasExceededCopyLagSize(previousSeg, totalLogSize, cumulativeSize, copyLagBytes); + if (copyLagMs != LogConfig.DEFAULT_REMOTE_COPY_LAG_MS + && copyLagBytes != LogConfig.DEFAULT_REMOTE_COPY_LAG_BYTES) { + return notExceededCopyLagTime(previousSeg, currentTimeMs, copyLagMs) + && notExceededCopyLagSize(previousSeg, totalLogSize, cumulativeSize, copyLagBytes); + } + + if (copyLagMs != LogConfig.DEFAULT_REMOTE_COPY_LAG_MS) { + return notExceededCopyLagTime(previousSeg, currentTimeMs, copyLagMs); + } + + return notExceededCopyLagSize(previousSeg, totalLogSize, cumulativeSize, copyLagBytes); } - private boolean hasExceededCopyLagTime(LogSegment segment, long currentTimeMs, long copyLagMs) { + private boolean notExceededCopyLagTime(LogSegment segment, long currentTimeMs, long copyLagMs) { try { long segmentAgeMs = currentTimeMs - segment.largestTimestamp(); boolean exceeded = segmentAgeMs >= copyLagMs; logger.debug("{} eligible for upload by time? {} (segment age {} ms, copy lag {} ms)", segment, exceeded, segmentAgeMs, copyLagMs); - return exceeded; + return !exceeded; } catch (IOException e) { logger.warn("Failed to get largest timestamp for segment {}, take it as eligible for upload based on time", segment, e); return true; } } - private boolean hasExceededCopyLagSize(LogSegment segment, long totalLogSize, long cumulativeSize, long copyLagBytes) { + private boolean notExceededCopyLagSize(LogSegment segment, long totalLogSize, long cumulativeSize, long copyLagBytes) { long sizeLagBytes = totalLogSize - cumulativeSize; boolean exceeded = sizeLagBytes >= copyLagBytes; logger.debug("{} eligible for upload by size? {} (size lag {} bytes, copy lag {} bytes, totalLogSize={}, cumulativeSize={})", segment, exceeded, sizeLagBytes, copyLagBytes, totalLogSize, cumulativeSize); - return exceeded; + return !exceeded; } public void copyLogSegmentsToRemote(UnifiedLog log) throws InterruptedException, RetriableRemoteStorageException { From a23d2ea3d59c18bf53066b72ad6033f241f42b01 Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 3 Apr 2026 17:05:18 +0800 Subject: [PATCH 086/110] Refactor copy lag checks in RemoteLogManager --- .../server/log/remote/storage/RemoteLogManager.java | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index 9c0e374c3828e..2305393cba845 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -959,15 +959,12 @@ private boolean delayCopy(LogConfig logConfig, LogSegment previousSeg, long curr return false; } - if (copyLagMs == LogConfig.DEFAULT_REMOTE_COPY_LAG_MS - && copyLagBytes == LogConfig.DEFAULT_REMOTE_COPY_LAG_BYTES) { + if (copyLagMs == LogConfig.DEFAULT_REMOTE_COPY_LAG_MS && copyLagBytes == LogConfig.DEFAULT_REMOTE_COPY_LAG_BYTES) { return false; } - if (copyLagMs != LogConfig.DEFAULT_REMOTE_COPY_LAG_MS - && copyLagBytes != LogConfig.DEFAULT_REMOTE_COPY_LAG_BYTES) { - return notExceededCopyLagTime(previousSeg, currentTimeMs, copyLagMs) - && notExceededCopyLagSize(previousSeg, totalLogSize, cumulativeSize, copyLagBytes); + if (copyLagMs != LogConfig.DEFAULT_REMOTE_COPY_LAG_MS && copyLagBytes != LogConfig.DEFAULT_REMOTE_COPY_LAG_BYTES) { + return notExceededCopyLagTime(previousSeg, currentTimeMs, copyLagMs) && notExceededCopyLagSize(previousSeg, totalLogSize, cumulativeSize, copyLagBytes); } if (copyLagMs != LogConfig.DEFAULT_REMOTE_COPY_LAG_MS) { From 3d966a14fd1a7c5ba7f9256084815cf069177103 Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 3 Apr 2026 18:35:04 +0800 Subject: [PATCH 087/110] Add log configure unit test Signed-off-by: Jian --- .../scala/unit/kafka/log/LogConfigTest.scala | 55 +++++++++++++++++++ 1 file changed, 55 insertions(+) diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index 7431131eaffa0..78b168a0afede 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -259,6 +259,61 @@ class LogConfigTest { doTestInvalidLocalLogRetentionProps(2000L, -1, 100, 1000L) } + @Test + def testInvalidRemoteCopyLagMsWhenEffectiveLocalRetentionMsIsUnlimited(): Unit = { + val props = new util.HashMap[String, String]() + props.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") + props.put(TopicConfig.RETENTION_MS_CONFIG, "-1") + props.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, "-2") + props.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, "-1") + + val exception = assertThrows(classOf[ConfigException], () => validateTopicLogConfig(props)) + assertTrue(exception.getMessage.contains(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG)) + } + + @Test + def testInvalidRemoteCopyLagMsWhenGreaterThanEffectiveLocalRetentionMs(): Unit = { + val props = new util.HashMap[String, String]() + props.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") + props.put(TopicConfig.RETENTION_MS_CONFIG, "1000") + props.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, "-2") + props.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, "1001") + + val exception = assertThrows(classOf[ConfigException], () => validateTopicLogConfig(props)) + assertTrue(exception.getMessage.contains(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG)) + } + + @Test + def testInvalidRemoteCopyLagBytesWhenEffectiveLocalRetentionBytesIsUnlimited(): Unit = { + val props = new util.HashMap[String, String]() + props.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") + props.put(TopicConfig.RETENTION_BYTES_CONFIG, "-1") + props.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, "-2") + props.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, "-1") + + val exception = assertThrows(classOf[ConfigException], () => validateTopicLogConfig(props)) + assertTrue(exception.getMessage.contains(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG)) + } + + @Test + def testInvalidRemoteCopyLagBytesWhenGreaterThanEffectiveLocalRetentionBytes(): Unit = { + val props = new util.HashMap[String, String]() + props.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") + props.put(TopicConfig.RETENTION_BYTES_CONFIG, "1000") + props.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, "-2") + props.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, "1001") + + val exception = assertThrows(classOf[ConfigException], () => validateTopicLogConfig(props)) + assertTrue(exception.getMessage.contains(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG)) + } + + private def validateTopicLogConfig(props: util.Map[String, String]): Unit = { + val kafkaProps = TestUtils.createDummyBrokerConfig() + kafkaProps.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, "true") + val kafkaConfig = KafkaConfig.fromProps(kafkaProps) + LogConfig.validate(util.Map.of, props, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled) + } + private def doTestInvalidLocalLogRetentionProps(localRetentionMs: Long, localRetentionBytes: Int, retentionBytes: Int, From fba4cea6e3f2220195899c2171c2742057aedf8f Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 3 Apr 2026 19:20:42 +0800 Subject: [PATCH 088/110] Add log configure unit test Signed-off-by: Jian --- .../server/DynamicBrokerConfigTest.scala | 68 +++++++++++++++++++ 1 file changed, 68 insertions(+) diff --git a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala index 012993c7763c4..cfa014d699d78 100755 --- a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala @@ -770,6 +770,49 @@ class DynamicBrokerConfigTest { verifyIncorrectLogLocalRetentionProps(2000L, 1000L, -1, 100) } + @Test + def testDynamicRemoteCopyLagThrowsOnIncorrectConfig(): Unit = { + // effective local retention ms = -1 and remote copy lag ms = -1 is invalid + verifyIncorrectRemoteCopyLagProps( + retentionMs = -1L, + logLocalRetentionMs = -2L, + remoteCopyLagMs = -1L, + retentionBytes = 1000L, + logLocalRetentionBytes = -2L, + remoteCopyLagBytes = 100L + ) + + // remote copy lag ms cannot exceed effective local retention ms + verifyIncorrectRemoteCopyLagProps( + retentionMs = 1000L, + logLocalRetentionMs = -2L, + remoteCopyLagMs = 1001L, + retentionBytes = 1000L, + logLocalRetentionBytes = -2L, + remoteCopyLagBytes = 100L + ) + + // effective local retention bytes = -1 and remote copy lag bytes = -1 is invalid + verifyIncorrectRemoteCopyLagProps( + retentionMs = 1000L, + logLocalRetentionMs = -2L, + remoteCopyLagMs = 100L, + retentionBytes = -1L, + logLocalRetentionBytes = -2L, + remoteCopyLagBytes = -1L + ) + + // remote copy lag bytes cannot exceed effective local retention bytes + verifyIncorrectRemoteCopyLagProps( + retentionMs = 1000L, + logLocalRetentionMs = -2L, + remoteCopyLagMs = 100L, + retentionBytes = 1000L, + logLocalRetentionBytes = -2L, + remoteCopyLagBytes = 1001L + ) + } + @Test def testDynamicRemoteFetchMaxWaitMsConfig(): Unit = { val props = TestUtils.createBrokerConfig(0, port = 8181) @@ -1024,6 +1067,31 @@ class DynamicBrokerConfigTest { assertThrows(classOf[ConfigException], () => config.dynamicConfig.validate(newProps, perBrokerConfig = true)) } + def verifyIncorrectRemoteCopyLagProps(retentionMs: Long, + logLocalRetentionMs: Long, + remoteCopyLagMs: Long, + retentionBytes: Long, + logLocalRetentionBytes: Long, + remoteCopyLagBytes: Long): Unit = { + val props = TestUtils.createBrokerConfig(0, port = 8181) + props.put(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, retentionMs.toString) + props.put(ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG, retentionBytes.toString) + val config = KafkaConfig(props) + val dynamicLogConfig = new DynamicLogConfig(mock(classOf[LogManager]), mock(classOf[DirectoryEventHandler])) + config.dynamicConfig.initialize(None) + config.dynamicConfig.addBrokerReconfigurable(dynamicLogConfig) + + val newProps = new Properties() + newProps.put(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP, logLocalRetentionMs.toString) + newProps.put(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_MS_PROP, remoteCopyLagMs.toString) + newProps.put(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP, logLocalRetentionBytes.toString) + newProps.put(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_BYTES_PROP, remoteCopyLagBytes.toString) + // validate default config + assertThrows(classOf[ConfigException], () => config.dynamicConfig.validate(newProps, perBrokerConfig = false)) + // validate per broker config + assertThrows(classOf[ConfigException], () => config.dynamicConfig.validate(newProps, perBrokerConfig = true)) + } + class DynamicLogConfigContext(origProps: Properties) { val config = KafkaConfig(origProps) val serverMock = Mockito.mock(classOf[BrokerServer]) From 0a7038bccec0bae6ef38852aafccdc46dfd3a7ab Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 3 Apr 2026 19:27:51 +0800 Subject: [PATCH 089/110] Add log configure unit test Signed-off-by: Jian --- .../server/DynamicBrokerConfigTest.scala | 50 +++++++++---------- 1 file changed, 25 insertions(+), 25 deletions(-) diff --git a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala index cfa014d699d78..a787071de4b57 100755 --- a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala @@ -813,6 +813,31 @@ class DynamicBrokerConfigTest { ) } + def verifyIncorrectRemoteCopyLagProps(retentionMs: Long, + logLocalRetentionMs: Long, + remoteCopyLagMs: Long, + retentionBytes: Long, + logLocalRetentionBytes: Long, + remoteCopyLagBytes: Long): Unit = { + val props = TestUtils.createBrokerConfig(0, port = 8181) + props.put(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, retentionMs.toString) + props.put(ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG, retentionBytes.toString) + val config = KafkaConfig(props) + val dynamicLogConfig = new DynamicLogConfig(mock(classOf[LogManager]), mock(classOf[DirectoryEventHandler])) + config.dynamicConfig.initialize(None) + config.dynamicConfig.addBrokerReconfigurable(dynamicLogConfig) + + val newProps = new Properties() + newProps.put(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP, logLocalRetentionMs.toString) + newProps.put(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_MS_PROP, remoteCopyLagMs.toString) + newProps.put(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP, logLocalRetentionBytes.toString) + newProps.put(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_BYTES_PROP, remoteCopyLagBytes.toString) + // validate default config + assertThrows(classOf[ConfigException], () => config.dynamicConfig.validate(newProps, perBrokerConfig = false)) + // validate per broker config + assertThrows(classOf[ConfigException], () => config.dynamicConfig.validate(newProps, perBrokerConfig = true)) + } + @Test def testDynamicRemoteFetchMaxWaitMsConfig(): Unit = { val props = TestUtils.createBrokerConfig(0, port = 8181) @@ -1067,31 +1092,6 @@ class DynamicBrokerConfigTest { assertThrows(classOf[ConfigException], () => config.dynamicConfig.validate(newProps, perBrokerConfig = true)) } - def verifyIncorrectRemoteCopyLagProps(retentionMs: Long, - logLocalRetentionMs: Long, - remoteCopyLagMs: Long, - retentionBytes: Long, - logLocalRetentionBytes: Long, - remoteCopyLagBytes: Long): Unit = { - val props = TestUtils.createBrokerConfig(0, port = 8181) - props.put(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, retentionMs.toString) - props.put(ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG, retentionBytes.toString) - val config = KafkaConfig(props) - val dynamicLogConfig = new DynamicLogConfig(mock(classOf[LogManager]), mock(classOf[DirectoryEventHandler])) - config.dynamicConfig.initialize(None) - config.dynamicConfig.addBrokerReconfigurable(dynamicLogConfig) - - val newProps = new Properties() - newProps.put(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP, logLocalRetentionMs.toString) - newProps.put(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_MS_PROP, remoteCopyLagMs.toString) - newProps.put(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP, logLocalRetentionBytes.toString) - newProps.put(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_BYTES_PROP, remoteCopyLagBytes.toString) - // validate default config - assertThrows(classOf[ConfigException], () => config.dynamicConfig.validate(newProps, perBrokerConfig = false)) - // validate per broker config - assertThrows(classOf[ConfigException], () => config.dynamicConfig.validate(newProps, perBrokerConfig = true)) - } - class DynamicLogConfigContext(origProps: Properties) { val config = KafkaConfig(origProps) val serverMock = Mockito.mock(classOf[BrokerServer]) From f377844cfa53afbe3112f5868bffc964d42f39be Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 3 Apr 2026 19:56:12 +0800 Subject: [PATCH 090/110] Fix one bug and add more tests Signed-off-by: Jian --- .../log/remote/storage/RemoteLogManager.java | 4 +- .../remote/storage/RemoteLogManagerTest.java | 156 ++++++++++++++++++ 2 files changed, 158 insertions(+), 2 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index 2305393cba845..8c8d5d7f2c56f 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -964,7 +964,7 @@ private boolean delayCopy(LogConfig logConfig, LogSegment previousSeg, long curr } if (copyLagMs != LogConfig.DEFAULT_REMOTE_COPY_LAG_MS && copyLagBytes != LogConfig.DEFAULT_REMOTE_COPY_LAG_BYTES) { - return notExceededCopyLagTime(previousSeg, currentTimeMs, copyLagMs) && notExceededCopyLagSize(previousSeg, totalLogSize, cumulativeSize, copyLagBytes); + return notExceededCopyLagTime(previousSeg, currentTimeMs, copyLagMs) && notExceededCopyLagSize(previousSeg, totalLogSize, cumulativeSize, copyLagBytes); } if (copyLagMs != LogConfig.DEFAULT_REMOTE_COPY_LAG_MS) { @@ -983,7 +983,7 @@ private boolean notExceededCopyLagTime(LogSegment segment, long currentTimeMs, l return !exceeded; } catch (IOException e) { logger.warn("Failed to get largest timestamp for segment {}, take it as eligible for upload based on time", segment, e); - return true; + return false; } } diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java index 234bb0f06f4ef..46f0f99a933e2 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java @@ -23,6 +23,7 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.common.compress.Compression; import org.apache.kafka.common.config.AbstractConfig; +import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.errors.ReplicaNotAvailableException; import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.metrics.Metrics; @@ -2160,6 +2161,161 @@ public void testCandidateLogSegmentsSkipsSegmentsAfterLastStableOffset() { assertEquals(expected, actual); } + @Test + public void testCandidateLogSegmentsDelayUploadWhenRemoteCopyLagMsNotExceeded() throws IOException { + UnifiedLog log = mock(UnifiedLog.class); + LogSegment segment1 = mock(LogSegment.class); + LogSegment segment2 = mock(LogSegment.class); + LogSegment activeSegment = mock(LogSegment.class); + + when(segment1.baseOffset()).thenReturn(5L); + when(segment2.baseOffset()).thenReturn(10L); + when(activeSegment.baseOffset()).thenReturn(15L); + when(segment1.size()).thenReturn(100); + when(segment2.size()).thenReturn(100); + when(activeSegment.size()).thenReturn(100); + + Map logProps = new HashMap<>(); + logProps.put(TopicConfig.RETENTION_MS_CONFIG, 10_000L); + logProps.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, 100L); + LogConfig logConfig = new LogConfig(logProps); + when(log.config()).thenReturn(logConfig); + when(log.logSegments(5L, Long.MAX_VALUE)).thenReturn(List.of(segment1, segment2, activeSegment)); + + time.sleep(1000L); + when(segment1.largestTimestamp()).thenReturn(time.milliseconds() - 50L); + RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, 128); + List actual = task.candidateLogSegments(log, 5L, 20L); + assertTrue(actual.isEmpty()); + } + + @Test + public void testCandidateLogSegmentsUploadWhenRemoteCopyLagMsReachedBoundary() throws IOException { + UnifiedLog log = mock(UnifiedLog.class); + LogSegment segment1 = mock(LogSegment.class); + LogSegment segment2 = mock(LogSegment.class); + LogSegment activeSegment = mock(LogSegment.class); + + when(segment1.baseOffset()).thenReturn(5L); + when(segment2.baseOffset()).thenReturn(10L); + when(activeSegment.baseOffset()).thenReturn(15L); + when(segment1.size()).thenReturn(100); + when(segment2.size()).thenReturn(100); + when(activeSegment.size()).thenReturn(100); + + Map logProps = new HashMap<>(); + logProps.put(TopicConfig.RETENTION_MS_CONFIG, 10_000L); + logProps.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, 100L); + LogConfig logConfig = new LogConfig(logProps); + when(log.config()).thenReturn(logConfig); + when(log.logSegments(5L, Long.MAX_VALUE)).thenReturn(List.of(segment1, segment2, activeSegment)); + + time.sleep(1000L); + when(segment1.largestTimestamp()).thenReturn(time.milliseconds() - 100L); + when(segment2.largestTimestamp()).thenReturn(time.milliseconds() - 50L); + RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, 128); + List expected = + List.of( + new RemoteLogManager.EnrichedLogSegment(segment1, 10L) + ); + List actual = task.candidateLogSegments(log, 5L, 20L); + assertEquals(expected, actual); + } + + @Test + public void testCandidateLogSegmentsDelayUploadWhenRemoteCopyLagBytesNotExceeded() { + UnifiedLog log = mock(UnifiedLog.class); + LogSegment segment1 = mock(LogSegment.class); + LogSegment segment2 = mock(LogSegment.class); + LogSegment activeSegment = mock(LogSegment.class); + + when(segment1.baseOffset()).thenReturn(5L); + when(segment2.baseOffset()).thenReturn(10L); + when(activeSegment.baseOffset()).thenReturn(15L); + when(segment1.size()).thenReturn(40); + when(segment2.size()).thenReturn(30); + when(activeSegment.size()).thenReturn(20); + + Map logProps = new HashMap<>(); + logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, 10_000L); + logProps.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, 60L); + LogConfig logConfig = new LogConfig(logProps); + when(log.config()).thenReturn(logConfig); + when(log.logSegments(5L, Long.MAX_VALUE)).thenReturn(List.of(segment1, segment2, activeSegment)); + + RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, 128); + List actual = task.candidateLogSegments(log, 5L, 20L); + assertTrue(actual.isEmpty()); + } + + @Test + public void testCandidateLogSegmentsUploadWhenEitherLagConditionExceeded() throws IOException { + UnifiedLog log = mock(UnifiedLog.class); + LogSegment segment1 = mock(LogSegment.class); + LogSegment segment2 = mock(LogSegment.class); + LogSegment activeSegment = mock(LogSegment.class); + + when(segment1.baseOffset()).thenReturn(5L); + when(segment2.baseOffset()).thenReturn(10L); + when(activeSegment.baseOffset()).thenReturn(15L); + when(segment1.size()).thenReturn(40); + when(segment2.size()).thenReturn(30); + when(activeSegment.size()).thenReturn(20); + + Map logProps = new HashMap<>(); + logProps.put(TopicConfig.RETENTION_MS_CONFIG, 10_000L); + logProps.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, 100L); + logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, 10_000L); + logProps.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, 60L); + LogConfig logConfig = new LogConfig(logProps); + when(log.config()).thenReturn(logConfig); + when(log.logSegments(5L, Long.MAX_VALUE)).thenReturn(List.of(segment1, segment2, activeSegment)); + + time.sleep(1000L); + when(segment1.largestTimestamp()).thenReturn(time.milliseconds() - 100L); + when(segment2.largestTimestamp()).thenReturn(time.milliseconds() - 20L); + RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, 128); + List expected = + List.of( + new RemoteLogManager.EnrichedLogSegment(segment1, 10L) + ); + List actual = task.candidateLogSegments(log, 5L, 20L); + assertEquals(expected, actual); + } + + @Test + public void testCandidateLogSegmentsUploadWhenLargestTimestampLookupFails() throws IOException { + UnifiedLog log = mock(UnifiedLog.class); + LogSegment segment1 = mock(LogSegment.class); + LogSegment segment2 = mock(LogSegment.class); + LogSegment activeSegment = mock(LogSegment.class); + + when(segment1.baseOffset()).thenReturn(5L); + when(segment2.baseOffset()).thenReturn(10L); + when(activeSegment.baseOffset()).thenReturn(15L); + when(segment1.size()).thenReturn(100); + when(segment2.size()).thenReturn(100); + when(activeSegment.size()).thenReturn(100); + + Map logProps = new HashMap<>(); + logProps.put(TopicConfig.RETENTION_MS_CONFIG, 10_000L); + logProps.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, 100L); + LogConfig logConfig = new LogConfig(logProps); + when(log.config()).thenReturn(logConfig); + when(segment1.largestTimestamp()).thenThrow(new IOException("failed-to-read-largest-timestamp")); + when(log.logSegments(5L, Long.MAX_VALUE)).thenReturn(List.of(segment1, segment2, activeSegment)); + + time.sleep(1000L); + when(segment2.largestTimestamp()).thenReturn(time.milliseconds() - 50L); + RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, 128); + List expected = + List.of( + new RemoteLogManager.EnrichedLogSegment(segment1, 10L) + ); + List actual = task.candidateLogSegments(log, 5L, 20L); + assertEquals(expected, actual); + } + @SuppressWarnings("unchecked") @Test public void testRemoteSizeData() { From 02819b2f72546acd161b21453b95eab3fbfeaeb4 Mon Sep 17 00:00:00 2001 From: Jian Date: Sat, 4 Apr 2026 08:38:20 +0800 Subject: [PATCH 091/110] Using real constants instead of magic number Signed-off-by: Jian --- .../remote/storage/RemoteLogManagerTest.java | 32 +++++++++---------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java index 46f0f99a933e2..0277fd6c9ac58 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java @@ -551,7 +551,7 @@ private void assertCopyExpectedLogSegmentsToRemote(long oldSegmentStartOffset, assertEquals(0, brokerTopicStats.allTopicsStats().remoteCopyBytesRate().count()); assertEquals(0, brokerTopicStats.allTopicsStats().failedRemoteCopyRequestRate().count()); - RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, 128); + RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_METADATA_CUSTOM_METADATA_MAX_BYTES); task.copyLogSegmentsToRemote(mockLog); // verify remoteLogMetadataManager did add the expected RemoteLogSegmentMetadata @@ -752,7 +752,7 @@ void testFailedCopyShouldDeleteTheDanglingSegment() throws Exception { // throw exception when copyLogSegmentData when(remoteStorageManager.copyLogSegmentData(any(RemoteLogSegmentMetadata.class), any(LogSegmentData.class))) .thenThrow(new RemoteStorageException("test")); - RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, 128); + RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_METADATA_CUSTOM_METADATA_MAX_BYTES); task.copyLogSegmentsToRemote(mockLog); ArgumentCaptor remoteLogSegmentMetadataArg = ArgumentCaptor.forClass(RemoteLogSegmentMetadata.class); @@ -839,7 +839,7 @@ void testFailedCopyWithRetriableExceptionShouldNotDeleteTheDanglingSegment() thr // throw retriable exception when copyLogSegmentData when(remoteStorageManager.copyLogSegmentData(any(RemoteLogSegmentMetadata.class), any(LogSegmentData.class))) .thenThrow(new RetriableRemoteStorageException("test-retriable")); - RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, 128); + RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_METADATA_CUSTOM_METADATA_MAX_BYTES); assertThrows(RetriableRemoteStorageException.class, () -> task.copyLogSegmentsToRemote(mockLog)); ArgumentCaptor remoteLogSegmentMetadataArg = ArgumentCaptor.forClass(RemoteLogSegmentMetadata.class); @@ -1317,7 +1317,7 @@ void testMetricsUpdateOnCopyLogSegmentsFailure() throws Exception { // Verify aggregate metrics assertEquals(0, brokerTopicStats.allTopicsStats().remoteCopyRequestRate().count()); assertEquals(0, brokerTopicStats.allTopicsStats().failedRemoteCopyRequestRate().count()); - RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, 128); + RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_METADATA_CUSTOM_METADATA_MAX_BYTES); task.copyLogSegmentsToRemote(mockLog); // Verify we attempted to copy log segment metadata to remote storage @@ -1368,7 +1368,7 @@ void testRLMTaskDoesNotUploadSegmentsWhenRemoteLogMetadataManagerIsNotInitialize assertEquals(0, brokerTopicStats.allTopicsStats().remoteCopyRequestRate().count()); assertEquals(0, brokerTopicStats.allTopicsStats().failedRemoteCopyRequestRate().count()); - RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, 128); + RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_METADATA_CUSTOM_METADATA_MAX_BYTES); task.run(); // verify the remoteLogMetadataManager never add any metadata and remoteStorageManager never copy log segments @@ -2126,7 +2126,7 @@ public void testCandidateLogSegmentsSkipsActiveSegment() { when(log.logSegments(5L, Long.MAX_VALUE)).thenReturn(List.of(segment1, segment2, activeSegment)); - RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, 128); + RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_METADATA_CUSTOM_METADATA_MAX_BYTES); List expected = List.of( new RemoteLogManager.EnrichedLogSegment(segment1, 10L), @@ -2151,7 +2151,7 @@ public void testCandidateLogSegmentsSkipsSegmentsAfterLastStableOffset() { when(log.logSegments(5L, Long.MAX_VALUE)).thenReturn(List.of(segment1, segment2, segment3, activeSegment)); - RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, 128); + RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_METADATA_CUSTOM_METADATA_MAX_BYTES); List expected = List.of( new RemoteLogManager.EnrichedLogSegment(segment1, 10L), @@ -2184,7 +2184,7 @@ public void testCandidateLogSegmentsDelayUploadWhenRemoteCopyLagMsNotExceeded() time.sleep(1000L); when(segment1.largestTimestamp()).thenReturn(time.milliseconds() - 50L); - RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, 128); + RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_METADATA_CUSTOM_METADATA_MAX_BYTES); List actual = task.candidateLogSegments(log, 5L, 20L); assertTrue(actual.isEmpty()); } @@ -2213,7 +2213,7 @@ public void testCandidateLogSegmentsUploadWhenRemoteCopyLagMsReachedBoundary() t time.sleep(1000L); when(segment1.largestTimestamp()).thenReturn(time.milliseconds() - 100L); when(segment2.largestTimestamp()).thenReturn(time.milliseconds() - 50L); - RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, 128); + RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_METADATA_CUSTOM_METADATA_MAX_BYTES); List expected = List.of( new RemoteLogManager.EnrichedLogSegment(segment1, 10L) @@ -2243,7 +2243,7 @@ public void testCandidateLogSegmentsDelayUploadWhenRemoteCopyLagBytesNotExceeded when(log.config()).thenReturn(logConfig); when(log.logSegments(5L, Long.MAX_VALUE)).thenReturn(List.of(segment1, segment2, activeSegment)); - RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, 128); + RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_METADATA_CUSTOM_METADATA_MAX_BYTES); List actual = task.candidateLogSegments(log, 5L, 20L); assertTrue(actual.isEmpty()); } @@ -2274,7 +2274,7 @@ public void testCandidateLogSegmentsUploadWhenEitherLagConditionExceeded() throw time.sleep(1000L); when(segment1.largestTimestamp()).thenReturn(time.milliseconds() - 100L); when(segment2.largestTimestamp()).thenReturn(time.milliseconds() - 20L); - RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, 128); + RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_METADATA_CUSTOM_METADATA_MAX_BYTES); List expected = List.of( new RemoteLogManager.EnrichedLogSegment(segment1, 10L) @@ -2307,7 +2307,7 @@ public void testCandidateLogSegmentsUploadWhenLargestTimestampLookupFails() thro time.sleep(1000L); when(segment2.largestTimestamp()).thenReturn(time.milliseconds() - 50L); - RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, 128); + RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_METADATA_CUSTOM_METADATA_MAX_BYTES); List expected = List.of( new RemoteLogManager.EnrichedLogSegment(segment1, 10L) @@ -2730,7 +2730,7 @@ public RemoteLogMetadataManager createRemoteLogMetadataManager() { return remoteLogMetadataManager; } }) { - RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, 128); + RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_METADATA_CUSTOM_METADATA_MAX_BYTES); task.copyLogSegmentsToRemote(mockLog); assertEquals(600L, logStartOffset.get()); } @@ -2867,7 +2867,7 @@ long findLogStartOffset(TopicIdPartition topicIdPartition, UnifiedLog log) { LogConfig mockLogConfig = new LogConfig(logProps); when(mockLog.config()).thenReturn(mockLogConfig); - RemoteLogManager.RLMCopyTask copyTask = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, 128); + RemoteLogManager.RLMCopyTask copyTask = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_METADATA_CUSTOM_METADATA_MAX_BYTES); Thread copyThread = new Thread(() -> { try { copyTask.copyLogSegmentsToRemote(mockLog); @@ -4076,7 +4076,7 @@ private RemoteLogManager.RLMCopyTask setupRLMTask(boolean quotaExceeded) throws when(rlmCopyQuotaManager.getThrottleTimeMs()).thenReturn(quotaExceeded ? 1000L : 0L); doNothing().when(rlmCopyQuotaManager).record(anyInt()); - return remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, 128); + return remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_METADATA_CUSTOM_METADATA_MAX_BYTES); } @Test @@ -4146,7 +4146,7 @@ public void testCopyThrottling() throws Exception { when(rlmCopyQuotaManager.getThrottleTimeMs()).thenReturn(0L, 1000L); doNothing().when(rlmCopyQuotaManager).record(anyInt()); - RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, 128); + RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_METADATA_CUSTOM_METADATA_MAX_BYTES); // Verify that the copy operation times out, since the second segment cannot be copied due to quota being exceeded assertThrows(AssertionFailedError.class, () -> assertTimeoutPreemptively(Duration.ofMillis(200), () -> task.copyLogSegmentsToRemote(mockLog))); From 3d7c5c9f9db1168bcfa0ec85903e42e1e6714371 Mon Sep 17 00:00:00 2001 From: Jian Date: Sat, 4 Apr 2026 08:59:35 +0800 Subject: [PATCH 092/110] Add more unit tests Signed-off-by: Jian --- .../remote/storage/RemoteLogManagerTest.java | 287 ++++++++++++++++++ 1 file changed, 287 insertions(+) diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java index 0277fd6c9ac58..7cab3e8ba7b17 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java @@ -2248,6 +2248,260 @@ public void testCandidateLogSegmentsDelayUploadWhenRemoteCopyLagBytesNotExceeded assertTrue(actual.isEmpty()); } + @Test + public void testCandidateLogSegmentsUploadWhenRemoteCopyLagBytesReachedBoundary() { + UnifiedLog log = mock(UnifiedLog.class); + LogSegment segment1 = mock(LogSegment.class); + LogSegment segment2 = mock(LogSegment.class); + LogSegment activeSegment = mock(LogSegment.class); + + when(segment1.baseOffset()).thenReturn(5L); + when(segment2.baseOffset()).thenReturn(10L); + when(activeSegment.baseOffset()).thenReturn(15L); + when(segment1.size()).thenReturn(40); + when(segment2.size()).thenReturn(30); + when(activeSegment.size()).thenReturn(20); + + Map logProps = new HashMap<>(); + logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, 10_000L); + logProps.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, 50L); + LogConfig logConfig = new LogConfig(logProps); + when(log.config()).thenReturn(logConfig); + when(log.logSegments(5L, Long.MAX_VALUE)).thenReturn(List.of(segment1, segment2, activeSegment)); + + RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_METADATA_CUSTOM_METADATA_MAX_BYTES); + List expected = + List.of( + new RemoteLogManager.EnrichedLogSegment(segment1, 10L) + ); + List actual = task.candidateLogSegments(log, 5L, 20L); + assertEquals(expected, actual); + } + + @Test + public void testCandidateLogSegmentsUploadWhenBothRemoteCopyLagConfigsAreDefault() { + UnifiedLog log = mock(UnifiedLog.class); + LogSegment segment1 = mock(LogSegment.class); + LogSegment segment2 = mock(LogSegment.class); + LogSegment activeSegment = mock(LogSegment.class); + + when(segment1.baseOffset()).thenReturn(5L); + when(segment2.baseOffset()).thenReturn(10L); + when(activeSegment.baseOffset()).thenReturn(15L); + when(segment1.size()).thenReturn(40); + when(segment2.size()).thenReturn(30); + when(activeSegment.size()).thenReturn(20); + + Map logProps = new HashMap<>(); + logProps.put(TopicConfig.RETENTION_MS_CONFIG, 10_000L); + logProps.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, LogConfig.DEFAULT_REMOTE_COPY_LAG_MS); + logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, 10_000L); + logProps.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, LogConfig.DEFAULT_REMOTE_COPY_LAG_BYTES); + LogConfig logConfig = new LogConfig(logProps); + when(log.config()).thenReturn(logConfig); + when(log.logSegments(5L, Long.MAX_VALUE)).thenReturn(List.of(segment1, segment2, activeSegment)); + + RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_METADATA_CUSTOM_METADATA_MAX_BYTES); + List expected = + List.of( + new RemoteLogManager.EnrichedLogSegment(segment1, 10L), + new RemoteLogManager.EnrichedLogSegment(segment2, 15L) + ); + List actual = task.candidateLogSegments(log, 5L, 20L); + assertEquals(expected, actual); + } + + @Test + public void testCandidateLogSegmentsUploadWhenRemoteCopyLagConfigsAreNotSet() { + UnifiedLog log = mock(UnifiedLog.class); + LogSegment segment1 = mock(LogSegment.class); + LogSegment segment2 = mock(LogSegment.class); + LogSegment activeSegment = mock(LogSegment.class); + + when(segment1.baseOffset()).thenReturn(5L); + when(segment2.baseOffset()).thenReturn(10L); + when(activeSegment.baseOffset()).thenReturn(15L); + when(segment1.size()).thenReturn(40); + when(segment2.size()).thenReturn(30); + when(activeSegment.size()).thenReturn(20); + + Map logProps = new HashMap<>(); + logProps.put(TopicConfig.RETENTION_MS_CONFIG, 10_000L); + logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, 10_000L); + LogConfig logConfig = new LogConfig(logProps); + when(log.config()).thenReturn(logConfig); + when(log.logSegments(5L, Long.MAX_VALUE)).thenReturn(List.of(segment1, segment2, activeSegment)); + + RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_METADATA_CUSTOM_METADATA_MAX_BYTES); + List expected = + List.of( + new RemoteLogManager.EnrichedLogSegment(segment1, 10L), + new RemoteLogManager.EnrichedLogSegment(segment2, 15L) + ); + List actual = task.candidateLogSegments(log, 5L, 20L); + assertEquals(expected, actual); + } + + @Test + public void testCandidateLogSegmentsUploadWhenEitherRemoteCopyLagConfigIsZero() { + UnifiedLog log = mock(UnifiedLog.class); + LogSegment segment1 = mock(LogSegment.class); + LogSegment segment2 = mock(LogSegment.class); + LogSegment activeSegment = mock(LogSegment.class); + + when(segment1.baseOffset()).thenReturn(5L); + when(segment2.baseOffset()).thenReturn(10L); + when(activeSegment.baseOffset()).thenReturn(15L); + when(segment1.size()).thenReturn(40); + when(segment2.size()).thenReturn(30); + when(activeSegment.size()).thenReturn(20); + + Map logProps = new HashMap<>(); + logProps.put(TopicConfig.RETENTION_MS_CONFIG, 10_000L); + logProps.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, 0L); + logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, 10_000L); + logProps.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, 60L); + LogConfig logConfig = new LogConfig(logProps); + when(log.config()).thenReturn(logConfig); + when(log.logSegments(5L, Long.MAX_VALUE)).thenReturn(List.of(segment1, segment2, activeSegment)); + + RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_METADATA_CUSTOM_METADATA_MAX_BYTES); + List expected = + List.of( + new RemoteLogManager.EnrichedLogSegment(segment1, 10L), + new RemoteLogManager.EnrichedLogSegment(segment2, 15L) + ); + List actual = task.candidateLogSegments(log, 5L, 20L); + assertEquals(expected, actual); + } + + @Test + public void testCandidateLogSegmentsUploadWhenRemoteCopyLagMsUsesLocalRetention() throws IOException { + UnifiedLog log = mock(UnifiedLog.class); + LogSegment segment1 = mock(LogSegment.class); + LogSegment segment2 = mock(LogSegment.class); + LogSegment activeSegment = mock(LogSegment.class); + + when(segment1.baseOffset()).thenReturn(5L); + when(segment2.baseOffset()).thenReturn(10L); + when(activeSegment.baseOffset()).thenReturn(15L); + when(segment1.size()).thenReturn(40); + when(segment2.size()).thenReturn(30); + when(activeSegment.size()).thenReturn(20); + + Map logProps = new HashMap<>(); + logProps.put(TopicConfig.RETENTION_MS_CONFIG, 10_000L); + logProps.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, 100L); + logProps.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, LogConfig.MAX_REMOTE_COPY_LAG_MS); + LogConfig logConfig = new LogConfig(logProps); + when(log.config()).thenReturn(logConfig); + when(log.logSegments(5L, Long.MAX_VALUE)).thenReturn(List.of(segment1, segment2, activeSegment)); + + time.sleep(1000L); + // remote.copy.lag.ms = -1 resolves to local retention ms (100), boundary should upload. + when(segment1.largestTimestamp()).thenReturn(time.milliseconds() - 100L); + when(segment2.largestTimestamp()).thenReturn(time.milliseconds() - 50L); + + RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_METADATA_CUSTOM_METADATA_MAX_BYTES); + List expected = + List.of( + new RemoteLogManager.EnrichedLogSegment(segment1, 10L) + ); + List actual = task.candidateLogSegments(log, 5L, 20L); + assertEquals(expected, actual); + } + + @Test + public void testCandidateLogSegmentsDelayUploadWhenRemoteCopyLagMsUsesLocalRetention() throws IOException { + UnifiedLog log = mock(UnifiedLog.class); + LogSegment segment1 = mock(LogSegment.class); + LogSegment segment2 = mock(LogSegment.class); + LogSegment activeSegment = mock(LogSegment.class); + + when(segment1.baseOffset()).thenReturn(5L); + when(segment2.baseOffset()).thenReturn(10L); + when(activeSegment.baseOffset()).thenReturn(15L); + when(segment1.size()).thenReturn(40); + when(segment2.size()).thenReturn(30); + when(activeSegment.size()).thenReturn(20); + + Map logProps = new HashMap<>(); + logProps.put(TopicConfig.RETENTION_MS_CONFIG, 10_000L); + logProps.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, 100L); + logProps.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, LogConfig.MAX_REMOTE_COPY_LAG_MS); + LogConfig logConfig = new LogConfig(logProps); + when(log.config()).thenReturn(logConfig); + when(log.logSegments(5L, Long.MAX_VALUE)).thenReturn(List.of(segment1, segment2, activeSegment)); + + time.sleep(1000L); + // remote.copy.lag.ms = -1 resolves to local retention ms (100), age=50ms < 100ms => delay. + when(segment1.largestTimestamp()).thenReturn(time.milliseconds() - 50L); + + RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_METADATA_CUSTOM_METADATA_MAX_BYTES); + List actual = task.candidateLogSegments(log, 5L, 20L); + assertTrue(actual.isEmpty()); + } + + @Test + public void testCandidateLogSegmentsUploadWhenRemoteCopyLagBytesUsesLocalRetention() { + UnifiedLog log = mock(UnifiedLog.class); + LogSegment segment1 = mock(LogSegment.class); + LogSegment segment2 = mock(LogSegment.class); + LogSegment activeSegment = mock(LogSegment.class); + + when(segment1.baseOffset()).thenReturn(5L); + when(segment2.baseOffset()).thenReturn(10L); + when(activeSegment.baseOffset()).thenReturn(15L); + when(segment1.size()).thenReturn(40); + when(segment2.size()).thenReturn(30); + when(activeSegment.size()).thenReturn(20); + + Map logProps = new HashMap<>(); + logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, 10_000L); + logProps.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, 50L); + logProps.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, LogConfig.MAX_REMOTE_COPY_LAG_BYTES); + LogConfig logConfig = new LogConfig(logProps); + when(log.config()).thenReturn(logConfig); + when(log.logSegments(5L, Long.MAX_VALUE)).thenReturn(List.of(segment1, segment2, activeSegment)); + + // remote.copy.lag.bytes = -1 resolves to local retention bytes (50), boundary should upload. + RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_METADATA_CUSTOM_METADATA_MAX_BYTES); + List expected = + List.of( + new RemoteLogManager.EnrichedLogSegment(segment1, 10L) + ); + List actual = task.candidateLogSegments(log, 5L, 20L); + assertEquals(expected, actual); + } + + @Test + public void testCandidateLogSegmentsDelayUploadWhenRemoteCopyLagBytesUsesLocalRetention() { + UnifiedLog log = mock(UnifiedLog.class); + LogSegment segment1 = mock(LogSegment.class); + LogSegment segment2 = mock(LogSegment.class); + LogSegment activeSegment = mock(LogSegment.class); + + when(segment1.baseOffset()).thenReturn(5L); + when(segment2.baseOffset()).thenReturn(10L); + when(activeSegment.baseOffset()).thenReturn(15L); + when(segment1.size()).thenReturn(40); + when(segment2.size()).thenReturn(30); + when(activeSegment.size()).thenReturn(20); + + Map logProps = new HashMap<>(); + logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, 10_000L); + logProps.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, 60L); + logProps.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, LogConfig.MAX_REMOTE_COPY_LAG_BYTES); + LogConfig logConfig = new LogConfig(logProps); + when(log.config()).thenReturn(logConfig); + when(log.logSegments(5L, Long.MAX_VALUE)).thenReturn(List.of(segment1, segment2, activeSegment)); + + // remote.copy.lag.bytes = -1 resolves to local retention bytes (60), sizeLag=50 < 60 => delay. + RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_METADATA_CUSTOM_METADATA_MAX_BYTES); + List actual = task.candidateLogSegments(log, 5L, 20L); + assertTrue(actual.isEmpty()); + } + @Test public void testCandidateLogSegmentsUploadWhenEitherLagConditionExceeded() throws IOException { UnifiedLog log = mock(UnifiedLog.class); @@ -2283,6 +2537,39 @@ public void testCandidateLogSegmentsUploadWhenEitherLagConditionExceeded() throw assertEquals(expected, actual); } + @Test + public void testCandidateLogSegmentsDelayUploadWhenBothLagConditionsNotExceeded() throws IOException { + UnifiedLog log = mock(UnifiedLog.class); + LogSegment segment1 = mock(LogSegment.class); + LogSegment segment2 = mock(LogSegment.class); + LogSegment activeSegment = mock(LogSegment.class); + + when(segment1.baseOffset()).thenReturn(5L); + when(segment2.baseOffset()).thenReturn(10L); + when(activeSegment.baseOffset()).thenReturn(15L); + when(segment1.size()).thenReturn(40); + when(segment2.size()).thenReturn(30); + when(activeSegment.size()).thenReturn(20); + + Map logProps = new HashMap<>(); + logProps.put(TopicConfig.RETENTION_MS_CONFIG, 10_000L); + logProps.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, 100L); + logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, 10_000L); + logProps.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, 60L); + LogConfig logConfig = new LogConfig(logProps); + when(log.config()).thenReturn(logConfig); + when(log.logSegments(5L, Long.MAX_VALUE)).thenReturn(List.of(segment1, segment2, activeSegment)); + + time.sleep(1000L); + // Both lag checks are not exceeded: age=50ms < 100ms and sizeLag=50bytes < 60bytes. + when(segment1.largestTimestamp()).thenReturn(time.milliseconds() - 50L); + + RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask( + leaderTopicIdPartition, RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_METADATA_CUSTOM_METADATA_MAX_BYTES); + List actual = task.candidateLogSegments(log, 5L, 20L); + assertTrue(actual.isEmpty()); + } + @Test public void testCandidateLogSegmentsUploadWhenLargestTimestampLookupFails() throws IOException { UnifiedLog log = mock(UnifiedLog.class); From 79d95875d25223f544a29b0a7ffd184d5412727f Mon Sep 17 00:00:00 2001 From: Jian Date: Sat, 4 Apr 2026 09:05:54 +0800 Subject: [PATCH 093/110] improve the unit test Signed-off-by: Jian --- .../kafka/server/log/remote/storage/RemoteLogManagerTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java index 7cab3e8ba7b17..8a7e30fb813b9 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java @@ -2526,7 +2526,8 @@ public void testCandidateLogSegmentsUploadWhenEitherLagConditionExceeded() throw when(log.logSegments(5L, Long.MAX_VALUE)).thenReturn(List.of(segment1, segment2, activeSegment)); time.sleep(1000L); - when(segment1.largestTimestamp()).thenReturn(time.milliseconds() - 100L); + // segment1: time lag exceeded (101ms >= 100ms), size lag not exceeded (50bytes < 60bytes) => upload. + when(segment1.largestTimestamp()).thenReturn(time.milliseconds() - 101L); when(segment2.largestTimestamp()).thenReturn(time.milliseconds() - 20L); RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_METADATA_CUSTOM_METADATA_MAX_BYTES); List expected = From d85551ed6ce2147c89df861350f0daf009287eeb Mon Sep 17 00:00:00 2001 From: Jian Date: Tue, 5 May 2026 19:01:00 +0800 Subject: [PATCH 094/110] Fix typos in TopicConfig documentation --- .../main/java/org/apache/kafka/common/config/TopicConfig.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index 0dfa68004407d..90043c50ea262 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -105,7 +105,7 @@ public class TopicConfig { public static final String REMOTE_COPY_LAG_MS_DOC = "Controls how long to delay uploading segments to remote storage. " + "When set to -2 (default), no delay check based on local retention ms. " + "When set to 0, segments are uploaded as soon as they are eligible without any delay check. " + - "When set to a positive value (ms), a segment can't become eligible for upload util the time since the latest record in the segment reaches the value. " + + "When set to a positive value (ms), a segment can't become eligible for upload until the time since the latest record in the segment reaches the value. " + "The value should not exceed the real local retention ms except the latter is retained indefinitely (-1). " + "When set to -1, resolves to the real local retention ms as maximum delay. " + "If the real local retention ms is configured as infinite, -1 is treated as an invalid configuration. " + @@ -115,7 +115,7 @@ public class TopicConfig { public static final String REMOTE_COPY_LAG_BYTES_DOC = "Controls size-based delay for uploading segments to remote storage. " + "When set to -2 (default), no delay check based on local retention bytes. " + "When set to 0, segments are uploaded as soon as they are eligible without any delay check. " + - "When set to a positive value (bytes), a segment can't become eligible for upload util the total bytes of log data after the segment reach the value. " + + "When set to a positive value (bytes), a segment can't become eligible for upload until the total bytes of log data after the segment reach the value. " + "The value should not exceed the real local retention bytes except the latter is retained indefinitely (-1). " + "When set to -1, resolves to the real local retention bytes as maximum delay. " + "If the real local retention bytes is configured as infinite, -1 is treated as an invalid configuration. " + From bc5b3954aaf40ce3c66fc3b38db594ca2cf0c6f4 Mon Sep 17 00:00:00 2001 From: Jian Date: Tue, 5 May 2026 19:02:38 +0800 Subject: [PATCH 095/110] Fix typos in RemoteLogManagerConfig documentation --- .../server/log/remote/storage/RemoteLogManagerConfig.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java index 7c5924cc8524a..18f8b57eaa659 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java @@ -172,7 +172,7 @@ public final class RemoteLogManagerConfig { public static final String LOG_REMOTE_COPY_LAG_MS_DOC = "Controls how long to delay uploading segments to remote storage. " + "When set to -2 (default), no delay check based on local retention ms. " + "When set to 0, segments are uploaded as soon as they are eligible without any delay check. " + - "When set to a positive value (ms), a segment can't become eligible for upload util the time since the latest record in the segment reaches the value. " + + "When set to a positive value (ms), a segment can't become eligible for upload until the time since the latest record in the segment reaches the value. " + "The value should not exceed the real local retention ms except the latter is retained indefinitely (-1). " + "When set to -1, resolves to the real local retention ms as maximum delay. " + "If the real local retention ms is configured as infinite, -1 is treated as an invalid configuration. " + @@ -183,7 +183,7 @@ public final class RemoteLogManagerConfig { public static final String LOG_REMOTE_COPY_LAG_BYTES_DOC = "Controls size-based delay for uploading segments to remote storage. " + "When set to -2 (default), no delay check based on local retention bytes. " + "When set to 0, segments are uploaded as soon as they are eligible without any delay check. " + - "When set to a positive value (bytes), a segment can't become eligible for upload util the total bytes of log data after the segment reach the value. " + + "When set to a positive value (bytes), a segment can't become eligible for upload until the total bytes of log data after the segment reach the value. " + "The value should not exceed the real local retention bytes except the latter is retained indefinitely (-1). " + "When set to -1, resolves to the real local retention bytes as maximum delay. " + "If the real local retention bytes is configured as infinite, -1 is treated as an invalid configuration. " + From 1122220a87fa41115decb033bb4b95abdfebce6f Mon Sep 17 00:00:00 2001 From: Jian Date: Tue, 5 May 2026 19:13:22 +0800 Subject: [PATCH 096/110] judge logger.isDebugEnabled() before print debug Signed-off-by: Jian --- .../log/remote/storage/RemoteLogManager.java | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index 3e1b56d218665..b1a55307c8be7 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -953,8 +953,10 @@ private boolean delayCopy(LogConfig logConfig, LogSegment previousSeg, long curr long copyLagMs = logConfig.remoteCopyLagMs(); long copyLagBytes = logConfig.remoteCopyLagBytes(); - logger.debug("delayCopy check for segment {}: copyLagMs={}, copyLagBytes={}, currentTimeMs={}, totalLogSize={}, cumulativeSize={}, sizeLagBytes={}", - previousSeg, copyLagMs, copyLagBytes, currentTimeMs, totalLogSize, cumulativeSize, totalLogSize - cumulativeSize); + if (logger.isDebugEnabled()) { + logger.debug("delayCopy check for segment {}: copyLagMs={}, copyLagBytes={}, currentTimeMs={}, totalLogSize={}, cumulativeSize={}, sizeLagBytes={}", + previousSeg, copyLagMs, copyLagBytes, currentTimeMs, totalLogSize, cumulativeSize, totalLogSize - cumulativeSize); + } if (copyLagMs == 0 || copyLagBytes == 0) { return false; @@ -979,8 +981,10 @@ private boolean notExceededCopyLagTime(LogSegment segment, long currentTimeMs, l try { long segmentAgeMs = currentTimeMs - segment.largestTimestamp(); boolean exceeded = segmentAgeMs >= copyLagMs; - logger.debug("{} eligible for upload by time? {} (segment age {} ms, copy lag {} ms)", - segment, exceeded, segmentAgeMs, copyLagMs); + if (logger.isDebugEnabled()) { + logger.debug("{} eligible for upload by time? {} (segment age {} ms, copy lag {} ms)", + segment, exceeded, segmentAgeMs, copyLagMs); + } return !exceeded; } catch (IOException e) { logger.warn("Failed to get largest timestamp for segment {}, take it as eligible for upload based on time", segment, e); @@ -991,8 +995,10 @@ private boolean notExceededCopyLagTime(LogSegment segment, long currentTimeMs, l private boolean notExceededCopyLagSize(LogSegment segment, long totalLogSize, long cumulativeSize, long copyLagBytes) { long sizeLagBytes = totalLogSize - cumulativeSize; boolean exceeded = sizeLagBytes >= copyLagBytes; - logger.debug("{} eligible for upload by size? {} (size lag {} bytes, copy lag {} bytes, totalLogSize={}, cumulativeSize={})", - segment, exceeded, sizeLagBytes, copyLagBytes, totalLogSize, cumulativeSize); + if (logger.isDebugEnabled()) { + logger.debug("{} eligible for upload by size? {} (size lag {} bytes, copy lag {} bytes, totalLogSize={}, cumulativeSize={})", + segment, exceeded, sizeLagBytes, copyLagBytes, totalLogSize, cumulativeSize); + } return !exceeded; } From 2fa65e7c08bab354d6497dfa89029c8ff2957989 Mon Sep 17 00:00:00 2001 From: Jian Date: Tue, 5 May 2026 19:16:39 +0800 Subject: [PATCH 097/110] change debug to trace Signed-off-by: Jian --- .../server/log/remote/storage/RemoteLogManager.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index b1a55307c8be7..4361acae3977b 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -953,8 +953,8 @@ private boolean delayCopy(LogConfig logConfig, LogSegment previousSeg, long curr long copyLagMs = logConfig.remoteCopyLagMs(); long copyLagBytes = logConfig.remoteCopyLagBytes(); - if (logger.isDebugEnabled()) { - logger.debug("delayCopy check for segment {}: copyLagMs={}, copyLagBytes={}, currentTimeMs={}, totalLogSize={}, cumulativeSize={}, sizeLagBytes={}", + if (logger.isTraceEnabled()) { + logger.trace("delayCopy check for segment {}: copyLagMs={}, copyLagBytes={}, currentTimeMs={}, totalLogSize={}, cumulativeSize={}, sizeLagBytes={}", previousSeg, copyLagMs, copyLagBytes, currentTimeMs, totalLogSize, cumulativeSize, totalLogSize - cumulativeSize); } @@ -981,8 +981,8 @@ private boolean notExceededCopyLagTime(LogSegment segment, long currentTimeMs, l try { long segmentAgeMs = currentTimeMs - segment.largestTimestamp(); boolean exceeded = segmentAgeMs >= copyLagMs; - if (logger.isDebugEnabled()) { - logger.debug("{} eligible for upload by time? {} (segment age {} ms, copy lag {} ms)", + if (logger.isTraceEnabled()) { + logger.trace("{} eligible for upload by time? {} (segment age {} ms, copy lag {} ms)", segment, exceeded, segmentAgeMs, copyLagMs); } return !exceeded; @@ -995,8 +995,8 @@ private boolean notExceededCopyLagTime(LogSegment segment, long currentTimeMs, l private boolean notExceededCopyLagSize(LogSegment segment, long totalLogSize, long cumulativeSize, long copyLagBytes) { long sizeLagBytes = totalLogSize - cumulativeSize; boolean exceeded = sizeLagBytes >= copyLagBytes; - if (logger.isDebugEnabled()) { - logger.debug("{} eligible for upload by size? {} (size lag {} bytes, copy lag {} bytes, totalLogSize={}, cumulativeSize={})", + if (logger.isTraceEnabled()) { + logger.trace("{} eligible for upload by size? {} (size lag {} bytes, copy lag {} bytes, totalLogSize={}, cumulativeSize={})", segment, exceeded, sizeLagBytes, copyLagBytes, totalLogSize, cumulativeSize); } return !exceeded; From e1a9c99afe371fa423e08afc5a7a4d81986a0fd0 Mon Sep 17 00:00:00 2001 From: Jian Date: Tue, 5 May 2026 19:21:12 +0800 Subject: [PATCH 098/110] improve the code for config according to code review Signed-off-by: Jian --- .../server/log/remote/storage/RemoteLogManagerConfig.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java index 18f8b57eaa659..d73141e075e56 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java @@ -372,13 +372,13 @@ public static ConfigDef configDef() { .define(LOG_REMOTE_COPY_LAG_MS_PROP, LONG, DEFAULT_LOG_REMOTE_COPY_LAG_MS, - atLeast(-2), + atLeast(DEFAULT_LOG_REMOTE_COPY_LAG_MS), MEDIUM, LOG_REMOTE_COPY_LAG_MS_DOC) .define(LOG_REMOTE_COPY_LAG_BYTES_PROP, LONG, DEFAULT_LOG_REMOTE_COPY_LAG_BYTES, - atLeast(-2), + atLeast(DEFAULT_LOG_REMOTE_COPY_LAG_BYTES), MEDIUM, LOG_REMOTE_COPY_LAG_BYTES_DOC) .define(REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP, From 6d481c59a13c4af7b759d4a471c56361195414ae Mon Sep 17 00:00:00 2001 From: Jian Date: Thu, 7 May 2026 20:07:51 +0800 Subject: [PATCH 099/110] add more validation configure Signed-off-by: Jian --- .../kafka/server/DynamicBrokerConfig.scala | 16 ++++++++ .../scala/unit/kafka/log/LogConfigTest.scala | 40 +++++++++++++++++++ .../server/DynamicBrokerConfigTest.scala | 20 ++++++++++ .../storage/internals/log/LogConfig.java | 14 +++++++ 4 files changed, 90 insertions(+) diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index 073cdf209a5a2..f5f9c18f0b874 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -584,6 +584,14 @@ class DynamicLogConfig(logManager: LogManager, directoryEventHandler: DirectoryE val logLocalRetentionMs = newConfig.remoteLogManagerConfig.logLocalRetentionMs val effectiveLocalRetentionMs = if (logLocalRetentionMs == -2L) logRetentionMs else logLocalRetentionMs val logRemoteCopyLagMs = newConfig.remoteLogManagerConfig.logRemoteCopyLagMs + val logRemoteCopyLagBytes = newConfig.remoteLogManagerConfig.logRemoteCopyLagBytes + if (logRemoteCopyLagMs == LogConfig.DEFAULT_REMOTE_COPY_LAG_MS && + logRemoteCopyLagBytes != LogConfig.DEFAULT_REMOTE_COPY_LAG_BYTES && + effectiveLocalRetentionMs >= 0L) { + throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_MS_PROP, logRemoteCopyLagMs, + s"Value must not be -2 when effective ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP} is non-negative " + + s"unless ${RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_BYTES_PROP} is also -2.") + } if (effectiveLocalRetentionMs == -1L && logRemoteCopyLagMs == -1L) { throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_MS_PROP, logRemoteCopyLagMs, s"Value must not be -1 when effective ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP} is -1") @@ -599,6 +607,14 @@ class DynamicLogConfig(logManager: LogManager, directoryEventHandler: DirectoryE val logLocalRetentionBytes = newConfig.remoteLogManagerConfig.logLocalRetentionBytes val effectiveLocalRetentionBytes = if (logLocalRetentionBytes == -2L) logRetentionBytes else logLocalRetentionBytes val logRemoteCopyLagBytes = newConfig.remoteLogManagerConfig.logRemoteCopyLagBytes + val logRemoteCopyLagMs = newConfig.remoteLogManagerConfig.logRemoteCopyLagMs + if (logRemoteCopyLagBytes == LogConfig.DEFAULT_REMOTE_COPY_LAG_BYTES && + logRemoteCopyLagMs != LogConfig.DEFAULT_REMOTE_COPY_LAG_MS && + effectiveLocalRetentionBytes >= 0L) { + throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_BYTES_PROP, logRemoteCopyLagBytes, + s"Value must not be -2 when effective ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP} is non-negative " + + s"unless ${RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_MS_PROP} is also -2.") + } if (effectiveLocalRetentionBytes == -1L && logRemoteCopyLagBytes == -1L) { throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_BYTES_PROP, logRemoteCopyLagBytes, s"Value must not be -1 when effective ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP} is -1") diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index b67b93ec5ae9a..6143eb0334bc3 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -308,6 +308,46 @@ class LogConfigTest { assertTrue(exception.getMessage.contains(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG)) } + @Test + def testInvalidRemoteCopyLagMsWhenOnlyTimeLagCheckIsDisabled(): Unit = { + val props = new util.HashMap[String, String]() + props.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") + props.put(TopicConfig.RETENTION_MS_CONFIG, "1000") + props.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, "500") + props.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, "-2") + props.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, "100") + + val exception = assertThrows(classOf[ConfigException], () => validateTopicLogConfig(props)) + assertTrue(exception.getMessage.contains(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG)) + } + + @Test + def testInvalidRemoteCopyLagBytesWhenOnlySizeLagCheckIsDisabled(): Unit = { + val props = new util.HashMap[String, String]() + props.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") + props.put(TopicConfig.RETENTION_BYTES_CONFIG, "1000") + props.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, "500") + props.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, "100") + props.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, "-2") + + val exception = assertThrows(classOf[ConfigException], () => validateTopicLogConfig(props)) + assertTrue(exception.getMessage.contains(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG)) + } + + @Test + def testValidRemoteCopyLagWhenBothLagChecksAreDisabled(): Unit = { + val props = new util.HashMap[String, String]() + props.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") + props.put(TopicConfig.RETENTION_MS_CONFIG, "1000") + props.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, "-2") + props.put(TopicConfig.RETENTION_BYTES_CONFIG, "1000") + props.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, "-2") + props.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, "-2") + props.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, "-2") + + validateTopicLogConfig(props) + } + private def validateTopicLogConfig(props: util.Map[String, String]): Unit = { val kafkaProps = TestUtils.createDummyBrokerConfig() kafkaProps.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, "true") diff --git a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala index 0346a7f66d14c..4e8a4a8b17ae0 100755 --- a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala @@ -811,6 +811,26 @@ class DynamicBrokerConfigTest { logLocalRetentionBytes = -2L, remoteCopyLagBytes = 1001L ) + + // when local retention ms is explicitly configured, remote copy lag ms cannot be disabled alone + verifyIncorrectRemoteCopyLagProps( + retentionMs = 1000L, + logLocalRetentionMs = 500L, + remoteCopyLagMs = -2L, + retentionBytes = 1000L, + logLocalRetentionBytes = -2L, + remoteCopyLagBytes = 100L + ) + + // when local retention bytes is explicitly configured, remote copy lag bytes cannot be disabled alone + verifyIncorrectRemoteCopyLagProps( + retentionMs = 1000L, + logLocalRetentionMs = -2L, + remoteCopyLagMs = 100L, + retentionBytes = 1000L, + logLocalRetentionBytes = 500L, + remoteCopyLagBytes = -2L + ) } def verifyIncorrectRemoteCopyLagProps(retentionMs: Long, diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index c89bb77d82fa8..cc71cf8c9c6ec 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -635,7 +635,14 @@ private static void validateRemoteCopyLagTime(Map props) { Long retentionMs = (Long) props.get(TopicConfig.RETENTION_MS_CONFIG); Long localRetentionMs = (Long) props.get(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG); Long remoteCopyLagMs = (Long) props.get(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG); + Long remoteCopyLagBytes = (Long) props.get(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG); long effectiveLocalRetentionMs = localRetentionMs == -2 ? retentionMs : localRetentionMs; + if (remoteCopyLagMs == DEFAULT_REMOTE_COPY_LAG_MS && remoteCopyLagBytes != DEFAULT_REMOTE_COPY_LAG_BYTES + && effectiveLocalRetentionMs >= 0) { + String message = String.format("Value must not be -2 when effective %s is non-negative unless %s is also -2.", + TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG); + throw new ConfigException(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, remoteCopyLagMs, message); + } if (effectiveLocalRetentionMs == -1 && remoteCopyLagMs == -1) { String message = String.format("Value must not be -1 when effective %s is -1", TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG); @@ -653,7 +660,14 @@ private static void validateRemoteCopyLagSize(Map props) { Long retentionBytes = (Long) props.get(TopicConfig.RETENTION_BYTES_CONFIG); Long localRetentionBytes = (Long) props.get(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG); Long remoteCopyLagBytes = (Long) props.get(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG); + Long remoteCopyLagMs = (Long) props.get(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG); long effectiveLocalRetentionBytes = localRetentionBytes == -2 ? retentionBytes : localRetentionBytes; + if (remoteCopyLagBytes == DEFAULT_REMOTE_COPY_LAG_BYTES && remoteCopyLagMs != DEFAULT_REMOTE_COPY_LAG_MS + && effectiveLocalRetentionBytes >= 0) { + String message = String.format("Value must not be -2 when effective %s is non-negative unless %s is also -2.", + TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, TopicConfig.REMOTE_COPY_LAG_MS_CONFIG); + throw new ConfigException(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, remoteCopyLagBytes, message); + } if (effectiveLocalRetentionBytes == -1 && remoteCopyLagBytes == -1) { String message = String.format("Value must not be -1 when effective %s is -1", TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG); From b302dac348b95218493f0adec7c51cdc6bf3af69 Mon Sep 17 00:00:00 2001 From: Jian Date: Thu, 7 May 2026 20:19:32 +0800 Subject: [PATCH 100/110] add document for Constraint Signed-off-by: Jian --- .../main/java/org/apache/kafka/common/config/TopicConfig.java | 4 ++++ .../server/log/remote/storage/RemoteLogManagerConfig.java | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index 3dd563c45ed4e..8a84c0fc08c59 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -109,6 +109,8 @@ public class TopicConfig { "The value should not exceed the real local retention ms except the latter is retained indefinitely (-1). " + "When set to -1, resolves to the real local retention ms as maximum delay. " + "If the real local retention ms is configured as infinite, -1 is treated as an invalid configuration. " + + "Constraint: if local.retention.ms is explicitly set to a non-negative value, remote.copy.lag.ms cannot be -2 unless remote.copy.lag.bytes is also -2. " + + "Otherwise, it may break time-based delete policy. " + "For how the real local retention time is computed, see local.retention.ms."; public static final String REMOTE_COPY_LAG_BYTES_CONFIG = "remote.copy.lag.bytes"; @@ -119,6 +121,8 @@ public class TopicConfig { "The value should not exceed the real local retention bytes except the latter is retained indefinitely (-1). " + "When set to -1, resolves to the real local retention bytes as maximum delay. " + "If the real local retention bytes is configured as infinite, -1 is treated as an invalid configuration. " + + "Constraint: if local.retention.bytes is explicitly set to a non-negative value, remote.copy.lag.bytes cannot be -2 unless remote.copy.lag.ms is also -2. " + + "Otherwise, it may break size-based delete policy. " + "For how the real local retention size is computed, see local.retention.bytes."; public static final String REMOTE_LOG_DELETE_ON_DISABLE_CONFIG = "remote.log.delete.on.disable"; diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java index d73141e075e56..ce54ace75e2e0 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java @@ -176,6 +176,8 @@ public final class RemoteLogManagerConfig { "The value should not exceed the real local retention ms except the latter is retained indefinitely (-1). " + "When set to -1, resolves to the real local retention ms as maximum delay. " + "If the real local retention ms is configured as infinite, -1 is treated as an invalid configuration. " + + "Constraint: if log.local.retention.ms is explicitly set to a non-negative value, log.remote.copy.lag.ms cannot be -2 unless log.remote.copy.lag.bytes is also -2. " + + "Otherwise, it may break time-based delete policy. " + "For how the real local retention time is computed, see log.local.retention.ms."; public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_MS = -2L; @@ -187,6 +189,8 @@ public final class RemoteLogManagerConfig { "The value should not exceed the real local retention bytes except the latter is retained indefinitely (-1). " + "When set to -1, resolves to the real local retention bytes as maximum delay. " + "If the real local retention bytes is configured as infinite, -1 is treated as an invalid configuration. " + + "Constraint: if log.local.retention.bytes is explicitly set to a non-negative value, log.remote.copy.lag.bytes cannot be -2 unless log.remote.copy.lag.ms is also -2. " + + "Otherwise, it may break size-based delete policy. " + "For how the real local retention size is computed, see log.local.retention.bytes."; public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_BYTES = -2L; From 31c686aff0cd6b610a5f5a7b876dbec4305ebf19 Mon Sep 17 00:00:00 2001 From: Jian Date: Thu, 7 May 2026 23:23:38 +0800 Subject: [PATCH 101/110] update document Signed-off-by: Jian --- .../kafka/common/config/TopicConfig.java | 18 ++++-------------- .../remote/storage/RemoteLogManagerConfig.java | 18 ++++-------------- 2 files changed, 8 insertions(+), 28 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index 8a84c0fc08c59..2df0078b62122 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -103,26 +103,16 @@ public class TopicConfig { public static final String REMOTE_COPY_LAG_MS_CONFIG = "remote.copy.lag.ms"; public static final String REMOTE_COPY_LAG_MS_DOC = "Controls how long to delay uploading segments to remote storage. " + - "When set to -2 (default), no delay check based on local retention ms. " + - "When set to 0, segments are uploaded as soon as they are eligible without any delay check. " + + "When set to 0, immediate upload when time-based retention is used; otherwise no time-based delay check. " + "When set to a positive value (ms), a segment can't become eligible for upload until the time since the latest record in the segment reaches the value. " + - "The value should not exceed the real local retention ms except the latter is retained indefinitely (-1). " + - "When set to -1, resolves to the real local retention ms as maximum delay. " + - "If the real local retention ms is configured as infinite, -1 is treated as an invalid configuration. " + - "Constraint: if local.retention.ms is explicitly set to a non-negative value, remote.copy.lag.ms cannot be -2 unless remote.copy.lag.bytes is also -2. " + - "Otherwise, it may break time-based delete policy. " + + "The value should not exceed the real local retention ms. " + "For how the real local retention time is computed, see local.retention.ms."; public static final String REMOTE_COPY_LAG_BYTES_CONFIG = "remote.copy.lag.bytes"; public static final String REMOTE_COPY_LAG_BYTES_DOC = "Controls size-based delay for uploading segments to remote storage. " + - "When set to -2 (default), no delay check based on local retention bytes. " + - "When set to 0, segments are uploaded as soon as they are eligible without any delay check. " + + "When set to 0, immediate upload when size-based retention is used; otherwise no size-based delay check. " + "When set to a positive value (bytes), a segment can't become eligible for upload until the total bytes of log data after the segment reach the value. " + - "The value should not exceed the real local retention bytes except the latter is retained indefinitely (-1). " + - "When set to -1, resolves to the real local retention bytes as maximum delay. " + - "If the real local retention bytes is configured as infinite, -1 is treated as an invalid configuration. " + - "Constraint: if local.retention.bytes is explicitly set to a non-negative value, remote.copy.lag.bytes cannot be -2 unless remote.copy.lag.ms is also -2. " + - "Otherwise, it may break size-based delete policy. " + + "The value should not exceed the real local retention bytes. " + "For how the real local retention size is computed, see local.retention.bytes."; public static final String REMOTE_LOG_DELETE_ON_DISABLE_CONFIG = "remote.log.delete.on.disable"; diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java index ce54ace75e2e0..13e885251cab1 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java @@ -170,27 +170,17 @@ public final class RemoteLogManagerConfig { public static final String LOG_REMOTE_COPY_LAG_MS_PROP = "log.remote.copy.lag.ms"; public static final String LOG_REMOTE_COPY_LAG_MS_DOC = "Controls how long to delay uploading segments to remote storage. " + - "When set to -2 (default), no delay check based on local retention ms. " + - "When set to 0, segments are uploaded as soon as they are eligible without any delay check. " + + "When set to 0, immediate upload when time-based retention is used; otherwise no time-based delay check. " + "When set to a positive value (ms), a segment can't become eligible for upload until the time since the latest record in the segment reaches the value. " + - "The value should not exceed the real local retention ms except the latter is retained indefinitely (-1). " + - "When set to -1, resolves to the real local retention ms as maximum delay. " + - "If the real local retention ms is configured as infinite, -1 is treated as an invalid configuration. " + - "Constraint: if log.local.retention.ms is explicitly set to a non-negative value, log.remote.copy.lag.ms cannot be -2 unless log.remote.copy.lag.bytes is also -2. " + - "Otherwise, it may break time-based delete policy. " + + "The value should not exceed the real local retention ms. " + "For how the real local retention time is computed, see log.local.retention.ms."; public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_MS = -2L; public static final String LOG_REMOTE_COPY_LAG_BYTES_PROP = "log.remote.copy.lag.bytes"; public static final String LOG_REMOTE_COPY_LAG_BYTES_DOC = "Controls size-based delay for uploading segments to remote storage. " + - "When set to -2 (default), no delay check based on local retention bytes. " + - "When set to 0, segments are uploaded as soon as they are eligible without any delay check. " + + "When set to 0, immediate upload when size-based retention is used; otherwise no size-based delay check. " + "When set to a positive value (bytes), a segment can't become eligible for upload until the total bytes of log data after the segment reach the value. " + - "The value should not exceed the real local retention bytes except the latter is retained indefinitely (-1). " + - "When set to -1, resolves to the real local retention bytes as maximum delay. " + - "If the real local retention bytes is configured as infinite, -1 is treated as an invalid configuration. " + - "Constraint: if log.local.retention.bytes is explicitly set to a non-negative value, log.remote.copy.lag.bytes cannot be -2 unless log.remote.copy.lag.ms is also -2. " + - "Otherwise, it may break size-based delete policy. " + + "The value should not exceed the real local retention bytes. " + "For how the real local retention size is computed, see log.local.retention.bytes."; public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_BYTES = -2L; From 8036c81abca0333ccb5716313dd82c923494ae9e Mon Sep 17 00:00:00 2001 From: Jian Date: Thu, 7 May 2026 23:36:29 +0800 Subject: [PATCH 102/110] remove the logic for -1 and -2 Signed-off-by: Jian --- .../kafka/server/DynamicBrokerConfig.scala | 22 ------------ .../log/remote/storage/RemoteLogManager.java | 6 ++-- .../storage/RemoteLogManagerConfig.java | 4 +-- .../storage/internals/log/LogConfig.java | 34 +++---------------- .../remote/storage/RemoteLogManagerTest.java | 8 ++--- 5 files changed, 13 insertions(+), 61 deletions(-) diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index f5f9c18f0b874..ba1d98c31e038 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -585,17 +585,6 @@ class DynamicLogConfig(logManager: LogManager, directoryEventHandler: DirectoryE val effectiveLocalRetentionMs = if (logLocalRetentionMs == -2L) logRetentionMs else logLocalRetentionMs val logRemoteCopyLagMs = newConfig.remoteLogManagerConfig.logRemoteCopyLagMs val logRemoteCopyLagBytes = newConfig.remoteLogManagerConfig.logRemoteCopyLagBytes - if (logRemoteCopyLagMs == LogConfig.DEFAULT_REMOTE_COPY_LAG_MS && - logRemoteCopyLagBytes != LogConfig.DEFAULT_REMOTE_COPY_LAG_BYTES && - effectiveLocalRetentionMs >= 0L) { - throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_MS_PROP, logRemoteCopyLagMs, - s"Value must not be -2 when effective ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP} is non-negative " + - s"unless ${RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_BYTES_PROP} is also -2.") - } - if (effectiveLocalRetentionMs == -1L && logRemoteCopyLagMs == -1L) { - throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_MS_PROP, logRemoteCopyLagMs, - s"Value must not be -1 when effective ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP} is -1") - } if (logRemoteCopyLagMs > 0L && effectiveLocalRetentionMs >= 0L && logRemoteCopyLagMs > effectiveLocalRetentionMs) { throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_MS_PROP, logRemoteCopyLagMs, s"Value must not exceed ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP} (effective value: $effectiveLocalRetentionMs)") @@ -608,17 +597,6 @@ class DynamicLogConfig(logManager: LogManager, directoryEventHandler: DirectoryE val effectiveLocalRetentionBytes = if (logLocalRetentionBytes == -2L) logRetentionBytes else logLocalRetentionBytes val logRemoteCopyLagBytes = newConfig.remoteLogManagerConfig.logRemoteCopyLagBytes val logRemoteCopyLagMs = newConfig.remoteLogManagerConfig.logRemoteCopyLagMs - if (logRemoteCopyLagBytes == LogConfig.DEFAULT_REMOTE_COPY_LAG_BYTES && - logRemoteCopyLagMs != LogConfig.DEFAULT_REMOTE_COPY_LAG_MS && - effectiveLocalRetentionBytes >= 0L) { - throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_BYTES_PROP, logRemoteCopyLagBytes, - s"Value must not be -2 when effective ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP} is non-negative " + - s"unless ${RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_MS_PROP} is also -2.") - } - if (effectiveLocalRetentionBytes == -1L && logRemoteCopyLagBytes == -1L) { - throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_BYTES_PROP, logRemoteCopyLagBytes, - s"Value must not be -1 when effective ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP} is -1") - } if (logRemoteCopyLagBytes > 0L && effectiveLocalRetentionBytes >= 0L && logRemoteCopyLagBytes > effectiveLocalRetentionBytes) { throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_BYTES_PROP, logRemoteCopyLagBytes, s"Value must not exceed ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP} (effective value: $effectiveLocalRetentionBytes)") diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index 4361acae3977b..9c778675206d9 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -962,15 +962,15 @@ private boolean delayCopy(LogConfig logConfig, LogSegment previousSeg, long curr return false; } - if (copyLagMs == LogConfig.DEFAULT_REMOTE_COPY_LAG_MS && copyLagBytes == LogConfig.DEFAULT_REMOTE_COPY_LAG_BYTES) { + if (copyLagMs == -2 && copyLagBytes == -2) { return false; } - if (copyLagMs != LogConfig.DEFAULT_REMOTE_COPY_LAG_MS && copyLagBytes != LogConfig.DEFAULT_REMOTE_COPY_LAG_BYTES) { + if (copyLagMs != -2 && copyLagBytes != -2) { return notExceededCopyLagTime(previousSeg, currentTimeMs, copyLagMs) && notExceededCopyLagSize(previousSeg, totalLogSize, cumulativeSize, copyLagBytes); } - if (copyLagMs != LogConfig.DEFAULT_REMOTE_COPY_LAG_MS) { + if (copyLagMs != -2) { return notExceededCopyLagTime(previousSeg, currentTimeMs, copyLagMs); } diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java index 13e885251cab1..3d59fe45584b1 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java @@ -174,7 +174,7 @@ public final class RemoteLogManagerConfig { "When set to a positive value (ms), a segment can't become eligible for upload until the time since the latest record in the segment reaches the value. " + "The value should not exceed the real local retention ms. " + "For how the real local retention time is computed, see log.local.retention.ms."; - public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_MS = -2L; + public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_MS = 0L; public static final String LOG_REMOTE_COPY_LAG_BYTES_PROP = "log.remote.copy.lag.bytes"; public static final String LOG_REMOTE_COPY_LAG_BYTES_DOC = "Controls size-based delay for uploading segments to remote storage. " + @@ -182,7 +182,7 @@ public final class RemoteLogManagerConfig { "When set to a positive value (bytes), a segment can't become eligible for upload until the total bytes of log data after the segment reach the value. " + "The value should not exceed the real local retention bytes. " + "For how the real local retention size is computed, see log.local.retention.bytes."; - public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_BYTES = -2L; + public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_BYTES = 0L; public static final String REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP = "remote.log.manager.copy.max.bytes.per.second"; public static final String REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_DOC = "The maximum number of bytes that can be copied from local storage to remote storage per second. " + diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index cc71cf8c9c6ec..1d9c22535fa9d 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -144,10 +144,8 @@ public Optional serverConfigName(String configName) { public static final boolean DEFAULT_REMOTE_LOG_DELETE_ON_DISABLE_CONFIG = false; public static final long DEFAULT_LOCAL_RETENTION_BYTES = -2; // It indicates the value to be derived from RetentionBytes public static final long DEFAULT_LOCAL_RETENTION_MS = -2; // It indicates the value to be derived from RetentionMs - public static final long DEFAULT_REMOTE_COPY_LAG_MS = -2; // It indicates no delay check based on local retention ms - public static final long DEFAULT_REMOTE_COPY_LAG_BYTES = -2; // It indicates no delay check based on local retention bytes - public static final long MAX_REMOTE_COPY_LAG_MS = -1; // It indicates the value depends on local retention ms - public static final long MAX_REMOTE_COPY_LAG_BYTES = -1; // It indicates the value depends on local retention bytes + public static final long DEFAULT_REMOTE_COPY_LAG_MS = 0; + public static final long DEFAULT_REMOTE_COPY_LAG_BYTES = 0; public static final String INTERNAL_SEGMENT_BYTES_CONFIG = "internal.segment.bytes"; public static final String INTERNAL_SEGMENT_BYTES_DOC = "The maximum size of a single log file. This should be used for testing only."; @@ -420,11 +418,11 @@ public Boolean remoteLogCopyDisable() { public long remoteCopyLagMs() { - return remoteLogConfig.remoteCopyLagMs == MAX_REMOTE_COPY_LAG_MS ? localRetentionMs() : remoteLogConfig.remoteCopyLagMs; + return remoteLogConfig.remoteCopyLagMs; } public long remoteCopyLagBytes() { - return remoteLogConfig.remoteCopyLagBytes == MAX_REMOTE_COPY_LAG_BYTES ? localRetentionBytes() : remoteLogConfig.remoteCopyLagBytes; + return remoteLogConfig.remoteCopyLagBytes; } public long localRetentionMs() { @@ -635,19 +633,7 @@ private static void validateRemoteCopyLagTime(Map props) { Long retentionMs = (Long) props.get(TopicConfig.RETENTION_MS_CONFIG); Long localRetentionMs = (Long) props.get(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG); Long remoteCopyLagMs = (Long) props.get(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG); - Long remoteCopyLagBytes = (Long) props.get(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG); long effectiveLocalRetentionMs = localRetentionMs == -2 ? retentionMs : localRetentionMs; - if (remoteCopyLagMs == DEFAULT_REMOTE_COPY_LAG_MS && remoteCopyLagBytes != DEFAULT_REMOTE_COPY_LAG_BYTES - && effectiveLocalRetentionMs >= 0) { - String message = String.format("Value must not be -2 when effective %s is non-negative unless %s is also -2.", - TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG); - throw new ConfigException(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, remoteCopyLagMs, message); - } - if (effectiveLocalRetentionMs == -1 && remoteCopyLagMs == -1) { - String message = String.format("Value must not be -1 when effective %s is -1", - TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG); - throw new ConfigException(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, remoteCopyLagMs, message); - } if (remoteCopyLagMs > 0 && effectiveLocalRetentionMs >= 0 && remoteCopyLagMs > effectiveLocalRetentionMs) { String message = String.format("Value must not exceed %s (effective value: %d)", @@ -660,19 +646,7 @@ private static void validateRemoteCopyLagSize(Map props) { Long retentionBytes = (Long) props.get(TopicConfig.RETENTION_BYTES_CONFIG); Long localRetentionBytes = (Long) props.get(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG); Long remoteCopyLagBytes = (Long) props.get(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG); - Long remoteCopyLagMs = (Long) props.get(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG); long effectiveLocalRetentionBytes = localRetentionBytes == -2 ? retentionBytes : localRetentionBytes; - if (remoteCopyLagBytes == DEFAULT_REMOTE_COPY_LAG_BYTES && remoteCopyLagMs != DEFAULT_REMOTE_COPY_LAG_MS - && effectiveLocalRetentionBytes >= 0) { - String message = String.format("Value must not be -2 when effective %s is non-negative unless %s is also -2.", - TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, TopicConfig.REMOTE_COPY_LAG_MS_CONFIG); - throw new ConfigException(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, remoteCopyLagBytes, message); - } - if (effectiveLocalRetentionBytes == -1 && remoteCopyLagBytes == -1) { - String message = String.format("Value must not be -1 when effective %s is -1", - TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG); - throw new ConfigException(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, remoteCopyLagBytes, message); - } if (remoteCopyLagBytes > 0 && effectiveLocalRetentionBytes >= 0 && remoteCopyLagBytes > effectiveLocalRetentionBytes) { String message = String.format("Value must not exceed %s (effective value: %d)", diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java index c4b178b98c8a8..8ac0d686c4394 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java @@ -2383,7 +2383,7 @@ public void testCandidateLogSegmentsUploadWhenRemoteCopyLagMsUsesLocalRetention( Map logProps = new HashMap<>(); logProps.put(TopicConfig.RETENTION_MS_CONFIG, 10_000L); logProps.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, 100L); - logProps.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, LogConfig.MAX_REMOTE_COPY_LAG_MS); + logProps.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, -1L); LogConfig logConfig = new LogConfig(logProps); when(log.config()).thenReturn(logConfig); when(log.logSegments(5L, Long.MAX_VALUE)).thenReturn(List.of(segment1, segment2, activeSegment)); @@ -2419,7 +2419,7 @@ public void testCandidateLogSegmentsDelayUploadWhenRemoteCopyLagMsUsesLocalReten Map logProps = new HashMap<>(); logProps.put(TopicConfig.RETENTION_MS_CONFIG, 10_000L); logProps.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, 100L); - logProps.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, LogConfig.MAX_REMOTE_COPY_LAG_MS); + logProps.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, -1L); LogConfig logConfig = new LogConfig(logProps); when(log.config()).thenReturn(logConfig); when(log.logSegments(5L, Long.MAX_VALUE)).thenReturn(List.of(segment1, segment2, activeSegment)); @@ -2450,7 +2450,7 @@ public void testCandidateLogSegmentsUploadWhenRemoteCopyLagBytesUsesLocalRetenti Map logProps = new HashMap<>(); logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, 10_000L); logProps.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, 50L); - logProps.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, LogConfig.MAX_REMOTE_COPY_LAG_BYTES); + logProps.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, -1L); LogConfig logConfig = new LogConfig(logProps); when(log.config()).thenReturn(logConfig); when(log.logSegments(5L, Long.MAX_VALUE)).thenReturn(List.of(segment1, segment2, activeSegment)); @@ -2482,7 +2482,7 @@ public void testCandidateLogSegmentsDelayUploadWhenRemoteCopyLagBytesUsesLocalRe Map logProps = new HashMap<>(); logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, 10_000L); logProps.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, 60L); - logProps.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, LogConfig.MAX_REMOTE_COPY_LAG_BYTES); + logProps.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, -1L); LogConfig logConfig = new LogConfig(logProps); when(log.config()).thenReturn(logConfig); when(log.logSegments(5L, Long.MAX_VALUE)).thenReturn(List.of(segment1, segment2, activeSegment)); From abd05638969f17b7b37fe5cb6b03f2ee45c46d1b Mon Sep 17 00:00:00 2001 From: Jian Date: Thu, 7 May 2026 23:37:42 +0800 Subject: [PATCH 103/110] change to 0 Signed-off-by: Jian --- .../org/apache/kafka/storage/internals/log/LogConfig.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 1d9c22535fa9d..5d9856ece9df1 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -255,8 +255,8 @@ public Optional serverConfigName(String configName) { .define(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, LONG, DEFAULT_LOCAL_RETENTION_BYTES, atLeast(-2), MEDIUM, TopicConfig.LOCAL_LOG_RETENTION_BYTES_DOC) .define(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_COPY_DISABLE_DOC) - .define(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, LONG, DEFAULT_REMOTE_COPY_LAG_MS, atLeast(-2), MEDIUM, TopicConfig.REMOTE_COPY_LAG_MS_DOC) - .define(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, LONG, DEFAULT_REMOTE_COPY_LAG_BYTES, atLeast(-2), MEDIUM, TopicConfig.REMOTE_COPY_LAG_BYTES_DOC) + .define(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, LONG, DEFAULT_REMOTE_COPY_LAG_MS, atLeast(0), MEDIUM, TopicConfig.REMOTE_COPY_LAG_MS_DOC) + .define(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, LONG, DEFAULT_REMOTE_COPY_LAG_BYTES, atLeast(0), MEDIUM, TopicConfig.REMOTE_COPY_LAG_BYTES_DOC) .define(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_DOC) .define(TopicConfig.ERRORS_DEADLETTERQUEUE_GROUP_ENABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.ERRORS_DEADLETTERQUEUE_GROUP_ENABLE_DOC) .defineInternal(INTERNAL_SEGMENT_BYTES_CONFIG, INT, null, null, MEDIUM, INTERNAL_SEGMENT_BYTES_DOC); From 33944d40c7028ce22e674ba8ad174c71640abc7c Mon Sep 17 00:00:00 2001 From: Jian Date: Thu, 7 May 2026 23:50:14 +0800 Subject: [PATCH 104/110] remove useless test Signed-off-by: Jian --- .../scala/unit/kafka/log/LogConfigTest.scala | 50 ------------------- 1 file changed, 50 deletions(-) diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index 6143eb0334bc3..45592ffc19c91 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -260,18 +260,6 @@ class LogConfigTest { doTestInvalidLocalLogRetentionProps(2000L, -1, 100, 1000L) } - @Test - def testInvalidRemoteCopyLagMsWhenEffectiveLocalRetentionMsIsUnlimited(): Unit = { - val props = new util.HashMap[String, String]() - props.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") - props.put(TopicConfig.RETENTION_MS_CONFIG, "-1") - props.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, "-2") - props.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, "-1") - - val exception = assertThrows(classOf[ConfigException], () => validateTopicLogConfig(props)) - assertTrue(exception.getMessage.contains(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG)) - } - @Test def testInvalidRemoteCopyLagMsWhenGreaterThanEffectiveLocalRetentionMs(): Unit = { val props = new util.HashMap[String, String]() @@ -284,18 +272,6 @@ class LogConfigTest { assertTrue(exception.getMessage.contains(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG)) } - @Test - def testInvalidRemoteCopyLagBytesWhenEffectiveLocalRetentionBytesIsUnlimited(): Unit = { - val props = new util.HashMap[String, String]() - props.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") - props.put(TopicConfig.RETENTION_BYTES_CONFIG, "-1") - props.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, "-2") - props.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, "-1") - - val exception = assertThrows(classOf[ConfigException], () => validateTopicLogConfig(props)) - assertTrue(exception.getMessage.contains(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG)) - } - @Test def testInvalidRemoteCopyLagBytesWhenGreaterThanEffectiveLocalRetentionBytes(): Unit = { val props = new util.HashMap[String, String]() @@ -308,32 +284,6 @@ class LogConfigTest { assertTrue(exception.getMessage.contains(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG)) } - @Test - def testInvalidRemoteCopyLagMsWhenOnlyTimeLagCheckIsDisabled(): Unit = { - val props = new util.HashMap[String, String]() - props.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") - props.put(TopicConfig.RETENTION_MS_CONFIG, "1000") - props.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, "500") - props.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, "-2") - props.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, "100") - - val exception = assertThrows(classOf[ConfigException], () => validateTopicLogConfig(props)) - assertTrue(exception.getMessage.contains(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG)) - } - - @Test - def testInvalidRemoteCopyLagBytesWhenOnlySizeLagCheckIsDisabled(): Unit = { - val props = new util.HashMap[String, String]() - props.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") - props.put(TopicConfig.RETENTION_BYTES_CONFIG, "1000") - props.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, "500") - props.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, "100") - props.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, "-2") - - val exception = assertThrows(classOf[ConfigException], () => validateTopicLogConfig(props)) - assertTrue(exception.getMessage.contains(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG)) - } - @Test def testValidRemoteCopyLagWhenBothLagChecksAreDisabled(): Unit = { val props = new util.HashMap[String, String]() From ec8ff4fce8de0a0da66314abfe8cd927e57230e7 Mon Sep 17 00:00:00 2001 From: Jian Date: Thu, 7 May 2026 23:53:30 +0800 Subject: [PATCH 105/110] correct the unit test Signed-off-by: Jian --- core/src/test/scala/unit/kafka/log/LogConfigTest.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index 45592ffc19c91..f5c3241dd9b2e 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -73,8 +73,8 @@ class LogConfigTest { case TopicConfig.COMPRESSION_ZSTD_LEVEL_CONFIG => assertPropertyInvalid(name, "not_a_number", "-0.1") case TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG => assertPropertyInvalid(name, "not_a_number", "remove", "0") case TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG => assertPropertyInvalid(name, "not_a_number", "remove", "0") - case TopicConfig.REMOTE_COPY_LAG_MS_CONFIG => assertPropertyInvalid(name, "not_a_number", "-3") - case TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG => assertPropertyInvalid(name, "not_a_number", "-3") + case TopicConfig.REMOTE_COPY_LAG_MS_CONFIG => assertPropertyInvalid(name, "not_a_number", "-1") + case TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG => assertPropertyInvalid(name, "not_a_number", "-1") case TopicConfig.ERRORS_DEADLETTERQUEUE_GROUP_ENABLE_CONFIG => assertPropertyInvalid(name, "not_a_boolean") case LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG => // no op From 06a93df50ead5b9a130a467b6d1954b7a04802df Mon Sep 17 00:00:00 2001 From: Jian Date: Thu, 7 May 2026 23:57:44 +0800 Subject: [PATCH 106/110] correct the unit test Signed-off-by: Jian --- core/src/test/scala/unit/kafka/log/LogConfigTest.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index f5c3241dd9b2e..fbeed4cfeb2a7 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -292,8 +292,8 @@ class LogConfigTest { props.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, "-2") props.put(TopicConfig.RETENTION_BYTES_CONFIG, "1000") props.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, "-2") - props.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, "-2") - props.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, "-2") + props.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, "0") + props.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, "0") validateTopicLogConfig(props) } From a20e4d2fc04f3207aaa51b6feddd38346d746eac Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 8 May 2026 00:01:09 +0800 Subject: [PATCH 107/110] correct the unit test Signed-off-by: Jian --- .../server/DynamicBrokerConfigTest.scala | 39 ------------------- .../unit/kafka/server/KafkaConfigTest.scala | 4 +- 2 files changed, 2 insertions(+), 41 deletions(-) diff --git a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala index 4e8a4a8b17ae0..581af6827933e 100755 --- a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala @@ -772,16 +772,6 @@ class DynamicBrokerConfigTest { @Test def testDynamicRemoteCopyLagThrowsOnIncorrectConfig(): Unit = { - // effective local retention ms = -1 and remote copy lag ms = -1 is invalid - verifyIncorrectRemoteCopyLagProps( - retentionMs = -1L, - logLocalRetentionMs = -2L, - remoteCopyLagMs = -1L, - retentionBytes = 1000L, - logLocalRetentionBytes = -2L, - remoteCopyLagBytes = 100L - ) - // remote copy lag ms cannot exceed effective local retention ms verifyIncorrectRemoteCopyLagProps( retentionMs = 1000L, @@ -792,16 +782,6 @@ class DynamicBrokerConfigTest { remoteCopyLagBytes = 100L ) - // effective local retention bytes = -1 and remote copy lag bytes = -1 is invalid - verifyIncorrectRemoteCopyLagProps( - retentionMs = 1000L, - logLocalRetentionMs = -2L, - remoteCopyLagMs = 100L, - retentionBytes = -1L, - logLocalRetentionBytes = -2L, - remoteCopyLagBytes = -1L - ) - // remote copy lag bytes cannot exceed effective local retention bytes verifyIncorrectRemoteCopyLagProps( retentionMs = 1000L, @@ -812,25 +792,6 @@ class DynamicBrokerConfigTest { remoteCopyLagBytes = 1001L ) - // when local retention ms is explicitly configured, remote copy lag ms cannot be disabled alone - verifyIncorrectRemoteCopyLagProps( - retentionMs = 1000L, - logLocalRetentionMs = 500L, - remoteCopyLagMs = -2L, - retentionBytes = 1000L, - logLocalRetentionBytes = -2L, - remoteCopyLagBytes = 100L - ) - - // when local retention bytes is explicitly configured, remote copy lag bytes cannot be disabled alone - verifyIncorrectRemoteCopyLagProps( - retentionMs = 1000L, - logLocalRetentionMs = -2L, - remoteCopyLagMs = 100L, - retentionBytes = 1000L, - logLocalRetentionBytes = 500L, - remoteCopyLagBytes = -2L - ) } def verifyIncorrectRemoteCopyLagProps(retentionMs: Long, diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 97aeecc47c1ed..8000d9a7d0aa8 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -1047,8 +1047,8 @@ class KafkaConfigTest { case RemoteLogManagerConfig.REMOTE_LOG_READER_MAX_PENDING_TASKS_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", 0, -1) case RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", -3) case RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", -3) - case RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_MS_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", "-3") - case RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_BYTES_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", "-3") + case RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_MS_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", "-1") + case RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_BYTES_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", "-1") /** New group coordinator configs */ case GroupCoordinatorConfig.GROUP_COORDINATOR_NUM_THREADS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", 0, -1) From 3cd8b26d8f49001260281813700893deb60730d6 Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 8 May 2026 00:11:14 +0800 Subject: [PATCH 108/110] correct the unit test Signed-off-by: Jian --- .../server/log/remote/storage/RemoteLogManagerTest.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java index 8ac0d686c4394..cff32a5de3b9b 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java @@ -2383,7 +2383,7 @@ public void testCandidateLogSegmentsUploadWhenRemoteCopyLagMsUsesLocalRetention( Map logProps = new HashMap<>(); logProps.put(TopicConfig.RETENTION_MS_CONFIG, 10_000L); logProps.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, 100L); - logProps.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, -1L); + logProps.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, 100L); LogConfig logConfig = new LogConfig(logProps); when(log.config()).thenReturn(logConfig); when(log.logSegments(5L, Long.MAX_VALUE)).thenReturn(List.of(segment1, segment2, activeSegment)); @@ -2419,7 +2419,7 @@ public void testCandidateLogSegmentsDelayUploadWhenRemoteCopyLagMsUsesLocalReten Map logProps = new HashMap<>(); logProps.put(TopicConfig.RETENTION_MS_CONFIG, 10_000L); logProps.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, 100L); - logProps.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, -1L); + logProps.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, 100L); LogConfig logConfig = new LogConfig(logProps); when(log.config()).thenReturn(logConfig); when(log.logSegments(5L, Long.MAX_VALUE)).thenReturn(List.of(segment1, segment2, activeSegment)); @@ -2450,7 +2450,7 @@ public void testCandidateLogSegmentsUploadWhenRemoteCopyLagBytesUsesLocalRetenti Map logProps = new HashMap<>(); logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, 10_000L); logProps.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, 50L); - logProps.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, -1L); + logProps.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, 50L); LogConfig logConfig = new LogConfig(logProps); when(log.config()).thenReturn(logConfig); when(log.logSegments(5L, Long.MAX_VALUE)).thenReturn(List.of(segment1, segment2, activeSegment)); @@ -2482,7 +2482,7 @@ public void testCandidateLogSegmentsDelayUploadWhenRemoteCopyLagBytesUsesLocalRe Map logProps = new HashMap<>(); logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, 10_000L); logProps.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, 60L); - logProps.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, -1L); + logProps.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, 60L); LogConfig logConfig = new LogConfig(logProps); when(log.config()).thenReturn(logConfig); when(log.logSegments(5L, Long.MAX_VALUE)).thenReturn(List.of(segment1, segment2, activeSegment)); From 8a0a031935fe61e11cb9dedc0df8203fe4ecd556 Mon Sep 17 00:00:00 2001 From: Jian Date: Fri, 8 May 2026 00:28:29 +0800 Subject: [PATCH 109/110] correct the implement Signed-off-by: Jian --- .../server/log/remote/storage/RemoteLogManager.java | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index 9c778675206d9..fbe8edbb3299b 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -958,19 +958,18 @@ private boolean delayCopy(LogConfig logConfig, LogSegment previousSeg, long curr previousSeg, copyLagMs, copyLagBytes, currentTimeMs, totalLogSize, cumulativeSize, totalLogSize - cumulativeSize); } - if (copyLagMs == 0 || copyLagBytes == 0) { - return false; - } + boolean needCheckCopyLagMs = copyLagMs > 0; + boolean needCheckCopyLagBytes = copyLagBytes > 0; - if (copyLagMs == -2 && copyLagBytes == -2) { + if (!needCheckCopyLagMs && !needCheckCopyLagBytes) { return false; } - if (copyLagMs != -2 && copyLagBytes != -2) { + if (needCheckCopyLagMs && needCheckCopyLagBytes) { return notExceededCopyLagTime(previousSeg, currentTimeMs, copyLagMs) && notExceededCopyLagSize(previousSeg, totalLogSize, cumulativeSize, copyLagBytes); } - if (copyLagMs != -2) { + if (needCheckCopyLagBytes) { return notExceededCopyLagTime(previousSeg, currentTimeMs, copyLagMs); } From f26f152f12695ae9569f074c9ae804a96b3c6f79 Mon Sep 17 00:00:00 2001 From: Jian Date: Thu, 7 May 2026 23:36:29 +0800 Subject: [PATCH 110/110] remove the logic for -1 and -2 Signed-off-by: Jian --- .../kafka/server/DynamicBrokerConfig.scala | 22 ------- .../scala/unit/kafka/log/LogConfigTest.scala | 58 ++----------------- .../server/DynamicBrokerConfigTest.scala | 39 ------------- .../unit/kafka/server/KafkaConfigTest.scala | 4 +- .../log/remote/storage/RemoteLogManager.java | 11 ++-- .../storage/RemoteLogManagerConfig.java | 4 +- .../storage/internals/log/LogConfig.java | 38 ++---------- .../remote/storage/RemoteLogManagerTest.java | 8 +-- 8 files changed, 23 insertions(+), 161 deletions(-) diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index f5f9c18f0b874..ba1d98c31e038 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -585,17 +585,6 @@ class DynamicLogConfig(logManager: LogManager, directoryEventHandler: DirectoryE val effectiveLocalRetentionMs = if (logLocalRetentionMs == -2L) logRetentionMs else logLocalRetentionMs val logRemoteCopyLagMs = newConfig.remoteLogManagerConfig.logRemoteCopyLagMs val logRemoteCopyLagBytes = newConfig.remoteLogManagerConfig.logRemoteCopyLagBytes - if (logRemoteCopyLagMs == LogConfig.DEFAULT_REMOTE_COPY_LAG_MS && - logRemoteCopyLagBytes != LogConfig.DEFAULT_REMOTE_COPY_LAG_BYTES && - effectiveLocalRetentionMs >= 0L) { - throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_MS_PROP, logRemoteCopyLagMs, - s"Value must not be -2 when effective ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP} is non-negative " + - s"unless ${RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_BYTES_PROP} is also -2.") - } - if (effectiveLocalRetentionMs == -1L && logRemoteCopyLagMs == -1L) { - throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_MS_PROP, logRemoteCopyLagMs, - s"Value must not be -1 when effective ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP} is -1") - } if (logRemoteCopyLagMs > 0L && effectiveLocalRetentionMs >= 0L && logRemoteCopyLagMs > effectiveLocalRetentionMs) { throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_MS_PROP, logRemoteCopyLagMs, s"Value must not exceed ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP} (effective value: $effectiveLocalRetentionMs)") @@ -608,17 +597,6 @@ class DynamicLogConfig(logManager: LogManager, directoryEventHandler: DirectoryE val effectiveLocalRetentionBytes = if (logLocalRetentionBytes == -2L) logRetentionBytes else logLocalRetentionBytes val logRemoteCopyLagBytes = newConfig.remoteLogManagerConfig.logRemoteCopyLagBytes val logRemoteCopyLagMs = newConfig.remoteLogManagerConfig.logRemoteCopyLagMs - if (logRemoteCopyLagBytes == LogConfig.DEFAULT_REMOTE_COPY_LAG_BYTES && - logRemoteCopyLagMs != LogConfig.DEFAULT_REMOTE_COPY_LAG_MS && - effectiveLocalRetentionBytes >= 0L) { - throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_BYTES_PROP, logRemoteCopyLagBytes, - s"Value must not be -2 when effective ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP} is non-negative " + - s"unless ${RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_MS_PROP} is also -2.") - } - if (effectiveLocalRetentionBytes == -1L && logRemoteCopyLagBytes == -1L) { - throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_BYTES_PROP, logRemoteCopyLagBytes, - s"Value must not be -1 when effective ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP} is -1") - } if (logRemoteCopyLagBytes > 0L && effectiveLocalRetentionBytes >= 0L && logRemoteCopyLagBytes > effectiveLocalRetentionBytes) { throw new ConfigException(RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_BYTES_PROP, logRemoteCopyLagBytes, s"Value must not exceed ${RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP} (effective value: $effectiveLocalRetentionBytes)") diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index 6143eb0334bc3..fbeed4cfeb2a7 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -73,8 +73,8 @@ class LogConfigTest { case TopicConfig.COMPRESSION_ZSTD_LEVEL_CONFIG => assertPropertyInvalid(name, "not_a_number", "-0.1") case TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG => assertPropertyInvalid(name, "not_a_number", "remove", "0") case TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG => assertPropertyInvalid(name, "not_a_number", "remove", "0") - case TopicConfig.REMOTE_COPY_LAG_MS_CONFIG => assertPropertyInvalid(name, "not_a_number", "-3") - case TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG => assertPropertyInvalid(name, "not_a_number", "-3") + case TopicConfig.REMOTE_COPY_LAG_MS_CONFIG => assertPropertyInvalid(name, "not_a_number", "-1") + case TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG => assertPropertyInvalid(name, "not_a_number", "-1") case TopicConfig.ERRORS_DEADLETTERQUEUE_GROUP_ENABLE_CONFIG => assertPropertyInvalid(name, "not_a_boolean") case LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG => // no op @@ -260,18 +260,6 @@ class LogConfigTest { doTestInvalidLocalLogRetentionProps(2000L, -1, 100, 1000L) } - @Test - def testInvalidRemoteCopyLagMsWhenEffectiveLocalRetentionMsIsUnlimited(): Unit = { - val props = new util.HashMap[String, String]() - props.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") - props.put(TopicConfig.RETENTION_MS_CONFIG, "-1") - props.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, "-2") - props.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, "-1") - - val exception = assertThrows(classOf[ConfigException], () => validateTopicLogConfig(props)) - assertTrue(exception.getMessage.contains(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG)) - } - @Test def testInvalidRemoteCopyLagMsWhenGreaterThanEffectiveLocalRetentionMs(): Unit = { val props = new util.HashMap[String, String]() @@ -284,18 +272,6 @@ class LogConfigTest { assertTrue(exception.getMessage.contains(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG)) } - @Test - def testInvalidRemoteCopyLagBytesWhenEffectiveLocalRetentionBytesIsUnlimited(): Unit = { - val props = new util.HashMap[String, String]() - props.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") - props.put(TopicConfig.RETENTION_BYTES_CONFIG, "-1") - props.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, "-2") - props.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, "-1") - - val exception = assertThrows(classOf[ConfigException], () => validateTopicLogConfig(props)) - assertTrue(exception.getMessage.contains(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG)) - } - @Test def testInvalidRemoteCopyLagBytesWhenGreaterThanEffectiveLocalRetentionBytes(): Unit = { val props = new util.HashMap[String, String]() @@ -308,32 +284,6 @@ class LogConfigTest { assertTrue(exception.getMessage.contains(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG)) } - @Test - def testInvalidRemoteCopyLagMsWhenOnlyTimeLagCheckIsDisabled(): Unit = { - val props = new util.HashMap[String, String]() - props.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") - props.put(TopicConfig.RETENTION_MS_CONFIG, "1000") - props.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, "500") - props.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, "-2") - props.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, "100") - - val exception = assertThrows(classOf[ConfigException], () => validateTopicLogConfig(props)) - assertTrue(exception.getMessage.contains(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG)) - } - - @Test - def testInvalidRemoteCopyLagBytesWhenOnlySizeLagCheckIsDisabled(): Unit = { - val props = new util.HashMap[String, String]() - props.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") - props.put(TopicConfig.RETENTION_BYTES_CONFIG, "1000") - props.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, "500") - props.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, "100") - props.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, "-2") - - val exception = assertThrows(classOf[ConfigException], () => validateTopicLogConfig(props)) - assertTrue(exception.getMessage.contains(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG)) - } - @Test def testValidRemoteCopyLagWhenBothLagChecksAreDisabled(): Unit = { val props = new util.HashMap[String, String]() @@ -342,8 +292,8 @@ class LogConfigTest { props.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, "-2") props.put(TopicConfig.RETENTION_BYTES_CONFIG, "1000") props.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, "-2") - props.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, "-2") - props.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, "-2") + props.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, "0") + props.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, "0") validateTopicLogConfig(props) } diff --git a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala index 4e8a4a8b17ae0..581af6827933e 100755 --- a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala @@ -772,16 +772,6 @@ class DynamicBrokerConfigTest { @Test def testDynamicRemoteCopyLagThrowsOnIncorrectConfig(): Unit = { - // effective local retention ms = -1 and remote copy lag ms = -1 is invalid - verifyIncorrectRemoteCopyLagProps( - retentionMs = -1L, - logLocalRetentionMs = -2L, - remoteCopyLagMs = -1L, - retentionBytes = 1000L, - logLocalRetentionBytes = -2L, - remoteCopyLagBytes = 100L - ) - // remote copy lag ms cannot exceed effective local retention ms verifyIncorrectRemoteCopyLagProps( retentionMs = 1000L, @@ -792,16 +782,6 @@ class DynamicBrokerConfigTest { remoteCopyLagBytes = 100L ) - // effective local retention bytes = -1 and remote copy lag bytes = -1 is invalid - verifyIncorrectRemoteCopyLagProps( - retentionMs = 1000L, - logLocalRetentionMs = -2L, - remoteCopyLagMs = 100L, - retentionBytes = -1L, - logLocalRetentionBytes = -2L, - remoteCopyLagBytes = -1L - ) - // remote copy lag bytes cannot exceed effective local retention bytes verifyIncorrectRemoteCopyLagProps( retentionMs = 1000L, @@ -812,25 +792,6 @@ class DynamicBrokerConfigTest { remoteCopyLagBytes = 1001L ) - // when local retention ms is explicitly configured, remote copy lag ms cannot be disabled alone - verifyIncorrectRemoteCopyLagProps( - retentionMs = 1000L, - logLocalRetentionMs = 500L, - remoteCopyLagMs = -2L, - retentionBytes = 1000L, - logLocalRetentionBytes = -2L, - remoteCopyLagBytes = 100L - ) - - // when local retention bytes is explicitly configured, remote copy lag bytes cannot be disabled alone - verifyIncorrectRemoteCopyLagProps( - retentionMs = 1000L, - logLocalRetentionMs = -2L, - remoteCopyLagMs = 100L, - retentionBytes = 1000L, - logLocalRetentionBytes = 500L, - remoteCopyLagBytes = -2L - ) } def verifyIncorrectRemoteCopyLagProps(retentionMs: Long, diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 97aeecc47c1ed..8000d9a7d0aa8 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -1047,8 +1047,8 @@ class KafkaConfigTest { case RemoteLogManagerConfig.REMOTE_LOG_READER_MAX_PENDING_TASKS_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", 0, -1) case RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", -3) case RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", -3) - case RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_MS_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", "-3") - case RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_BYTES_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", "-3") + case RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_MS_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", "-1") + case RemoteLogManagerConfig.LOG_REMOTE_COPY_LAG_BYTES_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", "-1") /** New group coordinator configs */ case GroupCoordinatorConfig.GROUP_COORDINATOR_NUM_THREADS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", 0, -1) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index 4361acae3977b..fbe8edbb3299b 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -958,19 +958,18 @@ private boolean delayCopy(LogConfig logConfig, LogSegment previousSeg, long curr previousSeg, copyLagMs, copyLagBytes, currentTimeMs, totalLogSize, cumulativeSize, totalLogSize - cumulativeSize); } - if (copyLagMs == 0 || copyLagBytes == 0) { - return false; - } + boolean needCheckCopyLagMs = copyLagMs > 0; + boolean needCheckCopyLagBytes = copyLagBytes > 0; - if (copyLagMs == LogConfig.DEFAULT_REMOTE_COPY_LAG_MS && copyLagBytes == LogConfig.DEFAULT_REMOTE_COPY_LAG_BYTES) { + if (!needCheckCopyLagMs && !needCheckCopyLagBytes) { return false; } - if (copyLagMs != LogConfig.DEFAULT_REMOTE_COPY_LAG_MS && copyLagBytes != LogConfig.DEFAULT_REMOTE_COPY_LAG_BYTES) { + if (needCheckCopyLagMs && needCheckCopyLagBytes) { return notExceededCopyLagTime(previousSeg, currentTimeMs, copyLagMs) && notExceededCopyLagSize(previousSeg, totalLogSize, cumulativeSize, copyLagBytes); } - if (copyLagMs != LogConfig.DEFAULT_REMOTE_COPY_LAG_MS) { + if (needCheckCopyLagBytes) { return notExceededCopyLagTime(previousSeg, currentTimeMs, copyLagMs); } diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java index 13e885251cab1..3d59fe45584b1 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java @@ -174,7 +174,7 @@ public final class RemoteLogManagerConfig { "When set to a positive value (ms), a segment can't become eligible for upload until the time since the latest record in the segment reaches the value. " + "The value should not exceed the real local retention ms. " + "For how the real local retention time is computed, see log.local.retention.ms."; - public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_MS = -2L; + public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_MS = 0L; public static final String LOG_REMOTE_COPY_LAG_BYTES_PROP = "log.remote.copy.lag.bytes"; public static final String LOG_REMOTE_COPY_LAG_BYTES_DOC = "Controls size-based delay for uploading segments to remote storage. " + @@ -182,7 +182,7 @@ public final class RemoteLogManagerConfig { "When set to a positive value (bytes), a segment can't become eligible for upload until the total bytes of log data after the segment reach the value. " + "The value should not exceed the real local retention bytes. " + "For how the real local retention size is computed, see log.local.retention.bytes."; - public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_BYTES = -2L; + public static final Long DEFAULT_LOG_REMOTE_COPY_LAG_BYTES = 0L; public static final String REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP = "remote.log.manager.copy.max.bytes.per.second"; public static final String REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_DOC = "The maximum number of bytes that can be copied from local storage to remote storage per second. " + diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index cc71cf8c9c6ec..5d9856ece9df1 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -144,10 +144,8 @@ public Optional serverConfigName(String configName) { public static final boolean DEFAULT_REMOTE_LOG_DELETE_ON_DISABLE_CONFIG = false; public static final long DEFAULT_LOCAL_RETENTION_BYTES = -2; // It indicates the value to be derived from RetentionBytes public static final long DEFAULT_LOCAL_RETENTION_MS = -2; // It indicates the value to be derived from RetentionMs - public static final long DEFAULT_REMOTE_COPY_LAG_MS = -2; // It indicates no delay check based on local retention ms - public static final long DEFAULT_REMOTE_COPY_LAG_BYTES = -2; // It indicates no delay check based on local retention bytes - public static final long MAX_REMOTE_COPY_LAG_MS = -1; // It indicates the value depends on local retention ms - public static final long MAX_REMOTE_COPY_LAG_BYTES = -1; // It indicates the value depends on local retention bytes + public static final long DEFAULT_REMOTE_COPY_LAG_MS = 0; + public static final long DEFAULT_REMOTE_COPY_LAG_BYTES = 0; public static final String INTERNAL_SEGMENT_BYTES_CONFIG = "internal.segment.bytes"; public static final String INTERNAL_SEGMENT_BYTES_DOC = "The maximum size of a single log file. This should be used for testing only."; @@ -257,8 +255,8 @@ public Optional serverConfigName(String configName) { .define(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, LONG, DEFAULT_LOCAL_RETENTION_BYTES, atLeast(-2), MEDIUM, TopicConfig.LOCAL_LOG_RETENTION_BYTES_DOC) .define(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_COPY_DISABLE_DOC) - .define(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, LONG, DEFAULT_REMOTE_COPY_LAG_MS, atLeast(-2), MEDIUM, TopicConfig.REMOTE_COPY_LAG_MS_DOC) - .define(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, LONG, DEFAULT_REMOTE_COPY_LAG_BYTES, atLeast(-2), MEDIUM, TopicConfig.REMOTE_COPY_LAG_BYTES_DOC) + .define(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, LONG, DEFAULT_REMOTE_COPY_LAG_MS, atLeast(0), MEDIUM, TopicConfig.REMOTE_COPY_LAG_MS_DOC) + .define(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, LONG, DEFAULT_REMOTE_COPY_LAG_BYTES, atLeast(0), MEDIUM, TopicConfig.REMOTE_COPY_LAG_BYTES_DOC) .define(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_DOC) .define(TopicConfig.ERRORS_DEADLETTERQUEUE_GROUP_ENABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.ERRORS_DEADLETTERQUEUE_GROUP_ENABLE_DOC) .defineInternal(INTERNAL_SEGMENT_BYTES_CONFIG, INT, null, null, MEDIUM, INTERNAL_SEGMENT_BYTES_DOC); @@ -420,11 +418,11 @@ public Boolean remoteLogCopyDisable() { public long remoteCopyLagMs() { - return remoteLogConfig.remoteCopyLagMs == MAX_REMOTE_COPY_LAG_MS ? localRetentionMs() : remoteLogConfig.remoteCopyLagMs; + return remoteLogConfig.remoteCopyLagMs; } public long remoteCopyLagBytes() { - return remoteLogConfig.remoteCopyLagBytes == MAX_REMOTE_COPY_LAG_BYTES ? localRetentionBytes() : remoteLogConfig.remoteCopyLagBytes; + return remoteLogConfig.remoteCopyLagBytes; } public long localRetentionMs() { @@ -635,19 +633,7 @@ private static void validateRemoteCopyLagTime(Map props) { Long retentionMs = (Long) props.get(TopicConfig.RETENTION_MS_CONFIG); Long localRetentionMs = (Long) props.get(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG); Long remoteCopyLagMs = (Long) props.get(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG); - Long remoteCopyLagBytes = (Long) props.get(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG); long effectiveLocalRetentionMs = localRetentionMs == -2 ? retentionMs : localRetentionMs; - if (remoteCopyLagMs == DEFAULT_REMOTE_COPY_LAG_MS && remoteCopyLagBytes != DEFAULT_REMOTE_COPY_LAG_BYTES - && effectiveLocalRetentionMs >= 0) { - String message = String.format("Value must not be -2 when effective %s is non-negative unless %s is also -2.", - TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG); - throw new ConfigException(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, remoteCopyLagMs, message); - } - if (effectiveLocalRetentionMs == -1 && remoteCopyLagMs == -1) { - String message = String.format("Value must not be -1 when effective %s is -1", - TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG); - throw new ConfigException(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, remoteCopyLagMs, message); - } if (remoteCopyLagMs > 0 && effectiveLocalRetentionMs >= 0 && remoteCopyLagMs > effectiveLocalRetentionMs) { String message = String.format("Value must not exceed %s (effective value: %d)", @@ -660,19 +646,7 @@ private static void validateRemoteCopyLagSize(Map props) { Long retentionBytes = (Long) props.get(TopicConfig.RETENTION_BYTES_CONFIG); Long localRetentionBytes = (Long) props.get(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG); Long remoteCopyLagBytes = (Long) props.get(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG); - Long remoteCopyLagMs = (Long) props.get(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG); long effectiveLocalRetentionBytes = localRetentionBytes == -2 ? retentionBytes : localRetentionBytes; - if (remoteCopyLagBytes == DEFAULT_REMOTE_COPY_LAG_BYTES && remoteCopyLagMs != DEFAULT_REMOTE_COPY_LAG_MS - && effectiveLocalRetentionBytes >= 0) { - String message = String.format("Value must not be -2 when effective %s is non-negative unless %s is also -2.", - TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, TopicConfig.REMOTE_COPY_LAG_MS_CONFIG); - throw new ConfigException(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, remoteCopyLagBytes, message); - } - if (effectiveLocalRetentionBytes == -1 && remoteCopyLagBytes == -1) { - String message = String.format("Value must not be -1 when effective %s is -1", - TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG); - throw new ConfigException(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, remoteCopyLagBytes, message); - } if (remoteCopyLagBytes > 0 && effectiveLocalRetentionBytes >= 0 && remoteCopyLagBytes > effectiveLocalRetentionBytes) { String message = String.format("Value must not exceed %s (effective value: %d)", diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java index c4b178b98c8a8..cff32a5de3b9b 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java @@ -2383,7 +2383,7 @@ public void testCandidateLogSegmentsUploadWhenRemoteCopyLagMsUsesLocalRetention( Map logProps = new HashMap<>(); logProps.put(TopicConfig.RETENTION_MS_CONFIG, 10_000L); logProps.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, 100L); - logProps.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, LogConfig.MAX_REMOTE_COPY_LAG_MS); + logProps.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, 100L); LogConfig logConfig = new LogConfig(logProps); when(log.config()).thenReturn(logConfig); when(log.logSegments(5L, Long.MAX_VALUE)).thenReturn(List.of(segment1, segment2, activeSegment)); @@ -2419,7 +2419,7 @@ public void testCandidateLogSegmentsDelayUploadWhenRemoteCopyLagMsUsesLocalReten Map logProps = new HashMap<>(); logProps.put(TopicConfig.RETENTION_MS_CONFIG, 10_000L); logProps.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, 100L); - logProps.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, LogConfig.MAX_REMOTE_COPY_LAG_MS); + logProps.put(TopicConfig.REMOTE_COPY_LAG_MS_CONFIG, 100L); LogConfig logConfig = new LogConfig(logProps); when(log.config()).thenReturn(logConfig); when(log.logSegments(5L, Long.MAX_VALUE)).thenReturn(List.of(segment1, segment2, activeSegment)); @@ -2450,7 +2450,7 @@ public void testCandidateLogSegmentsUploadWhenRemoteCopyLagBytesUsesLocalRetenti Map logProps = new HashMap<>(); logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, 10_000L); logProps.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, 50L); - logProps.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, LogConfig.MAX_REMOTE_COPY_LAG_BYTES); + logProps.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, 50L); LogConfig logConfig = new LogConfig(logProps); when(log.config()).thenReturn(logConfig); when(log.logSegments(5L, Long.MAX_VALUE)).thenReturn(List.of(segment1, segment2, activeSegment)); @@ -2482,7 +2482,7 @@ public void testCandidateLogSegmentsDelayUploadWhenRemoteCopyLagBytesUsesLocalRe Map logProps = new HashMap<>(); logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, 10_000L); logProps.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, 60L); - logProps.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, LogConfig.MAX_REMOTE_COPY_LAG_BYTES); + logProps.put(TopicConfig.REMOTE_COPY_LAG_BYTES_CONFIG, 60L); LogConfig logConfig = new LogConfig(logProps); when(log.config()).thenReturn(logConfig); when(log.logSegments(5L, Long.MAX_VALUE)).thenReturn(List.of(segment1, segment2, activeSegment));