[
https://issues.apache.org/jira/browse/SPARK-12177?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15174092#comment-15174092
]
Cody Koeninger commented on SPARK-12177:
----------------------------------------
My thoughts so far
Must-haves:
- The major new feature of the kafka 0.9 consumer that may justify upgrading is
security.
- No one (as far as I can tell) is actually doing integration testing of these
existing PRs using the new kafka security features.
We need actual manual integration testing and benchmarking, ideally with
production loads.
- Actually making security features usable at low latencies is probably going
to require caching consumers on the executors, not just the driver.
The current direct stream managed without caching, but you don't want to be
doing handshaking/getting a new token/whatever every XXX ms batch.
This is going to require redesigning the iterator and getPreferredLocations.
- Even if handshaking doesn't turn out to be that bad, getting full benefit of
the new consumer's pipelining will require caching it between batches.
Nice to haves that should be considered up front:
- There are some long standing minor features that would be easier to do with
the new consumer
(dynamic topicpartitions, making it easier to control offsetrange generation
policy in general, making it easier to commit offsets into kafka)
- Those minor features are going to require minor redesign of the Dstream, and
the interface for creating it.
>From what I can tell, the existing PRs and commentary on them don't address
>any of this (don't get me wrong, I appreciate the work).
Until that stuff is addressed, I think contemplating replacing the existing
consumers is premature.
Now that I've got a production 0.9 cluster going, I'm certainly willing to help
work on it.
But it doesn't seem like there's consensus (or even committer fiat, which I'd
also be fine with) on the approach for dealing with 0.8 vs 0.9.
My strong preference on the bikeshed's color:
- Leave the existing 0.8 consumer integration subproject exactly as is.
It's known working, should still work with 0.9 brokers, and the maintenance (if
any) necessary to make it work with spark 2.0 should be minor.
Changing the existing consumer to factor out functionality makes it harder to
verify that the PR isn't breaking something, so don't do it.
The substantive new features would be hard / impossible to backport, so don't
do it.
- Make an entirely new subproject, with a differently-named artifact, for the
0.9 consumer integration.
Don't name all the packages or classes in the new subproject NewKafkaWhatever,
it's unnecessary.
Because of broker differences, someone's not going to be likely to use both
artifacts in one project.
Different signatures will make it fail at compile time if they use the wrong
one.
- Don't copy everything wholesale from the existing subproject, only copy those
pieces of the direct stream that still make sense, and modify them.
Cleanly factoring out common functionality from the existing subproject would
be awkward given the actual differences in the consumer.
A lot of the existing pieces don't make sense at all for the new consumer
(KafkaCluster as a wrapper should just go away, for instance)
- Don't try to hide the fact that the consumer is different from end users.
It really is different, and exposing that will give them more power (e.g. topic
subscription by pattern).
That's just my 2 cents, I'm willing to help out regardless of the direction
taken, but I think we need some committer weigh in on direction, especially
given there are now 4 separate PRs relating to this issue. [~tdas]
[~srowen][~vanzin] any thoughts?
> 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]