Re: [VOTE] KIP-396: Add Commit/List Offsets Operations to AdminClient

2019-08-14 Thread Jungtaek Lim
+1 (non-binding) I found it very useful for Spark's case. (Discussion on KIP-505 described it.) Thanks for driving the effort! 2019년 8월 14일 (수) 오후 8:49, Mickael Maison 님이 작성: > Hi Guozhang, > > Thanks for taking a look. > > 1. Right, I updated the titles of the code blocks > > 2. Yes that's a g

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

2019-08-12 Thread Jungtaek Lim
r not), but I also respect the lifecycle of KIP in Kafka community. On Tue, Aug 13, 2019 at 12:46 PM Jungtaek Lim wrote: > > > On Tue, Aug 13, 2019 at 10:01 AM Colin McCabe wrote: > >> On Mon, Aug 12, 2019, at 14:54, Jungtaek Lim wrote: >> > Thanks for the feedbacks

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

2019-08-12 Thread Jungtaek Lim
On Tue, Aug 13, 2019 at 10:01 AM Colin McCabe wrote: > On Mon, Aug 12, 2019, at 14:54, Jungtaek Lim wrote: > > Thanks for the feedbacks Colin and Matthias. > > > > I agree with you regarding getting topics and partitions via AdminClient, > > just curious how much the

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

2019-08-12 Thread Jungtaek Lim
of a single request though. > > Also, what offset information are you interested in? Because you don't > commit any offsets to Kafka, but store it in Spark's checkpoint, it's > unclear what information you are looking for? > > -Matthias > > > On 8/12/19 2:53 PM

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

2019-08-12 Thread Jungtaek Lim
gned to be used internally) which end users may not indicate how to leverage the class. IMHO, exposing assignment metadata update on consumer sounds simpler. Thanks, Jungtaek Lim (HeartSaVioR) On Tue, Aug 13, 2019 at 7:51 AM Harsha Chintalapani wrote: > Hi Jungtaek, >Have

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

2019-08-12 Thread Jungtaek Lim
. Instead, Spark should use > AdminClient to find out what partitions exist and where, manage their > offsets, and so on. There are some KIPs under discussion now that would add > the necessary APIs for managing offsets. > > > > Best, > > Colin > > > > On Mon, A

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

2019-08-12 Thread Jungtaek Lim
ConsumerRebalanceListener` via > > > `subscribe()` to get informed when the group rebalances > > > > > > As you pointed out, using pattern subscription metadata can change if > > > topic are added/deleted. However, each metadata change will triggering > a > >

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

2019-08-11 Thread Jungtaek Lim
s known one. There's only one known approach doing this, calling `poll`, but Spark is not interested on returned records, so there's a need for a hack `poll(0)`, and Kafka deprecated the API. This KIP proposes to support this as official approach. On Sun, Aug 11, 2019 at 8:18 PM Jungtaek

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 > c

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

2019-08-11 Thread Jungtaek Lim
des. 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 if I'm not missing here, there's no way to accomplish this. Please guide me if I'm missing something. Than

Re: Alternative of poll(0) without pulling records

2019-08-11 Thread Jungtaek Lim
y to accomplish this. Please correct me if I'm missing something. Thanks, Jungtaek Lim (HeartSaVioR) On Sat, Aug 10, 2019 at 7:32 AM Colin McCabe wrote: > Hi Gabor, > > What is it that you want to do here? If you just want to check that the > partitions exist, but not fetch any d

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

2019-08-08 Thread Jungtaek Lim
tion vs boolean, etc.) on Kafka project. Thanks in advance! Jungtaek Lim (HeartSaVioR)

[jira] [Created] (KAFKA-8776) Add new public method to only update assignment metadata in consumer

2019-08-08 Thread Jungtaek Lim (JIRA)
Jungtaek Lim created KAFKA-8776: --- Summary: Add new public method to only update assignment metadata in consumer Key: KAFKA-8776 URL: https://issues.apache.org/jira/browse/KAFKA-8776 Project: Kafka

Re: Ask for contributor access and write permission on wiki

2019-08-08 Thread Jungtaek Lim
Thanks Harsha for granting permission, now I can create subpage on wiki. -Jungtaek Lim (HeartSaVioR) On Fri, Aug 9, 2019 at 11:22 AM Harsha Chintalapani wrote: > Hi Jungtaek, > Gave you permissions on wiki. Please check. > Thansk, > Harsha > > > On Thu, Au

Ask for contributor access and write permission on wiki

2019-08-08 Thread Jungtaek Lim
Hi devs, I'd like to give a shot to make first contribution on Kafka community, as I initiated thread on needs a new public API for metadata update only [1]. Could you please grant me contributor in JIRA as well as write permission on wiki page? Thanks in advance! Jungtaek Lim (HeartSaVio

Re: Alternative of poll(0) without pulling records

2019-08-08 Thread Jungtaek Lim
Thanks Viktor for guiding me through this! I would initiate new thread to ask edit permission on wiki. Once I got permission I'll come up with simple KIP page and initiate discussion thread. Thanks again, Jungtaek Lim On Thu, Aug 8, 2019 at 9:42 PM Viktor Somogyi-Vass wrote: > Hey

Re: Alternative of poll(0) without pulling records

2019-08-07 Thread Jungtaek Lim
If we just wanted to remove deprecation and let both co-exist, that would be also viable, though `poll(0)` is still a hack and it would be ideal to provide official approach to do so. On Wed, Aug 7, 2019 at 4:24 PM Jungtaek Lim wrote: > Hi devs, > > I'm trying to replace deprec

Alternative of poll(0) without pulling records

2019-08-07 Thread Jungtaek Lim
xpose `updateAssignmentMetadataIfNeeded` to the public API, maybe with renaming as `waitForAssignment` which was proposed in KIP-288 if it feels too long. What do you think? If it sounds feasible I'd like to try out contribution on this. I'm new to contribute Kafka community, so not sur

Re: [ANNOUNCE] New Committer: Manikumar Reddy

2018-10-16 Thread Jungtaek Lim
Congrats Mani! On Tue, 16 Oct 2018 at 1:45 PM Abhimanyu Nagrath wrote: > Congratulations Manikumar > > On Tue, Oct 16, 2018 at 10:09 AM Satish Duggana > wrote: > > > Congratulations Mani! > > > > > > On Fri, Oct 12, 2018 at 9:41 PM Colin McCabe wrote: > > > > > > Congratulations, Manikumar! We

Provide clients Header implementation without accessing internal package

2018-09-04 Thread Jungtaek Lim
ed if we really want to hide the implementation and let end users only refer Header interface. If that's only missing spot I'm happy to take it forward. (I might request contributor role then.) If that's by intention please also let me know that we can discuss about it. Thanks! Jungtaek Lim (HeartSaVioR)

Re: [DISCUSS] KIP-80: Kafka REST Server

2016-10-16 Thread Jungtaek Lim
pers inspiring and adopting it to their projects. Hopes that Kafka community continuously drives the transparency model among the ASF projects, and beyond. - Jungtaek Lim (HeartSaVioR) 2016년 10월 17일 (월) 오전 7:56, Jay Kreps 님이 작성: Hey Nacho, Yeah, I think it is definitely a call we have to m

[jira] [Commented] (KAFKA-1974) NPE in SelectorTest on trunk

2015-03-20 Thread Jungtaek Lim (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1974?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14372557#comment-14372557 ] Jungtaek Lim commented on KAFKA-1974: - I agree with [~aauradkar], but I'