kafka-users mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Evan Chan ...@ooyala.com>
Subject Re: Non-blocking Kafka stream iterators
Date Tue, 22 Jan 2013 20:57:27 GMT
Hi Jay,

Actually, it's mostly the ability to easily cross-build;   also the ease of
understanding the code (less code to grok) and implementing alternatives (I
guess all of those falls under cleanliness).

thanks,
Evan


On Tue, Jan 22, 2013 at 12:47 PM, Jay Kreps <jay.kreps@gmail.com> wrote:

> Hi Evan,
>
> Makes sense. Is your goal in separating the client shrinking the jar size?
> or just general cleanliness?
>
> -Jay
>
>
> On Tue, Jan 22, 2013 at 10:53 AM, Evan Chan <ev@ooyala.com> wrote:
>
> > Jay,
> >
> > Comments inlined.
> >
> > On Tue, Jan 22, 2013 at 10:15 AM, Jay Kreps <jay.kreps@gmail.com> wrote:
> >
> > > Hey Evan,
> > >
> > > Great points, some comments:
> > > - Not sure if I understand what you mean by separating consumer and
> main
> > > logic.
> > >
> >
> > I just meant having a separate Scala/Java client jar, so it's more
> > lightweight and easier to build independently.... kind of like the
> > consumers for the other languages.
> >
> >
> > > - Yes, cross-building, I think this is in progress now for kafka as a
> > whole
> > > so it should be in either 0.8 or 0.8.1
> > > - Yes, forgot to mention offset initialization, but that is definitely
> > > needed.
> > >
> > > For the hasNext functionality, even that is not very good since if you
> > have
> > > two streams and want to take the next message from either you would
> have
> > to
> > > busy wait calling hasNext on both in a loop.
> > >
> > > An alternative would be something like
> > > val client = new ConsumerClient(topics, config)
> > > client.select(timeout: Long): Iterator[MessageAndMetadata]
> > >
> > > This method would have no internal threading. It would scatter-gather
> > over
> > > the topic/partitions assigned to this consumer (whether they are
> > statically
> > > or dynamically assigned would be specified in the config). The select
> > call
> > > would internally just do an epoll/select on all the connections and
> > return
> > > the first message set it gets back or an empty list if it hits the
> > timeout
> > > and no one has responded.
> > >
> >
> > Hm, I like that API actually.  It would definitely be more flexible.
> >
> >
> > >
> > > This api is less intuitive then the blocking iterator, but more
> flexible
> > > and enables a better, faster implementation. There would be no threads
> > > aside from the client's thread. It allows non-blocking or blocking
> > > consumption. And it generalizes easily to consuming from many
> > > topics/partitions simultaneously.
> > >
> > > We could implement an iterator like wrapper for this to ease the
> > transition
> > > that just used this api under the covers.
> > >
> > > Anyhow this is a ways out, and we haven't really had any proposals or
> > > discussions on it, but this is what I was thinking.
> > >
> > > -Jay
> > >
> > >
> > >
> > >
> > > On Tue, Jan 22, 2013 at 9:37 AM, Evan Chan <ev@ooyala.com> wrote:
> > >
> > > > Jay,
> > > >
> > > > For the consumer:
> > > > - Separation of the consumer logic from the main logic
> > > > - Making it easier to build the consumer for different versions of
> > Scala
> > > > (say 2.10)
> > > > - Make it easier to read from any offset you want, while being able
> to
> > > keep
> > > > partition management features
> > > > - Better support for Akka and other non-blocking / event-based
> > frameworks
> > > > (instead of a timeout, implement true hasNext functionality, for
> > example)
> > > >
> > > > thanks,
> > > > Evan
> > > >
> > > >
> > > > On Mon, Jan 21, 2013 at 9:27 AM, Jay Kreps <jay.kreps@gmail.com>
> > wrote:
> > > >
> > > > > It's worth mentioning that we are interested in exploring potential
> > > > > generalizations of the producer and consumer API, but as a
> practical
> > > > matter
> > > > > most of the committers are working on getting a stable 0.8 release
> > out
> > > > the
> > > > > door. So an improved consumer and producer api would be a 0.9
> > feature.
> > > > >
> > > > > If you have a concrete thing you are trying to do now that is
> awkward
> > > it
> > > > > would be great to hear about the use case.
> > > > >
> > > > > Possible goals of improving the apis and client impls would include
> > the
> > > > > following:
> > > > >
> > > > > Producer:
> > > > > 1. Include the offset in the information returned to the producer
> > > > > 2. Pipeline producer requests to improve throughput for synchronous
> > > > > production
> > > > >
> > > > > Consumer
> > > > > 1. Simplify api while supporting various advanced use cases like
> > > > > multi-stream consumption
> > > > > 2. Make partition assignment optional and server-side (this is
> > > currently
> > > > > the difference between the zk consumer and the simple consumer)
> > > > > 3. Make offset management optional
> > > > > 4. Remove threading from the consumer
> > > > > 5. Simplify consumer memory management
> > > > >
> > > > > -Jay
> > > > >
> > > > >
> > > > >
> > > > >
> > > > > On Mon, Jan 21, 2013 at 8:05 AM, Jun Rao <junrao@gmail.com>
wrote:
> > > > >
> > > > > > No, but you can implement it in your application.
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > > On Sun, Jan 20, 2013 at 11:02 PM, navneet sharma <
> > > > > > navneetsharma0505@gmail.com> wrote:
> > > > > >
> > > > > > > Is there any property to make consumer work for lets say
only
> 10
> > > mins
> > > > > (ie
> > > > > > > some kind of timer)
> > > > > > > So, i want to close the consumer after 10 mins reading
from
> > > broker..
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Navneet Sharma
> > > > > > >
> > > > > > >
> > > > > > > On Mon, Jan 21, 2013 at 11:48 AM, Ryan LeCompte <
> > > lecompte@gmail.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Perfect. Thanks Jun!
> > > > > > > >
> > > > > > > > Ryan
> > > > > > > >
> > > > > > > >
> > > > > > > > On Sun, Jan 20, 2013 at 10:16 PM, Jun Rao <junrao@gmail.com>
> > > > wrote:
> > > > > > > >
> > > > > > > > > Yes, see consumer.timeout.ms in
> > > > > > > > http://kafka.apache.org/configuration.html
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > >
> > > > > > > > > Jun
> > > > > > > > >
> > > > > > > > > On Sun, Jan 20, 2013 at 7:14 PM, Ryan LeCompte
<
> > > > lecompte@gmail.com
> > > > > >
> > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hello all,
> > > > > > > > > >
> > > > > > > > > > Does Kafka still only support blocking stream
iterstors?
> It
> > > > would
> > > > > > be
> > > > > > > > > great
> > > > > > > > > > to pass a timeout or have a poll() operation
for fetching
> > > > items.
> > > > > > > Right
> > > > > > > > > now
> > > > > > > > > > I'm always blocking in this call:
> > > > > > > > > > for (m <- stream) ...
> > > > > > > > > >
> > > > > > > > > > Thanks!
> > > > > > > > > >
> > > > > > > > > > Ryan
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > --
> > > > *Evan Chan*
> > > > Senior Software Engineer |
> > > > ev@ooyala.com | (650) 996-4600
> > > > www.ooyala.com | blog <http://www.ooyala.com/blog> |
> > > > @ooyala<http://www.twitter.com/ooyala>
> > > >
> > >
> >
> >
> >
> > --
> > --
> > *Evan Chan*
> > Senior Software Engineer |
> > ev@ooyala.com | (650) 996-4600
> > www.ooyala.com | blog <http://www.ooyala.com/blog> |
> > @ooyala<http://www.twitter.com/ooyala>
> >
>



-- 
--
*Evan Chan*
Senior Software Engineer |
ev@ooyala.com | (650) 996-4600
www.ooyala.com | blog <http://www.ooyala.com/blog> |
@ooyala<http://www.twitter.com/ooyala>

Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message