[ https://issues.apache.org/jira/browse/SPARK-12177?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15174149#comment-15174149 ]
Mark Grover commented on SPARK-12177: ------------------------------------- Thanks Cody, I appreciate your thoughts. I have been keeping most of my commentary on the PRs but I will post some parts of it here for the sake of argument. bq. 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. Agreed. The code in [my PR for the new security API|https://github.com/apache/spark/pull/10953/files] was integration tested by me against a distributed Kafka and ZK cluster, albeit manually. Working on adding automated integration tests is on my list of things to do, however, that PR is bit rotting because it's blocked by [the PR|https://github.com/apache/spark/pull/11143] to upgrade Kafka to 0.9.0. Your comment about caching consumers on executors is an excellent one. I haven't invested much time there because the way I was thinking of doing this was in several steps: 1. Upgrade Kafka to 0.9 (with or without 0.8 support, pending decision on https://github.com/apache/spark/pull/11143) 2. Add support for the new consumer API (https://github.com/apache/spark/pull/10953/files) 3. Add Kerberos/SASL support for authentication and SSL support for encryption over wire. This work is blocked until delegation token support is added in Kafka (https://issues.apache.org/jira/browse/KAFKA-1696). I have been following that design discussion closely Kafka mailing list. Thanks for sharing your preference. I understand where you are coming from, and think that's reasonable. I had gotten feedback to the contrary on [this PR|https://github.com/apache/spark/pull/10953/files] so I changed my original implementation which had separate subprojects, to all be in the same project. I don't mind changing it back, especially if we are going to keep 0.8 support. Related to not hiding the fact that the consumer is new, is concerned: I agree with you, KafkaUtils, for example has exposed TopicAndPartition, MessageAndMetadata classes. And, I think we may have to expose their new API equivalent TopicPartition and ConsumerRecord in KafkaUtils. In any case, I'd appreciate your help in moving this forward. I think the first step is to come to a resolution on https://github.com/apache/spark/pull/11143. Perhaps you, I, [~tdas] and anyone else who's interested could get on a call to sort this out? I will post the call details here so anyone would be able to join in. Other methods of communication work too, my goal is to move that conversation forward. > 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: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org