> Is making it optional a good solution? Or should we recover the snapshot
if not found before uploading it?

IMO, we should recover the snapshot if it is not found.

> But what if the topic is created before v2.8.0 and old log segments are
deleted, how could we recover all the producer snapshot for old logs?

We don't need the snapshot files of the deleted log segments. Only for the
segments which lack one.
The ProduceStateManager contains the complete state. We have to regenerate
the state from there.

@christolo...@gmail.com <christolo...@gmail.com>

You have filed https://issues.apache.org/jira/browse/KAFKA-15195 ticket to
generate the snapshots.
Could you please update the ticket with the details? Thanks!

On Thu, Apr 4, 2024 at 2:01 PM Luke Chen <show...@gmail.com> wrote:

> Hi Kamal,
>
> Thanks for sharing! I didn't know about the change before v2.8.
> If that's the case, then we have to reconsider the solution of this PR.
> Is making it optional a good solution? Or should we recover the snapshot
> if not found before uploading it?
> But what if the topic is created before v2.8.0 and old log segments are
> deleted, how could we recover all the producer snapshot for old logs?
>
> Thanks.
> Luke
>
>
> On Wed, Apr 3, 2024 at 11:59 PM Arpit Goyal <goyal.arpit...@gmail.com>
> wrote:
>
>> Thanks @Kamal Chandraprakash <kamal.chandraprak...@gmail.com>  Greg
>> Harris
>> I currently do not have detailed understanding on the behaviour when empty
>> producer snapshot  restored. I will try to test out the
>> behaviour.Meanwhile
>> I would request other community members if they can chime in and assist if
>> they are already aware of the behaviour mentioned above.
>> Thanks and Regards
>> Arpit Goyal
>> 8861094754
>>
>>
>> On Tue, Mar 26, 2024 at 4:04 PM Kamal Chandraprakash <
>> kamal.chandraprak...@gmail.com> wrote:
>>
>> > Hi,
>> >
>> > Sorry for the late reply. Greg has raised some good points:
>> >
>> > > Does an empty producer snapshot have the same behavior as a
>> > non-existent snapshot when restored?
>> >
>> > Producer snapshots maintain the status about the ongoing txn to either
>> > COMMIT/ABORT the transaction. In the older version (<2.8), we maintain
>> > the producer snapshots only for the recent segments. If such a topic
>> gets
>> > onboarded to tiered storage and the recently built replica becomes then
>> > leader,
>> > then it might break the producer.
>> >
>> > Assume there are 100 segments for a partition, and the producer
>> snapshots
>> > are available only for the recent 2 segments. Then, tiered storage is
>> > enabled
>> > for that topic, 90/100 segments are uploaded and local-log-segments are
>> > cleared upto segment 50. If a new follower builds the state from
>> remote, it
>> > will
>> > have the empty producer snapshots and start reading the data from the
>> > leader
>> > from segment-51. If a transaction is started at segment-40, then it will
>> > break the
>> > client.
>> >
>> > We also have to check the impact of expiring producer-ids before the
>> > default
>> > expiration time of 7 days: *transactional.id.expiration.ms
>> > <http://transactional.id.expiration.ms>*
>> >
>> > > 2. Why were empty producer snapshots not backfilled for older data
>> > when clusters were upgraded from 2.8?
>> >
>> > https://github.com/apache/kafka/pull/7929 -- It was not required at
>> that
>> > time.
>> > With tiered storage, we need the snapshot file for each segment to
>> reliably
>> > build the follower state from remote storage.
>> >
>> > > 3. Do producer snapshots need to be available contiguously, or can
>> > earlier snapshots be empty while later segments do not exist?
>> >
>> > I assume you refer to "while later segments do exist". Each snapshot
>> file
>> > will contain
>> > the cumulative/complete data of all the previous segments. So, a recent
>> > segment
>> > snapshot is enough to build the producer state. We need to figure out a
>> > solution to
>> > build the complete producer state for replicas that built the state
>> using
>> > the remote.
>> >
>> > Arpit,
>> > We have to deep dive into each of them to come up with the proper
>> solution.
>> >
>> > --
>> > Kamal
>> >
>> >
>> > On Tue, Mar 26, 2024 at 3:55 AM Greg Harris
>> <greg.har...@aiven.io.invalid>
>> > wrote:
>> >
>> > > Hi Arpit,
>> > >
>> > > I think creating empty producer snapshots would be
>> > > backwards-compatible for the tiered storage plugins, but I'm not aware
>> > > of what the other compatibility/design concerns might be. Maybe you or
>> > > another reviewer can answer these questions:
>> > > 1. Does an empty producer snapshot have the same behavior as a
>> > > non-existent snapshot when restored?
>> > > 2. Why were empty producer snapshots not backfilled for older data
>> > > when clusters were upgraded from 2.8?
>> > > 3. Do producer snapshots need to be available contiguously, or can
>> > > earlier snapshots be empty while later segments do not exist?
>> > >
>> > > Thanks,
>> > > Greg
>> > >
>> > > On Sat, Mar 23, 2024 at 3:24 AM Arpit Goyal <goyal.arpit...@gmail.com
>> >
>> > > wrote:
>> > > >
>> > > > Yes Luke,
>> > > > I am also in favour of creating producer snapshot at run time if
>> > > > foundEmpty  as this would only be required for topics migrated from
>> <
>> > 2.8
>> > > > version. This will not break the existing contract with the plugin.
>> > Yes,
>> > > > metrics do not make sense here as of now.
>> > > > Greg, @Kamal Chandraprakash <kamal.chandraprak...@gmail.com>  WDYT
>> ?
>> > > > Arpit Goyal
>> > > > 8861094754
>> > > >
>> > > >
>> > > > On Sat, Mar 23, 2024 at 3:05 PM Luke Chen <show...@gmail.com>
>> wrote:
>> > > >
>> > > > > Hi Arpit,
>> > > > >
>> > > > > I'm in favor of creating an empty producer snapshot since it's
>> only
>> > for
>> > > > > topics <= v2.8.
>> > > > > About the metric, I don't know what we expect users to know.
>> > > > > I think we can implement with the empty producer snapshot method,
>> > > without
>> > > > > the metric.
>> > > > > And add them if users are requested it.
>> > > > > WDYT?
>> > > > >
>> > > > > Thank you.
>> > > > > Luke
>> > > > >
>> > > > > On Sat, Mar 23, 2024 at 1:24 PM Arpit Goyal <
>> > goyal.arpit...@gmail.com>
>> > > > > wrote:
>> > > > >
>> > > > > > Hi Team,
>> > > > > > Any further comments or suggestions on the possible approaches
>> > > discussed
>> > > > > > above.
>> > > > > >
>> > > > > > On Tue, Mar 19, 2024, 09:55 Arpit Goyal <
>> goyal.arpit...@gmail.com>
>> > > > > wrote:
>> > > > > >
>> > > > > > > @Luke Chen @Kamal Chandraprakash <
>> kamal.chandraprak...@gmail.com
>> > >
>> > > > > @Greg
>> > > > > > > Harris Any suggestion on the above two possible approaches.
>> > > > > > > On Sun, Mar 17, 2024, 13:36 Arpit Goyal <
>> > goyal.arpit...@gmail.com>
>> > > > > > wrote:
>> > > > > > >
>> > > > > > >>
>> > > > > > >>>>  In summary , There are two possible solution to handle the
>> > > above
>> > > > > > >> scenario when producer snapshot file found to be null
>> > > > > > >>
>> > > > > > >> 1. *Generate empty producer snapshot file at run time when
>> > copying
>> > > > > > >> LogSegment*
>> > > > > > >>
>> > > > > > >>
>> > > > > > >>    - This will not require any backward compatibility
>> > dependencies
>> > > > > with
>> > > > > > >>    the plugin.
>> > > > > > >>    - It preserves the contract i.e producerSnapshot files
>> should
>> > > be
>> > > > > > >>    mandatory.
>> > > > > > >>    - We could have a metric which helps us to assess how many
>> > > times
>> > > > > > >>    empty snapshot files have been created.
>> > > > > > >>
>> > > > > > >> 2.    *  Make producerSnapshot files optional *
>> > > > > > >>
>> > > > > > >>    - This would break the contract with the plugin and would
>> > > require
>> > > > > > >>    defining a set of approaches to handle it which is
>> mentioned
>> > > > > earlier
>> > > > > > in the
>> > > > > > >>    thread.
>> > > > > > >>    - If we make producer Snapshot optional , We would   not
>> be
>> > > > > handling
>> > > > > > >>    the error which @LukeChen mentioned when producerSnapshot
>> > > > > > >>    accidentally deleted a given segment. But this holds true
>> for
>> > > > > > >>    TransactionalIndex.
>> > > > > > >>    - The other question is do we really need to make the
>> field
>> > > > > optional.
>> > > > > > >>    The only case where this problem can occur is only when
>> the
>> > > topic
>> > > > > > migrated
>> > > > > > >>    from < 2.8 version.
>> > > > > > >>
>> > > > > > >>
>> > > > > >
>> > > > >
>> > >
>> >
>>
>

Reply via email to