For #1, I agree that obtaining good default is not trivial. We can revisit
in the future.

For #2, the table is readable.

Thanks

On Tue, Jul 3, 2018 at 4:23 PM, Lucas Wang <lucasatu...@gmail.com> wrote:

> @Ted
> For #1, it's probably hard to predict M since it also depends on the
> hardware.
> I'm not sure how to use the suggested formula for the default value if we
> don't know M.
> Also TO is the default timeout we want to figure out, and the formula seems
> to be recursive.
> I'd suggest we stay with the current default value of 300 milliseconds, and
> address it separately
> if it turns out to be a problem. What do you think?
>
> #2, please try this link and see if it works now:
> https://drive.google.com/file/d/1QbPDqfT59A2X4To2p3OfD5YeJR8aW
> DK7/view?usp=sharing
>
> Regards,
> Lucas
>
>
> On Mon, Jul 2, 2018 at 5:52 PM, Ted Yu <yuzhih...@gmail.com> wrote:
>
> > For #1, I don't know what would be good approximation for M.
> > Maybe use max((TO / 2) / N, M / N) as default value for poll timeout ?
> >
> > For #2, I don't see the picture in email :-)
> > Can you use third party website ?
> >
> > Thanks
> >
> > On Mon, Jul 2, 2018 at 5:17 PM, Lucas Wang <lucasatu...@gmail.com>
> wrote:
> >
> > > Hi Ted,
> > >
> > > 1. I'm neutral on making the poll timeout parameter configurable.
> > > Mainly because as a config, it could be confusing for operators who try
> > to
> > > choose a value for it.
> > >
> > > To understand the implication of this value better,
> > > let's use TO to represent the timeout value under discussion,
> > > M to denote the processing time of data requests,
> > > and N to be the number of io threads.
> > >
> > > - If the data request queue is empty and there is no incoming data
> > > requests,
> > >   all io threads should be blocked on the data request queue, and
> > >   the average delay for a controller request is (TO / 2) / N, and the
> > > worst case delay is TO.
> > > - If all IO threads are busy processing data requests, then the average
> > > latency for a controller request is M / N.
> > > - In the worst case, a controller request can just miss the train, and
> IO
> > > threads get blocked on data request queue
> > >   for TO, at the end of which they all receive a new incoming data
> > > request, the latency for the
> > >   controller request can be TO + M.
> > >
> > > Given the intricacies, what do you think about choosing a relatively
> > > meaningful value and stick with it,
> > > rather than exposing it as a config?
> > >
> > > 2. Sorry for losing the format of the table, I've attached it below as
> a
> > > picture
> > >
> > >
> > > Regards,
> > > Lucas
> > >
> > > On Fri, Jun 29, 2018 at 5:28 PM, Ted Yu <yuzhih...@gmail.com> wrote:
> > >
> > >> bq. which is hard coded to be 300 milliseconds
> > >>
> > >> Have you considered making the duration configurable ?
> > >>
> > >> The comparison at the end of your email seems to be copied where
> tabular
> > >> form is lost.
> > >> Do you mind posting that part again ?
> > >>
> > >> Thanks
> > >>
> > >> On Fri, Jun 29, 2018 at 4:53 PM, Lucas Wang <lucasatu...@gmail.com>
> > >> wrote:
> > >>
> > >> > Hi Jun,
> > >> >
> > >> > Thanks for your comments.
> > >> > 1. I just replied in the discussion thread about the positive change
> > >> this
> > >> > KIP can still bring
> > >> > if implemented on the latest trunk, which includes the async ZK
> > >> operations
> > >> > for KAFKA-5642.
> > >> > The evaluation is done using an integration test.
> > >> > In production, we have not upgraded to Kafka 1.1 yet, and the code
> we
> > >> are
> > >> > currently running does
> > >> > not include async ZK operations, therefore I don't have any real
> usage
> > >> > result.
> > >> >
> > >> > 2. Thanks for bringing this up. I haven't considered this setting,
> and
> > >> the
> > >> > existing proposal in this KIP
> > >> > would make data requests and controller requests share a memory poll
> > of
> > >> > size specified by the config
> > >> > queued.max.request.bytes. The downside is that if there is memory
> > >> pressure,
> > >> > controller requests may be blocked
> > >> > from being read from a socket and does not get prioritized at the
> > socket
> > >> > layer.
> > >> >
> > >> > If we have a separate bytes limit for the controller requests, I
> > imagine
> > >> > there would be a separate memory pool
> > >> > dedicated to controller requests. Also it requires the processors to
> > >> tell
> > >> > connections from a controller apart
> > >> > from connections from other brokers or clients, which would probably
> > >> > require a dedicated port for the controller?
> > >> > IMO, this change is mainly driven by the memory pressure, kind of an
> > >> > orthogonal issue, and we can address it with a separate KIP
> > >> > if desired. Please let me know what you think.
> > >> >
> > >> > 3. I plans to change the implementation of the method
> > >> > receiveRequest(timeout: Long) in the RequestChannel class as
> follows:
> > >> >
> > >> > val controllerRequest = controllerRequestQueue.poll()
> > >> > if (controllerRequest != null) {
> > >> >   controllerRequest
> > >> > } else {
> > >> >   dataRequestQueue.poll(timeout, TimeUnit.MILLISECONDS)
> > >> > }
> > >> >
> > >> > with this implementation, there is no need to explicitly choose a
> > >> request
> > >> > handler thread to wake up depending on
> > >> > the types of request enqueued, and if a controller request arrives
> > while
> > >> > some request handler threads are blocked on an empty data request
> > queue,
> > >> > they will simply timeout and call the receiveRequest method again.
> > >> >
> > >> > In terms of performance, it means that in the worst case, for a
> > >> controller
> > >> > request that just missed the receiveRequest call, it can be delayed
> > for
> > >> as
> > >> > long as
> > >> > the timeout parameter, which is hard coded to be 300 milliseconds.
> If
> > >> there
> > >> > is just one request handler thread, the average delay is
> > >> > 150 milliseconds assuming the chance of a controller request
> arriving
> > at
> > >> > any particular time is the same. With N request handler threads,
> > >> > the average delay is 150/N milliseconds, which does not seem to be a
> > >> > problem.
> > >> >
> > >> > We have considered waking up of request handler threads based on
> which
> > >> > queue the request handler threads are blocked,
> > >> > and that design was turned down because of its complexity. The
> design
> > >> can
> > >> > be found at here
> > >> > <https://cwiki.apache.org/confluence/display/KAFKA/Old+
> > >> > controller+request+queue+design>
> > >> > .
> > >> >
> > >> > If you mean a general purpose priority queue such as the
> > >> > java.util.PriorityQueue, we also have considered it and turned down
> > the
> > >> > design because
> > >> > - The readily available class java.util.PriorityQueue is unbounded
> and
> > >> > we'll need to implement a bounded version
> > >> > - We would still like to have the FIFO semantics on both the
> > controller
> > >> > request queue and data request queue, which conceptually does not
> fit
> > >> very
> > >> > well
> > >> > with a general purpose priority queue, e.g. we would probably need
> to
> > >> use
> > >> > the enqueue time to enforce FIFO semantics.
> > >> > - A typical operation on the priority queue is O(log n), whereas the
> > >> sample
> > >> > implementation above gives O(1) performance regardless of the size
> of
> > >> both
> > >> > queues.
> > >> >
> > >> > 4. For the two APIs sendRequest and receiveRequest, since we are
> only
> > >> > changing their implementation, not the API itself
> > >> > the two metrics will support two queues and the meaning of "Idle"
> > still
> > >> > holds:
> > >> >
> > >> >
> > >> >
> > >> >
> > >> >
> > >> >
> > >> > *Before this KIPAfter this KIPNetworkProcessorAvgIdlePercentidle =
> > >> blocked
> > >> > on selectnot idle includes being blocked on requestQueueidle =
> blocked
> > >> on
> > >> > selectnot idle includes being blocked on either controller request
> > >> queue or
> > >> > data request queueRequestHandlerAvgIdlePercentidle = blocked on
> > reading
> > >> > from requestQueue idle = taking a request from the controller
> request
> > >> > queue, or blocked on reading from the data request queue*
> > >> >
> > >> > Regards,
> > >> > Lucas
> > >> >
> > >> > On Fri, Jun 29, 2018 at 11:22 AM, Jun Rao <j...@confluent.io> wrote:
> > >> >
> > >> > > Hi, Lucas,
> > >> > >
> > >> > > Thanks for the KIP. A few comments below.
> > >> > >
> > >> > > 1. As Eno mentioned in the discussion thread, I am wondering how
> > much
> > >> of
> > >> > > this is still an issue after KAFKA-5642. With that fix, the
> requests
> > >> from
> > >> > > the controller to the brokers are batched in all the common cases.
> > >> Have
> > >> > you
> > >> > > deployed Kafka 1.1? What's the request queue time and the request
> > >> queue
> > >> > > size that you have observed in production?
> > >> > >
> > >> > > 2. For the request queue, currently we can also bound it by size
> > >> > > through queued.max.request.bytes. Should we consider the same for
> > the
> > >> > > control queue?
> > >> > >
> > >> > > 3. Implementation wise, currently the request handler threads just
> > >> block
> > >> > on
> > >> > > the request queue when the queue is empty. With two queues, it
> seems
> > >> that
> > >> > > we need to wake up a request handler thread blocked on one queue,
> > when
> > >> > > another queue gets a request? Have we considered just making the
> > >> request
> > >> > > queue a priority queue?
> > >> > >
> > >> > > 4. Related to 3, currently we have 2
> > >> > > metrics  NetworkProcessorAvgIdlePercent and
> > >> RequestHandlerAvgIdlePercent
> > >> > > that measure the utilization of the network and the request
> handler
> > >> > thread
> > >> > > pools. They are computed by measuring the amount of time waiting
> on
> > >> the
> > >> > > request queue. Will these 2 metrics be extended to support 2
> request
> > >> > > queues.
> > >> > >
> > >> > > Jun
> > >> > >
> > >> > >
> > >> > > On Mon, Jun 18, 2018 at 1:04 PM, Lucas Wang <
> lucasatu...@gmail.com>
> > >> > wrote:
> > >> > >
> > >> > > > Hi All,
> > >> > > >
> > >> > > > I've addressed a couple of comments in the discussion thread for
> > >> > KIP-291,
> > >> > > > and
> > >> > > > got no objections after making the changes. Therefore I would
> like
> > >> to
> > >> > > start
> > >> > > > the voting thread.
> > >> > > >
> > >> > > > KIP:
> > >> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-291%
> > >> > > > 3A+Have+separate+queues+for+control+requests+and+data+requests
> > >> > > >
> > >> > > > Thanks for your time!
> > >> > > > Lucas
> > >> > > >
> > >> > >
> > >> >
> > >>
> > >
> > >
> >
>

Reply via email to