Re: [PR] KAFKA-16452: Bound high-watermark offset to range between LLSO and LEO [kafka]

2024-05-01 Thread via GitHub
junrao commented on code in PR #15634: URL: https://github.com/apache/kafka/pull/15634#discussion_r1586840549 ## core/src/main/scala/kafka/log/UnifiedLog.scala: ## @@ -282,15 +282,15 @@ class UnifiedLog(@volatile var logStartOffset: Long, /** * Update high watermark

Re: [PR] KAFKA-16452: Bound high-watermark offset to range between LLSO and LEO [kafka]

2024-04-28 Thread via GitHub
kamalcph commented on code in PR #15634: URL: https://github.com/apache/kafka/pull/15634#discussion_r1582149811 ## core/src/main/scala/kafka/log/UnifiedLog.scala: ## @@ -282,15 +282,15 @@ class UnifiedLog(@volatile var logStartOffset: Long, /** * Update high watermark

Re: [PR] KAFKA-16452: Bound high-watermark offset to range between LLSO and LEO [kafka]

2024-04-28 Thread via GitHub
kamalcph commented on code in PR #15634: URL: https://github.com/apache/kafka/pull/15634#discussion_r1582144306 ## core/src/main/scala/kafka/log/UnifiedLog.scala: ## @@ -282,15 +282,15 @@ class UnifiedLog(@volatile var logStartOffset: Long, /** * Update high watermark

Re: [PR] KAFKA-16452: Bound high-watermark offset to range between LLSO and LEO [kafka]

2024-04-28 Thread via GitHub
kamalcph commented on code in PR #15634: URL: https://github.com/apache/kafka/pull/15634#discussion_r1582144306 ## core/src/main/scala/kafka/log/UnifiedLog.scala: ## @@ -282,15 +282,15 @@ class UnifiedLog(@volatile var logStartOffset: Long, /** * Update high watermark

Re: [PR] KAFKA-16452: Bound high-watermark offset to range between LLSO and LEO [kafka]

2024-04-28 Thread via GitHub
kamalcph commented on code in PR #15634: URL: https://github.com/apache/kafka/pull/15634#discussion_r1582144306 ## core/src/main/scala/kafka/log/UnifiedLog.scala: ## @@ -282,15 +282,15 @@ class UnifiedLog(@volatile var logStartOffset: Long, /** * Update high watermark

Re: [PR] KAFKA-16452: Bound high-watermark offset to range between LLSO and LEO [kafka]

2024-04-18 Thread via GitHub
kamalcph commented on code in PR #15634: URL: https://github.com/apache/kafka/pull/15634#discussion_r1571126880 ## core/src/main/scala/kafka/log/UnifiedLog.scala: ## @@ -282,15 +282,15 @@ class UnifiedLog(@volatile var logStartOffset: Long, /** * Update high watermark

Re: [PR] KAFKA-16452: Bound high-watermark offset to range between LLSO and LEO [kafka]

2024-04-18 Thread via GitHub
kamalcph commented on PR #15634: URL: https://github.com/apache/kafka/pull/15634#issuecomment-2064647379 Thanks @chia7712 for the review! > log-start-offset-checkpoint is missing and remote storage is enabled. The logStartOffset will be set to zero, and it seems be a potential issue

Re: [PR] KAFKA-16452: Bound high-watermark offset to range between LLSO and LEO [kafka]

2024-04-18 Thread via GitHub
chia7712 commented on PR #15634: URL: https://github.com/apache/kafka/pull/15634#issuecomment-2063727744 Sorry that the story I mentioned above seems be another issue. Let me have the summary about my thought. 1. `log-start-offset-checkpoint` is missing and remote storage is enabled.

Re: [PR] KAFKA-16452: Bound high-watermark offset to range between LLSO and LEO [kafka]

2024-04-18 Thread via GitHub
chia7712 commented on PR #15634: URL: https://github.com/apache/kafka/pull/15634#issuecomment-2063435567 > HWM is set to to localLogStartOffset in

Re: [PR] KAFKA-16452: Bound high-watermark offset to range between LLSO and LEO [kafka]

2024-04-17 Thread via GitHub
junrao commented on code in PR #15634: URL: https://github.com/apache/kafka/pull/15634#discussion_r1569562746 ## core/src/main/scala/kafka/log/UnifiedLog.scala: ## @@ -282,15 +282,15 @@ class UnifiedLog(@volatile var logStartOffset: Long, /** * Update high watermark

Re: [PR] KAFKA-16452: Bound high-watermark offset to range between LLSO and LEO [kafka]

2024-04-17 Thread via GitHub
kamalcph commented on code in PR #15634: URL: https://github.com/apache/kafka/pull/15634#discussion_r1569356239 ## core/src/main/scala/kafka/log/UnifiedLog.scala: ## @@ -1223,6 +1223,12 @@ class UnifiedLog(@volatile var logStartOffset: Long, s"but we only have log

Re: [PR] KAFKA-16452: Bound high-watermark offset to range between LLSO and LEO [kafka]

2024-04-17 Thread via GitHub
kamalcph commented on code in PR #15634: URL: https://github.com/apache/kafka/pull/15634#discussion_r1563885907 ## core/src/main/scala/kafka/log/UnifiedLog.scala: ## @@ -282,15 +282,15 @@ class UnifiedLog(@volatile var logStartOffset: Long, /** * Update high watermark

Re: [PR] KAFKA-16452: Bound high-watermark offset to range between LLSO and LEO [kafka]

2024-04-17 Thread via GitHub
kamalcph commented on code in PR #15634: URL: https://github.com/apache/kafka/pull/15634#discussion_r1569328393 ## core/src/main/scala/kafka/log/UnifiedLog.scala: ## @@ -282,15 +282,15 @@ class UnifiedLog(@volatile var logStartOffset: Long, /** * Update high watermark

Re: [PR] KAFKA-16452: Bound high-watermark offset to range between LLSO and LEO [kafka]

2024-04-17 Thread via GitHub
kamalcph commented on code in PR #15634: URL: https://github.com/apache/kafka/pull/15634#discussion_r1563885907 ## core/src/main/scala/kafka/log/UnifiedLog.scala: ## @@ -282,15 +282,15 @@ class UnifiedLog(@volatile var logStartOffset: Long, /** * Update high watermark

Re: [PR] KAFKA-16452: Bound high-watermark offset to range between LLSO and LEO [kafka]

2024-04-16 Thread via GitHub
junrao commented on code in PR #15634: URL: https://github.com/apache/kafka/pull/15634#discussion_r1567796843 ## core/src/main/scala/kafka/log/UnifiedLog.scala: ## @@ -282,15 +282,15 @@ class UnifiedLog(@volatile var logStartOffset: Long, /** * Update high watermark

Re: [PR] KAFKA-16452: Bound high-watermark offset to range between LLSO and LEO [kafka]

2024-04-16 Thread via GitHub
chia7712 commented on code in PR #15634: URL: https://github.com/apache/kafka/pull/15634#discussion_r1567414802 ## core/src/main/scala/kafka/log/UnifiedLog.scala: ## @@ -1223,6 +1223,12 @@ class UnifiedLog(@volatile var logStartOffset: Long, s"but we only have log

Re: [PR] KAFKA-16452: Bound high-watermark offset to range between LLSO and LEO [kafka]

2024-04-13 Thread via GitHub
kamalcph commented on code in PR #15634: URL: https://github.com/apache/kafka/pull/15634#discussion_r1563885907 ## core/src/main/scala/kafka/log/UnifiedLog.scala: ## @@ -282,15 +282,15 @@ class UnifiedLog(@volatile var logStartOffset: Long, /** * Update high watermark

Re: [PR] KAFKA-16452: Bound high-watermark offset to range between LLSO and LEO [kafka]

2024-04-13 Thread via GitHub
kamalcph commented on code in PR #15634: URL: https://github.com/apache/kafka/pull/15634#discussion_r1563885907 ## core/src/main/scala/kafka/log/UnifiedLog.scala: ## @@ -282,15 +282,15 @@ class UnifiedLog(@volatile var logStartOffset: Long, /** * Update high watermark

Re: [PR] KAFKA-16452: Bound high-watermark offset to range between LLSO and LEO [kafka]

2024-04-13 Thread via GitHub
kamalcph commented on code in PR #15634: URL: https://github.com/apache/kafka/pull/15634#discussion_r1563885907 ## core/src/main/scala/kafka/log/UnifiedLog.scala: ## @@ -282,15 +282,15 @@ class UnifiedLog(@volatile var logStartOffset: Long, /** * Update high watermark

Re: [PR] KAFKA-16452: Bound high-watermark offset to range between LLSO and LEO [kafka]

2024-04-12 Thread via GitHub
junrao commented on code in PR #15634: URL: https://github.com/apache/kafka/pull/15634#discussion_r1562991673 ## core/src/main/scala/kafka/log/UnifiedLog.scala: ## @@ -282,15 +282,15 @@ class UnifiedLog(@volatile var logStartOffset: Long, /** * Update high watermark

Re: [PR] KAFKA-16452: Bound high-watermark offset to range between LLSO and LEO [kafka]

2024-04-11 Thread via GitHub
kamalcph commented on PR #15634: URL: https://github.com/apache/kafka/pull/15634#issuecomment-2050934793 @junrao @showuon @divijvaidya Gentle bump to review the diff, thanks! -- This is an automated message from the Apache Git Service. To respond to the message, please log on to

Re: [PR] KAFKA-16452: Bound high-watermark offset to range between LLSO and LEO [kafka]

2024-04-09 Thread via GitHub
kamalcph commented on code in PR #15634: URL: https://github.com/apache/kafka/pull/15634#discussion_r1557089094 ## core/src/main/scala/kafka/log/UnifiedLog.scala: ## @@ -282,15 +282,15 @@ class UnifiedLog(@volatile var logStartOffset: Long, /** * Update high watermark

Re: [PR] KAFKA-16452: Bound high-watermark offset to range between LLSO and LEO [kafka]

2024-04-08 Thread via GitHub
junrao commented on code in PR #15634: URL: https://github.com/apache/kafka/pull/15634#discussion_r1556593959 ## core/src/main/scala/kafka/log/UnifiedLog.scala: ## @@ -282,15 +282,15 @@ class UnifiedLog(@volatile var logStartOffset: Long, /** * Update high watermark

Re: [PR] KAFKA-16452: Bound high-watermark offset to range between LLSO and LEO [kafka]

2024-04-01 Thread via GitHub
satishd commented on code in PR #15634: URL: https://github.com/apache/kafka/pull/15634#discussion_r1546167963 ## core/src/main/scala/kafka/log/UnifiedLog.scala: ## @@ -136,16 +136,16 @@ class UnifiedLog(@volatile var logStartOffset: Long, */ @volatile private var

Re: [PR] KAFKA-16452: Bound high-watermark offset to range between LLSO and LEO [kafka]

2024-04-01 Thread via GitHub
kamalcph commented on PR #15634: URL: https://github.com/apache/kafka/pull/15634#issuecomment-2029474481 > just curious. Does it happens only if remote storage is enabled? According to the description: > > > The follower sends the first FETCH request to the leader, the leader checks

Re: [PR] KAFKA-16452: Bound high-watermark offset to range between LLSO and LEO [kafka]

2024-04-01 Thread via GitHub
chia7712 commented on PR #15634: URL: https://github.com/apache/kafka/pull/15634#issuecomment-2029310080 just curious. Does it happens only if remote storage is enabled? According to the description: > The follower sends the first FETCH request to the leader, the leader checks

[PR] KAFKA-16452: Bound high-watermark offset to range between LLSO and LEO [kafka]

2024-03-31 Thread via GitHub
kamalcph opened a new pull request, #15634: URL: https://github.com/apache/kafka/pull/15634 Bound high-watermark offset between local-log-start-offset and log-end-offset: The high watermark should not go below the local-log-start offset. If the high watermark is less than the