[ 
https://issues.apache.org/jira/browse/SPARK-12177?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15176225#comment-15176225
 ] 

Mark Grover commented on SPARK-12177:
-------------------------------------

Let me clarify what I was saying:
There are 2 axes here - one is the new/old consumer API and other is the 
support for Kafka v0.8 and v0.9. Both Kafka v0.8 and v0.9 provide the old API, 
only v0.9 provides the new API.

bq. The fact that the 0.9 consumer is still considered beta by the Kafka 
project and that things are going to change in 0.10 is an argument for keeping 
the existing implementation as it is, not an argument for throwing it away 
prematurely. 
I totally agree with you, Cody, that the old API implementation is bug free and 
I am definitely not proposing to throw away that implementation. My proposal is 
that both the old implementation and the new will rely on depend against the 
same version of Kafka - that being 0.9.

Based on what I now understand (and please correct me if I am wrong), I think 
what you are proposing is:
Proposal 1:
2 subprojects - one with old implementation and one with new. The 'old' 
subproject will be built against Kafka 0.8, and will have it's own assembly and 
the new subproject will use the new API and will be built against Kafka 0.9 and 
will have it's own assembly.

And, what I am proposing is:
Proposal 2:
2 subprojects - one with old implementation and one with new. Both the 
implementations will be built against Kafka 0.9, they both end up in one single 
Kafka assembly artifact.

Pro of Proposal 1 is that folks who want to use the old implementation with 
Kafka 0.8 brokers can use it, without upgrading their brokers. Con of proposal 
1 is that it doesn't allow for re-use of any code between the old and new 
implementation. This can be a good thing if we don't want to share any code in 
the new implementation but there is a definitely a bunch of test code that I 
think, it'd be good to share.

Pro of Proposal 2 - test code, etc. can be shared, there will be a single 
artifact that folks would need to run the old direct stream implementation or 
the new one.
The con is, of course, that folks would have to upgrade their brokers to Kafka 
0.9, if they want to use Spark 2.0.

> Update KafkaDStreams to new Kafka 0.9 Consumer API
> --------------------------------------------------
>
>                 Key: SPARK-12177
>                 URL: https://issues.apache.org/jira/browse/SPARK-12177
>             Project: Spark
>          Issue Type: Improvement
>          Components: Streaming
>    Affects Versions: 1.6.0
>            Reporter: Nikita Tarasenko
>              Labels: consumer, kafka
>
> Kafka 0.9 already released and it introduce new consumer API that not 
> compatible with old one. So, I added new consumer api. I made separate 
> classes in package org.apache.spark.streaming.kafka.v09 with changed API. I 
> didn't remove old classes for more backward compatibility. User will not need 
> to change his old spark applications when he uprgade to new Spark version.
> Please rewiew my changes



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to