Hi All,
When tackling the issue outlined in
https://issues.apache.org/jira/browse/KAFKA-16088, which pertains to
transitioning from RemoteLogSegment to LocalLogSegment during log
compaction in a fetch request, I'm seeking some suggestions or guidance
from the community to advance

*Current Behaviour *

At a very high level The fetch request behaves incase of topics enabled
with tiered storage.

[image: Screenshot 2024-04-11 at 8.49.31 PM.png]

1. When a consumer client requests an offset that is not available in the
local log for the given partition, the broker throws an OffsetOutOfRange
error.
2. If the required offset falls within the range of the Remote Segment log,
we schedule a DelayedRemoteFetch request along with the successful results
of other partitions from the log, and the responseCallback.

*Current Issue*
In this scenario, the functionality is compromised when RemoteLogSegments
are log compacted. As we initiate offset reading from RemoteLogManager,
there's a possibility that we cannot find the required remote log segment
within the requested offset range because of log compaction. Ideally, the
system should then search for higher log segments for additional retrieval.
However, these higher segments are stored locally instead of remote.
Presently, there's no mechanism in place to transition the fetch request
from remote log segments to local log segments, resulting in empty records
being returned (Check the diagram above). Consequently, the consumer client
remains unable to increment the offset, erroneously perceiving that there
is no more data available.

The scenario has been discussed here
<https://github.com/apache/kafka/pull/15060#pullrequestreview-1802495064>
in detail.

*Possible Approaches and downside*

*Implement Mechanism of RemoteLogSegment to Local LogSegment*

1. Suppose we implement a mechanism to advance the fetch request from
Remote LogSegment to LocalLogSegment. However, during this process, there's
a possibility that the local segments are moved to Remote Storage. This
situation hints at a potential dead spiral loop, where we continuously
switch between local and remote segments and vice versa.

2. Handling the advancement of Fetch Request from remote segment to log
segment code-wise is complex, mainly because the flow is independent, and
there is no existing mechanism to manage this transition seamlessly.

*Use endoffset of the last RemoteLogSegmentMetadata *

Once we've determined that no RemoteLogSegment satisfies the fetch offset,
we can utilize the information regarding the next segment to inspect, which
is based on the end offset of the lastRemoteLogSegment iterated. This
information can be passed to the client along with an error message
resembling "OffsetOutOfRangeBecauseOfLogCompaction". Similar to the
advanced strategy options like "latest" or "earliest", we can advance the
next fetch request for the given partition to the required value passed to
the client.

1. In our current implementation, the subscription position is only
incremented under two conditions: when there are records in the response or
when the resetStrategy is configured.
2. The proposed design requires sending the next fetch offset in the
response to the client if this scenario occurs.


Please let me know if the community has any suggestions or directions to
offer.


Thanks and Regards
Arpit Goyal
8861094754

Reply via email to