Re: [DISCUSS] KIP-505 : Add new public method to only update assignment metadata in consumer

2019-08-11 Thread Satish Duggana
Hi Jungtaek, Thanks for the KIP. I have a couple of questions here. Is not Spark using Kafka's consumer group management across multiple consumers? Is Spark using KafkaConsumer#subscribe(Pattern pattern, ConsumerRebalanceListener listener) only to get all the topics for a pattern based subscriptio

Build failed in Jenkins: kafka-trunk-jdk11 #746

2019-08-11 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: Eliminate unnecessary map operations in RecordAccumulator.isMuted [github] KAFKA-8782: Close metrics in QuotaManagerTests (#7191) -- [...tru

Build failed in Jenkins: kafka-trunk-jdk8 #3844

2019-08-11 Thread Apache Jenkins Server
See Changes: [github] KAFKA-8782: Close metrics in QuotaManagerTests (#7191) -- [...truncated 2.58 MB...] org.apache.kafka.streams.test.ConsumerRecordFactoryTest > shoul

Build failed in Jenkins: kafka-trunk-jdk8 #3843

2019-08-11 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: Eliminate unnecessary map operations in RecordAccumulator.isMuted -- [...truncated 2.59 MB...] org.apache.kafka.connect.file.FileStreamSourc

Re: Dynamic configuration of interbroker SSL certificates

2019-08-11 Thread Michael Carter
Hi Rajini, I just thought I'd let you know I've sorted through my issue now. It turns out that I was doing something silly. The SSL handshake errors weren't caused by the brokers talking to themselves but were instead being initiated from an AdminClient I had running in a seperate process. Iron

Re: [DISCUSS] KIP-505 : Add new public method to only update assignment metadata in consumer

2019-08-11 Thread Matthias J. Sax
If am not sure if I fully understand yet. The fact, that Spark does not stores offsets in Kafka but as part of its own checkpoint mechanism seems to be orthogonal. Maybe I am missing something here. As you are using subscribe(), you use Kafka consumer group mechanism, that takes care of the assig

Build failed in Jenkins: kafka-trunk-jdk8 #3842

2019-08-11 Thread Apache Jenkins Server
See Changes: [rhauch] KAFKA-8550: Fix plugin loading of aliased converters in Connect (#6959) -- [...truncated 6.50 MB...] org.apache.kafka.streams.StreamsConfigTest > sh

Jenkins build is back to normal : kafka-2.1-jdk8 #221

2019-08-11 Thread Apache Jenkins Server
See

Build failed in Jenkins: kafka-2.2-jdk8 #158

2019-08-11 Thread Apache Jenkins Server
See Changes: [rhauch] KAFKA-8550: Fix plugin loading of aliased converters in Connect (#6959) -- [...truncated 2.75 MB...] kafka.coordinator.group.GroupCoordinatorTest > tes

Build failed in Jenkins: kafka-1.0-jdk7 #276

2019-08-11 Thread Apache Jenkins Server
See Changes: [rhauch] KAFKA-8550: Fix plugin loading of aliased converters in Connect (#6959) -- [...truncated 299.79 KB...] kafka.api.AuthorizerIntegrationTest > testFetchAl

Build failed in Jenkins: kafka-2.0-jdk8 #287

2019-08-11 Thread Apache Jenkins Server
See Changes: [rhauch] KAFKA-8550: Fix plugin loading of aliased converters in Connect (#6959) -- [...truncated 437.18 KB...] kafka.security.auth.ZkAuthorizationTest > testZk

[jira] [Resolved] (KAFKA-8550) Connector validation fails with aliased converters

2019-08-11 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8550?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-8550. -- Resolution: Fixed Thanks, [~ChrisEgerton]! > Connector validation fails with aliased converter

Re: [DISCUSS] KIP-505 : Add new public method to only update assignment metadata in consumer

2019-08-11 Thread Jungtaek Lim
Let me elaborate my explanation a bit more. Here we say about Apache Spark, but this will apply for everything which want to control offset of Kafka consumers. Spark is managing the committed offsets and the offsets which should be polled now. Topics and partitions as well. This is required as Spa

Re: Alternative of poll(0) without pulling records

2019-08-11 Thread Jungtaek Lim
Btw, I'd like to ask you to move on thread for KIP discussion, as it will make us reaching conclusion faster and have single channel to discuss. On Sun, Aug 11, 2019 at 8:16 PM Jungtaek Lim wrote: > So we have some use case which we don't just rely on everything what Kafka > consumer provides. W

Re: [DISCUSS] KIP-505 : Add new public method to only update assignment metadata in consumer

2019-08-11 Thread Jungtaek Lim
Sorry I didn't recognize you're also asking it here as well. I'm in favor of describing it in this discussion thread so the discussion itself can go forward. So copying my answer here: We have some use case which we don't just rely on everything what Kafka consumer provides. We want to know curren

Re: Alternative of poll(0) without pulling records

2019-08-11 Thread Jungtaek Lim
So we have some use case which we don't just rely on everything what Kafka consumer provides. We want to know current assignment on this consumer, and to get the latest assignment, we called the hack `poll(0)`. That said, we don't want to pull any records here, and there's no way to accomplish thi