Maybe I misunderstood this explanation.

Agreed. Spark relies on Kafka, Google Pub/Sub or any other messaging
systems to process the related streaming data via topic or topics and
present them to Spark. At this stage, Spark does not care to know how this
streaming data is produced. Spark relies on the appropriate API to read
data from Kafka or from Google Pub/Sub. For example if you are processing
temperature, you construct a streaming dataframe that subscribes to a topic
say temperature. As long as you have the correct jar files to
interface with Kafka and that takes care of compatibility, this should
work. In reality Kafka will be running on its own container(s) plus the
zookeeper containers if any. So as far as i can ascertain, the
discussion is about deploying the compatible APIs

HTH


   view my Linkedin profile
<https://www.linkedin.com/in/mich-talebzadeh-ph-d-5205b2/>


 https://en.everybodywiki.com/Mich_Talebzadeh



*Disclaimer:* Use it at your own risk. Any and all responsibility for any
loss, damage or destruction of data or any other property which may arise
from relying on this email's technical content is explicitly disclaimed.
The author will in no case be liable for any monetary damages arising from
such loss, damage or destruction.




On Wed, 23 Mar 2022 at 20:12, Jungtaek Lim <kabhwan.opensou...@gmail.com>
wrote:

> If it requires a Kafka broker update, we should not simply bump the
> version of Kafka client. Probably we should at least provide separate
> artifacts.
>
> We should not enforce the upgrade of other component just because we want
> to upgrade the dependency. At least it should not happen in minor versions
> of Spark. Hopefully that’s not a case.
>
> There’s a belief that Kafka client-broker compatibility is both backwards
> and forwards. That is true in many cases (that’s what Kafka excels to), but
> there seems to be the case it is not satisfied with specific configuration
> and specific setup of Kafka broker. E.g KIP-679.
>
> The less compatible config is going to turn on by default in 3.0 and will
> not work correctly with the specific setup of Kafka broker. So that is us
> who breaks things for specific setup, and my point is how much
> responsibility we should have to guide the end users to avoid the
> frustration.
>
> 2022년 3월 23일 (수) 오후 9:41, Sean Owen <sro...@gmail.com>님이 작성:
>
>> Well, yes, but if it requires a Kafka server-side update, it does, and
>> that is out of scope for us to document.
>> It is important that we document if and how (if we know) the client
>> update will impact existing Kafka installations (does it require a
>> server-side update or not?), and document the change itself for sure along
>> with any Spark-side migration notes.
>>
>> On Fri, Mar 18, 2022 at 8:47 PM Jungtaek Lim <
>> kabhwan.opensou...@gmail.com> wrote:
>>
>>> The thing is, it is “us” who upgrades Kafka client and makes possible
>>> divergence between client and broker in end users’ production env.
>>>
>>> Someone can claim that end users can downgrade the kafka-client artifact
>>> when building their app so that the version can be matched, but we don’t
>>> test anything against downgrading kafka-client version for kafka connector.
>>> That sounds to me we defer our work to end users.
>>>
>>> It sounds to me “someone” should refer to us, and then it is no longer a
>>> matter of “help”. It is a matter of “responsibility”, as you said.
>>>
>>> 2022년 3월 18일 (금) 오후 10:15, Sean Owen <sro...@gmail.com>님이 작성:
>>>
>>>> I think we can assume that someone upgrading Kafka will be responsible
>>>> for thinking through the breaking changes. We can help by listing anything
>>>> we know could affect Spark-Kafka usage and calling those out in a release
>>>> note, for sure. I don't think we need to get into items that would affect
>>>> Kafka usage itself; focus on the connector-related issues.
>>>>
>>>> On Fri, Mar 18, 2022 at 5: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