Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2024-01-14 Thread via GitHub
satishd merged PR #15060: URL: https://github.com/apache/kafka/pull/15060 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail:

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2024-01-14 Thread via GitHub
satishd commented on PR #15060: URL: https://github.com/apache/kafka/pull/15060#issuecomment-1891112974 Merging it to trunk as the test failures are unrelated to this change. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2024-01-13 Thread via GitHub
satishd commented on PR #15060: URL: https://github.com/apache/kafka/pull/15060#issuecomment-1890811313 @iit2009060 As we discussed earlier in the [comment](https://github.com/apache/kafka/pull/15060#issuecomment-1882487316), this issue is only applicable to compact policy and the

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2024-01-12 Thread via GitHub
iit2009060 commented on PR #15060: URL: https://github.com/apache/kafka/pull/15060#issuecomment-1890348438 @satishd @kamalcph @divijvaidya I am able to reproduce the issue which @satishd mentioned , I need to introduced a delay to the movement of segments to remote through a hacky

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2024-01-12 Thread via GitHub
iit2009060 commented on PR #15060: URL: https://github.com/apache/kafka/pull/15060#issuecomment-1888750316 > https://issues.apache.org/jira/browse/KAFKA-16088 Thanks @kamalcph for the discussion. @satishd I discussed with @kamalcph on the above cases 1. If auto offset

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2024-01-09 Thread via GitHub
iit2009060 commented on PR #15060: URL: https://github.com/apache/kafka/pull/15060#issuecomment-1882690345 > @iit2009060 In step 7, it is anticipated that setting the fetch offset to 1 should follow the specified process to trigger an `OutOfOffsetRangeError` with empty records as I

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2024-01-09 Thread via GitHub
satishd commented on PR #15060: URL: https://github.com/apache/kafka/pull/15060#issuecomment-1882660480 @iit2009060 In step 7, it is anticipated that setting the fetch offset to 1 should follow the specified process to trigger an `OutOfOffsetRangeError` with empty records as I mentioned

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2024-01-08 Thread via GitHub
iit2009060 commented on PR #15060: URL: https://github.com/apache/kafka/pull/15060#issuecomment-1882517442 > auto-offset-reset > @iit2009060 The behaviour that you mentioned in the [comment](https://github.com/apache/kafka/pull/15060#issuecomment-1879657273) seems to be

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2024-01-08 Thread via GitHub
satishd commented on PR #15060: URL: https://github.com/apache/kafka/pull/15060#issuecomment-1882487316 @iit2009060 The behaviour that you mentioned in the [comment](https://github.com/apache/kafka/pull/15060#issuecomment-1879657273) seems to be working as expected. The client seems to

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2024-01-08 Thread via GitHub
iit2009060 commented on PR #15060: URL: https://github.com/apache/kafka/pull/15060#issuecomment-1880890006 > @iit2009060 [KAFKA-15388](https://issues.apache.org/jira/browse/KAFKA-15388) is about addressing the fetch requests for tiered storage enabled topics that have compacted offsets in

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2024-01-08 Thread via GitHub
satishd commented on PR #15060: URL: https://github.com/apache/kafka/pull/15060#issuecomment-1880861009 @iit2009060 KAFKA-15388 is about addressing the fetch requests for tiered storage enabled topics that have compacted offsets in a partition before their cleanup policy is changed to

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2024-01-08 Thread via GitHub
iit2009060 commented on PR #15060: URL: https://github.com/apache/kafka/pull/15060#issuecomment-1880775216 > > @satishd @divijvaidya @kamalcph I am able to reproduce the above scenario using retention feature instead of log compaction. > > The overall problem is we are sending

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2024-01-07 Thread via GitHub
kamalcph commented on PR #15060: URL: https://github.com/apache/kafka/pull/15060#issuecomment-1879987059 > In general this is happening because The next fetch request offset increments only when there is a record list. But the Remote fetch implementation by default if not able to find the

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2024-01-06 Thread via GitHub
iit2009060 commented on PR #15060: URL: https://github.com/apache/kafka/pull/15060#issuecomment-1879980873 > @satishd @divijvaidya @kamalcph I am able to reproduce the above scenario using retention feature instead of log compaction. > > The overall problem is we are sending

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2024-01-06 Thread via GitHub
iit2009060 commented on PR #15060: URL: https://github.com/apache/kafka/pull/15060#issuecomment-1879657273 @satishd @divijvaidya @kamalcph I am able to reproduce the above scenario using retention feature instead of log compaction. The overall problem is we are sending

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2024-01-04 Thread via GitHub
iit2009060 commented on PR #15060: URL: https://github.com/apache/kafka/pull/15060#issuecomment-1877542832 > @iit2009060 You may have mistyped as controller but it does not have any role in the fetch path here. Remote fetch is executed through a specific purgatory with DelayedRemoteFetch

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2024-01-04 Thread via GitHub
iit2009060 commented on PR #15060: URL: https://github.com/apache/kafka/pull/15060#issuecomment-1877528171 > Thanks @iit2009060 for the PR. > > Let us say there are two segments in remote storage and subsequents segments in local storage. remote-seg-10[10, 20], remote-seg-21[21, 30]

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2024-01-04 Thread via GitHub
satishd commented on PR #15060: URL: https://github.com/apache/kafka/pull/15060#issuecomment-1877333095 @iit2009060 You may have mistyped as controller but it does not have any role in the fetch path here. Remote fetch is executed through a specific purgatory with DelayedRemoteFetch

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2024-01-03 Thread via GitHub
iit2009060 commented on PR #15060: URL: https://github.com/apache/kafka/pull/15060#issuecomment-1876338999 > Thanks @iit2009060 for the PR. > > Let us say there are two segments in remote storage and subsequents segments in local storage. remote-seg-10[10, 20], remote-seg-21[21, 30]

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2024-01-02 Thread via GitHub
iit2009060 commented on code in PR #15060: URL: https://github.com/apache/kafka/pull/15060#discussion_r1440153281 ## core/src/main/java/kafka/log/remote/RemoteLogManager.java: ## @@ -1340,6 +1346,10 @@ public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo)

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2024-01-02 Thread via GitHub
kamalcph commented on code in PR #15060: URL: https://github.com/apache/kafka/pull/15060#discussion_r1440146317 ## core/src/main/java/kafka/log/remote/RemoteLogManager.java: ## @@ -1340,6 +1346,10 @@ public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo)

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2024-01-02 Thread via GitHub
iit2009060 commented on PR #15060: URL: https://github.com/apache/kafka/pull/15060#issuecomment-1874937679 > LGTM, thanks for the patch! > > This patch handles the FETCH request for previously compacted topic segments uploaded to remote storage. We also have to go through the >

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2024-01-02 Thread via GitHub
iit2009060 commented on code in PR #15060: URL: https://github.com/apache/kafka/pull/15060#discussion_r1440133601 ## core/src/main/java/kafka/log/remote/RemoteLogManager.java: ## @@ -1289,15 +1289,21 @@ public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo)

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2024-01-02 Thread via GitHub
iit2009060 commented on code in PR #15060: URL: https://github.com/apache/kafka/pull/15060#discussion_r1440126550 ## core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java: ## @@ -2065,6 +2066,11 @@ public Optional fetchRemoteLogSegmentMetadata(TopicPar

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2024-01-02 Thread via GitHub
iit2009060 commented on code in PR #15060: URL: https://github.com/apache/kafka/pull/15060#discussion_r1440116425 ## core/src/main/java/kafka/log/remote/RemoteLogManager.java: ## @@ -1340,6 +1346,10 @@ public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo)

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2024-01-02 Thread via GitHub
iit2009060 commented on code in PR #15060: URL: https://github.com/apache/kafka/pull/15060#discussion_r1440115636 ## core/src/main/java/kafka/log/remote/RemoteLogManager.java: ## @@ -1340,6 +1346,10 @@ public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo)

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2024-01-02 Thread via GitHub
iit2009060 commented on code in PR #15060: URL: https://github.com/apache/kafka/pull/15060#discussion_r1440115636 ## core/src/main/java/kafka/log/remote/RemoteLogManager.java: ## @@ -1340,6 +1346,10 @@ public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo)

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2024-01-02 Thread via GitHub
kamalcph commented on code in PR #15060: URL: https://github.com/apache/kafka/pull/15060#discussion_r1440097317 ## core/src/main/java/kafka/log/remote/RemoteLogManager.java: ## @@ -1289,15 +1289,21 @@ public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo)

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2024-01-02 Thread via GitHub
kamalcph commented on code in PR #15060: URL: https://github.com/apache/kafka/pull/15060#discussion_r1440089289 ## core/src/main/java/kafka/log/remote/RemoteLogManager.java: ## @@ -1340,6 +1346,10 @@ public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo)

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2024-01-02 Thread via GitHub
iit2009060 commented on PR #15060: URL: https://github.com/apache/kafka/pull/15060#issuecomment-1874100254 @clolov Can we merge the request ? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2024-01-02 Thread via GitHub
clolov commented on code in PR #15060: URL: https://github.com/apache/kafka/pull/15060#discussion_r1439460528 ## core/src/main/java/kafka/log/remote/RemoteLogManager.java: ## @@ -1289,15 +1289,21 @@ public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo)

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2023-12-28 Thread via GitHub
iit2009060 commented on PR #15060: URL: https://github.com/apache/kafka/pull/15060#issuecomment-1870969987 @clolov @divijvaidya Do you have any additional review comments ? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2023-12-24 Thread via GitHub
iit2009060 commented on code in PR #15060: URL: https://github.com/apache/kafka/pull/15060#discussion_r1435854787 ## core/src/main/java/kafka/log/remote/RemoteLogManager.java: ## @@ -1289,15 +1289,21 @@ public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo)

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2023-12-24 Thread via GitHub
clolov commented on code in PR #15060: URL: https://github.com/apache/kafka/pull/15060#discussion_r1435781798 ## core/src/main/java/kafka/log/remote/RemoteLogManager.java: ## @@ -1289,15 +1289,21 @@ public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo)

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2023-12-22 Thread via GitHub
iit2009060 commented on code in PR #15060: URL: https://github.com/apache/kafka/pull/15060#discussion_r1435445309 ## core/src/main/java/kafka/log/remote/RemoteLogManager.java: ## @@ -1289,15 +1289,21 @@ public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo)

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2023-12-22 Thread via GitHub
iit2009060 commented on code in PR #15060: URL: https://github.com/apache/kafka/pull/15060#discussion_r1435445309 ## core/src/main/java/kafka/log/remote/RemoteLogManager.java: ## @@ -1289,15 +1289,21 @@ public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo)

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2023-12-22 Thread via GitHub
iit2009060 commented on code in PR #15060: URL: https://github.com/apache/kafka/pull/15060#discussion_r1435444785 ## core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java: ## @@ -2161,6 +2167,86 @@ RecordBatch findFirstBatch(RemoteLogInputStream remoteLogInputStream,

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2023-12-22 Thread via GitHub
iit2009060 commented on code in PR #15060: URL: https://github.com/apache/kafka/pull/15060#discussion_r1435445309 ## core/src/main/java/kafka/log/remote/RemoteLogManager.java: ## @@ -1289,15 +1289,21 @@ public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo)

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2023-12-22 Thread via GitHub
iit2009060 commented on code in PR #15060: URL: https://github.com/apache/kafka/pull/15060#discussion_r1435446137 ## core/src/main/java/kafka/log/remote/RemoteLogManager.java: ## @@ -1412,8 +1422,8 @@ private void collectAbortedTransactionInLocalSegments(long startOffset,

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2023-12-22 Thread via GitHub
iit2009060 commented on code in PR #15060: URL: https://github.com/apache/kafka/pull/15060#discussion_r1435445309 ## core/src/main/java/kafka/log/remote/RemoteLogManager.java: ## @@ -1289,15 +1289,21 @@ public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo)

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2023-12-22 Thread via GitHub
iit2009060 commented on code in PR #15060: URL: https://github.com/apache/kafka/pull/15060#discussion_r1435444785 ## core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java: ## @@ -2161,6 +2167,86 @@ RecordBatch findFirstBatch(RemoteLogInputStream remoteLogInputStream,

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2023-12-22 Thread via GitHub
clolov commented on code in PR #15060: URL: https://github.com/apache/kafka/pull/15060#discussion_r1435282790 ## core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java: ## @@ -2161,6 +2167,86 @@ RecordBatch findFirstBatch(RemoteLogInputStream remoteLogInputStream, long

Re: [PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2023-12-22 Thread via GitHub
clolov commented on PR #15060: URL: https://github.com/apache/kafka/pull/15060#issuecomment-1867443497 Thank you for the contribution! I will aim to provide a review throughout the day! -- This is an automated message from the Apache Git Service. To respond to the message, please log on

[PR] KAFKA-15388: Handling remote segment read in case of log compaction [kafka]

2023-12-21 Thread via GitHub
iit2009060 opened a new pull request, #15060: URL: https://github.com/apache/kafka/pull/15060 *More detailed description of your change, if necessary. The PR title and PR message become the squashed commit message, so use a separate comment to ping reviewers.* *Summary of