Hi Greg,
I do not have access to update the KIP , Divij is helping me to do it.
Meanwhile let me update your queries here.
Backward compatibility:
These changes will not impact the existing functionality as the existing
behaviour always expects producer snapshot files to be present for a given
segment. Making Producer Snapshot file optional helps to cover both the
scenario i.e. both existing and non existing of the producer snapshot file.
The getter of producer snapshot file would also be changed as described
below:
Current
/**
* @return Producer snapshot file until this segment.
*/
public Path producerSnapshotIndex() {
return producerSnapshotIndex;
}
Proposed
/**
* @return Producer snapshot file until this segment.
*/
public Optional<Path> producerSnapshotIndex() {
return producerSnapshotIndex;
}
Thanks and Regards
Arpit Goyal
8861094754
On Wed, Mar 13, 2024 at 9:25 PM Greg Harris <[email protected]>
wrote:
> Hi Arpit,
>
> Thanks for the KIP!
>
> I am not familiar with the necessity of producer snapshots, but your
> explanation sounds like this should be made optional.
>
> Can you expand the KIP to include the changes that need to be made to
> the constructor and getter, and explain more about backwards
> compatibility? From the description I can't tell if this change is
> backwards-compatible or not.
>
> Thanks,
> Greg
>
> On Wed, Mar 13, 2024 at 6:48 AM Arpit Goyal <[email protected]>
> wrote:
> >
> > Hi all,
> >
> > I just wanted to bump up this thread.
> >
> > The KIP introduces a really small change and it would not take much of
> the
> > time reviewing it. This change would enable kafka users to use tiered
> > storage features seamlessly for the topics migrated from < 2.8 version
> > which currently failed with NullPointerException.
> >
> > I am waiting for this KIP to get approved and then start working on it.
> >
> > On Mon, Mar 11, 2024, 14:26 Arpit Goyal <[email protected]>
> wrote:
> >
> > > Hi All,
> > > Just a Reminder, KIP-1026 is open for discussion.
> > > Thanks and Regards
> > > Arpit Goyal
> > > 8861094754
> > >
> > >
> > > On Sat, Mar 9, 2024 at 9:27 AM Arpit Goyal <[email protected]>
> > > wrote:
> > >
> > >> Hi All,
> > >>
> > >> I have created KIP-1026 for handling producerSnapshot empty scenarios
> > >> when the topic is upgraded from the kafka < 2.8 version.
> > >>
> > >>
> > >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1026%3A+Handling+producer+snapshot+when+upgrading+from+%3C+v2.8.0+for+Tiered+Storage
> > >>
> > >> Feedback and suggestions are welcome.
> > >>
> > >> Thanks and Regards
> > >> Arpit Goyal
> > >> 8861094754
> > >>
> > >
>