Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KAFKA-15388: Handling remote segment read in case of log compaction #15060

Merged
merged 4 commits into from
Jan 14, 2024

Conversation

iit2009060
Copy link
Contributor

@iit2009060 iit2009060 commented Dec 21, 2023

Problem
Fetching from remote log segment implementation does not handled log compaction cases. It always assumes record batch will exist in the required segment for the requested offset. But there is a possibility where the requested offset is the last offset of the segment and has been removed due to log compaction. Then it requires to iterate over the next higher segment for further data as it has been done for local segment fetch request.
Fixes

  1. Iterating over the higher remote segment if there is no relevant batch in the current remotelogsegment.

Testing Strategy:

  • Tested through unit test.
  • Reproduced Issue locally
    Steps followed for testing manually
  1. Created topic
    bin/kafka-topics.sh --bootstrap-server localhost:9092 --create --topic arpit
  2. Add bunch of config to enable quick creation of segments and enable log compaction.
    bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter --entity-type topics --entity-name arpit --add-config segment.bytes=100 bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter --entity-type topics --entity-name arpit --add-config cleanup.policy=compact
  3. Produce item into the topic
Screenshot 2023-12-21 at 11 24 21 PM
  1. Remote log is not enabled yet, and successfully fetch data from the local segments.
Screenshot 2023-12-21 at 11 24 41 PM 5. Enable remote log storage.

bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter --entity-type topics --entity-name arpit --delete-config cleanup.policy=compact

bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter --entity-type topics --entity-name arpit --add-config remote.storage.enable=true

  1. When rerun the consumer script with remote storage enable , consumer does not return any data. The remote log segment 0.log does not have any data because of log compaction but the current logic returns null instead of checking data on the higher segment.Screenshot for the reference.
Screenshot 2023-12-21 at 11 27 14 PM Screenshot 2023-12-21 at 11 29 55 PM 7. After fixing the logic , The remote fetch successfully iterate over the segments in order until it fetches the first batch available. Screenshot 2023-12-21 at 11 31 30 PM

cc @divijvaidya @satishd @showuon

@iit2009060 iit2009060 force-pushed the KAFKA-15388 branch 2 times, most recently from 1b245e1 to fc956c3 Compare December 22, 2023 01:42
@divijvaidya divijvaidya added the tiered-storage Related to the Tiered Storage feature label Dec 22, 2023
@clolov clolov self-assigned this Dec 22, 2023
@clolov
Copy link
Collaborator

clolov commented Dec 22, 2023

Thank you for the contribution! I will aim to provide a review throughout the day!

Copy link
Collaborator

@clolov clolov left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I made a first pass, let me know if the comments don't make sense - thanks once again!


private Optional<RemoteLogSegmentMetadata> findNextSegmentMetadata(RemoteLogSegmentMetadata segmentMetadata,
// visible for testing.
Optional<RemoteLogSegmentMetadata> findNextSegmentMetadata(RemoteLogSegmentMetadata segmentMetadata,
Option<LeaderEpochFileCache> leaderEpochFileCacheOption) throws RemoteStorageException {
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
Option<LeaderEpochFileCache> leaderEpochFileCacheOption) throws RemoteStorageException {
Option<LeaderEpochFileCache> leaderEpochFileCacheOption) throws RemoteStorageException {

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

);

when(rsmManager.fetchLogSegment(any(), anyInt())).thenReturn(fileInputStream);
when(remoteLogMetadataManager.remoteLogSegmentMetadata(any(), anyInt(), anyLong())).thenReturn(Optional.of(segmentMetadata), Optional.of(segmentMetadata));
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Isn't one Optional.of(segmentMetadata) enough?

Copy link
Contributor Author

@iit2009060 iit2009060 Dec 23, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for pointing out. This may be even not required as I already mocked fetchRemoteLogSegmentMetadata function.


int startPos = 0;
RecordBatch firstBatch = null;
while (firstBatch == null && rlsMetadataOptional.isPresent()) {
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am not super hung-up on this, but don't you just need to look forward once i.e. you have a guarantee that if you do not find the offset in this segment then you are bound to find it in the next, no? If this is the case can you just look in the next segment and not use a while loop?

Copy link
Contributor Author

@iit2009060 iit2009060 Dec 23, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@clolov We need to use a while loop as there may be a possibility ,the next segment in the iteration is also log compacted and we may need to further move until we find it. Check the example above I attached in the description PR where 0.log and 6.log both are log compacted fully and the next batch exist only in the 07.log.
The similar logic is used in fetching data from the log segment.

} else segmentOpt = segments.higherSegment(baseOffset)

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Huh, interesting, I shall circle back to this, maybe I am not as familiar with compaction as I thought I was. To be honest, I was expecting that the segments 0 and 6 will be merged into one by compaction - I was under the impression that it doesn't leave empty segments.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@clolov I believe this merging of segments happens in the background , but before the merging start we disable the compaction on the topic to make it ready for tiered storage because of which merging of segments never happens.

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Apologies for the delay, okay, this reasoning makes sense to me, but even if I am wrong the while loop won't really cause a performance impact, so I am happy with it!

@iit2009060
Copy link
Contributor Author

@clolov @divijvaidya Do you have any additional review comments ?

@iit2009060
Copy link
Contributor Author

iit2009060 commented Jan 2, 2024

@clolov Can we merge the request ?

@satishd satishd requested a review from kamalcph January 2, 2024 15:25
Copy link
Collaborator

@kamalcph kamalcph left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

  1. upload, deletion path and
  2. RemoteLogMetadataCache internal state

for both normal and unclean-leader-election scenarios.

core/src/main/java/kafka/log/remote/RemoteLogManager.java Outdated Show resolved Hide resolved
@@ -2065,6 +2066,11 @@ public Optional<RemoteLogSegmentMetadata> fetchRemoteLogSegmentMetadata(TopicPar
return Optional.of(segmentMetadata);
}

public Optional<RemoteLogSegmentMetadata> findNextSegmentMetadata(RemoteLogSegmentMetadata segmentMetadata,
Option<LeaderEpochFileCache> leaderEpochFileCacheOption) {
return Optional.ofNullable(null);
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we use Optional.empty() instead?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done.

LeaderEpochFileCache cache = mock(LeaderEpochFileCache.class);
when(cache.epochForOffset(anyLong())).thenReturn(OptionalInt.of(1));

when(remoteStorageManager.fetchLogSegment(any(RemoteLogSegmentMetadata.class), anyInt()))
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There are two remote-storage-manager: rsmManager (local) and remoteStorageManager (global). Can we discard the latter one?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

@iit2009060
Copy link
Contributor Author

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

  1. upload, deletion path and
  2. RemoteLogMetadataCache internal state

for both normal and unclean-leader-election scenarios.
https://issues.apache.org/jira/browse/KAFKA-15388
As mentioned in the description for upload and delete it will not be impacted.

@divijvaidya divijvaidya added the backport-candidate This pull request is a candidate to be backported to previous versions label Jan 3, 2024
Copy link
Contributor

@divijvaidya divijvaidya left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you for fixing this. Looks good to me. I will wait for CI to be stable before merging this.

Copy link
Member

@satishd satishd left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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] : offsets 25 to 30 are compacted.
local-seg-31[31, 40]
local-seg-41[41, 90]
local-seg-91[41, 99] (active segment)

Updated the above that the targeted offset can be in any of the following local log segments but not only limited to active segment.

When a fetch request comes for offsets within [25, 30] then it should move to the local segment as those offsets might have been compacted earlier. Did you also cover this scenario in this PR?

@iit2009060
Copy link
Contributor Author

iit2009060 commented Jan 4, 2024

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] : offsets 25 to 30 are compacted. local-seg-31[31, 40]

When a fetch request comes for offsets with in [25, 30] then it should move to the local segment as those offsets might have been compacted earlier. Did you also cover this scenario in this PR?

@satishd I have not tested this case explicitly.
In this case RemoteLogManager would be returning firstBatch as null and the controller(The class which is invoking RemoteLogManager read) should take care of reading the next segment locally. Let me try to reproduce this issue locally and update the behaviour.

@satishd
Copy link
Member

satishd commented Jan 4, 2024

@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 and read operation/callback is executed in a specific thread pool. It returns empty records when there is no data for a specific offset in the remote storage.

@iit2009060
Copy link
Contributor Author

iit2009060 commented Jan 4, 2024

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] : offsets 25 to 30 are compacted. local-seg-31[31, 40]

When a fetch request comes for offsets with in [25, 30] then it should move to the local segment as those offsets might have been compacted earlier. Did you also cover this scenario in this PR?

@satishd I am trying to reproduce the case when last offsets of the segment earlier than the active segment is compacted away.
I tried locally but not able to reproduce the above scenario ?
There always exist a last offset in the segment earlier than the active segment.
As per the article
https://towardsdatascience.com/log-compacted-topics-in-apache-kafka-b1aa1e4665a7
All records in the active segment are never compacted or use for compaction in the earlier segment.
Attaching screenshot for reference.

  1. Created topic test8
  2. With segment size 250 bytes so that max 3 offsets exist within a segment.
  3. First segment contains (0,1,2 offset) which also happen to be segment just before the active segment.
  4. Active segment contains (3,4 offset)
  5. The offset which got compacted is 1, but ideally offset 2 should also needs to get compacted to reproduce the above scenario.
Screenshot 2024-01-05 at 12 40 13 AM

@satishd Do you have a step in mind to regenerate the above scenario ?

@iit2009060
Copy link
Contributor Author

iit2009060 commented Jan 4, 2024

@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 and read operation/callback is executed in a specific thread pool. It returns empty records when there is no data for a specific offset in the remote storage.

Yes @satishd correct . I am still going through the code but need few inputs

  1. If we send records empty , How it determine the next offset to be fetched. Because in LogOffSetMetadata we always return the fetchOffset request. Or it keep requesting the fetchOffSet request in case of Empty records ?

@iit2009060
Copy link
Contributor Author

iit2009060 commented Jan 6, 2024

@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 MemoryRecords.Empty when unable to find offset even though active segments can still have the data.

Consider the below scenario

  1. create topic test8 with partition 0 and with remote storage enabled.
  2. Current status of topic (Offset 0,2,3,4)
Screenshot 2024-01-05 at 12 40 13 AM 3. When we make a fetch request with offset 1 Screenshot 2024-01-06 at 5 19 01 PM 4. To delete remote segments I have set up the configuration retention.ms=1000 Screenshot 2024-01-06 at 5 21 24 PM 5. Now once the data is deleted, reset the local and remote retention to 1 hour. Produce some data Screenshot 2024-01-06 at 5 22 59 PM 7. When I am trying to fetch offset 1 from the topic test 8 partition 0 , it never able to respond. Ideally it should pick the data from the active segment whose offset starts at 6. Screenshot 2024-01-06 at 5 24 44 PM 8. It happened because requestedoffset 1 is not greater than log.logStartOffset which is 6 Screenshot 2024-01-06 at 5 27 42 PM
  1. We create Empty Record Response.
Screenshot 2024-01-06 at 5 26 04 PM

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 records sents Empty Records.

We should create separate JIRA to track this .The issue is not specific to log compaction but it will happen whenever we sent Records Empty from remote storage.

@iit2009060
Copy link
Contributor Author

@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 MemoryRecords.Empty when unable to find offset even though active segments can still have the data.

Consider the below scenario

  1. create topic test8 with partition 0 and with remote storage enabled.
  2. Current status of topic (Offset 0,2,3,4)

Screenshot 2024-01-05 at 12 40 13 AM 3. When we make a fetch request with offset 1 Screenshot 2024-01-06 at 5 19 01 PM 4. To delete remote segments I have set up the configuration retention.ms=1000 Screenshot 2024-01-06 at 5 21 24 PM 5. Now once the data is deleted, reset the local and remote retention to 1 hour. Produce some data Screenshot 2024-01-06 at 5 22 59 PM 7. When I am trying to fetch offset 1 from the topic test 8 partition 0 , it never able to respond. Ideally it should pick the data from the active segment whose offset starts at 6. Screenshot 2024-01-06 at 5 24 44 PM 8. It happened because requestedoffset 1 is not greater than log.logStartOffset which is 6 Screenshot 2024-01-06 at 5 27 42 PM
9. We create Empty Record Response.

Screenshot 2024-01-06 at 5 26 04 PM 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 records sents Empty Records.

We should create separate JIRA to track this .The issue is not specific to log compaction but it will happen whenever we sent Records Empty from remote storage.

@satishd @divijvaidya I have created separate ticket to track the scenarios when RemoteFetch return Empty Records.
https://issues.apache.org/jira/browse/KAFKA-16088

@kamalcph
Copy link
Collaborator

kamalcph commented Jan 7, 2024

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 records sents Empty Records.

Empty records response is sent back to the consumer so that it can retry the request if needed. In a non-compacted topic, the records are expected to be present in the contiguous manner.

Shall we list the scenarios where the record offsets won't be in contiguous manner for regular (non-compacted) topic (or) write a unit test to reproduce this problem?

@iit2009060
Copy link
Contributor Author

iit2009060 commented Jan 8, 2024

@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 MemoryRecords.Empty when unable to find offset even though active segments can still have the data.
Consider the below scenario

  1. create topic test8 with partition 0 and with remote storage enabled.
  2. Current status of topic (Offset 0,2,3,4)

Screenshot 2024-01-05 at 12 40 13 AM 3. When we make a fetch request with offset 1 Screenshot 2024-01-06 at 5 19 01 PM 4. To delete remote segments I have set up the configuration retention.ms=1000 Screenshot 2024-01-06 at 5 21 24 PM 5. Now once the data is deleted, reset the local and remote retention to 1 hour. Produce some data Screenshot 2024-01-06 at 5 22 59 PM 7. When I am trying to fetch offset 1 from the topic test 8 partition 0 , it never able to respond. Ideally it should pick the data from the active segment whose offset starts at 6. Screenshot 2024-01-06 at 5 24 44 PM 8. It happened because requestedoffset 1 is not greater than log.logStartOffset which is 6 Screenshot 2024-01-06 at 5 27 42 PM
9. We create Empty Record Response.
Screenshot 2024-01-06 at 5 26 04 PM
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 records sents Empty Records.
We should create separate JIRA to track this .The issue is not specific to log compaction but it will happen whenever we sent Records Empty from remote storage.

@satishd @divijvaidya I have created separate ticket to track the scenarios when RemoteFetch return Empty Records. https://issues.apache.org/jira/browse/KAFKA-16088

@divijvaidya @satishd @kamalcph
I would like to propose merging the aforementioned changes, as they not only effectively address the specific scenario highlighted but also have the potential to tackle a broader range of issues. The situation @satishd brought up is indicative of a more general problem that could potentially manifest across various contexts, including scenarios involving retention, not just limited to log compaction. I created a separate ticket to track this. Let me know your thoughts.

@satishd
Copy link
Member

satishd commented Jan 8, 2024

@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 delete. This PR only covers if the next offset is available only within the remote log segments. But there is a case when the offset is available in the subsequent local segments(not only in active local segments but in any of the local only segments) that have not yet been copied to remote storage. The solution does not address KAFKA-15388 completely.

I am fine to merge this and have a followup PR for the mentioned scenario to resolve KAFKA-15388 addressing the compacted topics before tiered storage is enabled.

@iit2009060
Copy link
Contributor Author

iit2009060 commented Jan 8, 2024

@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 delete. This PR only covers if the next offset is available only within the remote log segments. But there is a case when the offset is available in the subsequent local segments(not only in active local segments but in any of the local only segments) that have not yet been copied to remote storage. The solution does not address KAFKA-15388 completely.

I am fine to merge this and have a followup PR for the mentioned scenario to resolve KAFKA-15388 addressing the compacted topics before tiered storage is enabled.

@satishd Yes, as you mentioned the current PR is not covering reading from local log segments ,i.e. data not available in RemoteLogSegments and the reason can be

  1. Log compaction
  2. Retention
    As the above specific scenario can be reproducible on multiple feature and not merely a specific issue because of log compaction.
    The intention behind creating a separate ticket is to provide a dedicated space for addressing the broader issue, encompassing scenarios beyond the scope of the current PR.
    However I am ok if we want to have a follow up PR to merge this, but I firmly believe that these modifications have the potential to serve as a comprehensive solution, resolving not only the immediate scenario but also catering to a wider range of cases.

@satishd
Copy link
Member

satishd commented Jan 9, 2024

@iit2009060
The behaviour that you mentioned in the comment seems to be working as expected. The client seems to be receiving OffsetOutOfRangeException. When a consumer client sends a request for fetch offset that is out of range then it receives offset out of range error and it sends the next request based on auto-offset-reset strategy that is set on the consumer client. It seems the default value for kafka-console-consumer is latest if not set through an auto.offset.reset property or does not use --from-beginning argument.
In your case, it may be trying to fetch the latest offset in the next request and there are no messages published to the topic beyond that offset as there may not be any more messages published after that offset. You can try running the same scenario of fetching out of range offset 1 with --from-beginning and observe the behavior.

We can syncup offline to understand your scenario better.

@iit2009060
Copy link
Contributor Author

iit2009060 commented Jan 9, 2024

auto-offset-reset

@iit2009060 The behaviour that you mentioned in the comment seems to be working as expected. The client seems to be receiving OffsetOutOfRangeException. When a consumer client sends a request for fetch offset that is out of range then it receives offset out of range error and it sends the next request based on auto-offset-reset strategy that is set on the consumer client. It seems the default value for kafka-console-consumer is latest if not set through an auto.offset.reset property or does not use --from-beginning argument. In your case, it may be trying to fetch the latest offset in the next request and there are no messages published to the topic beyond that offset as there may not be any more messages published after that offset. You can try running the same scenario of fetching out of range offset 1 with --from-beginning and observe the behavior.

We can syncup offline to understand your scenario better.

@satishd Check the 5th step mentioned in the comment where I produced some data. In the 7th step I made an offset request with 1 and 6 both ,6th worked fine but the request with offset 1 goes through the step

  1. handle OutOfOffSetRangeError
  2. It goes through the if condition mentioned in the step 8
  3. Then it goes through the else part
    } else { createLogReadResult(exception) }
  4. As mentioned in step 9 screenshot It creates empty record response with exception wrapped in the LogReadResult.

Default configuration is latest in the console consumer. Even for the latest configuration it should work?.
I will try with from-beginning option and update the behaviour.

@satishd
Copy link
Member

satishd commented Jan 9, 2024

@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 earlier.

I will try with from-beginning option and update the behaviour.

Sure.

Even for the latest configuration it should work?

latest option also should work. To validate this, run the step with the latest setting and concurrently run kafka-console-producer, generating additional messages beyond the latest offset where the OutOfOffsetRangeError was encountered. Subsequently, the consumer should be able to consume the newly produced sequence of messages.

@iit2009060
Copy link
Contributor Author

@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 earlier.

I will try with from-beginning option and update the behaviour.

Sure.

Even for the latest configuration it should work?

latest option also should work. To validate this, run the step with the latest setting and concurrently run kafka-console-producer, generating additional messages beyond the latest offset where the OutOfOffsetRangeError was encountered. Subsequently, the consumer should be able to consume the newly produced sequence of messages.

@satishd I will try to write a integration or unit test to depicts the scenario I mentioned. However as far I noticed ,the additional messages I reproduced in step 5 were not able to retrieve with fetch offset 1 request and were able to proceed with offset 6.

@iit2009060
Copy link
Contributor Author

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 reset is latest, we can still lose the messages on the case mentioned above. That is the expected behaviour with the "latest" configuration.
  2. But if auto offset is earliest , The issue never occurs.

@satishd We can merge the current PR if it is feasible. I will continue to work and see if i can reproduce the case mentioned by you around log compaction. And will open a separate PR once the issue is reproducible.

@iit2009060
Copy link
Contributor Author

iit2009060 commented Jan 13, 2024

@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 code.

  1. Created topic test 10
  2. Set segment bytes 100 so that each segment contain only one offset.
  3. Set clean up policy compact
  4. Produce some messages (1:1,1:2,1:3,1:4,1:5,1:6,1:7)
  5. Log compaction is in progress
  6. Delete log compaction policy from the topic
  7. Enable remote storage enable = true
  8. I introduced a code which copied remote segments whose base offset <= 1 in the RemoteLogManager so that we can have a situation where data in remote segments is not available , but available in local/active segments.
Screenshot 2024-01-13 at 12 01 47 PM
  1. In the local log segments directory 0 and 1 segment has been removed and moved to a remote log storage. The number of bytes is zero as data is removed because of log compaction. Observe local segments contains data only for offset >=2.
    Local log segments view
    Screenshot 2024-01-13 at 12 05 08 PM
    Remote log segments view
Screenshot 2024-01-13 at 12 05 00 PM Remote log contain only two segments 0 and 1 , both are empty because of log compaction.
  1. Execute consumer service
    bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 --topic test10 --offset 0 --partition 0 --property print.offset=true --property print.key=true --property print.value=true --consumer-property auto.offset.reset=earliest
    This command never proceeds as RemoteLogManager return empty records and we do not have a mechanism yet to handle cases when RemoteLogManager return empty records.
Screenshot 2024-01-13 at 12 20 15 PM Screenshot 2024-01-13 at 11 57 53 AM Screenshot 2024-01-13 at 11 58 26 AM
  1. Though reading from offset 2 works as it exists in local log segments.
    Uploading Screenshot 2024-01-13 at 12.42.51 PM.png…

@satishd @divijvaidya @kamalcph Let me know if you have any implementation suggestion to fix this use case .

@satishd
Copy link
Member

satishd commented Jan 14, 2024

@iit2009060 As we discussed earlier in the comment, this issue is only applicable to compact policy and the behaviour on retention policy is as expected.

As I mentioned in my earlier comment, I am fine to merge this PR which partially addresses the problem of compact policy by iterating through the remote log segments and we can have a followup PR on the remaining cases.

We can discuss the possible cases and solutions in KAFKA-16088

Copy link
Member

@satishd satishd left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM. It covers KAFKA-15388 partially by covering the next available segments if they exist in remote storage.

We have a follwup to cover local segments with KAFKA-16088

@satishd
Copy link
Member

satishd commented Jan 14, 2024

Merging it to trunk as the test failures are unrelated to this change.

@satishd satishd merged commit ef92dee into apache:trunk Jan 14, 2024
1 check failed
showuon pushed a commit to showuon/kafka that referenced this pull request Jan 22, 2024
…pache#15060)

Fetching from remote log segment implementation does not handle the topics that had retention policy as compact earlier and changed to delete. It always assumes record batch will exist in the required segment for the requested offset. But there is a possibility where the requested offset is the last offset of the segment and has been removed due to log compaction. Then it requires iterating over the next higher segment for further data as it has been done for local segment fetch request.

This change partially addresses the above problem by iterating through the remote log segments to find the respective segment for the target offset.

Reviewers: Satish Duggana <satishd@apache.org>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Divij Vaidya <diviv@amazon.com>, Christo Lolov <lolovc@amazon.com>
yyu1993 pushed a commit to yyu1993/kafka that referenced this pull request Feb 15, 2024
…pache#15060)

Fetching from remote log segment implementation does not handle the topics that had retention policy as compact earlier and changed to delete. It always assumes record batch will exist in the required segment for the requested offset. But there is a possibility where the requested offset is the last offset of the segment and has been removed due to log compaction. Then it requires iterating over the next higher segment for further data as it has been done for local segment fetch request.

This change partially addresses the above problem by iterating through the remote log segments to find the respective segment for the target offset.

Reviewers: Satish Duggana <satishd@apache.org>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Divij Vaidya <diviv@amazon.com>, Christo Lolov <lolovc@amazon.com>
clolov pushed a commit to clolov/kafka that referenced this pull request Apr 5, 2024
…pache#15060)

Fetching from remote log segment implementation does not handle the topics that had retention policy as compact earlier and changed to delete. It always assumes record batch will exist in the required segment for the requested offset. But there is a possibility where the requested offset is the last offset of the segment and has been removed due to log compaction. Then it requires iterating over the next higher segment for further data as it has been done for local segment fetch request.

This change partially addresses the above problem by iterating through the remote log segments to find the respective segment for the target offset.

Reviewers: Satish Duggana <satishd@apache.org>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Divij Vaidya <diviv@amazon.com>, Christo Lolov <lolovc@amazon.com>
Phuc-Hong-Tran pushed a commit to Phuc-Hong-Tran/kafka that referenced this pull request Jun 6, 2024
…pache#15060)

Fetching from remote log segment implementation does not handle the topics that had retention policy as compact earlier and changed to delete. It always assumes record batch will exist in the required segment for the requested offset. But there is a possibility where the requested offset is the last offset of the segment and has been removed due to log compaction. Then it requires iterating over the next higher segment for further data as it has been done for local segment fetch request.

This change partially addresses the above problem by iterating through the remote log segments to find the respective segment for the target offset.

Reviewers: Satish Duggana <satishd@apache.org>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Divij Vaidya <diviv@amazon.com>, Christo Lolov <lolovc@amazon.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
backport-candidate This pull request is a candidate to be backported to previous versions tiered-storage Related to the Tiered Storage feature
Projects
None yet
5 participants