As always, I hope that the direction of the discussion would be focusing on
the topic. Let’s avoid ourselves to be side-tracked. Please consider the
mail thread as full context and feel free to ask me if there is a lack of
information for you to provide a voice.

Thanks for the voice in previous mail btw!

2022년 3월 18일 (금) 오후 9:41, Gabor Somogyi <gabor.g.somo...@gmail.com>님이 작성:

> I've just read the related PR and seems like the situation is not so black
> and white as I've presumed purely from tech point of view...
>
> On Fri, 18 Mar 2022, 12:44 Gabor Somogyi, <gabor.g.somo...@gmail.com>
> wrote:
>
>> Hi Jungtaek,
>>
>> I've taken a deeper look at the issue and here are my findings.
>> As far as I'm concerned there are basically 2 ways with some minor
>> decorations:
>> * We care
>> * We don't care
>>
>> I'm pretty sure users are clever enough but setting the expectation that
>> all users are tracking Kafka KIPs one-by-one would be a hard requirement.
>> This implies that I would vote on the "We care" point, the only question
>> is how.
>>
>> Unless we have a specific reason for point 3 I wouldn't override default
>> configs. The reason behind is simple.
>> Kafka has it's strategic direction and going against it w/o good reason
>> is rarely a good idea (maybe we have such but that would be said out).
>>
>> I think when Kafka version upgrade happens we engineers are having a look
>> whether all the changes in the new version
>> are backward compatible or not so point 2 makes sense to me. Honestly I'm
>> drinking coffee with some of the Kafka devs
>> so I've never ever read through all the KIPs between releases because
>> they've told what's important to check :)
>>
>> Seems like my Kafka Spark compatibility gist is out-of-date so maybe I
>> need to invest some time to resurrect it:
>> https://gist.github.com/gaborgsomogyi/3476c32d69ff2087ed5d7d031653c7a9
>>
>> Hope my thoughts are helpful!
>>
>> BR,
>> G
>>
>>
>> On Fri, Mar 18, 2022 at 11:15 AM Jungtaek Lim <
>> kabhwan.opensou...@gmail.com> wrote:
>>
>>> CORRECTION: in option 2, we enumerate KIPs which may bring
>>> incompatibility with older brokers (not all KIPs).
>>>
>>> On Fri, Mar 18, 2022 at 7:12 PM Jungtaek Lim <
>>> kabhwan.opensou...@gmail.com> wrote:
>>>
>>>> Hi dev,
>>>>
>>>> I would like to initiate the discussion about how to deal with the
>>>> migration guide on upgrading Kafka to 3.1 (from 2.8.1) in upcoming Spark
>>>> 3.3.
>>>>
>>>> We didn't care much about the upgrade of Kafka dependency since our
>>>> belief on Kafka client has been that the new Kafka client version should
>>>> have no compatibility issues with older brokers. Based on semantic
>>>> versioning, upgrading major versions rings an alarm for me.
>>>>
>>>> I haven't gone through changes that happened between versions, but
>>>> found one KIP (KIP-679
>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-679%3A+Producer+will+enable+the+strongest+delivery+guarantee+by+default>)
>>>> which may not work with older brokers with specific setup. (It's described
>>>> in the "Compatibility, Deprecation, and Migration Plan" section of the 
>>>> KIP).
>>>>
>>>> This may not be problematic for the users who upgrade both client and
>>>> broker altogether, but end users of Spark may be unlikely the case.
>>>> Computation engines are relatively easier to upgrade. Storage systems
>>>> aren't. End users would think the components are independent.
>>>>
>>>> I looked through the notable changes in the Kafka doc, and it does
>>>> mention this KIP, but it just says the default config has changed and
>>>> doesn't mention about the impacts. There is a link to
>>>> KIP, that said, everyone needs to read through the KIP wiki page for
>>>> details.
>>>>
>>>> Based on the context, what would be the best way to notice end users
>>>> for the major version upgrade of Kafka? I can imagine several options
>>>> including...
>>>>
>>>> 1. Explicitly mention that Spark 3.3 upgrades Kafka to 3.1 with linking
>>>> the noticeable changes in the Kafka doc in the migration guide.
>>>> 2. Do 1 & spend more effort to read through all KIPs and check
>>>> "Compatibility, Deprecation, and Migration Plan" section, and enumerate all
>>>> KIPs (or even summarize) in the migration guide.
>>>> 3. Do 2 & actively override the default configs to be compatible with
>>>> older versions if the change of the default configs in Kafka 3.0 is
>>>> backward incompatible. End users should set these configs explicitly to
>>>> override them back.
>>>> 4. Do not care. End users can indicate the upgrade in the release note,
>>>> and we expect end users to actively check the notable changes (& KIPs) from
>>>> Kafka doc.
>>>> 5. Options not described above...
>>>>
>>>> Please take a look and provide your voice on this.
>>>>
>>>> Thanks,
>>>> Jungtaek Lim (HeartSaVioR)
>>>>
>>>> ps. Probably this would be applied to all non-bugfix versions of
>>>> dependency upgrades. We may still want to be pragmatic, e.g. pass-through
>>>> for minor versions, though.
>>>>
>>>

Reply via email to