kafka-users mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Guozhang Wang <wangg...@gmail.com>
Subject Re: Questions about Kafka 0.9 API changes
Date Tue, 23 Sep 2014 22:36:38 GMT
Hi Valentin,

I see your point. Would the following be work for you then: You can
maintain the broker metadata as you already did and then use a 0.9 kafka
consumer for each broker, and hence by calling subscribe / de-subscribe the
consumer would not close / re-connect to the broker if it is implemented
smartly enough to realize the newly subscribed topic is still on the
current connected broker?

Guozhang

On Tue, Sep 23, 2014 at 12:23 PM, Valentin <kafka-9999-vw@sblk.de> wrote:

> Hi Jun,
>
> yes, that would theoretically be possible, but it does not scale at all.
>
> I.e. in the current HTTP REST API use case, I have 5 connection pools on
> every tomcat server (as I have 5 brokers) and each connection pool holds
> upto 10 SimpleConsumer connections. So all in all I get a maximum of 50
> open connections per web application server. And with that I am able to
> handle most requests from HTTP consumers without having to open/close
> any new connections to a broker host.
>
> If I would now do the same implementation with the new Kafka 0.9 high
> level consumer, I would end up with >1000 connection pools (as I have
> >1000 topic partitions) and each of these connection pools would contain
> a number of consumer connections. So all in all, I would end up with
> thousands of connection objects per application server. Not really a
> viable approach :|
>
> Currently I am wondering what the rationale is for deprecating the
> SimpleConsumer API, if there are use cases which just work much better
> using it.
>
> Greetings
> Valentin
>
> On 23/09/14 18:16, Guozhang Wang wrote:
> > Hello,
> >
> > For your use case, with the new consumer you can still create a new
> > consumer instance for each  topic / partition, and remember the mapping
> of
> > topic / partition => consumer. The upon receiving the http request you
> can
> > then decide which consumer to use. Since the new consumer is single
> > threaded, creating this many new consumers is roughly the same cost with
> > the old simple consumer.
> >
> > Guozhang
> >
> > On Tue, Sep 23, 2014 at 2:32 AM, Valentin <kafka-9999-vw@sblk.de> wrote:
> >
> >>
> >> Hi Jun,
> >>
> >> On Mon, 22 Sep 2014 21:15:55 -0700, Jun Rao <junrao@gmail.com> wrote:
> >>> The new consumer api will also allow you to do what you want in a
> >>> SimpleConsumer (e.g., subscribe to a static set of partitions, control
> >>> initial offsets, etc), only more conveniently.
> >>
> >> Yeah, I have reviewed the available javadocs for the new Kafka 0.9
> >> consumer APIs.
> >> However, while they still allow me to do roughly what I want, I fear
> that
> >> they will result in an overall much worse performing implementation on
> my
> >> side.
> >> The main problem I have in my scenario is that consumer requests are
> >> coming in via stateless HTTP requests (each request is standalone and
> >> specifies topics+partitions+offsets to read data from) and I need to
> find a
> >> good way to do connection pooling to the Kafka backend for good
> >> performance. The SimpleConsumer would allow me to do that, an approach
> with
> >> the new Kafka 0.9 consumer API seems to have a lot more overhead.
> >>
> >> Basically, what I am looking for is a way to pool connections per Kafka
> >> broker host, independent of the topics/partitions/clients/..., so each
> >> Tomcat app server would keep N disjunctive connection pools, if I have N
> >> Kafka broker hosts.
> >> I would then keep some central metadata which tells me which hosts are
> the
> >> leaders for which topic+partition and for an incoming HTTP client
> request
> >> I'd just take a Kafka connection from the pool for that particular
> broker
> >> host, request the data and return the connection to the pool. This means
> >> that a Kafka broker host will get requests from lots of different end
> >> consumers via the same TCP connection (sequentially of course).
> >>
> >> With the new Kafka consumer API I would have to subscribe/unsubscribe
> from
> >> topics every time I take a connection from the pool and as the request
> may
> >> need go to a different broker host than the last one, that wouldn't even
> >> prevent all the connection/reconnection overhead. I guess I could create
> >> one dedicated connection pool per topic-partition, that way
> >> connection/reconnection overhead should be minimized, but that way I'd
> end
> >> up with hundreds of connection pools per app server, also not a good
> >> approach.
> >> All in all, the planned design of the new consumer API just doesn't seem
> >> to fit my use case well. Which is why I am a bit anxious about the
> >> SimpleConsumer API being deprecated.
> >>
> >> Or am I missing something here? Thanks!
> >>
> >> Greetings
> >> Valentin
>
>


-- 
-- Guozhang

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