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 <viktorsomo...@gmail.com>
wrote:

> Hey Jungtaek,
>
> Thanks for your interest, sometimes I also think such an API would be a
> good thing.
> I don't see any strong reasons neither in KIP-288 nor in KIP-266 why such
> an API shouldn't be created, so go ahead with it, although you'll need to
> create a short KIP for this as the KafkaConsumer class considered to be a
> public API.
>
> Best,
> Viktor
>
> On Wed, Aug 7, 2019 at 9:26 AM Jungtaek Lim <kabh...@gmail.com> wrote:
>
> > 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 <kabh...@gmail.com> wrote:
> >
> > > Hi devs,
> > >
> > > I'm trying to replace deprecated poll(long) with poll(Duration), and
> > > realized there's no alternative which behaves exactly same as poll(0),
> as
> > > poll(0) has been used as a hack to only update metadata instead of
> > pulling
> > > records. poll(Duration.ZERO) wouldn't behave same since even updating
> > > metadata will be timed-out. So now end users would need to give more
> > > timeout and even pull some records even they're only interested in
> > metadata.
> > >
> > > I looked back some KIPs which brought the change, and "discarded" KIP
> > > (KIP-288 [1]) actually proposed a new API which only pulls metadata.
> > > KIP-266 [2] is picked up instead but it didn't cover all the things
> what
> > > KIP-288 proposed. I'm seeing some doc explaining poll(0) hasn't been
> > > supported officially, but the hack has been widely used and they can't
> be
> > > ignored.
> > >
> > > Kafka test code itself relies on either deprecated poll(0),
> > > or updateAssignmentMetadataIfNeeded, which seems to be private API only
> > for
> > > testing.
> > > (Btw, I'd try out replacing poll(0) to updateAssignmentMetadataIfNeeded
> > as
> > > avoiding deprecated method - if it works I'll submit a PR.)
> > >
> > > I'm feeling that it would be ideal to expose
> > > `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 sure it would
> > > require a new KIP or not.
> > >
> > > Thanks,
> > > Jungtaek Lim (HeartSaVioR)
> > >
> > > 1.
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-288%3A+%5BDISCARDED%5D+Consumer.poll%28%29+timeout+semantic+change+and+new+waitForAssignment+method
> > > 2.
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-266%3A+Fix+consumer+indefinite+blocking+behavior
> > >
> > >
> >
> > --
> > Name : Jungtaek Lim
> > Blog : http://medium.com/@heartsavior
> > Twitter : http://twitter.com/heartsavior
> > LinkedIn : http://www.linkedin.com/in/heartsavior
> >
>


-- 
Name : Jungtaek Lim
Blog : http://medium.com/@heartsavior
Twitter : http://twitter.com/heartsavior
LinkedIn : http://www.linkedin.com/in/heartsavior

Reply via email to