Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-10-10 Thread Lucas Wang
Thanks for your review, Joel and Dong.
I've updated the KIP according to Dong's last comments.

Cheers!
Lucas

On Tue, Oct 9, 2018 at 10:06 PM Dong Lin  wrote:

> Hey Lucas,
>
> Thanks for the KIP. Looks good overall. +1
>
> I have two trivial comments which may be a bit useful to reader.
>
> - Can we include the default value for the new config in Public Interface
> section? Typically the default value of the new config is an important part
> of public interface and we usually specify it in the KIP's public interface
> section.
> - Can we change "whose default capacity is 20" to  "whose capacity is 20"
> in the section "How are controller requests handled over the dedicated
> connections"? The use of word "default" seems to suggest that this is
> configurable.
>
> Thanks,
> Dong
>
> On Mon, Jun 18, 2018 at 1:04 PM Lucas Wang  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
> >
>


Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-10-09 Thread Dong Lin
Hey Lucas,

Thanks for the KIP. Looks good overall. +1

I have two trivial comments which may be a bit useful to reader.

- Can we include the default value for the new config in Public Interface
section? Typically the default value of the new config is an important part
of public interface and we usually specify it in the KIP's public interface
section.
- Can we change "whose default capacity is 20" to  "whose capacity is 20"
in the section "How are controller requests handled over the dedicated
connections"? The use of word "default" seems to suggest that this is
configurable.

Thanks,
Dong

On Mon, Jun 18, 2018 at 1:04 PM Lucas Wang  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
>


Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-10-09 Thread Joel Koshy
+1
Thanks for the updated KIP.

On Tue, Oct 9, 2018 at 3:28 PM Lucas Wang  wrote:

> Thanks Jun, I've updated the KIP with the new names.
>
> Hi Joel, Becket, Dong, Ismael,
> Since you've reviewed this KIP in the past, can you please review it again?
> Thanks a lot!
>
> Lucas
>
> On Mon, Oct 8, 2018 at 6:10 PM Jun Rao  wrote:
>
>> Hi, Lucas,
>>
>> Yes, the new names sound good to me.
>>
>> Thanks,
>>
>> Jun
>>
>> On Fri, Oct 5, 2018 at 1:12 PM, Lucas Wang  wrote:
>>
>> > Thanks for the suggestion, Ismael. I like it.
>> >
>> > Jun,
>> > I'm excited to get the +1, thanks a lot!
>> > Meanwhile what do you feel about renaming the metrics and config to
>> >
>> > ControlPlaneRequestQueueSize
>> >
>> > ControlPlaneNetworkProcessorIdlePercent
>> >
>> > ControlPlaneRequestHandlerIdlePercent
>> >
>> > control.plane.listener.name
>> >
>> > ?
>> >
>> >
>> > Thanks,
>> >
>> > Lucas
>> >
>> > On Thu, Oct 4, 2018 at 11:38 AM Ismael Juma  wrote:
>> >
>> > > Have we considered control plane if we think control by itself is
>> > > ambiguous? I agree with the original concern that "controller" may be
>> > > confusing for something that affects all brokers.
>> > >
>> > > Ismael
>> > >
>> > >
>> > > On 4 Oct 2018 11:08 am, "Lucas Wang"  wrote:
>> > >
>> > > Thanks Jun. I've changed the KIP with the suggested 2 step upgrade.
>> > > Please take a look again when you have time.
>> > >
>> > > Regards,
>> > > Lucas
>> > >
>> > >
>> > > On Thu, Oct 4, 2018 at 10:06 AM Jun Rao  wrote:
>> > >
>> > > > Hi, Lucas,
>> > > >
>> > > > 200. That's a valid concern. So, we can probably just keep the
>> current
>> > > > name.
>> > > >
>> > > > 201. I am thinking that you would upgrade in the same way as
>> changing
>> > > > inter.broker.listener.name. This requires 2 rounds of rolling
>> restart.
>> > > In
>> > > > the first round, we add the controller endpoint to the listeners w/o
>> > > > setting controller.listener.name. In the second round, every broker
>> > sets
>> > > > controller.listener.name. At that point, the controller listener is
>> > > ready
>> > > > in every broker.
>> > > >
>> > > > Thanks,
>> > > >
>> > > > Jun
>> > > >
>> > > > On Tue, Oct 2, 2018 at 10:38 AM, Lucas Wang 
>> > > wrote:
>> > > >
>> > > > > Thanks for the further comments, Jun.
>> > > > >
>> > > > > 200. Currently in the code base, we have the term of
>> "ControlBatch"
>> > > > related
>> > > > > to
>> > > > > idempotent/transactional producing. Do you think it's a concern
>> for
>> > > > reusing
>> > > > > the term "control"?
>> > > > >
>> > > > > 201. It's not clear to me how it would work by following the same
>> > > > strategy
>> > > > > for "controller.listener.name".
>> > > > > Say the new controller has its "controller.listener.name" set to
>> the
>> > > > value
>> > > > > "CONTROLLER", and broker 1
>> > > > > has picked up this KIP by announcing
>> > > > > "endpoints": [
>> > > > > "CONTROLLER://broker1.example.com:9091",
>> > > > > "INTERNAL://broker1.example.com:9092",
>> > > > > "EXTERNAL://host1.example.com:9093"
>> > > > > ],
>> > > > >
>> > > > > while broker2 has not picked up the change, and is announcing
>> > > > > "endpoints": [
>> > > > > "INTERNAL://broker2.example.com:9092",
>> > > > > "EXTERNAL://host2.example.com:9093"
>> > > > > ],
>> > > > > to support both broker 1 for the new behavior and broker 2 for the
>> > old
>> > > > > behavior, it seems the controller must
>> > > > > check their published endpoints. Am I missing something?
>> > > > >
>> > > > > Thanks!
>> > > > > Lucas
>> > > > >
>> > > > > On Mon, Oct 1, 2018 at 6:29 PM Jun Rao  wrote:
>> > > > >
>> > > > > > Hi, Lucas,
>> > > > > >
>> > > > > > Sorry for the delay. The updated wiki looks good to me overall.
>> > Just
>> > > a
>> > > > > > couple more minor comments.
>> > > > > >
>> > > > > > 200.
>> > > kafka.network:name=ControllerRequestQueueSize,type=RequestChannel:
>> > > > > The
>> > > > > > name ControllerRequestQueueSize gives the impression that it's
>> only
>> > > for
>> > > > > the
>> > > > > > controller broker. Perhaps we can just rename all metrics and
>> > configs
>> > > > > from
>> > > > > > controller to control. This indicates that the threads and the
>> > queues
>> > > > are
>> > > > > > for the control requests (as oppose to data requests).
>> > > > > >
>> > > > > > 201. ": In this scenario, the
>> > controller
>> > > > will
>> > > > > > have the "controller.listener.name" config set to a value like
>> > > > > > "CONTROLLER", however the broker's exposed endpoints do not
>> have an
>> > > > entry
>> > > > > > corresponding to the new listener name. Hence the controller
>> should
>> > > > > > preserve the existing behavior by determining the endpoint using
>> > > > > > *inter-broker-listener-name *value. The end result should be the
>> > same
>> > > > > > behavior as today." Currently, the controller makes connections
>> > based
>> > > > on
>> > > > > > its local inter.broker.listener.name config without 

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-10-09 Thread Lucas Wang
Thanks Jun, I've updated the KIP with the new names.

Hi Joel, Becket, Dong, Ismael,
Since you've reviewed this KIP in the past, can you please review it again?
Thanks a lot!

Lucas

On Mon, Oct 8, 2018 at 6:10 PM Jun Rao  wrote:

> Hi, Lucas,
>
> Yes, the new names sound good to me.
>
> Thanks,
>
> Jun
>
> On Fri, Oct 5, 2018 at 1:12 PM, Lucas Wang  wrote:
>
> > Thanks for the suggestion, Ismael. I like it.
> >
> > Jun,
> > I'm excited to get the +1, thanks a lot!
> > Meanwhile what do you feel about renaming the metrics and config to
> >
> > ControlPlaneRequestQueueSize
> >
> > ControlPlaneNetworkProcessorIdlePercent
> >
> > ControlPlaneRequestHandlerIdlePercent
> >
> > control.plane.listener.name
> >
> > ?
> >
> >
> > Thanks,
> >
> > Lucas
> >
> > On Thu, Oct 4, 2018 at 11:38 AM Ismael Juma  wrote:
> >
> > > Have we considered control plane if we think control by itself is
> > > ambiguous? I agree with the original concern that "controller" may be
> > > confusing for something that affects all brokers.
> > >
> > > Ismael
> > >
> > >
> > > On 4 Oct 2018 11:08 am, "Lucas Wang"  wrote:
> > >
> > > Thanks Jun. I've changed the KIP with the suggested 2 step upgrade.
> > > Please take a look again when you have time.
> > >
> > > Regards,
> > > Lucas
> > >
> > >
> > > On Thu, Oct 4, 2018 at 10:06 AM Jun Rao  wrote:
> > >
> > > > Hi, Lucas,
> > > >
> > > > 200. That's a valid concern. So, we can probably just keep the
> current
> > > > name.
> > > >
> > > > 201. I am thinking that you would upgrade in the same way as changing
> > > > inter.broker.listener.name. This requires 2 rounds of rolling
> restart.
> > > In
> > > > the first round, we add the controller endpoint to the listeners w/o
> > > > setting controller.listener.name. In the second round, every broker
> > sets
> > > > controller.listener.name. At that point, the controller listener is
> > > ready
> > > > in every broker.
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > > On Tue, Oct 2, 2018 at 10:38 AM, Lucas Wang 
> > > wrote:
> > > >
> > > > > Thanks for the further comments, Jun.
> > > > >
> > > > > 200. Currently in the code base, we have the term of "ControlBatch"
> > > > related
> > > > > to
> > > > > idempotent/transactional producing. Do you think it's a concern for
> > > > reusing
> > > > > the term "control"?
> > > > >
> > > > > 201. It's not clear to me how it would work by following the same
> > > > strategy
> > > > > for "controller.listener.name".
> > > > > Say the new controller has its "controller.listener.name" set to
> the
> > > > value
> > > > > "CONTROLLER", and broker 1
> > > > > has picked up this KIP by announcing
> > > > > "endpoints": [
> > > > > "CONTROLLER://broker1.example.com:9091",
> > > > > "INTERNAL://broker1.example.com:9092",
> > > > > "EXTERNAL://host1.example.com:9093"
> > > > > ],
> > > > >
> > > > > while broker2 has not picked up the change, and is announcing
> > > > > "endpoints": [
> > > > > "INTERNAL://broker2.example.com:9092",
> > > > > "EXTERNAL://host2.example.com:9093"
> > > > > ],
> > > > > to support both broker 1 for the new behavior and broker 2 for the
> > old
> > > > > behavior, it seems the controller must
> > > > > check their published endpoints. Am I missing something?
> > > > >
> > > > > Thanks!
> > > > > Lucas
> > > > >
> > > > > On Mon, Oct 1, 2018 at 6:29 PM Jun Rao  wrote:
> > > > >
> > > > > > Hi, Lucas,
> > > > > >
> > > > > > Sorry for the delay. The updated wiki looks good to me overall.
> > Just
> > > a
> > > > > > couple more minor comments.
> > > > > >
> > > > > > 200.
> > > kafka.network:name=ControllerRequestQueueSize,type=RequestChannel:
> > > > > The
> > > > > > name ControllerRequestQueueSize gives the impression that it's
> only
> > > for
> > > > > the
> > > > > > controller broker. Perhaps we can just rename all metrics and
> > configs
> > > > > from
> > > > > > controller to control. This indicates that the threads and the
> > queues
> > > > are
> > > > > > for the control requests (as oppose to data requests).
> > > > > >
> > > > > > 201. ": In this scenario, the
> > controller
> > > > will
> > > > > > have the "controller.listener.name" config set to a value like
> > > > > > "CONTROLLER", however the broker's exposed endpoints do not have
> an
> > > > entry
> > > > > > corresponding to the new listener name. Hence the controller
> should
> > > > > > preserve the existing behavior by determining the endpoint using
> > > > > > *inter-broker-listener-name *value. The end result should be the
> > same
> > > > > > behavior as today." Currently, the controller makes connections
> > based
> > > > on
> > > > > > its local inter.broker.listener.name config without checking the
> > > > target
> > > > > > broker's ZK registration. For consistency, perhaps we can just
> > follow
> > > > the
> > > > > > same strategy for controller.listener.name. This existing
> behavior
> > > > seems
> > > > > > simpler to understand and has 

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-10-08 Thread Jun Rao
Hi, Lucas,

Yes, the new names sound good to me.

Thanks,

Jun

On Fri, Oct 5, 2018 at 1:12 PM, Lucas Wang  wrote:

> Thanks for the suggestion, Ismael. I like it.
>
> Jun,
> I'm excited to get the +1, thanks a lot!
> Meanwhile what do you feel about renaming the metrics and config to
>
> ControlPlaneRequestQueueSize
>
> ControlPlaneNetworkProcessorIdlePercent
>
> ControlPlaneRequestHandlerIdlePercent
>
> control.plane.listener.name
>
> ?
>
>
> Thanks,
>
> Lucas
>
> On Thu, Oct 4, 2018 at 11:38 AM Ismael Juma  wrote:
>
> > Have we considered control plane if we think control by itself is
> > ambiguous? I agree with the original concern that "controller" may be
> > confusing for something that affects all brokers.
> >
> > Ismael
> >
> >
> > On 4 Oct 2018 11:08 am, "Lucas Wang"  wrote:
> >
> > Thanks Jun. I've changed the KIP with the suggested 2 step upgrade.
> > Please take a look again when you have time.
> >
> > Regards,
> > Lucas
> >
> >
> > On Thu, Oct 4, 2018 at 10:06 AM Jun Rao  wrote:
> >
> > > Hi, Lucas,
> > >
> > > 200. That's a valid concern. So, we can probably just keep the current
> > > name.
> > >
> > > 201. I am thinking that you would upgrade in the same way as changing
> > > inter.broker.listener.name. This requires 2 rounds of rolling restart.
> > In
> > > the first round, we add the controller endpoint to the listeners w/o
> > > setting controller.listener.name. In the second round, every broker
> sets
> > > controller.listener.name. At that point, the controller listener is
> > ready
> > > in every broker.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Tue, Oct 2, 2018 at 10:38 AM, Lucas Wang 
> > wrote:
> > >
> > > > Thanks for the further comments, Jun.
> > > >
> > > > 200. Currently in the code base, we have the term of "ControlBatch"
> > > related
> > > > to
> > > > idempotent/transactional producing. Do you think it's a concern for
> > > reusing
> > > > the term "control"?
> > > >
> > > > 201. It's not clear to me how it would work by following the same
> > > strategy
> > > > for "controller.listener.name".
> > > > Say the new controller has its "controller.listener.name" set to the
> > > value
> > > > "CONTROLLER", and broker 1
> > > > has picked up this KIP by announcing
> > > > "endpoints": [
> > > > "CONTROLLER://broker1.example.com:9091",
> > > > "INTERNAL://broker1.example.com:9092",
> > > > "EXTERNAL://host1.example.com:9093"
> > > > ],
> > > >
> > > > while broker2 has not picked up the change, and is announcing
> > > > "endpoints": [
> > > > "INTERNAL://broker2.example.com:9092",
> > > > "EXTERNAL://host2.example.com:9093"
> > > > ],
> > > > to support both broker 1 for the new behavior and broker 2 for the
> old
> > > > behavior, it seems the controller must
> > > > check their published endpoints. Am I missing something?
> > > >
> > > > Thanks!
> > > > Lucas
> > > >
> > > > On Mon, Oct 1, 2018 at 6:29 PM Jun Rao  wrote:
> > > >
> > > > > Hi, Lucas,
> > > > >
> > > > > Sorry for the delay. The updated wiki looks good to me overall.
> Just
> > a
> > > > > couple more minor comments.
> > > > >
> > > > > 200.
> > kafka.network:name=ControllerRequestQueueSize,type=RequestChannel:
> > > > The
> > > > > name ControllerRequestQueueSize gives the impression that it's only
> > for
> > > > the
> > > > > controller broker. Perhaps we can just rename all metrics and
> configs
> > > > from
> > > > > controller to control. This indicates that the threads and the
> queues
> > > are
> > > > > for the control requests (as oppose to data requests).
> > > > >
> > > > > 201. ": In this scenario, the
> controller
> > > will
> > > > > have the "controller.listener.name" config set to a value like
> > > > > "CONTROLLER", however the broker's exposed endpoints do not have an
> > > entry
> > > > > corresponding to the new listener name. Hence the controller should
> > > > > preserve the existing behavior by determining the endpoint using
> > > > > *inter-broker-listener-name *value. The end result should be the
> same
> > > > > behavior as today." Currently, the controller makes connections
> based
> > > on
> > > > > its local inter.broker.listener.name config without checking the
> > > target
> > > > > broker's ZK registration. For consistency, perhaps we can just
> follow
> > > the
> > > > > same strategy for controller.listener.name. This existing behavior
> > > seems
> > > > > simpler to understand and has the benefit of catching inconsistent
> > > > configs
> > > > > across brokers.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jun
> > > > >
> > > > > On Mon, Oct 1, 2018 at 8:43 AM, Lucas Wang 
> > > > wrote:
> > > > >
> > > > > > Hi Jun,
> > > > > >
> > > > > > Sorry to bother you again. Can you please take a look at the wiki
> > > again
> > > > > > when you have time?
> > > > > >
> > > > > > Thanks a lot!
> > > > > > Lucas
> > > > > >
> > > > > > On Wed, Sep 19, 2018 at 3:57 PM Lucas Wang <
> lucasatu...@gmail.com>
> > > > > wrote:
> > 

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-10-05 Thread Lucas Wang
Thanks for the suggestion, Ismael. I like it.

Jun,
I'm excited to get the +1, thanks a lot!
Meanwhile what do you feel about renaming the metrics and config to

ControlPlaneRequestQueueSize

ControlPlaneNetworkProcessorIdlePercent

ControlPlaneRequestHandlerIdlePercent

control.plane.listener.name

?


Thanks,

Lucas

On Thu, Oct 4, 2018 at 11:38 AM Ismael Juma  wrote:

> Have we considered control plane if we think control by itself is
> ambiguous? I agree with the original concern that "controller" may be
> confusing for something that affects all brokers.
>
> Ismael
>
>
> On 4 Oct 2018 11:08 am, "Lucas Wang"  wrote:
>
> Thanks Jun. I've changed the KIP with the suggested 2 step upgrade.
> Please take a look again when you have time.
>
> Regards,
> Lucas
>
>
> On Thu, Oct 4, 2018 at 10:06 AM Jun Rao  wrote:
>
> > Hi, Lucas,
> >
> > 200. That's a valid concern. So, we can probably just keep the current
> > name.
> >
> > 201. I am thinking that you would upgrade in the same way as changing
> > inter.broker.listener.name. This requires 2 rounds of rolling restart.
> In
> > the first round, we add the controller endpoint to the listeners w/o
> > setting controller.listener.name. In the second round, every broker sets
> > controller.listener.name. At that point, the controller listener is
> ready
> > in every broker.
> >
> > Thanks,
> >
> > Jun
> >
> > On Tue, Oct 2, 2018 at 10:38 AM, Lucas Wang 
> wrote:
> >
> > > Thanks for the further comments, Jun.
> > >
> > > 200. Currently in the code base, we have the term of "ControlBatch"
> > related
> > > to
> > > idempotent/transactional producing. Do you think it's a concern for
> > reusing
> > > the term "control"?
> > >
> > > 201. It's not clear to me how it would work by following the same
> > strategy
> > > for "controller.listener.name".
> > > Say the new controller has its "controller.listener.name" set to the
> > value
> > > "CONTROLLER", and broker 1
> > > has picked up this KIP by announcing
> > > "endpoints": [
> > > "CONTROLLER://broker1.example.com:9091",
> > > "INTERNAL://broker1.example.com:9092",
> > > "EXTERNAL://host1.example.com:9093"
> > > ],
> > >
> > > while broker2 has not picked up the change, and is announcing
> > > "endpoints": [
> > > "INTERNAL://broker2.example.com:9092",
> > > "EXTERNAL://host2.example.com:9093"
> > > ],
> > > to support both broker 1 for the new behavior and broker 2 for the old
> > > behavior, it seems the controller must
> > > check their published endpoints. Am I missing something?
> > >
> > > Thanks!
> > > Lucas
> > >
> > > On Mon, Oct 1, 2018 at 6:29 PM Jun Rao  wrote:
> > >
> > > > Hi, Lucas,
> > > >
> > > > Sorry for the delay. The updated wiki looks good to me overall. Just
> a
> > > > couple more minor comments.
> > > >
> > > > 200.
> kafka.network:name=ControllerRequestQueueSize,type=RequestChannel:
> > > The
> > > > name ControllerRequestQueueSize gives the impression that it's only
> for
> > > the
> > > > controller broker. Perhaps we can just rename all metrics and configs
> > > from
> > > > controller to control. This indicates that the threads and the queues
> > are
> > > > for the control requests (as oppose to data requests).
> > > >
> > > > 201. ": In this scenario, the controller
> > will
> > > > have the "controller.listener.name" config set to a value like
> > > > "CONTROLLER", however the broker's exposed endpoints do not have an
> > entry
> > > > corresponding to the new listener name. Hence the controller should
> > > > preserve the existing behavior by determining the endpoint using
> > > > *inter-broker-listener-name *value. The end result should be the same
> > > > behavior as today." Currently, the controller makes connections based
> > on
> > > > its local inter.broker.listener.name config without checking the
> > target
> > > > broker's ZK registration. For consistency, perhaps we can just follow
> > the
> > > > same strategy for controller.listener.name. This existing behavior
> > seems
> > > > simpler to understand and has the benefit of catching inconsistent
> > > configs
> > > > across brokers.
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > > On Mon, Oct 1, 2018 at 8:43 AM, Lucas Wang 
> > > wrote:
> > > >
> > > > > Hi Jun,
> > > > >
> > > > > Sorry to bother you again. Can you please take a look at the wiki
> > again
> > > > > when you have time?
> > > > >
> > > > > Thanks a lot!
> > > > > Lucas
> > > > >
> > > > > On Wed, Sep 19, 2018 at 3:57 PM Lucas Wang 
> > > > wrote:
> > > > >
> > > > > > Hi Jun,
> > > > > >
> > > > > > Thanks a lot for the detailed explanation.
> > > > > > I've restored the wiki to a previous version that does not
> require
> > > > config
> > > > > > changes,
> > > > > > and keeps the current behavior with the proposed changes turned
> off
> > > by
> > > > > > default.
> > > > > > I'd appreciate it if you can review it again.
> > > > > >
> > > > > > Thanks!
> > > > > > Lucas
> > > > > >
> > > > > > On 

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-10-05 Thread Jun Rao
Hi, Lucas,

Thanks for the updated KIP. About the name of the metric, since it's tagged
by type=RequestChannel, it's probably fine to use the proposed name. So, +1
for the KIP.

Jun

On Thu, Oct 4, 2018 at 10:31 AM, Lucas Wang  wrote:

> Thanks Jun. I've changed the KIP with the suggested 2 step upgrade.
> Please take a look again when you have time.
>
> Regards,
> Lucas
>
> On Thu, Oct 4, 2018 at 10:06 AM Jun Rao  wrote:
>
> > Hi, Lucas,
> >
> > 200. That's a valid concern. So, we can probably just keep the current
> > name.
> >
> > 201. I am thinking that you would upgrade in the same way as changing
> > inter.broker.listener.name. This requires 2 rounds of rolling restart.
> In
> > the first round, we add the controller endpoint to the listeners w/o
> > setting controller.listener.name. In the second round, every broker sets
> > controller.listener.name. At that point, the controller listener is
> ready
> > in every broker.
> >
> > Thanks,
> >
> > Jun
> >
> > On Tue, Oct 2, 2018 at 10:38 AM, Lucas Wang 
> wrote:
> >
> > > Thanks for the further comments, Jun.
> > >
> > > 200. Currently in the code base, we have the term of "ControlBatch"
> > related
> > > to
> > > idempotent/transactional producing. Do you think it's a concern for
> > reusing
> > > the term "control"?
> > >
> > > 201. It's not clear to me how it would work by following the same
> > strategy
> > > for "controller.listener.name".
> > > Say the new controller has its "controller.listener.name" set to the
> > value
> > > "CONTROLLER", and broker 1
> > > has picked up this KIP by announcing
> > > "endpoints": [
> > > "CONTROLLER://broker1.example.com:9091",
> > > "INTERNAL://broker1.example.com:9092",
> > > "EXTERNAL://host1.example.com:9093"
> > > ],
> > >
> > > while broker2 has not picked up the change, and is announcing
> > > "endpoints": [
> > > "INTERNAL://broker2.example.com:9092",
> > > "EXTERNAL://host2.example.com:9093"
> > > ],
> > > to support both broker 1 for the new behavior and broker 2 for the old
> > > behavior, it seems the controller must
> > > check their published endpoints. Am I missing something?
> > >
> > > Thanks!
> > > Lucas
> > >
> > > On Mon, Oct 1, 2018 at 6:29 PM Jun Rao  wrote:
> > >
> > > > Hi, Lucas,
> > > >
> > > > Sorry for the delay. The updated wiki looks good to me overall. Just
> a
> > > > couple more minor comments.
> > > >
> > > > 200. kafka.network:name=ControllerRequestQueueSize,
> type=RequestChannel:
> > > The
> > > > name ControllerRequestQueueSize gives the impression that it's only
> for
> > > the
> > > > controller broker. Perhaps we can just rename all metrics and configs
> > > from
> > > > controller to control. This indicates that the threads and the queues
> > are
> > > > for the control requests (as oppose to data requests).
> > > >
> > > > 201. ": In this scenario, the controller
> > will
> > > > have the "controller.listener.name" config set to a value like
> > > > "CONTROLLER", however the broker's exposed endpoints do not have an
> > entry
> > > > corresponding to the new listener name. Hence the controller should
> > > > preserve the existing behavior by determining the endpoint using
> > > > *inter-broker-listener-name *value. The end result should be the same
> > > > behavior as today." Currently, the controller makes connections based
> > on
> > > > its local inter.broker.listener.name config without checking the
> > target
> > > > broker's ZK registration. For consistency, perhaps we can just follow
> > the
> > > > same strategy for controller.listener.name. This existing behavior
> > seems
> > > > simpler to understand and has the benefit of catching inconsistent
> > > configs
> > > > across brokers.
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > > On Mon, Oct 1, 2018 at 8:43 AM, Lucas Wang 
> > > wrote:
> > > >
> > > > > Hi Jun,
> > > > >
> > > > > Sorry to bother you again. Can you please take a look at the wiki
> > again
> > > > > when you have time?
> > > > >
> > > > > Thanks a lot!
> > > > > Lucas
> > > > >
> > > > > On Wed, Sep 19, 2018 at 3:57 PM Lucas Wang 
> > > > wrote:
> > > > >
> > > > > > Hi Jun,
> > > > > >
> > > > > > Thanks a lot for the detailed explanation.
> > > > > > I've restored the wiki to a previous version that does not
> require
> > > > config
> > > > > > changes,
> > > > > > and keeps the current behavior with the proposed changes turned
> off
> > > by
> > > > > > default.
> > > > > > I'd appreciate it if you can review it again.
> > > > > >
> > > > > > Thanks!
> > > > > > Lucas
> > > > > >
> > > > > > On Tue, Sep 18, 2018 at 1:48 PM Jun Rao 
> wrote:
> > > > > >
> > > > > >> Hi, Lucas,
> > > > > >>
> > > > > >> When upgrading to a minor release, I think the expectation is
> > that a
> > > > > user
> > > > > >> wouldn't need to make any config changes, other than the usual
> > > > > >> inter.broker.protocol. If we require other config changes during
> > an
> > > > > >> upgrade, then it's 

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-10-04 Thread Ismael Juma
Have we considered control plane if we think control by itself is
ambiguous? I agree with the original concern that "controller" may be
confusing for something that affects all brokers.

Ismael


On 4 Oct 2018 11:08 am, "Lucas Wang"  wrote:

Thanks Jun. I've changed the KIP with the suggested 2 step upgrade.
Please take a look again when you have time.

Regards,
Lucas


On Thu, Oct 4, 2018 at 10:06 AM Jun Rao  wrote:

> Hi, Lucas,
>
> 200. That's a valid concern. So, we can probably just keep the current
> name.
>
> 201. I am thinking that you would upgrade in the same way as changing
> inter.broker.listener.name. This requires 2 rounds of rolling restart. In
> the first round, we add the controller endpoint to the listeners w/o
> setting controller.listener.name. In the second round, every broker sets
> controller.listener.name. At that point, the controller listener is ready
> in every broker.
>
> Thanks,
>
> Jun
>
> On Tue, Oct 2, 2018 at 10:38 AM, Lucas Wang  wrote:
>
> > Thanks for the further comments, Jun.
> >
> > 200. Currently in the code base, we have the term of "ControlBatch"
> related
> > to
> > idempotent/transactional producing. Do you think it's a concern for
> reusing
> > the term "control"?
> >
> > 201. It's not clear to me how it would work by following the same
> strategy
> > for "controller.listener.name".
> > Say the new controller has its "controller.listener.name" set to the
> value
> > "CONTROLLER", and broker 1
> > has picked up this KIP by announcing
> > "endpoints": [
> > "CONTROLLER://broker1.example.com:9091",
> > "INTERNAL://broker1.example.com:9092",
> > "EXTERNAL://host1.example.com:9093"
> > ],
> >
> > while broker2 has not picked up the change, and is announcing
> > "endpoints": [
> > "INTERNAL://broker2.example.com:9092",
> > "EXTERNAL://host2.example.com:9093"
> > ],
> > to support both broker 1 for the new behavior and broker 2 for the old
> > behavior, it seems the controller must
> > check their published endpoints. Am I missing something?
> >
> > Thanks!
> > Lucas
> >
> > On Mon, Oct 1, 2018 at 6:29 PM Jun Rao  wrote:
> >
> > > Hi, Lucas,
> > >
> > > Sorry for the delay. The updated wiki looks good to me overall. Just a
> > > couple more minor comments.
> > >
> > > 200.
kafka.network:name=ControllerRequestQueueSize,type=RequestChannel:
> > The
> > > name ControllerRequestQueueSize gives the impression that it's only
for
> > the
> > > controller broker. Perhaps we can just rename all metrics and configs
> > from
> > > controller to control. This indicates that the threads and the queues
> are
> > > for the control requests (as oppose to data requests).
> > >
> > > 201. ": In this scenario, the controller
> will
> > > have the "controller.listener.name" config set to a value like
> > > "CONTROLLER", however the broker's exposed endpoints do not have an
> entry
> > > corresponding to the new listener name. Hence the controller should
> > > preserve the existing behavior by determining the endpoint using
> > > *inter-broker-listener-name *value. The end result should be the same
> > > behavior as today." Currently, the controller makes connections based
> on
> > > its local inter.broker.listener.name config without checking the
> target
> > > broker's ZK registration. For consistency, perhaps we can just follow
> the
> > > same strategy for controller.listener.name. This existing behavior
> seems
> > > simpler to understand and has the benefit of catching inconsistent
> > configs
> > > across brokers.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Mon, Oct 1, 2018 at 8:43 AM, Lucas Wang 
> > wrote:
> > >
> > > > Hi Jun,
> > > >
> > > > Sorry to bother you again. Can you please take a look at the wiki
> again
> > > > when you have time?
> > > >
> > > > Thanks a lot!
> > > > Lucas
> > > >
> > > > On Wed, Sep 19, 2018 at 3:57 PM Lucas Wang 
> > > wrote:
> > > >
> > > > > Hi Jun,
> > > > >
> > > > > Thanks a lot for the detailed explanation.
> > > > > I've restored the wiki to a previous version that does not require
> > > config
> > > > > changes,
> > > > > and keeps the current behavior with the proposed changes turned
off
> > by
> > > > > default.
> > > > > I'd appreciate it if you can review it again.
> > > > >
> > > > > Thanks!
> > > > > Lucas
> > > > >
> > > > > On Tue, Sep 18, 2018 at 1:48 PM Jun Rao  wrote:
> > > > >
> > > > >> Hi, Lucas,
> > > > >>
> > > > >> When upgrading to a minor release, I think the expectation is
> that a
> > > > user
> > > > >> wouldn't need to make any config changes, other than the usual
> > > > >> inter.broker.protocol. If we require other config changes during
> an
> > > > >> upgrade, then it's probably better to do that in a major release.
> > > > >>
> > > > >> Regarding your proposal, I think removing host/advertised_host in
> > > favor
> > > > of
> > > > >> listeners:advertised_listeners seems useful regardless of this
> KIP.
> > > > >> However, that can probably wait until a major 

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-10-04 Thread Lucas Wang
Thanks Jun. I've changed the KIP with the suggested 2 step upgrade.
Please take a look again when you have time.

Regards,
Lucas

On Thu, Oct 4, 2018 at 10:06 AM Jun Rao  wrote:

> Hi, Lucas,
>
> 200. That's a valid concern. So, we can probably just keep the current
> name.
>
> 201. I am thinking that you would upgrade in the same way as changing
> inter.broker.listener.name. This requires 2 rounds of rolling restart. In
> the first round, we add the controller endpoint to the listeners w/o
> setting controller.listener.name. In the second round, every broker sets
> controller.listener.name. At that point, the controller listener is ready
> in every broker.
>
> Thanks,
>
> Jun
>
> On Tue, Oct 2, 2018 at 10:38 AM, Lucas Wang  wrote:
>
> > Thanks for the further comments, Jun.
> >
> > 200. Currently in the code base, we have the term of "ControlBatch"
> related
> > to
> > idempotent/transactional producing. Do you think it's a concern for
> reusing
> > the term "control"?
> >
> > 201. It's not clear to me how it would work by following the same
> strategy
> > for "controller.listener.name".
> > Say the new controller has its "controller.listener.name" set to the
> value
> > "CONTROLLER", and broker 1
> > has picked up this KIP by announcing
> > "endpoints": [
> > "CONTROLLER://broker1.example.com:9091",
> > "INTERNAL://broker1.example.com:9092",
> > "EXTERNAL://host1.example.com:9093"
> > ],
> >
> > while broker2 has not picked up the change, and is announcing
> > "endpoints": [
> > "INTERNAL://broker2.example.com:9092",
> > "EXTERNAL://host2.example.com:9093"
> > ],
> > to support both broker 1 for the new behavior and broker 2 for the old
> > behavior, it seems the controller must
> > check their published endpoints. Am I missing something?
> >
> > Thanks!
> > Lucas
> >
> > On Mon, Oct 1, 2018 at 6:29 PM Jun Rao  wrote:
> >
> > > Hi, Lucas,
> > >
> > > Sorry for the delay. The updated wiki looks good to me overall. Just a
> > > couple more minor comments.
> > >
> > > 200. kafka.network:name=ControllerRequestQueueSize,type=RequestChannel:
> > The
> > > name ControllerRequestQueueSize gives the impression that it's only for
> > the
> > > controller broker. Perhaps we can just rename all metrics and configs
> > from
> > > controller to control. This indicates that the threads and the queues
> are
> > > for the control requests (as oppose to data requests).
> > >
> > > 201. ": In this scenario, the controller
> will
> > > have the "controller.listener.name" config set to a value like
> > > "CONTROLLER", however the broker's exposed endpoints do not have an
> entry
> > > corresponding to the new listener name. Hence the controller should
> > > preserve the existing behavior by determining the endpoint using
> > > *inter-broker-listener-name *value. The end result should be the same
> > > behavior as today." Currently, the controller makes connections based
> on
> > > its local inter.broker.listener.name config without checking the
> target
> > > broker's ZK registration. For consistency, perhaps we can just follow
> the
> > > same strategy for controller.listener.name. This existing behavior
> seems
> > > simpler to understand and has the benefit of catching inconsistent
> > configs
> > > across brokers.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Mon, Oct 1, 2018 at 8:43 AM, Lucas Wang 
> > wrote:
> > >
> > > > Hi Jun,
> > > >
> > > > Sorry to bother you again. Can you please take a look at the wiki
> again
> > > > when you have time?
> > > >
> > > > Thanks a lot!
> > > > Lucas
> > > >
> > > > On Wed, Sep 19, 2018 at 3:57 PM Lucas Wang 
> > > wrote:
> > > >
> > > > > Hi Jun,
> > > > >
> > > > > Thanks a lot for the detailed explanation.
> > > > > I've restored the wiki to a previous version that does not require
> > > config
> > > > > changes,
> > > > > and keeps the current behavior with the proposed changes turned off
> > by
> > > > > default.
> > > > > I'd appreciate it if you can review it again.
> > > > >
> > > > > Thanks!
> > > > > Lucas
> > > > >
> > > > > On Tue, Sep 18, 2018 at 1:48 PM Jun Rao  wrote:
> > > > >
> > > > >> Hi, Lucas,
> > > > >>
> > > > >> When upgrading to a minor release, I think the expectation is
> that a
> > > > user
> > > > >> wouldn't need to make any config changes, other than the usual
> > > > >> inter.broker.protocol. If we require other config changes during
> an
> > > > >> upgrade, then it's probably better to do that in a major release.
> > > > >>
> > > > >> Regarding your proposal, I think removing host/advertised_host in
> > > favor
> > > > of
> > > > >> listeners:advertised_listeners seems useful regardless of this
> KIP.
> > > > >> However, that can probably wait until a major release.
> > > > >>
> > > > >> As for the controller listener, I am not sure if one has to set
> it.
> > To
> > > > >> make
> > > > >> a cluster healthy, one sort of have to make sure that the request
> > > queue
> > > > is
> > > > >> never 

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-10-04 Thread Jun Rao
Hi, Lucas,

200. That's a valid concern. So, we can probably just keep the current name.

201. I am thinking that you would upgrade in the same way as changing
inter.broker.listener.name. This requires 2 rounds of rolling restart. In
the first round, we add the controller endpoint to the listeners w/o
setting controller.listener.name. In the second round, every broker sets
controller.listener.name. At that point, the controller listener is ready
in every broker.

Thanks,

Jun

On Tue, Oct 2, 2018 at 10:38 AM, Lucas Wang  wrote:

> Thanks for the further comments, Jun.
>
> 200. Currently in the code base, we have the term of "ControlBatch" related
> to
> idempotent/transactional producing. Do you think it's a concern for reusing
> the term "control"?
>
> 201. It's not clear to me how it would work by following the same strategy
> for "controller.listener.name".
> Say the new controller has its "controller.listener.name" set to the value
> "CONTROLLER", and broker 1
> has picked up this KIP by announcing
> "endpoints": [
> "CONTROLLER://broker1.example.com:9091",
> "INTERNAL://broker1.example.com:9092",
> "EXTERNAL://host1.example.com:9093"
> ],
>
> while broker2 has not picked up the change, and is announcing
> "endpoints": [
> "INTERNAL://broker2.example.com:9092",
> "EXTERNAL://host2.example.com:9093"
> ],
> to support both broker 1 for the new behavior and broker 2 for the old
> behavior, it seems the controller must
> check their published endpoints. Am I missing something?
>
> Thanks!
> Lucas
>
> On Mon, Oct 1, 2018 at 6:29 PM Jun Rao  wrote:
>
> > Hi, Lucas,
> >
> > Sorry for the delay. The updated wiki looks good to me overall. Just a
> > couple more minor comments.
> >
> > 200. kafka.network:name=ControllerRequestQueueSize,type=RequestChannel:
> The
> > name ControllerRequestQueueSize gives the impression that it's only for
> the
> > controller broker. Perhaps we can just rename all metrics and configs
> from
> > controller to control. This indicates that the threads and the queues are
> > for the control requests (as oppose to data requests).
> >
> > 201. ": In this scenario, the controller will
> > have the "controller.listener.name" config set to a value like
> > "CONTROLLER", however the broker's exposed endpoints do not have an entry
> > corresponding to the new listener name. Hence the controller should
> > preserve the existing behavior by determining the endpoint using
> > *inter-broker-listener-name *value. The end result should be the same
> > behavior as today." Currently, the controller makes connections based on
> > its local inter.broker.listener.name config without checking the target
> > broker's ZK registration. For consistency, perhaps we can just follow the
> > same strategy for controller.listener.name. This existing behavior seems
> > simpler to understand and has the benefit of catching inconsistent
> configs
> > across brokers.
> >
> > Thanks,
> >
> > Jun
> >
> > On Mon, Oct 1, 2018 at 8:43 AM, Lucas Wang 
> wrote:
> >
> > > Hi Jun,
> > >
> > > Sorry to bother you again. Can you please take a look at the wiki again
> > > when you have time?
> > >
> > > Thanks a lot!
> > > Lucas
> > >
> > > On Wed, Sep 19, 2018 at 3:57 PM Lucas Wang 
> > wrote:
> > >
> > > > Hi Jun,
> > > >
> > > > Thanks a lot for the detailed explanation.
> > > > I've restored the wiki to a previous version that does not require
> > config
> > > > changes,
> > > > and keeps the current behavior with the proposed changes turned off
> by
> > > > default.
> > > > I'd appreciate it if you can review it again.
> > > >
> > > > Thanks!
> > > > Lucas
> > > >
> > > > On Tue, Sep 18, 2018 at 1:48 PM Jun Rao  wrote:
> > > >
> > > >> Hi, Lucas,
> > > >>
> > > >> When upgrading to a minor release, I think the expectation is that a
> > > user
> > > >> wouldn't need to make any config changes, other than the usual
> > > >> inter.broker.protocol. If we require other config changes during an
> > > >> upgrade, then it's probably better to do that in a major release.
> > > >>
> > > >> Regarding your proposal, I think removing host/advertised_host in
> > favor
> > > of
> > > >> listeners:advertised_listeners seems useful regardless of this KIP.
> > > >> However, that can probably wait until a major release.
> > > >>
> > > >> As for the controller listener, I am not sure if one has to set it.
> To
> > > >> make
> > > >> a cluster healthy, one sort of have to make sure that the request
> > queue
> > > is
> > > >> never full and no request will be sitting in the request queue for
> > long.
> > > >> If
> > > >> one does that, setting the controller listener may not be necessary.
> > On
> > > >> the
> > > >> flip side, even if one sets the controller listener, but the request
> > > queue
> > > >> and the request time for the data part are still high, the cluster
> may
> > > >> still not be healthy. Given that we have already started the 2.1
> > release
> > > >> planning, perhaps we can 

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-10-02 Thread Lucas Wang
Thanks for the further comments, Jun.

200. Currently in the code base, we have the term of "ControlBatch" related
to
idempotent/transactional producing. Do you think it's a concern for reusing
the term "control"?

201. It's not clear to me how it would work by following the same strategy
for "controller.listener.name".
Say the new controller has its "controller.listener.name" set to the value
"CONTROLLER", and broker 1
has picked up this KIP by announcing
"endpoints": [
"CONTROLLER://broker1.example.com:9091",
"INTERNAL://broker1.example.com:9092",
"EXTERNAL://host1.example.com:9093"
],

while broker2 has not picked up the change, and is announcing
"endpoints": [
"INTERNAL://broker2.example.com:9092",
"EXTERNAL://host2.example.com:9093"
],
to support both broker 1 for the new behavior and broker 2 for the old
behavior, it seems the controller must
check their published endpoints. Am I missing something?

Thanks!
Lucas

On Mon, Oct 1, 2018 at 6:29 PM Jun Rao  wrote:

> Hi, Lucas,
>
> Sorry for the delay. The updated wiki looks good to me overall. Just a
> couple more minor comments.
>
> 200. kafka.network:name=ControllerRequestQueueSize,type=RequestChannel: The
> name ControllerRequestQueueSize gives the impression that it's only for the
> controller broker. Perhaps we can just rename all metrics and configs from
> controller to control. This indicates that the threads and the queues are
> for the control requests (as oppose to data requests).
>
> 201. ": In this scenario, the controller will
> have the "controller.listener.name" config set to a value like
> "CONTROLLER", however the broker's exposed endpoints do not have an entry
> corresponding to the new listener name. Hence the controller should
> preserve the existing behavior by determining the endpoint using
> *inter-broker-listener-name *value. The end result should be the same
> behavior as today." Currently, the controller makes connections based on
> its local inter.broker.listener.name config without checking the target
> broker's ZK registration. For consistency, perhaps we can just follow the
> same strategy for controller.listener.name. This existing behavior seems
> simpler to understand and has the benefit of catching inconsistent configs
> across brokers.
>
> Thanks,
>
> Jun
>
> On Mon, Oct 1, 2018 at 8:43 AM, Lucas Wang  wrote:
>
> > Hi Jun,
> >
> > Sorry to bother you again. Can you please take a look at the wiki again
> > when you have time?
> >
> > Thanks a lot!
> > Lucas
> >
> > On Wed, Sep 19, 2018 at 3:57 PM Lucas Wang 
> wrote:
> >
> > > Hi Jun,
> > >
> > > Thanks a lot for the detailed explanation.
> > > I've restored the wiki to a previous version that does not require
> config
> > > changes,
> > > and keeps the current behavior with the proposed changes turned off by
> > > default.
> > > I'd appreciate it if you can review it again.
> > >
> > > Thanks!
> > > Lucas
> > >
> > > On Tue, Sep 18, 2018 at 1:48 PM Jun Rao  wrote:
> > >
> > >> Hi, Lucas,
> > >>
> > >> When upgrading to a minor release, I think the expectation is that a
> > user
> > >> wouldn't need to make any config changes, other than the usual
> > >> inter.broker.protocol. If we require other config changes during an
> > >> upgrade, then it's probably better to do that in a major release.
> > >>
> > >> Regarding your proposal, I think removing host/advertised_host in
> favor
> > of
> > >> listeners:advertised_listeners seems useful regardless of this KIP.
> > >> However, that can probably wait until a major release.
> > >>
> > >> As for the controller listener, I am not sure if one has to set it. To
> > >> make
> > >> a cluster healthy, one sort of have to make sure that the request
> queue
> > is
> > >> never full and no request will be sitting in the request queue for
> long.
> > >> If
> > >> one does that, setting the controller listener may not be necessary.
> On
> > >> the
> > >> flip side, even if one sets the controller listener, but the request
> > queue
> > >> and the request time for the data part are still high, the cluster may
> > >> still not be healthy. Given that we have already started the 2.1
> release
> > >> planning, perhaps we can start with not requiring the controller
> > listener.
> > >> If this is indeed something that everyone wants to set, we can make
> it a
> > >> required config in a major release.
> > >>
> > >> Thanks,
> > >>
> > >> Jun
> > >>
> > >> On Tue, Sep 11, 2018 at 3:46 PM, Lucas Wang 
> > >> wrote:
> > >>
> > >> > @Jun Rao 
> > >> >
> > >> > I made the recent config changes after thinking about the default
> > >> behavior
> > >> > for adopting this KIP.
> > >> > I think there are basically two options:
> > >> > 1. By default, the behavior proposed in this KIP is turned off, and
> > >> > operators can turn it
> > >> > on by adding the "controller.listener.name" config and entries in
> the
> > >> > "listeners" and "advertised.listeners" list.
> > >> > If no 

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-10-01 Thread Jun Rao
Hi, Lucas,

Sorry for the delay. The updated wiki looks good to me overall. Just a
couple more minor comments.

200. kafka.network:name=ControllerRequestQueueSize,type=RequestChannel: The
name ControllerRequestQueueSize gives the impression that it's only for the
controller broker. Perhaps we can just rename all metrics and configs from
controller to control. This indicates that the threads and the queues are
for the control requests (as oppose to data requests).

201. ": In this scenario, the controller will
have the "controller.listener.name" config set to a value like
"CONTROLLER", however the broker's exposed endpoints do not have an entry
corresponding to the new listener name. Hence the controller should
preserve the existing behavior by determining the endpoint using
*inter-broker-listener-name *value. The end result should be the same
behavior as today." Currently, the controller makes connections based on
its local inter.broker.listener.name config without checking the target
broker's ZK registration. For consistency, perhaps we can just follow the
same strategy for controller.listener.name. This existing behavior seems
simpler to understand and has the benefit of catching inconsistent configs
across brokers.

Thanks,

Jun

On Mon, Oct 1, 2018 at 8:43 AM, Lucas Wang  wrote:

> Hi Jun,
>
> Sorry to bother you again. Can you please take a look at the wiki again
> when you have time?
>
> Thanks a lot!
> Lucas
>
> On Wed, Sep 19, 2018 at 3:57 PM Lucas Wang  wrote:
>
> > Hi Jun,
> >
> > Thanks a lot for the detailed explanation.
> > I've restored the wiki to a previous version that does not require config
> > changes,
> > and keeps the current behavior with the proposed changes turned off by
> > default.
> > I'd appreciate it if you can review it again.
> >
> > Thanks!
> > Lucas
> >
> > On Tue, Sep 18, 2018 at 1:48 PM Jun Rao  wrote:
> >
> >> Hi, Lucas,
> >>
> >> When upgrading to a minor release, I think the expectation is that a
> user
> >> wouldn't need to make any config changes, other than the usual
> >> inter.broker.protocol. If we require other config changes during an
> >> upgrade, then it's probably better to do that in a major release.
> >>
> >> Regarding your proposal, I think removing host/advertised_host in favor
> of
> >> listeners:advertised_listeners seems useful regardless of this KIP.
> >> However, that can probably wait until a major release.
> >>
> >> As for the controller listener, I am not sure if one has to set it. To
> >> make
> >> a cluster healthy, one sort of have to make sure that the request queue
> is
> >> never full and no request will be sitting in the request queue for long.
> >> If
> >> one does that, setting the controller listener may not be necessary. On
> >> the
> >> flip side, even if one sets the controller listener, but the request
> queue
> >> and the request time for the data part are still high, the cluster may
> >> still not be healthy. Given that we have already started the 2.1 release
> >> planning, perhaps we can start with not requiring the controller
> listener.
> >> If this is indeed something that everyone wants to set, we can make it a
> >> required config in a major release.
> >>
> >> Thanks,
> >>
> >> Jun
> >>
> >> On Tue, Sep 11, 2018 at 3:46 PM, Lucas Wang 
> >> wrote:
> >>
> >> > @Jun Rao 
> >> >
> >> > I made the recent config changes after thinking about the default
> >> behavior
> >> > for adopting this KIP.
> >> > I think there are basically two options:
> >> > 1. By default, the behavior proposed in this KIP is turned off, and
> >> > operators can turn it
> >> > on by adding the "controller.listener.name" config and entries in the
> >> > "listeners" and "advertised.listeners" list.
> >> > If no "controller.listener.name" is added, it'll be the *same as*
> the "
> >> > inter.broker.listener.name",
> >> > and the proposed feature is effectively turned off.
> >> > This has been the assumption in the KIP writeup before.
> >> >
> >> > 2. By default, the behavior proposed in this KIP is turned on, and
> >> > operators are forced to
> >> > recognize the proposed change if their "listeners" config is set (this
> >> is
> >> > most likely in production environments),
> >> > by allocating a new port for controller connections, and adding a new
> >> > endpoint to the "listeners" config.
> >> > For cases where "listeners" is not set explicitly,
> >> > there needs to be a default value for it that includes the controller
> >> > listener name,
> >> > e.g. "PLAINTEXT://:9092,CONTROLLER://:9091"
> >> >
> >> > I chose to go with option 2 since as author of this KIP,
> >> > I naturally think in the long run, it's worth the effort to adopt this
> >> > feature,
> >> > in order to prevent issues under circumstances listed in the
> motivation
> >> > section.
> >> >
> >> > 100, following the argument above, I want to enforce the separation
> >> > between controller
> >> > and data plane requests. Hence the "controller.listener.name" should
> >> > never be the 

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-10-01 Thread Lucas Wang
Hi Jun,

Sorry to bother you again. Can you please take a look at the wiki again
when you have time?

Thanks a lot!
Lucas

On Wed, Sep 19, 2018 at 3:57 PM Lucas Wang  wrote:

> Hi Jun,
>
> Thanks a lot for the detailed explanation.
> I've restored the wiki to a previous version that does not require config
> changes,
> and keeps the current behavior with the proposed changes turned off by
> default.
> I'd appreciate it if you can review it again.
>
> Thanks!
> Lucas
>
> On Tue, Sep 18, 2018 at 1:48 PM Jun Rao  wrote:
>
>> Hi, Lucas,
>>
>> When upgrading to a minor release, I think the expectation is that a user
>> wouldn't need to make any config changes, other than the usual
>> inter.broker.protocol. If we require other config changes during an
>> upgrade, then it's probably better to do that in a major release.
>>
>> Regarding your proposal, I think removing host/advertised_host in favor of
>> listeners:advertised_listeners seems useful regardless of this KIP.
>> However, that can probably wait until a major release.
>>
>> As for the controller listener, I am not sure if one has to set it. To
>> make
>> a cluster healthy, one sort of have to make sure that the request queue is
>> never full and no request will be sitting in the request queue for long.
>> If
>> one does that, setting the controller listener may not be necessary. On
>> the
>> flip side, even if one sets the controller listener, but the request queue
>> and the request time for the data part are still high, the cluster may
>> still not be healthy. Given that we have already started the 2.1 release
>> planning, perhaps we can start with not requiring the controller listener.
>> If this is indeed something that everyone wants to set, we can make it a
>> required config in a major release.
>>
>> Thanks,
>>
>> Jun
>>
>> On Tue, Sep 11, 2018 at 3:46 PM, Lucas Wang 
>> wrote:
>>
>> > @Jun Rao 
>> >
>> > I made the recent config changes after thinking about the default
>> behavior
>> > for adopting this KIP.
>> > I think there are basically two options:
>> > 1. By default, the behavior proposed in this KIP is turned off, and
>> > operators can turn it
>> > on by adding the "controller.listener.name" config and entries in the
>> > "listeners" and "advertised.listeners" list.
>> > If no "controller.listener.name" is added, it'll be the *same as* the "
>> > inter.broker.listener.name",
>> > and the proposed feature is effectively turned off.
>> > This has been the assumption in the KIP writeup before.
>> >
>> > 2. By default, the behavior proposed in this KIP is turned on, and
>> > operators are forced to
>> > recognize the proposed change if their "listeners" config is set (this
>> is
>> > most likely in production environments),
>> > by allocating a new port for controller connections, and adding a new
>> > endpoint to the "listeners" config.
>> > For cases where "listeners" is not set explicitly,
>> > there needs to be a default value for it that includes the controller
>> > listener name,
>> > e.g. "PLAINTEXT://:9092,CONTROLLER://:9091"
>> >
>> > I chose to go with option 2 since as author of this KIP,
>> > I naturally think in the long run, it's worth the effort to adopt this
>> > feature,
>> > in order to prevent issues under circumstances listed in the motivation
>> > section.
>> >
>> > 100, following the argument above, I want to enforce the separation
>> > between controller
>> > and data plane requests. Hence the "controller.listener.name" should
>> > never be the same
>> > as the "inter.broker.listener.name", which is intended for data plane
>> > requests.
>> >
>> > 101, the default value for "listeners" will be
>> > "PLAINTEXT://:9092,CONTROLLER://:9091",
>> > making values of "host", and "port" not being used under any config
>> > settings.
>> > And the default value for "advertised.listeners" will be derived from
>> > "listeners",
>> > making the values of "advertised.host", and "advertised.port" not being
>> > used any more.
>> >
>> > 102, for upgrading, a single broker that has "listeners" and/or
>> > "advertised.listeners" set,
>> > must add a new endpoint for the CONTROLLER listener name, or end up
>> using
>> > the default listeners "PLAINTEXT://:9092,CONTROLLER://:9091".
>> > During rolling upgrade, in cases of  +  or
>> >   + 
>> > we still need to fall back to the current behavior. However after the
>> > rolling upgrade is done,
>> > it is guaranteed that the controller plane and data plane are separated,
>> > given
>> > the "controller.listener.name" must be different from "
>> > inter.broker.listener.name".
>> >
>> > @Ismael Juma 
>> > Thanks for pointing that out. I did not know that.
>> > However my question is if the argument above makes sense, and my code
>> > change
>> > causes the configs "host", "port", "advertised.host", "advertised.port"
>> to
>> > be not used under any circumstance,
>> > then it's no different from removing them.
>> > Anyway if there is still a concern about removing them, is there a new
>> > major 

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-09-19 Thread Lucas Wang
Hi Jun,

Thanks a lot for the detailed explanation.
I've restored the wiki to a previous version that does not require config
changes,
and keeps the current behavior with the proposed changes turned off by
default.
I'd appreciate it if you can review it again.

Thanks!
Lucas

On Tue, Sep 18, 2018 at 1:48 PM Jun Rao  wrote:

> Hi, Lucas,
>
> When upgrading to a minor release, I think the expectation is that a user
> wouldn't need to make any config changes, other than the usual
> inter.broker.protocol. If we require other config changes during an
> upgrade, then it's probably better to do that in a major release.
>
> Regarding your proposal, I think removing host/advertised_host in favor of
> listeners:advertised_listeners seems useful regardless of this KIP.
> However, that can probably wait until a major release.
>
> As for the controller listener, I am not sure if one has to set it. To make
> a cluster healthy, one sort of have to make sure that the request queue is
> never full and no request will be sitting in the request queue for long. If
> one does that, setting the controller listener may not be necessary. On the
> flip side, even if one sets the controller listener, but the request queue
> and the request time for the data part are still high, the cluster may
> still not be healthy. Given that we have already started the 2.1 release
> planning, perhaps we can start with not requiring the controller listener.
> If this is indeed something that everyone wants to set, we can make it a
> required config in a major release.
>
> Thanks,
>
> Jun
>
> On Tue, Sep 11, 2018 at 3:46 PM, Lucas Wang  wrote:
>
> > @Jun Rao 
> >
> > I made the recent config changes after thinking about the default
> behavior
> > for adopting this KIP.
> > I think there are basically two options:
> > 1. By default, the behavior proposed in this KIP is turned off, and
> > operators can turn it
> > on by adding the "controller.listener.name" config and entries in the
> > "listeners" and "advertised.listeners" list.
> > If no "controller.listener.name" is added, it'll be the *same as* the "
> > inter.broker.listener.name",
> > and the proposed feature is effectively turned off.
> > This has been the assumption in the KIP writeup before.
> >
> > 2. By default, the behavior proposed in this KIP is turned on, and
> > operators are forced to
> > recognize the proposed change if their "listeners" config is set (this is
> > most likely in production environments),
> > by allocating a new port for controller connections, and adding a new
> > endpoint to the "listeners" config.
> > For cases where "listeners" is not set explicitly,
> > there needs to be a default value for it that includes the controller
> > listener name,
> > e.g. "PLAINTEXT://:9092,CONTROLLER://:9091"
> >
> > I chose to go with option 2 since as author of this KIP,
> > I naturally think in the long run, it's worth the effort to adopt this
> > feature,
> > in order to prevent issues under circumstances listed in the motivation
> > section.
> >
> > 100, following the argument above, I want to enforce the separation
> > between controller
> > and data plane requests. Hence the "controller.listener.name" should
> > never be the same
> > as the "inter.broker.listener.name", which is intended for data plane
> > requests.
> >
> > 101, the default value for "listeners" will be
> > "PLAINTEXT://:9092,CONTROLLER://:9091",
> > making values of "host", and "port" not being used under any config
> > settings.
> > And the default value for "advertised.listeners" will be derived from
> > "listeners",
> > making the values of "advertised.host", and "advertised.port" not being
> > used any more.
> >
> > 102, for upgrading, a single broker that has "listeners" and/or
> > "advertised.listeners" set,
> > must add a new endpoint for the CONTROLLER listener name, or end up using
> > the default listeners "PLAINTEXT://:9092,CONTROLLER://:9091".
> > During rolling upgrade, in cases of  +  or
> >   + 
> > we still need to fall back to the current behavior. However after the
> > rolling upgrade is done,
> > it is guaranteed that the controller plane and data plane are separated,
> > given
> > the "controller.listener.name" must be different from "
> > inter.broker.listener.name".
> >
> > @Ismael Juma 
> > Thanks for pointing that out. I did not know that.
> > However my question is if the argument above makes sense, and my code
> > change
> > causes the configs "host", "port", "advertised.host", "advertised.port"
> to
> > be not used under any circumstance,
> > then it's no different from removing them.
> > Anyway if there is still a concern about removing them, is there a new
> > major new version
> > now or in the future where I can remove them?
> >
> > Thanks!
> > Lucas
> >
> > On Mon, Sep 10, 2018 at 1:30 PM Ismael Juma  wrote:
> >
> >> To be clear, we can only remove configs in major new versions.
> Otherwise,
> >> we can only deprecate them.
> >>
> >> Ismael
> >>
> >> On Mon, Sep 10, 2018 at 

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-09-18 Thread Jun Rao
Hi, Lucas,

When upgrading to a minor release, I think the expectation is that a user
wouldn't need to make any config changes, other than the usual
inter.broker.protocol. If we require other config changes during an
upgrade, then it's probably better to do that in a major release.

Regarding your proposal, I think removing host/advertised_host in favor of
listeners:advertised_listeners seems useful regardless of this KIP.
However, that can probably wait until a major release.

As for the controller listener, I am not sure if one has to set it. To make
a cluster healthy, one sort of have to make sure that the request queue is
never full and no request will be sitting in the request queue for long. If
one does that, setting the controller listener may not be necessary. On the
flip side, even if one sets the controller listener, but the request queue
and the request time for the data part are still high, the cluster may
still not be healthy. Given that we have already started the 2.1 release
planning, perhaps we can start with not requiring the controller listener.
If this is indeed something that everyone wants to set, we can make it a
required config in a major release.

Thanks,

Jun

On Tue, Sep 11, 2018 at 3:46 PM, Lucas Wang  wrote:

> @Jun Rao 
>
> I made the recent config changes after thinking about the default behavior
> for adopting this KIP.
> I think there are basically two options:
> 1. By default, the behavior proposed in this KIP is turned off, and
> operators can turn it
> on by adding the "controller.listener.name" config and entries in the
> "listeners" and "advertised.listeners" list.
> If no "controller.listener.name" is added, it'll be the *same as* the "
> inter.broker.listener.name",
> and the proposed feature is effectively turned off.
> This has been the assumption in the KIP writeup before.
>
> 2. By default, the behavior proposed in this KIP is turned on, and
> operators are forced to
> recognize the proposed change if their "listeners" config is set (this is
> most likely in production environments),
> by allocating a new port for controller connections, and adding a new
> endpoint to the "listeners" config.
> For cases where "listeners" is not set explicitly,
> there needs to be a default value for it that includes the controller
> listener name,
> e.g. "PLAINTEXT://:9092,CONTROLLER://:9091"
>
> I chose to go with option 2 since as author of this KIP,
> I naturally think in the long run, it's worth the effort to adopt this
> feature,
> in order to prevent issues under circumstances listed in the motivation
> section.
>
> 100, following the argument above, I want to enforce the separation
> between controller
> and data plane requests. Hence the "controller.listener.name" should
> never be the same
> as the "inter.broker.listener.name", which is intended for data plane
> requests.
>
> 101, the default value for "listeners" will be
> "PLAINTEXT://:9092,CONTROLLER://:9091",
> making values of "host", and "port" not being used under any config
> settings.
> And the default value for "advertised.listeners" will be derived from
> "listeners",
> making the values of "advertised.host", and "advertised.port" not being
> used any more.
>
> 102, for upgrading, a single broker that has "listeners" and/or
> "advertised.listeners" set,
> must add a new endpoint for the CONTROLLER listener name, or end up using
> the default listeners "PLAINTEXT://:9092,CONTROLLER://:9091".
> During rolling upgrade, in cases of  +  or
>   + 
> we still need to fall back to the current behavior. However after the
> rolling upgrade is done,
> it is guaranteed that the controller plane and data plane are separated,
> given
> the "controller.listener.name" must be different from "
> inter.broker.listener.name".
>
> @Ismael Juma 
> Thanks for pointing that out. I did not know that.
> However my question is if the argument above makes sense, and my code
> change
> causes the configs "host", "port", "advertised.host", "advertised.port" to
> be not used under any circumstance,
> then it's no different from removing them.
> Anyway if there is still a concern about removing them, is there a new
> major new version
> now or in the future where I can remove them?
>
> Thanks!
> Lucas
>
> On Mon, Sep 10, 2018 at 1:30 PM Ismael Juma  wrote:
>
>> To be clear, we can only remove configs in major new versions. Otherwise,
>> we can only deprecate them.
>>
>> Ismael
>>
>> On Mon, Sep 10, 2018 at 10:47 AM Jun Rao  wrote:
>>
>> > Hi, Lucas,
>> >
>> > For the network idlePct, your understanding is correct. Currently,
>> > networkIdlePct metric is calculated as the average of (1 - io-ratio) in
>> the
>> > selector of all network threads.
>> >
>> > The metrics part looks good to me in the updated KIP.
>> >
>> > I am not still not quite sure about the configs.
>> >
>> > 100. "Whenever the "controller.listener.name" is set, upon broker
>> startup,
>> > we will validate its value and make sure it's different from the
>> > 

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-09-11 Thread Lucas Wang
@Jun Rao 

I made the recent config changes after thinking about the default behavior
for adopting this KIP.
I think there are basically two options:
1. By default, the behavior proposed in this KIP is turned off, and
operators can turn it
on by adding the "controller.listener.name" config and entries in the
"listeners" and "advertised.listeners" list.
If no "controller.listener.name" is added, it'll be the *same as* the "
inter.broker.listener.name",
and the proposed feature is effectively turned off.
This has been the assumption in the KIP writeup before.

2. By default, the behavior proposed in this KIP is turned on, and
operators are forced to
recognize the proposed change if their "listeners" config is set (this is
most likely in production environments),
by allocating a new port for controller connections, and adding a new
endpoint to the "listeners" config.
For cases where "listeners" is not set explicitly,
there needs to be a default value for it that includes the controller
listener name,
e.g. "PLAINTEXT://:9092,CONTROLLER://:9091"

I chose to go with option 2 since as author of this KIP,
I naturally think in the long run, it's worth the effort to adopt this
feature,
in order to prevent issues under circumstances listed in the motivation
section.

100, following the argument above, I want to enforce the separation between
controller
and data plane requests. Hence the "controller.listener.name" should never
be the same
as the "inter.broker.listener.name", which is intended for data plane
requests.

101, the default value for "listeners" will be
"PLAINTEXT://:9092,CONTROLLER://:9091",
making values of "host", and "port" not being used under any config
settings.
And the default value for "advertised.listeners" will be derived from
"listeners",
making the values of "advertised.host", and "advertised.port" not being
used any more.

102, for upgrading, a single broker that has "listeners" and/or
"advertised.listeners" set,
must add a new endpoint for the CONTROLLER listener name, or end up using
the default listeners "PLAINTEXT://:9092,CONTROLLER://:9091".
During rolling upgrade, in cases of  +  or   + 
we still need to fall back to the current behavior. However after the
rolling upgrade is done,
it is guaranteed that the controller plane and data plane are separated,
given
the "controller.listener.name" must be different from "
inter.broker.listener.name".

@Ismael Juma 
Thanks for pointing that out. I did not know that.
However my question is if the argument above makes sense, and my code change
causes the configs "host", "port", "advertised.host", "advertised.port" to
be not used under any circumstance,
then it's no different from removing them.
Anyway if there is still a concern about removing them, is there a new
major new version
now or in the future where I can remove them?

Thanks!
Lucas

On Mon, Sep 10, 2018 at 1:30 PM Ismael Juma  wrote:

> To be clear, we can only remove configs in major new versions. Otherwise,
> we can only deprecate them.
>
> Ismael
>
> On Mon, Sep 10, 2018 at 10:47 AM Jun Rao  wrote:
>
> > Hi, Lucas,
> >
> > For the network idlePct, your understanding is correct. Currently,
> > networkIdlePct metric is calculated as the average of (1 - io-ratio) in
> the
> > selector of all network threads.
> >
> > The metrics part looks good to me in the updated KIP.
> >
> > I am not still not quite sure about the configs.
> >
> > 100. "Whenever the "controller.listener.name" is set, upon broker
> startup,
> > we will validate its value and make sure it's different from the
> > *inter-broker-listener-name *value." Does that mean that
> > controller.listener.name has to be different from
> > inter.broker.listener.name?
> > That seems limiting.
> >
> > 101. The KIP says that advertised.listeners and listeners will now have a
> > different default value including controller. Could you document what the
> > default value looks like?
> >
> > 102. About removing the the following configs. How does that affect the
> > upgrade path? Do we now expect a user to add a new config when upgrading
> > from an old version to a new one?
> > host
> > port
> > advertised.host
> > advertised.port
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Thu, Sep 6, 2018 at 5:14 PM, Lucas Wang 
> wrote:
> >
> > > @Jun Rao 
> > >
> > > One clarification, currently on the selector level, we have the
> > > io-wait-ratio metric.
> > > For the new controller *network* thread, we can use it directly for
> > > IdlePct, instead of using 1- io-ratio,
> > > so that the logic is similar to the current average IdlePct for network
> > > threads. Is that correct?
> > >
> > > I've revised the KIP by adding two new metrics for measuring the
> IdlePct
> > > for the two additional threads.
> > > Please take a look again. Thanks!
> > >
> > > Lucas
> > >
> > >
> > >
> > >
> > >
> > > On Wed, Sep 5, 2018 at 5:01 PM Jun Rao  wrote:
> > >
> > > > Hi, Lucas,
> > > >
> > > > Thanks for the updated KIP.
> > > >
> > > > For monitoring the network 

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-09-10 Thread Ismael Juma
To be clear, we can only remove configs in major new versions. Otherwise,
we can only deprecate them.

Ismael

On Mon, Sep 10, 2018 at 10:47 AM Jun Rao  wrote:

> Hi, Lucas,
>
> For the network idlePct, your understanding is correct. Currently,
> networkIdlePct metric is calculated as the average of (1 - io-ratio) in the
> selector of all network threads.
>
> The metrics part looks good to me in the updated KIP.
>
> I am not still not quite sure about the configs.
>
> 100. "Whenever the "controller.listener.name" is set, upon broker startup,
> we will validate its value and make sure it's different from the
> *inter-broker-listener-name *value." Does that mean that
> controller.listener.name has to be different from
> inter.broker.listener.name?
> That seems limiting.
>
> 101. The KIP says that advertised.listeners and listeners will now have a
> different default value including controller. Could you document what the
> default value looks like?
>
> 102. About removing the the following configs. How does that affect the
> upgrade path? Do we now expect a user to add a new config when upgrading
> from an old version to a new one?
> host
> port
> advertised.host
> advertised.port
>
> Thanks,
>
> Jun
>
>
> On Thu, Sep 6, 2018 at 5:14 PM, Lucas Wang  wrote:
>
> > @Jun Rao 
> >
> > One clarification, currently on the selector level, we have the
> > io-wait-ratio metric.
> > For the new controller *network* thread, we can use it directly for
> > IdlePct, instead of using 1- io-ratio,
> > so that the logic is similar to the current average IdlePct for network
> > threads. Is that correct?
> >
> > I've revised the KIP by adding two new metrics for measuring the IdlePct
> > for the two additional threads.
> > Please take a look again. Thanks!
> >
> > Lucas
> >
> >
> >
> >
> >
> > On Wed, Sep 5, 2018 at 5:01 PM Jun Rao  wrote:
> >
> > > Hi, Lucas,
> > >
> > > Thanks for the updated KIP.
> > >
> > > For monitoring the network thread utilization for the control plane, we
> > > already have the metric io-ratio at the selector level (idlePct is 1 -
> > > io-ratio). So, we just need to give that selector a meaningful name.
> > >
> > > For monitoring the io thread utilization for the control plane, it's
> > > probably useful to have a separate metric for that. The controller
> > request
> > > queue size may not reflect the history in a window.
> > >
> > > Jun
> > >
> > > On Wed, Sep 5, 2018 at 3:38 PM, Lucas Wang 
> > wrote:
> > >
> > > > Thanks Jun for your quick response. It looks like I forgot to click
> the
> > > > "Update" button, :)
> > > > It's updated now.
> > > >
> > > > Regarding the idle ratio metrics for the additional threads, I
> > discussed
> > > > with Joel,
> > > > and think they are not as useful, and I added our reasoning in the
> last
> > > > paragraph of the
> > > > "How are controller requests handled over the dedicated connections?"
> > > > section.
> > > > On the other hand, we don't strongly oppose adding them if you think
> > they
> > > > are necessary.
> > > >
> > > > Thanks,
> > > > Lucas
> > > >
> > > >
> > > > On Wed, Sep 5, 2018 at 3:12 PM Jun Rao  wrote:
> > > >
> > > > > Hi, Lucas,
> > > > >
> > > > > Thanks for the reply. Have you actually updated the KIP? The wiki
> > says
> > > > that
> > > > > it's last updated on Aug. 22. and some of the changes that you
> > > mentioned
> > > > > (#1 and #3) are not there.
> > > > >
> > > > > Also, regarding Joel's comment on network/request idle ratio
> metrics,
> > > > could
> > > > > you comment on whether they include the new controller listener? If
> > > not,
> > > > do
> > > > > we need additional metrics to measure the utilization of the io
> > thread
> > > > for
> > > > > the control plane?
> > > > >
> > > > > Jun
> > > > >
> > > > > On Mon, Aug 27, 2018 at 6:26 PM, Lucas Wang  >
> > > > wrote:
> > > > >
> > > > > > Thanks for the comments, Jun.
> > > > > >
> > > > > > 1. I think the answer should be no, since the "
> > > > > inter.broker.listener.name"
> > > > > > are also used
> > > > > > for replication traffic, and merging these two types of request
> to
> > > the
> > > > > > single threaded tunnel
> > > > > > would defeat the purpose of this KIP and also hurt replication
> > > > > throughput.
> > > > > > So I think that means
> > > > > > we should validate to make sure when the new config is set, it's
> > > > > different
> > > > > > from "inter.broker.listener.name"
> > > > > > or "security.inter.broker.protocol", whichever is set.
> > > > > >
> > > > > > 2. Normally all broker configs in a given cluster are changed at
> > the
> > > > same
> > > > > > time. If there is a typo in the
> > > > > > controller.listener.name and it's not available in the endpoints
> > > list,
> > > > > we
> > > > > > could catch it, give an error
> > > > > > and block restart of the first broker in the cluster. With that,
> we
> > > > could
> > > > > > keep the current behavior
> > > > > > in the KIP write up that falls back to
> "inter.broker.listener.nam"
> 

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-09-10 Thread Jun Rao
Hi, Lucas,

For the network idlePct, your understanding is correct. Currently,
networkIdlePct metric is calculated as the average of (1 - io-ratio) in the
selector of all network threads.

The metrics part looks good to me in the updated KIP.

I am not still not quite sure about the configs.

100. "Whenever the "controller.listener.name" is set, upon broker startup,
we will validate its value and make sure it's different from the
*inter-broker-listener-name *value." Does that mean that
controller.listener.name has to be different from inter.broker.listener.name?
That seems limiting.

101. The KIP says that advertised.listeners and listeners will now have a
different default value including controller. Could you document what the
default value looks like?

102. About removing the the following configs. How does that affect the
upgrade path? Do we now expect a user to add a new config when upgrading
from an old version to a new one?
host
port
advertised.host
advertised.port

Thanks,

Jun


On Thu, Sep 6, 2018 at 5:14 PM, Lucas Wang  wrote:

> @Jun Rao 
>
> One clarification, currently on the selector level, we have the
> io-wait-ratio metric.
> For the new controller *network* thread, we can use it directly for
> IdlePct, instead of using 1- io-ratio,
> so that the logic is similar to the current average IdlePct for network
> threads. Is that correct?
>
> I've revised the KIP by adding two new metrics for measuring the IdlePct
> for the two additional threads.
> Please take a look again. Thanks!
>
> Lucas
>
>
>
>
>
> On Wed, Sep 5, 2018 at 5:01 PM Jun Rao  wrote:
>
> > Hi, Lucas,
> >
> > Thanks for the updated KIP.
> >
> > For monitoring the network thread utilization for the control plane, we
> > already have the metric io-ratio at the selector level (idlePct is 1 -
> > io-ratio). So, we just need to give that selector a meaningful name.
> >
> > For monitoring the io thread utilization for the control plane, it's
> > probably useful to have a separate metric for that. The controller
> request
> > queue size may not reflect the history in a window.
> >
> > Jun
> >
> > On Wed, Sep 5, 2018 at 3:38 PM, Lucas Wang 
> wrote:
> >
> > > Thanks Jun for your quick response. It looks like I forgot to click the
> > > "Update" button, :)
> > > It's updated now.
> > >
> > > Regarding the idle ratio metrics for the additional threads, I
> discussed
> > > with Joel,
> > > and think they are not as useful, and I added our reasoning in the last
> > > paragraph of the
> > > "How are controller requests handled over the dedicated connections?"
> > > section.
> > > On the other hand, we don't strongly oppose adding them if you think
> they
> > > are necessary.
> > >
> > > Thanks,
> > > Lucas
> > >
> > >
> > > On Wed, Sep 5, 2018 at 3:12 PM Jun Rao  wrote:
> > >
> > > > Hi, Lucas,
> > > >
> > > > Thanks for the reply. Have you actually updated the KIP? The wiki
> says
> > > that
> > > > it's last updated on Aug. 22. and some of the changes that you
> > mentioned
> > > > (#1 and #3) are not there.
> > > >
> > > > Also, regarding Joel's comment on network/request idle ratio metrics,
> > > could
> > > > you comment on whether they include the new controller listener? If
> > not,
> > > do
> > > > we need additional metrics to measure the utilization of the io
> thread
> > > for
> > > > the control plane?
> > > >
> > > > Jun
> > > >
> > > > On Mon, Aug 27, 2018 at 6:26 PM, Lucas Wang 
> > > wrote:
> > > >
> > > > > Thanks for the comments, Jun.
> > > > >
> > > > > 1. I think the answer should be no, since the "
> > > > inter.broker.listener.name"
> > > > > are also used
> > > > > for replication traffic, and merging these two types of request to
> > the
> > > > > single threaded tunnel
> > > > > would defeat the purpose of this KIP and also hurt replication
> > > > throughput.
> > > > > So I think that means
> > > > > we should validate to make sure when the new config is set, it's
> > > > different
> > > > > from "inter.broker.listener.name"
> > > > > or "security.inter.broker.protocol", whichever is set.
> > > > >
> > > > > 2. Normally all broker configs in a given cluster are changed at
> the
> > > same
> > > > > time. If there is a typo in the
> > > > > controller.listener.name and it's not available in the endpoints
> > list,
> > > > we
> > > > > could catch it, give an error
> > > > > and block restart of the first broker in the cluster. With that, we
> > > could
> > > > > keep the current behavior
> > > > > in the KIP write up that falls back to "inter.broker.listener.nam"
> > when
> > > > the
> > > > > "controller.listener.name"
> > > > > is not found during the migration phase of this KIP. Thoughts?
> > > > >
> > > > > 3. That makes sense, and I've changed it.
> > > > >
> > > > > Thanks,
> > > > > Lucas
> > > > >
> > > > > On Thu, Aug 23, 2018 at 3:46 PM Jun Rao  wrote:
> > > > >
> > > > > > Hi, Lucas,
> > > > > >
> > > > > > Sorry for the delay. The new proposal looks good to me overall. A
> > few
> > > > > minor
> > > > > 

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-09-06 Thread Lucas Wang
@Jun Rao 

One clarification, currently on the selector level, we have the
io-wait-ratio metric.
For the new controller *network* thread, we can use it directly for
IdlePct, instead of using 1- io-ratio,
so that the logic is similar to the current average IdlePct for network
threads. Is that correct?

I've revised the KIP by adding two new metrics for measuring the IdlePct
for the two additional threads.
Please take a look again. Thanks!

Lucas





On Wed, Sep 5, 2018 at 5:01 PM Jun Rao  wrote:

> Hi, Lucas,
>
> Thanks for the updated KIP.
>
> For monitoring the network thread utilization for the control plane, we
> already have the metric io-ratio at the selector level (idlePct is 1 -
> io-ratio). So, we just need to give that selector a meaningful name.
>
> For monitoring the io thread utilization for the control plane, it's
> probably useful to have a separate metric for that. The controller request
> queue size may not reflect the history in a window.
>
> Jun
>
> On Wed, Sep 5, 2018 at 3:38 PM, Lucas Wang  wrote:
>
> > Thanks Jun for your quick response. It looks like I forgot to click the
> > "Update" button, :)
> > It's updated now.
> >
> > Regarding the idle ratio metrics for the additional threads, I discussed
> > with Joel,
> > and think they are not as useful, and I added our reasoning in the last
> > paragraph of the
> > "How are controller requests handled over the dedicated connections?"
> > section.
> > On the other hand, we don't strongly oppose adding them if you think they
> > are necessary.
> >
> > Thanks,
> > Lucas
> >
> >
> > On Wed, Sep 5, 2018 at 3:12 PM Jun Rao  wrote:
> >
> > > Hi, Lucas,
> > >
> > > Thanks for the reply. Have you actually updated the KIP? The wiki says
> > that
> > > it's last updated on Aug. 22. and some of the changes that you
> mentioned
> > > (#1 and #3) are not there.
> > >
> > > Also, regarding Joel's comment on network/request idle ratio metrics,
> > could
> > > you comment on whether they include the new controller listener? If
> not,
> > do
> > > we need additional metrics to measure the utilization of the io thread
> > for
> > > the control plane?
> > >
> > > Jun
> > >
> > > On Mon, Aug 27, 2018 at 6:26 PM, Lucas Wang 
> > wrote:
> > >
> > > > Thanks for the comments, Jun.
> > > >
> > > > 1. I think the answer should be no, since the "
> > > inter.broker.listener.name"
> > > > are also used
> > > > for replication traffic, and merging these two types of request to
> the
> > > > single threaded tunnel
> > > > would defeat the purpose of this KIP and also hurt replication
> > > throughput.
> > > > So I think that means
> > > > we should validate to make sure when the new config is set, it's
> > > different
> > > > from "inter.broker.listener.name"
> > > > or "security.inter.broker.protocol", whichever is set.
> > > >
> > > > 2. Normally all broker configs in a given cluster are changed at the
> > same
> > > > time. If there is a typo in the
> > > > controller.listener.name and it's not available in the endpoints
> list,
> > > we
> > > > could catch it, give an error
> > > > and block restart of the first broker in the cluster. With that, we
> > could
> > > > keep the current behavior
> > > > in the KIP write up that falls back to "inter.broker.listener.nam"
> when
> > > the
> > > > "controller.listener.name"
> > > > is not found during the migration phase of this KIP. Thoughts?
> > > >
> > > > 3. That makes sense, and I've changed it.
> > > >
> > > > Thanks,
> > > > Lucas
> > > >
> > > > On Thu, Aug 23, 2018 at 3:46 PM Jun Rao  wrote:
> > > >
> > > > > Hi, Lucas,
> > > > >
> > > > > Sorry for the delay. The new proposal looks good to me overall. A
> few
> > > > minor
> > > > > comments below.
> > > > >
> > > > > 1. It's possible that listener.name.for.controller is set, but set
> to
> > > the
> > > > > same value as inter.broker.listener.name. In that case, should we
> > > have a
> > > > > single network thread and the request handling thread for that
> > > listener?
> > > > >
> > > > > 2. Currently, the controller always picks the listener specified by
> > > > > inter.broker.listener.name even if the listener name is not
> present
> > in
> > > > the
> > > > > receiving broker. This KIP proposes a slightly different approach
> for
> > > > > picking listener.name.for.controller only when the receiving end
> has
> > > the
> > > > > listener and switches listener.name.for.controller otherwise. There
> > are
> > > > > some tradeoffs between the two approaches. To change the inter
> broker
> > > > > listener, the former requires 2 steps: (1) adding the new listener
> to
> > > > > listener list in every broker and (2) changing
> > > > > listener.name.for.controller.
> > > > > The latter can do both changes in 1 step. On the hand, if
> > > > > listener.name.for.controller
> > > > > is mis-configured, the former will report an error and the latter
> > will
> > > > hide
> > > > > it (so the user may not know the misconfiguration). It seems that
> we
> > > > should

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-09-05 Thread Jun Rao
Hi, Lucas,

Thanks for the updated KIP.

For monitoring the network thread utilization for the control plane, we
already have the metric io-ratio at the selector level (idlePct is 1 -
io-ratio). So, we just need to give that selector a meaningful name.

For monitoring the io thread utilization for the control plane, it's
probably useful to have a separate metric for that. The controller request
queue size may not reflect the history in a window.

Jun

On Wed, Sep 5, 2018 at 3:38 PM, Lucas Wang  wrote:

> Thanks Jun for your quick response. It looks like I forgot to click the
> "Update" button, :)
> It's updated now.
>
> Regarding the idle ratio metrics for the additional threads, I discussed
> with Joel,
> and think they are not as useful, and I added our reasoning in the last
> paragraph of the
> "How are controller requests handled over the dedicated connections?"
> section.
> On the other hand, we don't strongly oppose adding them if you think they
> are necessary.
>
> Thanks,
> Lucas
>
>
> On Wed, Sep 5, 2018 at 3:12 PM Jun Rao  wrote:
>
> > Hi, Lucas,
> >
> > Thanks for the reply. Have you actually updated the KIP? The wiki says
> that
> > it's last updated on Aug. 22. and some of the changes that you mentioned
> > (#1 and #3) are not there.
> >
> > Also, regarding Joel's comment on network/request idle ratio metrics,
> could
> > you comment on whether they include the new controller listener? If not,
> do
> > we need additional metrics to measure the utilization of the io thread
> for
> > the control plane?
> >
> > Jun
> >
> > On Mon, Aug 27, 2018 at 6:26 PM, Lucas Wang 
> wrote:
> >
> > > Thanks for the comments, Jun.
> > >
> > > 1. I think the answer should be no, since the "
> > inter.broker.listener.name"
> > > are also used
> > > for replication traffic, and merging these two types of request to the
> > > single threaded tunnel
> > > would defeat the purpose of this KIP and also hurt replication
> > throughput.
> > > So I think that means
> > > we should validate to make sure when the new config is set, it's
> > different
> > > from "inter.broker.listener.name"
> > > or "security.inter.broker.protocol", whichever is set.
> > >
> > > 2. Normally all broker configs in a given cluster are changed at the
> same
> > > time. If there is a typo in the
> > > controller.listener.name and it's not available in the endpoints list,
> > we
> > > could catch it, give an error
> > > and block restart of the first broker in the cluster. With that, we
> could
> > > keep the current behavior
> > > in the KIP write up that falls back to "inter.broker.listener.nam" when
> > the
> > > "controller.listener.name"
> > > is not found during the migration phase of this KIP. Thoughts?
> > >
> > > 3. That makes sense, and I've changed it.
> > >
> > > Thanks,
> > > Lucas
> > >
> > > On Thu, Aug 23, 2018 at 3:46 PM Jun Rao  wrote:
> > >
> > > > Hi, Lucas,
> > > >
> > > > Sorry for the delay. The new proposal looks good to me overall. A few
> > > minor
> > > > comments below.
> > > >
> > > > 1. It's possible that listener.name.for.controller is set, but set to
> > the
> > > > same value as inter.broker.listener.name. In that case, should we
> > have a
> > > > single network thread and the request handling thread for that
> > listener?
> > > >
> > > > 2. Currently, the controller always picks the listener specified by
> > > > inter.broker.listener.name even if the listener name is not present
> in
> > > the
> > > > receiving broker. This KIP proposes a slightly different approach for
> > > > picking listener.name.for.controller only when the receiving end has
> > the
> > > > listener and switches listener.name.for.controller otherwise. There
> are
> > > > some tradeoffs between the two approaches. To change the inter broker
> > > > listener, the former requires 2 steps: (1) adding the new listener to
> > > > listener list in every broker and (2) changing
> > > > listener.name.for.controller.
> > > > The latter can do both changes in 1 step. On the hand, if
> > > > listener.name.for.controller
> > > > is mis-configured, the former will report an error and the latter
> will
> > > hide
> > > > it (so the user may not know the misconfiguration). It seems that we
> > > should
> > > > pick one approach to handle both listener.name.for.controller and
> > > > inter.broker.listener.name consistently. To me, the former seems
> > > slightly
> > > > better.
> > > >
> > > > 3. To be consistent with the existing naming, should
> > > > listener.name.for.controller
> > > > be controller.listener.name?
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > >
> > > > On Thu, Aug 9, 2018 at 3:21 PM, Lucas Wang 
> > > wrote:
> > > >
> > > > > Hi Jun and Joel,
> > > > >
> > > > > The KIP writeup has changed by quite a bit since your +1.
> > > > > Can you please take another review? Thanks a lot for your time!
> > > > >
> > > > > Lucas
> > > > >
> > > > > On Tue, Jul 17, 2018 at 10:33 AM, Joel Koshy 
> > > > wrote:
> > > > >
> > > > > > +1 on 

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-09-05 Thread Lucas Wang
Thanks Jun for your quick response. It looks like I forgot to click the
"Update" button, :)
It's updated now.

Regarding the idle ratio metrics for the additional threads, I discussed
with Joel,
and think they are not as useful, and I added our reasoning in the last
paragraph of the
"How are controller requests handled over the dedicated connections?"
section.
On the other hand, we don't strongly oppose adding them if you think they
are necessary.

Thanks,
Lucas


On Wed, Sep 5, 2018 at 3:12 PM Jun Rao  wrote:

> Hi, Lucas,
>
> Thanks for the reply. Have you actually updated the KIP? The wiki says that
> it's last updated on Aug. 22. and some of the changes that you mentioned
> (#1 and #3) are not there.
>
> Also, regarding Joel's comment on network/request idle ratio metrics, could
> you comment on whether they include the new controller listener? If not, do
> we need additional metrics to measure the utilization of the io thread for
> the control plane?
>
> Jun
>
> On Mon, Aug 27, 2018 at 6:26 PM, Lucas Wang  wrote:
>
> > Thanks for the comments, Jun.
> >
> > 1. I think the answer should be no, since the "
> inter.broker.listener.name"
> > are also used
> > for replication traffic, and merging these two types of request to the
> > single threaded tunnel
> > would defeat the purpose of this KIP and also hurt replication
> throughput.
> > So I think that means
> > we should validate to make sure when the new config is set, it's
> different
> > from "inter.broker.listener.name"
> > or "security.inter.broker.protocol", whichever is set.
> >
> > 2. Normally all broker configs in a given cluster are changed at the same
> > time. If there is a typo in the
> > controller.listener.name and it's not available in the endpoints list,
> we
> > could catch it, give an error
> > and block restart of the first broker in the cluster. With that, we could
> > keep the current behavior
> > in the KIP write up that falls back to "inter.broker.listener.nam" when
> the
> > "controller.listener.name"
> > is not found during the migration phase of this KIP. Thoughts?
> >
> > 3. That makes sense, and I've changed it.
> >
> > Thanks,
> > Lucas
> >
> > On Thu, Aug 23, 2018 at 3:46 PM Jun Rao  wrote:
> >
> > > Hi, Lucas,
> > >
> > > Sorry for the delay. The new proposal looks good to me overall. A few
> > minor
> > > comments below.
> > >
> > > 1. It's possible that listener.name.for.controller is set, but set to
> the
> > > same value as inter.broker.listener.name. In that case, should we
> have a
> > > single network thread and the request handling thread for that
> listener?
> > >
> > > 2. Currently, the controller always picks the listener specified by
> > > inter.broker.listener.name even if the listener name is not present in
> > the
> > > receiving broker. This KIP proposes a slightly different approach for
> > > picking listener.name.for.controller only when the receiving end has
> the
> > > listener and switches listener.name.for.controller otherwise. There are
> > > some tradeoffs between the two approaches. To change the inter broker
> > > listener, the former requires 2 steps: (1) adding the new listener to
> > > listener list in every broker and (2) changing
> > > listener.name.for.controller.
> > > The latter can do both changes in 1 step. On the hand, if
> > > listener.name.for.controller
> > > is mis-configured, the former will report an error and the latter will
> > hide
> > > it (so the user may not know the misconfiguration). It seems that we
> > should
> > > pick one approach to handle both listener.name.for.controller and
> > > inter.broker.listener.name consistently. To me, the former seems
> > slightly
> > > better.
> > >
> > > 3. To be consistent with the existing naming, should
> > > listener.name.for.controller
> > > be controller.listener.name?
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > >
> > > On Thu, Aug 9, 2018 at 3:21 PM, Lucas Wang 
> > wrote:
> > >
> > > > Hi Jun and Joel,
> > > >
> > > > The KIP writeup has changed by quite a bit since your +1.
> > > > Can you please take another review? Thanks a lot for your time!
> > > >
> > > > Lucas
> > > >
> > > > On Tue, Jul 17, 2018 at 10:33 AM, Joel Koshy 
> > > wrote:
> > > >
> > > > > +1 on the KIP.
> > > > >
> > > > > (I'm not sure we actually necessary to introduce the condition
> > > variables
> > > > > for the concern that Jun raised, but it's an implementation detail
> > that
> > > > we
> > > > > can defer to a discussion in the PR.)
> > > > >
> > > > > On Sat, Jul 14, 2018 at 10:45 PM, Lucas Wang <
> lucasatu...@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hi Jun,
> > > > > >
> > > > > > I agree by using the conditional variables, there is no need to
> add
> > > > such
> > > > > a
> > > > > > new config.
> > > > > > Also thanks for approving this KIP.
> > > > > >
> > > > > > Lucas
> > > > > >
> > > > >
> > > >
> > >
> >
>


Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-09-05 Thread Jun Rao
Hi, Lucas,

Thanks for the reply. Have you actually updated the KIP? The wiki says that
it's last updated on Aug. 22. and some of the changes that you mentioned
(#1 and #3) are not there.

Also, regarding Joel's comment on network/request idle ratio metrics, could
you comment on whether they include the new controller listener? If not, do
we need additional metrics to measure the utilization of the io thread for
the control plane?

Jun

On Mon, Aug 27, 2018 at 6:26 PM, Lucas Wang  wrote:

> Thanks for the comments, Jun.
>
> 1. I think the answer should be no, since the "inter.broker.listener.name"
> are also used
> for replication traffic, and merging these two types of request to the
> single threaded tunnel
> would defeat the purpose of this KIP and also hurt replication throughput.
> So I think that means
> we should validate to make sure when the new config is set, it's different
> from "inter.broker.listener.name"
> or "security.inter.broker.protocol", whichever is set.
>
> 2. Normally all broker configs in a given cluster are changed at the same
> time. If there is a typo in the
> controller.listener.name and it's not available in the endpoints list, we
> could catch it, give an error
> and block restart of the first broker in the cluster. With that, we could
> keep the current behavior
> in the KIP write up that falls back to "inter.broker.listener.nam" when the
> "controller.listener.name"
> is not found during the migration phase of this KIP. Thoughts?
>
> 3. That makes sense, and I've changed it.
>
> Thanks,
> Lucas
>
> On Thu, Aug 23, 2018 at 3:46 PM Jun Rao  wrote:
>
> > Hi, Lucas,
> >
> > Sorry for the delay. The new proposal looks good to me overall. A few
> minor
> > comments below.
> >
> > 1. It's possible that listener.name.for.controller is set, but set to the
> > same value as inter.broker.listener.name. In that case, should we have a
> > single network thread and the request handling thread for that listener?
> >
> > 2. Currently, the controller always picks the listener specified by
> > inter.broker.listener.name even if the listener name is not present in
> the
> > receiving broker. This KIP proposes a slightly different approach for
> > picking listener.name.for.controller only when the receiving end has the
> > listener and switches listener.name.for.controller otherwise. There are
> > some tradeoffs between the two approaches. To change the inter broker
> > listener, the former requires 2 steps: (1) adding the new listener to
> > listener list in every broker and (2) changing
> > listener.name.for.controller.
> > The latter can do both changes in 1 step. On the hand, if
> > listener.name.for.controller
> > is mis-configured, the former will report an error and the latter will
> hide
> > it (so the user may not know the misconfiguration). It seems that we
> should
> > pick one approach to handle both listener.name.for.controller and
> > inter.broker.listener.name consistently. To me, the former seems
> slightly
> > better.
> >
> > 3. To be consistent with the existing naming, should
> > listener.name.for.controller
> > be controller.listener.name?
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Thu, Aug 9, 2018 at 3:21 PM, Lucas Wang 
> wrote:
> >
> > > Hi Jun and Joel,
> > >
> > > The KIP writeup has changed by quite a bit since your +1.
> > > Can you please take another review? Thanks a lot for your time!
> > >
> > > Lucas
> > >
> > > On Tue, Jul 17, 2018 at 10:33 AM, Joel Koshy 
> > wrote:
> > >
> > > > +1 on the KIP.
> > > >
> > > > (I'm not sure we actually necessary to introduce the condition
> > variables
> > > > for the concern that Jun raised, but it's an implementation detail
> that
> > > we
> > > > can defer to a discussion in the PR.)
> > > >
> > > > On Sat, Jul 14, 2018 at 10:45 PM, Lucas Wang 
> > > > wrote:
> > > >
> > > > > Hi Jun,
> > > > >
> > > > > I agree by using the conditional variables, there is no need to add
> > > such
> > > > a
> > > > > new config.
> > > > > Also thanks for approving this KIP.
> > > > >
> > > > > Lucas
> > > > >
> > > >
> > >
> >
>


Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-08-27 Thread Lucas Wang
Thanks for the comments, Jun.

1. I think the answer should be no, since the "inter.broker.listener.name"
are also used
for replication traffic, and merging these two types of request to the
single threaded tunnel
would defeat the purpose of this KIP and also hurt replication throughput.
So I think that means
we should validate to make sure when the new config is set, it's different
from "inter.broker.listener.name"
or "security.inter.broker.protocol", whichever is set.

2. Normally all broker configs in a given cluster are changed at the same
time. If there is a typo in the
controller.listener.name and it's not available in the endpoints list, we
could catch it, give an error
and block restart of the first broker in the cluster. With that, we could
keep the current behavior
in the KIP write up that falls back to "inter.broker.listener.nam" when the
"controller.listener.name"
is not found during the migration phase of this KIP. Thoughts?

3. That makes sense, and I've changed it.

Thanks,
Lucas

On Thu, Aug 23, 2018 at 3:46 PM Jun Rao  wrote:

> Hi, Lucas,
>
> Sorry for the delay. The new proposal looks good to me overall. A few minor
> comments below.
>
> 1. It's possible that listener.name.for.controller is set, but set to the
> same value as inter.broker.listener.name. In that case, should we have a
> single network thread and the request handling thread for that listener?
>
> 2. Currently, the controller always picks the listener specified by
> inter.broker.listener.name even if the listener name is not present in the
> receiving broker. This KIP proposes a slightly different approach for
> picking listener.name.for.controller only when the receiving end has the
> listener and switches listener.name.for.controller otherwise. There are
> some tradeoffs between the two approaches. To change the inter broker
> listener, the former requires 2 steps: (1) adding the new listener to
> listener list in every broker and (2) changing
> listener.name.for.controller.
> The latter can do both changes in 1 step. On the hand, if
> listener.name.for.controller
> is mis-configured, the former will report an error and the latter will hide
> it (so the user may not know the misconfiguration). It seems that we should
> pick one approach to handle both listener.name.for.controller and
> inter.broker.listener.name consistently. To me, the former seems slightly
> better.
>
> 3. To be consistent with the existing naming, should
> listener.name.for.controller
> be controller.listener.name?
>
> Thanks,
>
> Jun
>
>
> On Thu, Aug 9, 2018 at 3:21 PM, Lucas Wang  wrote:
>
> > Hi Jun and Joel,
> >
> > The KIP writeup has changed by quite a bit since your +1.
> > Can you please take another review? Thanks a lot for your time!
> >
> > Lucas
> >
> > On Tue, Jul 17, 2018 at 10:33 AM, Joel Koshy 
> wrote:
> >
> > > +1 on the KIP.
> > >
> > > (I'm not sure we actually necessary to introduce the condition
> variables
> > > for the concern that Jun raised, but it's an implementation detail that
> > we
> > > can defer to a discussion in the PR.)
> > >
> > > On Sat, Jul 14, 2018 at 10:45 PM, Lucas Wang 
> > > wrote:
> > >
> > > > Hi Jun,
> > > >
> > > > I agree by using the conditional variables, there is no need to add
> > such
> > > a
> > > > new config.
> > > > Also thanks for approving this KIP.
> > > >
> > > > Lucas
> > > >
> > >
> >
>


Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-08-23 Thread Jun Rao
Hi, Lucas,

Sorry for the delay. The new proposal looks good to me overall. A few minor
comments below.

1. It's possible that listener.name.for.controller is set, but set to the
same value as inter.broker.listener.name. In that case, should we have a
single network thread and the request handling thread for that listener?

2. Currently, the controller always picks the listener specified by
inter.broker.listener.name even if the listener name is not present in the
receiving broker. This KIP proposes a slightly different approach for
picking listener.name.for.controller only when the receiving end has the
listener and switches listener.name.for.controller otherwise. There are
some tradeoffs between the two approaches. To change the inter broker
listener, the former requires 2 steps: (1) adding the new listener to
listener list in every broker and (2) changing listener.name.for.controller.
The latter can do both changes in 1 step. On the hand, if
listener.name.for.controller
is mis-configured, the former will report an error and the latter will hide
it (so the user may not know the misconfiguration). It seems that we should
pick one approach to handle both listener.name.for.controller and
inter.broker.listener.name consistently. To me, the former seems slightly
better.

3. To be consistent with the existing naming, should
listener.name.for.controller
be controller.listener.name?

Thanks,

Jun


On Thu, Aug 9, 2018 at 3:21 PM, Lucas Wang  wrote:

> Hi Jun and Joel,
>
> The KIP writeup has changed by quite a bit since your +1.
> Can you please take another review? Thanks a lot for your time!
>
> Lucas
>
> On Tue, Jul 17, 2018 at 10:33 AM, Joel Koshy  wrote:
>
> > +1 on the KIP.
> >
> > (I'm not sure we actually necessary to introduce the condition variables
> > for the concern that Jun raised, but it's an implementation detail that
> we
> > can defer to a discussion in the PR.)
> >
> > On Sat, Jul 14, 2018 at 10:45 PM, Lucas Wang 
> > wrote:
> >
> > > Hi Jun,
> > >
> > > I agree by using the conditional variables, there is no need to add
> such
> > a
> > > new config.
> > > Also thanks for approving this KIP.
> > >
> > > Lucas
> > >
> >
>


Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-08-09 Thread Lucas Wang
Hi Jun and Joel,

The KIP writeup has changed by quite a bit since your +1.
Can you please take another review? Thanks a lot for your time!

Lucas

On Tue, Jul 17, 2018 at 10:33 AM, Joel Koshy  wrote:

> +1 on the KIP.
>
> (I'm not sure we actually necessary to introduce the condition variables
> for the concern that Jun raised, but it's an implementation detail that we
> can defer to a discussion in the PR.)
>
> On Sat, Jul 14, 2018 at 10:45 PM, Lucas Wang 
> wrote:
>
> > Hi Jun,
> >
> > I agree by using the conditional variables, there is no need to add such
> a
> > new config.
> > Also thanks for approving this KIP.
> >
> > Lucas
> >
>


Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-07-17 Thread Joel Koshy
+1 on the KIP.

(I'm not sure we actually necessary to introduce the condition variables
for the concern that Jun raised, but it's an implementation detail that we
can defer to a discussion in the PR.)

On Sat, Jul 14, 2018 at 10:45 PM, Lucas Wang  wrote:

> Hi Jun,
>
> I agree by using the conditional variables, there is no need to add such a
> new config.
> Also thanks for approving this KIP.
>
> Lucas
>


Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-07-14 Thread Lucas Wang
Hi Jun,

I agree by using the conditional variables, there is no need to add such a
new config.
Also thanks for approving this KIP.

Lucas


Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-07-14 Thread Jun Rao
Hi, Lucas,

For 3, it has an impact to "io.thread.poll.timeout.ms", which is part of
the public interface. If we optimize the implication, it seems that there
is no need for this new config. Other than that, the KIP looks good to me.

Thanks,

Jun

On Thu, Jul 12, 2018 at 10:08 PM, Lucas Wang  wrote:

> Hi Jun,
>
> About 3, thanks for the clarification. I like your proposal in that it
> avoids the delay for controller requests when the data request queue is
> empty.
> In comparison, the approach I described earlier is simpler to understand
> and implement.
>
> Between these two I actually like your suggested approach better because
> in some cases the data request queue becoming empty can be a common
> scenario
> if the request handler average idle percent is high. Therefore the extra
> complexity is worth
> the effort.
>
> Either approach we choose, it does not affect public interfaces or the
> write up in the KIP,
> and we can discuss further in the PR.
>
> Thanks,
> Lucas
>
>
>
> On Wed, Jul 11, 2018 at 8:46 AM, Jun Rao  wrote:
>
> > Hi, Lucas,
> >
> > 2. Good point about not knowing the request type in memory pool. Looking
> at
> > the implementation. It seems that queued.max.request.bytes is orthogonal
> to
> > queued.max.requests. So, this seems fine.
> >
> > 3. The implementation that you suggested sounds good. It would be useful
> > not to unnecessarily delay the processing of a request up to 300ms. I was
> > thinking that we could have RequestChannel manage a Lock and a couple of
> > Conditions and have sendRequest()/receiveRequest() coordinate on the lock
> > and the conditions (similar to how ArrayBlockingQueue is implemented).
> This
> > way, any new request can wake up the blocked request handling threads
> > immediately.
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Fri, Jun 29, 2018 at 4:53 PM, Lucas Wang 
> 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
> > > 

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-07-12 Thread Lucas Wang
Hi Jun,

About 3, thanks for the clarification. I like your proposal in that it
avoids the delay for controller requests when the data request queue is
empty.
In comparison, the approach I described earlier is simpler to understand
and implement.

Between these two I actually like your suggested approach better because
in some cases the data request queue becoming empty can be a common scenario
if the request handler average idle percent is high. Therefore the extra
complexity is worth
the effort.

Either approach we choose, it does not affect public interfaces or the
write up in the KIP,
and we can discuss further in the PR.

Thanks,
Lucas



On Wed, Jul 11, 2018 at 8:46 AM, Jun Rao  wrote:

> Hi, Lucas,
>
> 2. Good point about not knowing the request type in memory pool. Looking at
> the implementation. It seems that queued.max.request.bytes is orthogonal to
> queued.max.requests. So, this seems fine.
>
> 3. The implementation that you suggested sounds good. It would be useful
> not to unnecessarily delay the processing of a request up to 300ms. I was
> thinking that we could have RequestChannel manage a Lock and a couple of
> Conditions and have sendRequest()/receiveRequest() coordinate on the lock
> and the conditions (similar to how ArrayBlockingQueue is implemented). This
> way, any new request can wake up the blocked request handling threads
> immediately.
>
> Thanks,
>
> Jun
>
>
> On Fri, Jun 29, 2018 at 4:53 PM, Lucas Wang  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
> >  > oller+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 

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-07-11 Thread Jun Rao
Hi, Lucas,

2. Good point about not knowing the request type in memory pool. Looking at
the implementation. It seems that queued.max.request.bytes is orthogonal to
queued.max.requests. So, this seems fine.

3. The implementation that you suggested sounds good. It would be useful
not to unnecessarily delay the processing of a request up to 300ms. I was
thinking that we could have RequestChannel manage a Lock and a couple of
Conditions and have sendRequest()/receiveRequest() coordinate on the lock
and the conditions (similar to how ArrayBlockingQueue is implemented). This
way, any new request can wake up the blocked request handling threads
immediately.

Thanks,

Jun


On Fri, Jun 29, 2018 at 4:53 PM, Lucas Wang  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
>  oller+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 

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-07-03 Thread Ted Yu
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  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  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 
> 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  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 
> > >> 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) {
> > >> >   

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-07-03 Thread Lucas Wang
@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/1QbPDqfT59A2X4To2p3OfD5YeJR8aWDK7/view?usp=sharing

Regards,
Lucas


On Mon, Jul 2, 2018 at 5:52 PM, Ted Yu  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  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  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 
> >> 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 

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-07-02 Thread Ted Yu
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  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  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 
>> 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
>> > 

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-07-02 Thread Lucas Wang
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  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  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
> >  > 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
> > 

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-06-29 Thread Ted Yu
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  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
>  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  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 

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-06-29 Thread Lucas Wang
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

.

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  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 

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-06-29 Thread Jun Rao
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  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
>


Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-06-20 Thread Harsha
+1

-Harsha

On Wed, Jun 20, 2018, at 5:15 AM, Thomas Crayford wrote:
> +1 (non-binding)
> 
> On Tue, Jun 19, 2018 at 8:20 PM, Lucas Wang  wrote:
> 
> > Hi Jun, Ismael,
> >
> > Can you please take a look when you get a chance? Thanks!
> >
> > Lucas
> >
> > On Mon, Jun 18, 2018 at 1:47 PM, Ted Yu  wrote:
> >
> > > +1
> > >
> > > On Mon, Jun 18, 2018 at 1:04 PM, Lucas Wang 
> > 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
> > > >
> > >
> >


Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-06-20 Thread Thomas Crayford
+1 (non-binding)

On Tue, Jun 19, 2018 at 8:20 PM, Lucas Wang  wrote:

> Hi Jun, Ismael,
>
> Can you please take a look when you get a chance? Thanks!
>
> Lucas
>
> On Mon, Jun 18, 2018 at 1:47 PM, Ted Yu  wrote:
>
> > +1
> >
> > On Mon, Jun 18, 2018 at 1:04 PM, Lucas Wang 
> 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
> > >
> >
>


Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-06-19 Thread Lucas Wang
Hi Jun, Ismael,

Can you please take a look when you get a chance? Thanks!

Lucas

On Mon, Jun 18, 2018 at 1:47 PM, Ted Yu  wrote:

> +1
>
> On Mon, Jun 18, 2018 at 1:04 PM, Lucas Wang  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
> >
>


Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-06-18 Thread Ted Yu
+1

On Mon, Jun 18, 2018 at 1:04 PM, Lucas Wang  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
>