cc: kafka-clients mailing list
On May 18, 2015 4:24 PM, "Warren Falk" <war...@warrenfalk.com> wrote:

> Thanks Guozhang,
>
> Actually your previous email was clear and I understood it.  Current broker
> design means that parallel requests require parallel connections.
>
> But I think you misunderstood me.  I am not asking if that is how the
> broker works, now, I am proposing that the broker should not work that way
> as it makes writing optimal client libraries impossible.  And so I am
> asking if a change to that could be considered (if someone submitted a
> patch).
>
> What I propose is that the request ordering guarantees of the broker be
> limited only to those guarantees which are useful (e.g. the order of
> produce requests on a single topic-partition, and whatever else might be
> useful).
>
> TCP connections have a non-zero overhead, and in fact, the underlying
> sockets, the multi-roundtrip handshake, (and especially the TLS handshake
> if supported in future versions), and congestion control/re-ordering
> algorithms are all quite expensive.  Using TCP connections as a way to "get
> serialization for free" is in fact not free at all.  This is the flaw that
> HTTP1.1 suffers from and what made HTTP2.0 necessary.
>
> The problem my original email mentioned is also known as "Head of Line
> Blocking".  (http://en.wikipedia.org/wiki/Head-of-line_blocking).
> HTTP1.1's solution to head-of-line-blocking is likewise "use multiple
> connections", but this is not a real solution, which is why HTTP2.0 is
> necessary and why I'm sending this long email, pleading that Kafka not
> repeat the mistake.
>
> Luckily, Kafka's network protocol appears to have been designed with the
> foresight of a future multiplexing feature in mind.  It already has a
> "correlation id" and that statement in the wiki that "only one connection
> should be necessary" suggests that this was a plan at one time (?).  (I.e.
> if every response can be paired with its request by virtue of the order it
> appears, then why waste bandwidth sending a useless correlation id?)
>
> So I'm not suggesting a change to the protocol, only to the behavior of the
> broker.  What I want to know is whether 'head of line blocking' has become
> a fundamental *requirement* of Kafka, or is it just a behavior that could
> be corrected (i.e. if I submitted a patch to remove it, could it be
> accepted?)
>
> Thanks for your thoughtful responses.  I really appreciate the time and
> effort you and the others put into Kafka.
>
> Warren
>
>
>
>
>
> On Mon, May 18, 2015 at 1:37 PM, Guozhang Wang <wangg...@gmail.com> wrote:
>
> > Hi Warren,
> >
> > Maybe I was a little misleading in my previous email:
> >
> > 1. The statement that "The server guarantees that on a single TCP
> > connection, requests will be processed in the order" is still valid. On a
> > single connection, all types of requests (produce/fetch/metadata) are
> > handled in order.
> >
> > 2. As for the clients, it is just a matter of whether or not you want to
> > have order preserving across different requests, which is orthogonal to
> the
> > protocol/server-side guarantees. The statement that "it should not
> > generally be necessary to maintain multiple connections to a single
> broker
> > from a single client instance" is a generally suggestion, not restriction
> > on client implementation. In fact, if your client acts as both a producer
> > and a consumer, and you do not need to make sure the produce and fetch
> > requests' ordering needs to be preserved, but rather only the ordering
> > within producer requests and the ordering within fetch requests to be
> > honored, then you'd better use separate channels for these two types of
> > requests, otherwise as you noticed a long pooling request can block
> > subsequent produce requests. As an example, the current Java consumer
> > client used a single channel for each broker, but people are discussing
> > about using another separate channel to the broker acting as the consumer
> > coordinator, etc. Again, this is just the client implementation details,
> of
> > how you would like to make use of the protocol guarantees while
> considering
> > performance, usability, etc.
> >
> > Guozhang
> >
> >
> > On Thu, May 14, 2015 at 3:34 PM, Warren Falk <war...@warrenfalk.com>
> > wrote:
> >
> > > The C# client (kafka-net) isn't written by me; I'm just working on it.
> > It
> > > has a separate producer and consumer, but the client is designed to
> > connect
> > > to each broker exactly once and then reuse (multiplex over) that one
> > > connection across all consumers and producers.
> > >
> > > It's a bit disappointing to see such a great feature of the Kafka
> > protocol
> > > be abandoned.  It seems such a shame to implement request/response
> > > correlation and turn around and incur the latency overhead of
> additional
> > > TCP handshakes anyway.  If requests didn't block (if the server
> > guaranteed
> > > ordering only per partition etc.) then there would seem to be no reason
> > to
> > > use separate channels.  Are we definitely giving up on that feature?
> > >
> > > I fear both C# clients will have to be mostly redesigned in light of
> > this,
> > > which is doubly unfortunate because the C# clients don't seem to have
> > > enough development momentum behind them as it is. Subsequently, Kafka
> use
> > > in .Net environments is still extremely rare.
> > >
> > >
> > > On Wed, May 13, 2015 at 11:55 AM, Guozhang Wang <wangg...@gmail.com>
> > > wrote:
> > >
> > > > Hello Warren,
> > > >
> > > > I seems your C# client is both a producer and a consumer. Then with
> the
> > > > behavior of the broker, your suspension is correct that a long
> pooling
> > > > fetch using the same TCP connection will block subsequent produce /
> > > > metadata requests.
> > > >
> > > > I think the statement that "it should not generally be necessary to
> > > > maintain multiple connections ..." is not valid anymore if your
> client
> > > acts
> > > > both as a producer and a consumer. In fact, in the 0.9 Java clients
> > > > (producer and consumer), we already could possibly maintain multiple
> > > > connections to a single broker even though the client only send
> produce
> > > or
> > > > fetch requests, because we need a separate channel for consumer
> > > > coordinator, etc, and we have also once discussed about using a
> > separate
> > > > channel for metadata refresh.
> > > >
> > > > So I think we should modify the above statement in the wiki. Thanks
> for
> > > > pointing out.
> > > >
> > > > Guozhang
> > > >
> > > > On Wed, May 13, 2015 at 7:44 AM, Warren Falk <war...@warrenfalk.com>
> > > > wrote:
> > > >
> > > > > I'm working on the C# client.  The current Kafka Protocol page says
> > > this:
> > > > >
> > > > > "it should not generally be necessary to maintain multiple
> > connections
> > > > to a
> > > > > single broker from a single client instance (i.e. connection
> > pooling)"
> > > > >
> > > > > But then says this:
> > > > >
> > > > > "The server guarantees that on a single TCP connection, requests
> will
> > > be
> > > > > processed in the order they are sent and responses will return in
> > that
> > > > > order as well".
> > > > >
> > > > > (
> > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol
> > > > > )
> > > > >
> > > > > Given that fetch requests can be long-polling, these two statements
> > are
> > > > > mutually exclusive, are they not?  E.g. if I issue a long polling
> > fetch
> > > > to
> > > > > one broker for a topic partition and then need to issue another
> > request
> > > > to
> > > > > that same broker for any other reason (fetch/produce/metadata), my
> > > second
> > > > > request will hang until my long poll times out.  (I either need to
> > use
> > > an
> > > > > unacceptably low poll timeout for the first request or I have to
> > accept
> > > > an
> > > > > unacceptably high latency for any second request to that broker,
> or I
> > > > have
> > > > > to implement connection pooling and/or multiple connections to a
> > single
> > > > > broker).
> > > > >
> > > > > Three things:
> > > > >
> > > > > 1. Am I just missing something obvious?
> > > > > 2. Is this changing in 0.9?  I know the consumer is getting a
> > redesign,
> > > > but
> > > > > is this broker issue addressed in some way?
> > > > > 3. Is this ordering over all requests even useful?
> > > > >
> > > > > On #3 the documentation goes on to say:  "The broker's request
> > > processing
> > > > > allows only a single in-flight request per connection in order to
> > > > guarantee
> > > > > this ordering"
> > > > >
> > > > > As far as I can tell, order preservation is valuable only for
> produce
> > > > > requests and only per topic-partition. What else?  But especially
> > once
> > > a
> > > > > fetch request goes to purgatory, why would the broker not continue
> > > > > processing other incoming requests?  (and of what actual use is a
> > > > > "correlation id" when all responses the server sends are always for
> > the
> > > > > oldest in-flight request?)
> > > > >
> > > > > This problem affects the C# client (kafka-net) among other things.
> > > > > Brilliantly, after the C# consumer returns fetched messages to its
> > > > caller,
> > > > > it immediately reissues another fetch request in the background,
> but
> > > this
> > > > > brilliance ends up backfiring because of the broker's behavior
> > > mentioned
> > > > > above.  Any attempt to publish to another topic while processing
> the
> > > > > consumed messages will have a mysterious sudden 95% drop in
> > performance
> > > > if
> > > > > the two topic partitions happen to be on the same broker.  The only
> > > > > solution seems to be to implement connection pooling.  This seems
> > > wrong.
> > > > >
> > > > > Despite the note that "it should not generally be necessary to
> > maintain
> > > > > multiple connections to a single broker", the java (scala)
> > > SimpleConsumer
> > > > > appears to make a separate connection for each instance.
> > > > >
> > > > > So is the correct solution to have the C# client try to
> transparently
> > > > > manage multiple connections to the broker, or is it to have the
> > broker
> > > > more
> > > > > intelligently use a single connection?
> > > > >
> > > > > Thanks in advance and my apologies if this has been discussed
> > elsewhere
> > > > on
> > > > > the list.  I searched but couldn't find anything.
> > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > -- Guozhang
> > > >
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
>

Reply via email to