Hi All, I was looking at one of the plugin implementation <https://github.com/Aiven-Open/tiered-storage-for-apache-kafka/blob/c8484843dbd146d3c6ff1fe955e790335bb10db6/core/src/main/java/io/aiven/kafka/tieredstorage/RemoteStorageManager.java#L292>s and Indeed it is expected to encounter a compilation error [image: Screenshot 2024-03-15 at 2.55.49 PM.png]
To overcome the backward compatibility issue , We have two options ? 1. Introduce a new field instead of modifying the existing one and then mark the existing field as deprecated.The plugin will gradually transition to using the new field as issues arise due to non-existence of the producer snapshot file. 2. As the interface is in an evolving phase , we can go ahead with the change and guide the upgrade process through the release documentation. @Luke Chen IMO The behaviour described in the question can also occur with a transaction index. In both cases there are scenarios where both files may or may not exist depending on the use case .This highlights a distinct process to validate which files should exist for a given segment based on the topic configuration. After the KIP , even if the producer snapshot file is mistakenly deleted, we would refrain from copying it and it holds true for the Transaction Index. @Kamal Chandraprakash <kamal.chandraprak...@gmail.com> Creating empty files may suffice for this use case, but it would lead to unnecessary file transfer on the RemoteStorage even when not required and impact performance. Thanks and Regards Arpit Goyal 8861094754 On Fri, Mar 15, 2024 at 2:23 PM Kamal Chandraprakash < kamal.chandraprak...@gmail.com> wrote: > Hi Arpit, > > Thanks for the KIP! > > There is an open ticket [1] to generate the empty producer snapshot for > segments which lacks one. > Tiered storage is supported from IBP 2.8-IV1 which mandates that all the > segments should have > the producer snapshots. If we make the producer snapshot optional, then we > have to handle all > the edge cases as Luke mentioned. > > Shall we propose to generate the empty producer snapshots instead? > > [1]: https://issues.apache.org/jira/browse/KAFKA-15195 > > On Fri, Mar 15, 2024 at 2:16 PM Luke Chen <show...@gmail.com> wrote: > > > Hi Arpit, > > > > Thanks for the KIP! > > > > I agree with Greg that we should make it clear about backward > > Compatibility. > > Since you don't have permission to edit the KIP, you could think about it > > and write in the email thread directly. > > > > Questions: > > 1. Could you explain what will happen if one partition created after > v2.8, > > which should upload the producer snapshot file, but somehow it didn't > > upload this file to remote storage (ex: the file is deleted accidentally > by > > user). Before this KIP, we'll throw exception when uploading the segment > > file. But what will happen after this KIP? > > > > > > Thanks. > > Luke > > > > On Fri, Mar 15, 2024 at 3:56 AM Greg Harris <greg.har...@aiven.io.invalid > > > > wrote: > > > > > Hi Arpit, > > > > > > Thanks for the clarification. Replying here without updating the KIP > > > is fine for now. > > > > > > I disagree with your evaluation of the backwards compatibility. If you > > > change the return type of a method, that breaks both source and binary > > > compatibility. > > > After upgrading, plugin implementations using this method would face > > > compilation errors. Implementations that were compiled against the old > > > interface will not be able to be loaded when the new interface is > > > present. > > > I see that the interface is marked Evolving which permits breaking > > > compatibility at minor releases, but that doesn't change the > > > compatibility of the change itself. > > > > > > Thanks, > > > Greg > > > > > > On Thu, Mar 14, 2024 at 8:55 AM Arpit Goyal <goyal.arpit...@gmail.com> > > > wrote: > > > > > > > > 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 > > <greg.har...@aiven.io.invalid > > > > > > > > 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 < > > goyal.arpit...@gmail.com> > > > > > 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 < > goyal.arpit...@gmail.com> > > > > > 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 < > > > goyal.arpit...@gmail.com> > > > > > > > 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 > > > > > > >> > > > > > > > > > > > > > > > > > >