Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-06-10 Thread Boyang Chen
That sounds great, thanks Jun!

On Wed, Jun 10, 2020 at 11:44 AM Jun Rao  wrote:

> Hi, Boyang,
>
> Thanks for the reply.
>
> For the metric, we just need to define a metric of meter type and of name
> NumRequestsForwardingToControllerPerSec. Meter exposes a few standard JMX
> attributes including an accumulated total and rates (
>
> https://metrics.dropwizard.io/2.2.0/apidocs/com/yammer/metrics/core/Meter.html
> ).
>
> Jun
>
> On Wed, Jun 10, 2020 at 10:38 AM Boyang Chen 
> wrote:
>
> > Thanks Jun for the suggestions! I have addressed suggestion and
> simplified
> > the metrics part.
> >
> > On Tue, Jun 9, 2020 at 5:46 PM Jun Rao  wrote:
> >
> > > Hi, Boyang,
> > >
> > > Thanks for the KIP. Just a few comments on the metrics.
> > >
> > > 1. It would be useful to document the full JMX metric names (package,
> > type,
> > > etc) of the new metrics. Also, for rates on the server side, we
> > > typically use Yammer Meter.
> > >
> > >  Sounds good.
> >
> > 2. For num-messages-redirected-rate, would num-requests-redirected-rate
> be
> > > better?
> > >
> > > Actually for the scope of this KIP, we are no longer needing to have a
> > separate tracking
> > of forwarded request rate, because the Envelope RPC is dropped.
> >
> >
> > > 3. num-client-forwarding-to-controller-rate: Is that based on clientId,
> > > client IP, client request version or sth else? How do you plan to
> > implement
> > > that since it seems to require tracking the current unique client set
> > > somehow. An alternative approach is to maintain a
> > > num-requests-redirected-rate metric with a client tag.
> > >
> > The clientId tag approach makes sense, will add to the KIP.
> >
> > Jun
> > >
> > >
> > >
> > > On Mon, Jun 8, 2020 at 9:36 AM Boyang Chen  >
> > > wrote:
> > >
> > > > Hey there,
> > > >
> > > > If no more question is raised, I will go ahead and start the vote
> > > shortly.
> > > >
> > > > On Thu, Jun 4, 2020 at 12:39 PM Boyang Chen <
> > reluctanthero...@gmail.com>
> > > > wrote:
> > > >
> > > > > Hey there,
> > > > >
> > > > > bumping this thread for any further KIP-590 discussion, since it's
> > been
> > > > > quiet for a while.
> > > > >
> > > > > Boyang
> > > > >
> > > > > On Thu, May 21, 2020 at 10:31 AM Boyang Chen <
> > > reluctanthero...@gmail.com
> > > > >
> > > > > wrote:
> > > > >
> > > > >> Thanks David, I agree the wording here is not clear, and the
> fellow
> > > > >> broker should just send a new CreateTopicRequest in this case.
> > > > >>
> > > > >> In the meantime, we had some offline discussion about the Envelope
> > API
> > > > as
> > > > >> well. Although it provides certain privileges like data embedding
> > and
> > > > >> principal embedding, it creates a security hole by letting a
> > malicious
> > > > user
> > > > >> impersonate any forwarding broker, thus pretending to be any admin
> > > user.
> > > > >> Passing the principal around also increases the vulnerability,
> > > compared
> > > > >> with other standard ways such as passing a verified token, but it
> is
> > > > >> unfortunately not fully supported with Kafka security.
> > > > >>
> > > > >> So for the security concerns, we are abandoning the Envelope
> > approach
> > > > and
> > > > >> fallback to just forward the raw admin requests. The
> authentication
> > > will
> > > > >> happen on the receiving broker side instead of on the controller,
> so
> > > > that
> > > > >> we are able to stripped off the principal fields and only include
> > the
> > > > >> principal in header as optional field for audit logging purpose.
> > > > >> Furthermore, we shall propose adding a separate endpoint for
> > > > >> broker-controller communication which should be recommended to
> > enable
> > > > >> secure connections so that a malicious client could not pretend to
> > be
> > > a
> > > > >> broker and perform impersonation.
> > > > >>
> > > > >> Let me know your thoughts.
> > > > >>
> > > > >> Best,
> > > > >> Boyang
> > > > >>
> > > > >> On Tue, May 19, 2020 at 12:17 AM David Jacot  >
> > > > wrote:
> > > > >>
> > > > >>> Hi Boyang,
> > > > >>>
> > > > >>> I've got another question regarding the auto topic creation case.
> > The
> > > > KIP
> > > > >>> says: "Currently the target broker shall just utilize its own ZK
> > > client
> > > > >>> to
> > > > >>> create
> > > > >>> internal topics, which is disallowed in the bridge release. For
> > above
> > > > >>> scenarios,
> > > > >>> non-controller broker shall just forward a CreateTopicRequest to
> > the
> > > > >>> controller
> > > > >>> instead and let controller take care of the rest, while waiting
> for
> > > the
> > > > >>> response
> > > > >>> in the meantime." There will be no request to forward in this
> case,
> > > > >>> right?
> > > > >>> Instead,
> > > > >>> a CreateTopicsRequest is created and sent to the controller node.
> > > > >>>
> > > > >>> When the CreateTopicsRequest is sent as a side effect of the
> > > > >>> MetadataRequest,
> > > > >>> it would be good to know the principal and the clientId in 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-06-10 Thread Jun Rao
Hi, Boyang,

Thanks for the reply.

For the metric, we just need to define a metric of meter type and of name
NumRequestsForwardingToControllerPerSec. Meter exposes a few standard JMX
attributes including an accumulated total and rates (
https://metrics.dropwizard.io/2.2.0/apidocs/com/yammer/metrics/core/Meter.html
).

Jun

On Wed, Jun 10, 2020 at 10:38 AM Boyang Chen 
wrote:

> Thanks Jun for the suggestions! I have addressed suggestion and simplified
> the metrics part.
>
> On Tue, Jun 9, 2020 at 5:46 PM Jun Rao  wrote:
>
> > Hi, Boyang,
> >
> > Thanks for the KIP. Just a few comments on the metrics.
> >
> > 1. It would be useful to document the full JMX metric names (package,
> type,
> > etc) of the new metrics. Also, for rates on the server side, we
> > typically use Yammer Meter.
> >
> >  Sounds good.
>
> 2. For num-messages-redirected-rate, would num-requests-redirected-rate be
> > better?
> >
> > Actually for the scope of this KIP, we are no longer needing to have a
> separate tracking
> of forwarded request rate, because the Envelope RPC is dropped.
>
>
> > 3. num-client-forwarding-to-controller-rate: Is that based on clientId,
> > client IP, client request version or sth else? How do you plan to
> implement
> > that since it seems to require tracking the current unique client set
> > somehow. An alternative approach is to maintain a
> > num-requests-redirected-rate metric with a client tag.
> >
> The clientId tag approach makes sense, will add to the KIP.
>
> Jun
> >
> >
> >
> > On Mon, Jun 8, 2020 at 9:36 AM Boyang Chen 
> > wrote:
> >
> > > Hey there,
> > >
> > > If no more question is raised, I will go ahead and start the vote
> > shortly.
> > >
> > > On Thu, Jun 4, 2020 at 12:39 PM Boyang Chen <
> reluctanthero...@gmail.com>
> > > wrote:
> > >
> > > > Hey there,
> > > >
> > > > bumping this thread for any further KIP-590 discussion, since it's
> been
> > > > quiet for a while.
> > > >
> > > > Boyang
> > > >
> > > > On Thu, May 21, 2020 at 10:31 AM Boyang Chen <
> > reluctanthero...@gmail.com
> > > >
> > > > wrote:
> > > >
> > > >> Thanks David, I agree the wording here is not clear, and the fellow
> > > >> broker should just send a new CreateTopicRequest in this case.
> > > >>
> > > >> In the meantime, we had some offline discussion about the Envelope
> API
> > > as
> > > >> well. Although it provides certain privileges like data embedding
> and
> > > >> principal embedding, it creates a security hole by letting a
> malicious
> > > user
> > > >> impersonate any forwarding broker, thus pretending to be any admin
> > user.
> > > >> Passing the principal around also increases the vulnerability,
> > compared
> > > >> with other standard ways such as passing a verified token, but it is
> > > >> unfortunately not fully supported with Kafka security.
> > > >>
> > > >> So for the security concerns, we are abandoning the Envelope
> approach
> > > and
> > > >> fallback to just forward the raw admin requests. The authentication
> > will
> > > >> happen on the receiving broker side instead of on the controller, so
> > > that
> > > >> we are able to stripped off the principal fields and only include
> the
> > > >> principal in header as optional field for audit logging purpose.
> > > >> Furthermore, we shall propose adding a separate endpoint for
> > > >> broker-controller communication which should be recommended to
> enable
> > > >> secure connections so that a malicious client could not pretend to
> be
> > a
> > > >> broker and perform impersonation.
> > > >>
> > > >> Let me know your thoughts.
> > > >>
> > > >> Best,
> > > >> Boyang
> > > >>
> > > >> On Tue, May 19, 2020 at 12:17 AM David Jacot 
> > > wrote:
> > > >>
> > > >>> Hi Boyang,
> > > >>>
> > > >>> I've got another question regarding the auto topic creation case.
> The
> > > KIP
> > > >>> says: "Currently the target broker shall just utilize its own ZK
> > client
> > > >>> to
> > > >>> create
> > > >>> internal topics, which is disallowed in the bridge release. For
> above
> > > >>> scenarios,
> > > >>> non-controller broker shall just forward a CreateTopicRequest to
> the
> > > >>> controller
> > > >>> instead and let controller take care of the rest, while waiting for
> > the
> > > >>> response
> > > >>> in the meantime." There will be no request to forward in this case,
> > > >>> right?
> > > >>> Instead,
> > > >>> a CreateTopicsRequest is created and sent to the controller node.
> > > >>>
> > > >>> When the CreateTopicsRequest is sent as a side effect of the
> > > >>> MetadataRequest,
> > > >>> it would be good to know the principal and the clientId in the
> > > controller
> > > >>> (quota,
> > > >>> audit, etc.). Do you plan to use the Envelope API for this case as
> > well
> > > >>> or
> > > >>> to call
> > > >>> the regular API directly? Another was to phrase it would be: Shall
> > the
> > > >>> internal
> > > >>> CreateTopicsRequest be sent with the original metadata (principal,
> > > >>> clientId, etc.)
> > > >>> of the 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-06-10 Thread Boyang Chen
Thanks Jun for the suggestions! I have addressed suggestion and simplified
the metrics part.

On Tue, Jun 9, 2020 at 5:46 PM Jun Rao  wrote:

> Hi, Boyang,
>
> Thanks for the KIP. Just a few comments on the metrics.
>
> 1. It would be useful to document the full JMX metric names (package, type,
> etc) of the new metrics. Also, for rates on the server side, we
> typically use Yammer Meter.
>
>  Sounds good.

2. For num-messages-redirected-rate, would num-requests-redirected-rate be
> better?
>
> Actually for the scope of this KIP, we are no longer needing to have a
separate tracking
of forwarded request rate, because the Envelope RPC is dropped.


> 3. num-client-forwarding-to-controller-rate: Is that based on clientId,
> client IP, client request version or sth else? How do you plan to implement
> that since it seems to require tracking the current unique client set
> somehow. An alternative approach is to maintain a
> num-requests-redirected-rate metric with a client tag.
>
The clientId tag approach makes sense, will add to the KIP.

Jun
>
>
>
> On Mon, Jun 8, 2020 at 9:36 AM Boyang Chen 
> wrote:
>
> > Hey there,
> >
> > If no more question is raised, I will go ahead and start the vote
> shortly.
> >
> > On Thu, Jun 4, 2020 at 12:39 PM Boyang Chen 
> > wrote:
> >
> > > Hey there,
> > >
> > > bumping this thread for any further KIP-590 discussion, since it's been
> > > quiet for a while.
> > >
> > > Boyang
> > >
> > > On Thu, May 21, 2020 at 10:31 AM Boyang Chen <
> reluctanthero...@gmail.com
> > >
> > > wrote:
> > >
> > >> Thanks David, I agree the wording here is not clear, and the fellow
> > >> broker should just send a new CreateTopicRequest in this case.
> > >>
> > >> In the meantime, we had some offline discussion about the Envelope API
> > as
> > >> well. Although it provides certain privileges like data embedding and
> > >> principal embedding, it creates a security hole by letting a malicious
> > user
> > >> impersonate any forwarding broker, thus pretending to be any admin
> user.
> > >> Passing the principal around also increases the vulnerability,
> compared
> > >> with other standard ways such as passing a verified token, but it is
> > >> unfortunately not fully supported with Kafka security.
> > >>
> > >> So for the security concerns, we are abandoning the Envelope approach
> > and
> > >> fallback to just forward the raw admin requests. The authentication
> will
> > >> happen on the receiving broker side instead of on the controller, so
> > that
> > >> we are able to stripped off the principal fields and only include the
> > >> principal in header as optional field for audit logging purpose.
> > >> Furthermore, we shall propose adding a separate endpoint for
> > >> broker-controller communication which should be recommended to enable
> > >> secure connections so that a malicious client could not pretend to be
> a
> > >> broker and perform impersonation.
> > >>
> > >> Let me know your thoughts.
> > >>
> > >> Best,
> > >> Boyang
> > >>
> > >> On Tue, May 19, 2020 at 12:17 AM David Jacot 
> > wrote:
> > >>
> > >>> Hi Boyang,
> > >>>
> > >>> I've got another question regarding the auto topic creation case. The
> > KIP
> > >>> says: "Currently the target broker shall just utilize its own ZK
> client
> > >>> to
> > >>> create
> > >>> internal topics, which is disallowed in the bridge release. For above
> > >>> scenarios,
> > >>> non-controller broker shall just forward a CreateTopicRequest to the
> > >>> controller
> > >>> instead and let controller take care of the rest, while waiting for
> the
> > >>> response
> > >>> in the meantime." There will be no request to forward in this case,
> > >>> right?
> > >>> Instead,
> > >>> a CreateTopicsRequest is created and sent to the controller node.
> > >>>
> > >>> When the CreateTopicsRequest is sent as a side effect of the
> > >>> MetadataRequest,
> > >>> it would be good to know the principal and the clientId in the
> > controller
> > >>> (quota,
> > >>> audit, etc.). Do you plan to use the Envelope API for this case as
> well
> > >>> or
> > >>> to call
> > >>> the regular API directly? Another was to phrase it would be: Shall
> the
> > >>> internal
> > >>> CreateTopicsRequest be sent with the original metadata (principal,
> > >>> clientId, etc.)
> > >>> of the MetadataRequest or as an admin request?
> > >>>
> > >>> Best,
> > >>> David
> > >>>
> > >>> On Fri, May 8, 2020 at 2:04 AM Guozhang Wang 
> > wrote:
> > >>>
> > >>> > Just to adds a bit more FYI here related to the last question from
> > >>> David:
> > >>> > in KIP-595 while implementing the new requests we are also adding a
> > >>> > "KafkaNetworkChannel" which is used for brokers to send vote /
> fetch
> > >>> > records, so besides the discussion on listeners I think
> > implementation
> > >>> wise
> > >>> > we can also consider consolidating a lot of those into the same
> > >>> call-trace
> > >>> > as well -- of course this is not related to public APIs so maybe
> just
> > >>> needs
> > >>> > 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-06-09 Thread Jun Rao
Hi, Boyang,

Thanks for the KIP. Just a few comments on the metrics.

1. It would be useful to document the full JMX metric names (package, type,
etc) of the new metrics. Also, for rates on the server side, we
typically use Yammer Meter.

2. For num-messages-redirected-rate, would num-requests-redirected-rate be
better?

3. num-client-forwarding-to-controller-rate: Is that based on clientId,
client IP, client request version or sth else? How do you plan to implement
that since it seems to require tracking the current unique client set
somehow. An alternative approach is to maintain a
num-requests-redirected-rate metric with a client tag.

Jun



On Mon, Jun 8, 2020 at 9:36 AM Boyang Chen 
wrote:

> Hey there,
>
> If no more question is raised, I will go ahead and start the vote shortly.
>
> On Thu, Jun 4, 2020 at 12:39 PM Boyang Chen 
> wrote:
>
> > Hey there,
> >
> > bumping this thread for any further KIP-590 discussion, since it's been
> > quiet for a while.
> >
> > Boyang
> >
> > On Thu, May 21, 2020 at 10:31 AM Boyang Chen  >
> > wrote:
> >
> >> Thanks David, I agree the wording here is not clear, and the fellow
> >> broker should just send a new CreateTopicRequest in this case.
> >>
> >> In the meantime, we had some offline discussion about the Envelope API
> as
> >> well. Although it provides certain privileges like data embedding and
> >> principal embedding, it creates a security hole by letting a malicious
> user
> >> impersonate any forwarding broker, thus pretending to be any admin user.
> >> Passing the principal around also increases the vulnerability, compared
> >> with other standard ways such as passing a verified token, but it is
> >> unfortunately not fully supported with Kafka security.
> >>
> >> So for the security concerns, we are abandoning the Envelope approach
> and
> >> fallback to just forward the raw admin requests. The authentication will
> >> happen on the receiving broker side instead of on the controller, so
> that
> >> we are able to stripped off the principal fields and only include the
> >> principal in header as optional field for audit logging purpose.
> >> Furthermore, we shall propose adding a separate endpoint for
> >> broker-controller communication which should be recommended to enable
> >> secure connections so that a malicious client could not pretend to be a
> >> broker and perform impersonation.
> >>
> >> Let me know your thoughts.
> >>
> >> Best,
> >> Boyang
> >>
> >> On Tue, May 19, 2020 at 12:17 AM David Jacot 
> wrote:
> >>
> >>> Hi Boyang,
> >>>
> >>> I've got another question regarding the auto topic creation case. The
> KIP
> >>> says: "Currently the target broker shall just utilize its own ZK client
> >>> to
> >>> create
> >>> internal topics, which is disallowed in the bridge release. For above
> >>> scenarios,
> >>> non-controller broker shall just forward a CreateTopicRequest to the
> >>> controller
> >>> instead and let controller take care of the rest, while waiting for the
> >>> response
> >>> in the meantime." There will be no request to forward in this case,
> >>> right?
> >>> Instead,
> >>> a CreateTopicsRequest is created and sent to the controller node.
> >>>
> >>> When the CreateTopicsRequest is sent as a side effect of the
> >>> MetadataRequest,
> >>> it would be good to know the principal and the clientId in the
> controller
> >>> (quota,
> >>> audit, etc.). Do you plan to use the Envelope API for this case as well
> >>> or
> >>> to call
> >>> the regular API directly? Another was to phrase it would be: Shall the
> >>> internal
> >>> CreateTopicsRequest be sent with the original metadata (principal,
> >>> clientId, etc.)
> >>> of the MetadataRequest or as an admin request?
> >>>
> >>> Best,
> >>> David
> >>>
> >>> On Fri, May 8, 2020 at 2:04 AM Guozhang Wang 
> wrote:
> >>>
> >>> > Just to adds a bit more FYI here related to the last question from
> >>> David:
> >>> > in KIP-595 while implementing the new requests we are also adding a
> >>> > "KafkaNetworkChannel" which is used for brokers to send vote / fetch
> >>> > records, so besides the discussion on listeners I think
> implementation
> >>> wise
> >>> > we can also consider consolidating a lot of those into the same
> >>> call-trace
> >>> > as well -- of course this is not related to public APIs so maybe just
> >>> needs
> >>> > to be coordinated among developments:
> >>> >
> >>> > 1. Broker -> Controller: ISR Change, Topic Creation, Admin Redirect
> >>> > (KIP-497).
> >>> > 2. Controller -> Broker: LeaderAndISR / MetadataUpdate; though these
> >>> are
> >>> > likely going to be deprecated post KIP-500.
> >>> > 3. Txn Coordinator -> Broker: TxnMarker
> >>> >
> >>> >
> >>> > Guozhang
> >>> >
> >>> > On Wed, May 6, 2020 at 8:58 PM Boyang Chen <
> reluctanthero...@gmail.com
> >>> >
> >>> > wrote:
> >>> >
> >>> > > Hey David,
> >>> > >
> >>> > > thanks for the feedbacks!
> >>> > >
> >>> > > On Wed, May 6, 2020 at 2:06 AM David Jacot 
> >>> wrote:
> >>> > >
> >>> > > > Hi Boyang,
> >>> > 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-06-08 Thread Boyang Chen
Hey there,

If no more question is raised, I will go ahead and start the vote shortly.

On Thu, Jun 4, 2020 at 12:39 PM Boyang Chen 
wrote:

> Hey there,
>
> bumping this thread for any further KIP-590 discussion, since it's been
> quiet for a while.
>
> Boyang
>
> On Thu, May 21, 2020 at 10:31 AM Boyang Chen 
> wrote:
>
>> Thanks David, I agree the wording here is not clear, and the fellow
>> broker should just send a new CreateTopicRequest in this case.
>>
>> In the meantime, we had some offline discussion about the Envelope API as
>> well. Although it provides certain privileges like data embedding and
>> principal embedding, it creates a security hole by letting a malicious user
>> impersonate any forwarding broker, thus pretending to be any admin user.
>> Passing the principal around also increases the vulnerability, compared
>> with other standard ways such as passing a verified token, but it is
>> unfortunately not fully supported with Kafka security.
>>
>> So for the security concerns, we are abandoning the Envelope approach and
>> fallback to just forward the raw admin requests. The authentication will
>> happen on the receiving broker side instead of on the controller, so that
>> we are able to stripped off the principal fields and only include the
>> principal in header as optional field for audit logging purpose.
>> Furthermore, we shall propose adding a separate endpoint for
>> broker-controller communication which should be recommended to enable
>> secure connections so that a malicious client could not pretend to be a
>> broker and perform impersonation.
>>
>> Let me know your thoughts.
>>
>> Best,
>> Boyang
>>
>> On Tue, May 19, 2020 at 12:17 AM David Jacot  wrote:
>>
>>> Hi Boyang,
>>>
>>> I've got another question regarding the auto topic creation case. The KIP
>>> says: "Currently the target broker shall just utilize its own ZK client
>>> to
>>> create
>>> internal topics, which is disallowed in the bridge release. For above
>>> scenarios,
>>> non-controller broker shall just forward a CreateTopicRequest to the
>>> controller
>>> instead and let controller take care of the rest, while waiting for the
>>> response
>>> in the meantime." There will be no request to forward in this case,
>>> right?
>>> Instead,
>>> a CreateTopicsRequest is created and sent to the controller node.
>>>
>>> When the CreateTopicsRequest is sent as a side effect of the
>>> MetadataRequest,
>>> it would be good to know the principal and the clientId in the controller
>>> (quota,
>>> audit, etc.). Do you plan to use the Envelope API for this case as well
>>> or
>>> to call
>>> the regular API directly? Another was to phrase it would be: Shall the
>>> internal
>>> CreateTopicsRequest be sent with the original metadata (principal,
>>> clientId, etc.)
>>> of the MetadataRequest or as an admin request?
>>>
>>> Best,
>>> David
>>>
>>> On Fri, May 8, 2020 at 2:04 AM Guozhang Wang  wrote:
>>>
>>> > Just to adds a bit more FYI here related to the last question from
>>> David:
>>> > in KIP-595 while implementing the new requests we are also adding a
>>> > "KafkaNetworkChannel" which is used for brokers to send vote / fetch
>>> > records, so besides the discussion on listeners I think implementation
>>> wise
>>> > we can also consider consolidating a lot of those into the same
>>> call-trace
>>> > as well -- of course this is not related to public APIs so maybe just
>>> needs
>>> > to be coordinated among developments:
>>> >
>>> > 1. Broker -> Controller: ISR Change, Topic Creation, Admin Redirect
>>> > (KIP-497).
>>> > 2. Controller -> Broker: LeaderAndISR / MetadataUpdate; though these
>>> are
>>> > likely going to be deprecated post KIP-500.
>>> > 3. Txn Coordinator -> Broker: TxnMarker
>>> >
>>> >
>>> > Guozhang
>>> >
>>> > On Wed, May 6, 2020 at 8:58 PM Boyang Chen >> >
>>> > wrote:
>>> >
>>> > > Hey David,
>>> > >
>>> > > thanks for the feedbacks!
>>> > >
>>> > > On Wed, May 6, 2020 at 2:06 AM David Jacot 
>>> wrote:
>>> > >
>>> > > > Hi Boyang,
>>> > > >
>>> > > > While re-reading the KIP, I've got few small questions/comments:
>>> > > >
>>> > > > 1. When auto topic creation is enabled, brokers will send a
>>> > > > CreateTopicRequest
>>> > > > to the controller instead of writing to ZK directly. It means that
>>> > > > creation of these
>>> > > > topics are subject to be rejected with an error if a
>>> CreateTopicPolicy
>>> > is
>>> > > > used. Today,
>>> > > > it bypasses the policy entirely. I suppose that clusters allowing
>>> auto
>>> > > > topic creation
>>> > > > don't have a policy in place so it is not a big deal. I suggest to
>>> call
>>> > > > out explicitly the
>>> > > > limitation in the KIP though.
>>> > > >
>>> > > > That's a good idea, will add to the KIP.
>>> > >
>>> > >
>>> > > > 2. In the same vein as my first point. How do you plan to handle
>>> errors
>>> > > > when internal
>>> > > > topics are created by a broker? Do you plan to retry retryable
>>> errors
>>> > > > indefinitely?
>>> > > >
>>> > 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-06-04 Thread Boyang Chen
Hey there,

bumping this thread for any further KIP-590 discussion, since it's been
quiet for a while.

Boyang

On Thu, May 21, 2020 at 10:31 AM Boyang Chen 
wrote:

> Thanks David, I agree the wording here is not clear, and the fellow broker
> should just send a new CreateTopicRequest in this case.
>
> In the meantime, we had some offline discussion about the Envelope API as
> well. Although it provides certain privileges like data embedding and
> principal embedding, it creates a security hole by letting a malicious user
> impersonate any forwarding broker, thus pretending to be any admin user.
> Passing the principal around also increases the vulnerability, compared
> with other standard ways such as passing a verified token, but it is
> unfortunately not fully supported with Kafka security.
>
> So for the security concerns, we are abandoning the Envelope approach and
> fallback to just forward the raw admin requests. The authentication will
> happen on the receiving broker side instead of on the controller, so that
> we are able to stripped off the principal fields and only include the
> principal in header as optional field for audit logging purpose.
> Furthermore, we shall propose adding a separate endpoint for
> broker-controller communication which should be recommended to enable
> secure connections so that a malicious client could not pretend to be a
> broker and perform impersonation.
>
> Let me know your thoughts.
>
> Best,
> Boyang
>
> On Tue, May 19, 2020 at 12:17 AM David Jacot  wrote:
>
>> Hi Boyang,
>>
>> I've got another question regarding the auto topic creation case. The KIP
>> says: "Currently the target broker shall just utilize its own ZK client to
>> create
>> internal topics, which is disallowed in the bridge release. For above
>> scenarios,
>> non-controller broker shall just forward a CreateTopicRequest to the
>> controller
>> instead and let controller take care of the rest, while waiting for the
>> response
>> in the meantime." There will be no request to forward in this case, right?
>> Instead,
>> a CreateTopicsRequest is created and sent to the controller node.
>>
>> When the CreateTopicsRequest is sent as a side effect of the
>> MetadataRequest,
>> it would be good to know the principal and the clientId in the controller
>> (quota,
>> audit, etc.). Do you plan to use the Envelope API for this case as well or
>> to call
>> the regular API directly? Another was to phrase it would be: Shall the
>> internal
>> CreateTopicsRequest be sent with the original metadata (principal,
>> clientId, etc.)
>> of the MetadataRequest or as an admin request?
>>
>> Best,
>> David
>>
>> On Fri, May 8, 2020 at 2:04 AM Guozhang Wang  wrote:
>>
>> > Just to adds a bit more FYI here related to the last question from
>> David:
>> > in KIP-595 while implementing the new requests we are also adding a
>> > "KafkaNetworkChannel" which is used for brokers to send vote / fetch
>> > records, so besides the discussion on listeners I think implementation
>> wise
>> > we can also consider consolidating a lot of those into the same
>> call-trace
>> > as well -- of course this is not related to public APIs so maybe just
>> needs
>> > to be coordinated among developments:
>> >
>> > 1. Broker -> Controller: ISR Change, Topic Creation, Admin Redirect
>> > (KIP-497).
>> > 2. Controller -> Broker: LeaderAndISR / MetadataUpdate; though these are
>> > likely going to be deprecated post KIP-500.
>> > 3. Txn Coordinator -> Broker: TxnMarker
>> >
>> >
>> > Guozhang
>> >
>> > On Wed, May 6, 2020 at 8:58 PM Boyang Chen 
>> > wrote:
>> >
>> > > Hey David,
>> > >
>> > > thanks for the feedbacks!
>> > >
>> > > On Wed, May 6, 2020 at 2:06 AM David Jacot 
>> wrote:
>> > >
>> > > > Hi Boyang,
>> > > >
>> > > > While re-reading the KIP, I've got few small questions/comments:
>> > > >
>> > > > 1. When auto topic creation is enabled, brokers will send a
>> > > > CreateTopicRequest
>> > > > to the controller instead of writing to ZK directly. It means that
>> > > > creation of these
>> > > > topics are subject to be rejected with an error if a
>> CreateTopicPolicy
>> > is
>> > > > used. Today,
>> > > > it bypasses the policy entirely. I suppose that clusters allowing
>> auto
>> > > > topic creation
>> > > > don't have a policy in place so it is not a big deal. I suggest to
>> call
>> > > > out explicitly the
>> > > > limitation in the KIP though.
>> > > >
>> > > > That's a good idea, will add to the KIP.
>> > >
>> > >
>> > > > 2. In the same vein as my first point. How do you plan to handle
>> errors
>> > > > when internal
>> > > > topics are created by a broker? Do you plan to retry retryable
>> errors
>> > > > indefinitely?
>> > > >
>> > > > I checked a bit on the admin client handling of the create topic
>> RPC.
>> > It
>> > > seems that
>> > > the only retriable exceptions at the moment are NOT_CONTROLLER and
>> > > REQUEST_TIMEOUT.
>> > > So I guess we just need to retry on these exceptions?
>> > >
>> > >
>> > > > 3. Could you 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-05-21 Thread Boyang Chen
Thanks David, I agree the wording here is not clear, and the fellow broker
should just send a new CreateTopicRequest in this case.

In the meantime, we had some offline discussion about the Envelope API as
well. Although it provides certain privileges like data embedding and
principal embedding, it creates a security hole by letting a malicious user
impersonate any forwarding broker, thus pretending to be any admin user.
Passing the principal around also increases the vulnerability, compared
with other standard ways such as passing a verified token, but it is
unfortunately not fully supported with Kafka security.

So for the security concerns, we are abandoning the Envelope approach and
fallback to just forward the raw admin requests. The authentication will
happen on the receiving broker side instead of on the controller, so that
we are able to stripped off the principal fields and only include the
principal in header as optional field for audit logging purpose.
Furthermore, we shall propose adding a separate endpoint for
broker-controller communication which should be recommended to enable
secure connections so that a malicious client could not pretend to be a
broker and perform impersonation.

Let me know your thoughts.

Best,
Boyang

On Tue, May 19, 2020 at 12:17 AM David Jacot  wrote:

> Hi Boyang,
>
> I've got another question regarding the auto topic creation case. The KIP
> says: "Currently the target broker shall just utilize its own ZK client to
> create
> internal topics, which is disallowed in the bridge release. For above
> scenarios,
> non-controller broker shall just forward a CreateTopicRequest to the
> controller
> instead and let controller take care of the rest, while waiting for the
> response
> in the meantime." There will be no request to forward in this case, right?
> Instead,
> a CreateTopicsRequest is created and sent to the controller node.
>
> When the CreateTopicsRequest is sent as a side effect of the
> MetadataRequest,
> it would be good to know the principal and the clientId in the controller
> (quota,
> audit, etc.). Do you plan to use the Envelope API for this case as well or
> to call
> the regular API directly? Another was to phrase it would be: Shall the
> internal
> CreateTopicsRequest be sent with the original metadata (principal,
> clientId, etc.)
> of the MetadataRequest or as an admin request?
>
> Best,
> David
>
> On Fri, May 8, 2020 at 2:04 AM Guozhang Wang  wrote:
>
> > Just to adds a bit more FYI here related to the last question from David:
> > in KIP-595 while implementing the new requests we are also adding a
> > "KafkaNetworkChannel" which is used for brokers to send vote / fetch
> > records, so besides the discussion on listeners I think implementation
> wise
> > we can also consider consolidating a lot of those into the same
> call-trace
> > as well -- of course this is not related to public APIs so maybe just
> needs
> > to be coordinated among developments:
> >
> > 1. Broker -> Controller: ISR Change, Topic Creation, Admin Redirect
> > (KIP-497).
> > 2. Controller -> Broker: LeaderAndISR / MetadataUpdate; though these are
> > likely going to be deprecated post KIP-500.
> > 3. Txn Coordinator -> Broker: TxnMarker
> >
> >
> > Guozhang
> >
> > On Wed, May 6, 2020 at 8:58 PM Boyang Chen 
> > wrote:
> >
> > > Hey David,
> > >
> > > thanks for the feedbacks!
> > >
> > > On Wed, May 6, 2020 at 2:06 AM David Jacot 
> wrote:
> > >
> > > > Hi Boyang,
> > > >
> > > > While re-reading the KIP, I've got few small questions/comments:
> > > >
> > > > 1. When auto topic creation is enabled, brokers will send a
> > > > CreateTopicRequest
> > > > to the controller instead of writing to ZK directly. It means that
> > > > creation of these
> > > > topics are subject to be rejected with an error if a
> CreateTopicPolicy
> > is
> > > > used. Today,
> > > > it bypasses the policy entirely. I suppose that clusters allowing
> auto
> > > > topic creation
> > > > don't have a policy in place so it is not a big deal. I suggest to
> call
> > > > out explicitly the
> > > > limitation in the KIP though.
> > > >
> > > > That's a good idea, will add to the KIP.
> > >
> > >
> > > > 2. In the same vein as my first point. How do you plan to handle
> errors
> > > > when internal
> > > > topics are created by a broker? Do you plan to retry retryable errors
> > > > indefinitely?
> > > >
> > > > I checked a bit on the admin client handling of the create topic RPC.
> > It
> > > seems that
> > > the only retriable exceptions at the moment are NOT_CONTROLLER and
> > > REQUEST_TIMEOUT.
> > > So I guess we just need to retry on these exceptions?
> > >
> > >
> > > > 3. Could you clarify which listener will be used for the internal
> > > requests?
> > > > Do you plan
> > > > to use the control plane listener or perhaps the inter-broker
> listener?
> > > >
> > > > As we discussed in the KIP, currently the internal design for
> > > broker->controller channel has not been
> > > done yet, and I feel it 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-05-19 Thread David Jacot
Hi Boyang,

I've got another question regarding the auto topic creation case. The KIP
says: "Currently the target broker shall just utilize its own ZK client to
create
internal topics, which is disallowed in the bridge release. For above
scenarios,
non-controller broker shall just forward a CreateTopicRequest to the
controller
instead and let controller take care of the rest, while waiting for the
response
in the meantime." There will be no request to forward in this case, right?
Instead,
a CreateTopicsRequest is created and sent to the controller node.

When the CreateTopicsRequest is sent as a side effect of the
MetadataRequest,
it would be good to know the principal and the clientId in the controller
(quota,
audit, etc.). Do you plan to use the Envelope API for this case as well or
to call
the regular API directly? Another was to phrase it would be: Shall the
internal
CreateTopicsRequest be sent with the original metadata (principal,
clientId, etc.)
of the MetadataRequest or as an admin request?

Best,
David

On Fri, May 8, 2020 at 2:04 AM Guozhang Wang  wrote:

> Just to adds a bit more FYI here related to the last question from David:
> in KIP-595 while implementing the new requests we are also adding a
> "KafkaNetworkChannel" which is used for brokers to send vote / fetch
> records, so besides the discussion on listeners I think implementation wise
> we can also consider consolidating a lot of those into the same call-trace
> as well -- of course this is not related to public APIs so maybe just needs
> to be coordinated among developments:
>
> 1. Broker -> Controller: ISR Change, Topic Creation, Admin Redirect
> (KIP-497).
> 2. Controller -> Broker: LeaderAndISR / MetadataUpdate; though these are
> likely going to be deprecated post KIP-500.
> 3. Txn Coordinator -> Broker: TxnMarker
>
>
> Guozhang
>
> On Wed, May 6, 2020 at 8:58 PM Boyang Chen 
> wrote:
>
> > Hey David,
> >
> > thanks for the feedbacks!
> >
> > On Wed, May 6, 2020 at 2:06 AM David Jacot  wrote:
> >
> > > Hi Boyang,
> > >
> > > While re-reading the KIP, I've got few small questions/comments:
> > >
> > > 1. When auto topic creation is enabled, brokers will send a
> > > CreateTopicRequest
> > > to the controller instead of writing to ZK directly. It means that
> > > creation of these
> > > topics are subject to be rejected with an error if a CreateTopicPolicy
> is
> > > used. Today,
> > > it bypasses the policy entirely. I suppose that clusters allowing auto
> > > topic creation
> > > don't have a policy in place so it is not a big deal. I suggest to call
> > > out explicitly the
> > > limitation in the KIP though.
> > >
> > > That's a good idea, will add to the KIP.
> >
> >
> > > 2. In the same vein as my first point. How do you plan to handle errors
> > > when internal
> > > topics are created by a broker? Do you plan to retry retryable errors
> > > indefinitely?
> > >
> > > I checked a bit on the admin client handling of the create topic RPC.
> It
> > seems that
> > the only retriable exceptions at the moment are NOT_CONTROLLER and
> > REQUEST_TIMEOUT.
> > So I guess we just need to retry on these exceptions?
> >
> >
> > > 3. Could you clarify which listener will be used for the internal
> > requests?
> > > Do you plan
> > > to use the control plane listener or perhaps the inter-broker listener?
> > >
> > > As we discussed in the KIP, currently the internal design for
> > broker->controller channel has not been
> > done yet, and I feel it makes sense to consolidate redirect RPC and
> > internal topic creation RPC to this future channel,
> > which are details to be filled in the near future, right now some
> > controller refactoring effort is still WIP.
> >
> >
> > > Thanks,
> > > David
> > >
> > > On Mon, May 4, 2020 at 9:37 AM Sönke Liebau
> > >  wrote:
> > >
> > > > Ah, I see, thanks for the clarification!
> > > >
> > > > Shouldn't be an issue I think. My understanding of KIPs was always
> that
> > > > they are mostly intended as a place to discuss and agree changes up
> > > front,
> > > > whereas tracking the actual releases that things go into should be
> > > handled
> > > > in Jira.
> > > > So maybe we just create new jiras for any subsequent work and either
> > link
> > > > those or make them subtasks (even though this jira is already a
> subtask
> > > > itself), that should allow us to properly track all releases that
> work
> > > goes
> > > > into.
> > > >
> > > > Thanks for your work on this!!
> > > >
> > > > Best,
> > > > Sönke
> > > >
> > > >
> > > > On Sat, 2 May 2020 at 00:31, Boyang Chen  >
> > > > wrote:
> > > >
> > > > > Sure thing Sonke, what I suggest is that usual KIPs get accepted to
> > go
> > > > into
> > > > > next release. It could span for a couple of releases because of
> > > > engineering
> > > > > time, but no change has to be shipped in specific future releases,
> > like
> > > > the
> > > > > backward incompatible change for KafkaPrincipal. But I guess it's
> not
> > > > > really a blocker, as long as we 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-05-07 Thread Guozhang Wang
Just to adds a bit more FYI here related to the last question from David:
in KIP-595 while implementing the new requests we are also adding a
"KafkaNetworkChannel" which is used for brokers to send vote / fetch
records, so besides the discussion on listeners I think implementation wise
we can also consider consolidating a lot of those into the same call-trace
as well -- of course this is not related to public APIs so maybe just needs
to be coordinated among developments:

1. Broker -> Controller: ISR Change, Topic Creation, Admin Redirect
(KIP-497).
2. Controller -> Broker: LeaderAndISR / MetadataUpdate; though these are
likely going to be deprecated post KIP-500.
3. Txn Coordinator -> Broker: TxnMarker


Guozhang

On Wed, May 6, 2020 at 8:58 PM Boyang Chen 
wrote:

> Hey David,
>
> thanks for the feedbacks!
>
> On Wed, May 6, 2020 at 2:06 AM David Jacot  wrote:
>
> > Hi Boyang,
> >
> > While re-reading the KIP, I've got few small questions/comments:
> >
> > 1. When auto topic creation is enabled, brokers will send a
> > CreateTopicRequest
> > to the controller instead of writing to ZK directly. It means that
> > creation of these
> > topics are subject to be rejected with an error if a CreateTopicPolicy is
> > used. Today,
> > it bypasses the policy entirely. I suppose that clusters allowing auto
> > topic creation
> > don't have a policy in place so it is not a big deal. I suggest to call
> > out explicitly the
> > limitation in the KIP though.
> >
> > That's a good idea, will add to the KIP.
>
>
> > 2. In the same vein as my first point. How do you plan to handle errors
> > when internal
> > topics are created by a broker? Do you plan to retry retryable errors
> > indefinitely?
> >
> > I checked a bit on the admin client handling of the create topic RPC. It
> seems that
> the only retriable exceptions at the moment are NOT_CONTROLLER and
> REQUEST_TIMEOUT.
> So I guess we just need to retry on these exceptions?
>
>
> > 3. Could you clarify which listener will be used for the internal
> requests?
> > Do you plan
> > to use the control plane listener or perhaps the inter-broker listener?
> >
> > As we discussed in the KIP, currently the internal design for
> broker->controller channel has not been
> done yet, and I feel it makes sense to consolidate redirect RPC and
> internal topic creation RPC to this future channel,
> which are details to be filled in the near future, right now some
> controller refactoring effort is still WIP.
>
>
> > Thanks,
> > David
> >
> > On Mon, May 4, 2020 at 9:37 AM Sönke Liebau
> >  wrote:
> >
> > > Ah, I see, thanks for the clarification!
> > >
> > > Shouldn't be an issue I think. My understanding of KIPs was always that
> > > they are mostly intended as a place to discuss and agree changes up
> > front,
> > > whereas tracking the actual releases that things go into should be
> > handled
> > > in Jira.
> > > So maybe we just create new jiras for any subsequent work and either
> link
> > > those or make them subtasks (even though this jira is already a subtask
> > > itself), that should allow us to properly track all releases that work
> > goes
> > > into.
> > >
> > > Thanks for your work on this!!
> > >
> > > Best,
> > > Sönke
> > >
> > >
> > > On Sat, 2 May 2020 at 00:31, Boyang Chen 
> > > wrote:
> > >
> > > > Sure thing Sonke, what I suggest is that usual KIPs get accepted to
> go
> > > into
> > > > next release. It could span for a couple of releases because of
> > > engineering
> > > > time, but no change has to be shipped in specific future releases,
> like
> > > the
> > > > backward incompatible change for KafkaPrincipal. But I guess it's not
> > > > really a blocker, as long as we stated clearly in the KIP how we are
> > > going
> > > > to roll things out, and let it partially finish in 2.6.
> > > >
> > > > Boyang
> > > >
> > > > On Fri, May 1, 2020 at 2:32 PM Sönke Liebau
> > > >  wrote:
> > > >
> > > > > Hi Boyang,
> > > > >
> > > > > thanks for the update, sounds reasonable to me. Making it a
> breaking
> > > > change
> > > > > is definitely the safer route to go.
> > > > >
> > > > > Just one quick question regarding your mail, I didn't fully
> > understand
> > > > what
> > > > > you mean by "I think this is the first time we need to introduce a
> > KIP
> > > > > without having it
> > > > > fully accepted in next release."  - could you perhaps explain that
> > some
> > > > > more very briefly?
> > > > >
> > > > > Best regards,
> > > > > Sönke
> > > > >
> > > > >
> > > > >
> > > > > On Fri, 1 May 2020 at 23:03, Boyang Chen <
> reluctanthero...@gmail.com
> > >
> > > > > wrote:
> > > > >
> > > > > > Hey Tom,
> > > > > >
> > > > > > thanks for the suggestion. As long as we could correctly
> serialize
> > > the
> > > > > > principal and embed in the Envelope, I think we could still
> > leverage
> > > > the
> > > > > > controller to do the client request authentication. Although this
> > > pays
> > > > an
> > > > > > extra round trip if the authorization is doomed to 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-05-06 Thread Boyang Chen
Hey David,

thanks for the feedbacks!

On Wed, May 6, 2020 at 2:06 AM David Jacot  wrote:

> Hi Boyang,
>
> While re-reading the KIP, I've got few small questions/comments:
>
> 1. When auto topic creation is enabled, brokers will send a
> CreateTopicRequest
> to the controller instead of writing to ZK directly. It means that
> creation of these
> topics are subject to be rejected with an error if a CreateTopicPolicy is
> used. Today,
> it bypasses the policy entirely. I suppose that clusters allowing auto
> topic creation
> don't have a policy in place so it is not a big deal. I suggest to call
> out explicitly the
> limitation in the KIP though.
>
> That's a good idea, will add to the KIP.


> 2. In the same vein as my first point. How do you plan to handle errors
> when internal
> topics are created by a broker? Do you plan to retry retryable errors
> indefinitely?
>
> I checked a bit on the admin client handling of the create topic RPC. It
seems that
the only retriable exceptions at the moment are NOT_CONTROLLER and
REQUEST_TIMEOUT.
So I guess we just need to retry on these exceptions?


> 3. Could you clarify which listener will be used for the internal requests?
> Do you plan
> to use the control plane listener or perhaps the inter-broker listener?
>
> As we discussed in the KIP, currently the internal design for
broker->controller channel has not been
done yet, and I feel it makes sense to consolidate redirect RPC and
internal topic creation RPC to this future channel,
which are details to be filled in the near future, right now some
controller refactoring effort is still WIP.


> Thanks,
> David
>
> On Mon, May 4, 2020 at 9:37 AM Sönke Liebau
>  wrote:
>
> > Ah, I see, thanks for the clarification!
> >
> > Shouldn't be an issue I think. My understanding of KIPs was always that
> > they are mostly intended as a place to discuss and agree changes up
> front,
> > whereas tracking the actual releases that things go into should be
> handled
> > in Jira.
> > So maybe we just create new jiras for any subsequent work and either link
> > those or make them subtasks (even though this jira is already a subtask
> > itself), that should allow us to properly track all releases that work
> goes
> > into.
> >
> > Thanks for your work on this!!
> >
> > Best,
> > Sönke
> >
> >
> > On Sat, 2 May 2020 at 00:31, Boyang Chen 
> > wrote:
> >
> > > Sure thing Sonke, what I suggest is that usual KIPs get accepted to go
> > into
> > > next release. It could span for a couple of releases because of
> > engineering
> > > time, but no change has to be shipped in specific future releases, like
> > the
> > > backward incompatible change for KafkaPrincipal. But I guess it's not
> > > really a blocker, as long as we stated clearly in the KIP how we are
> > going
> > > to roll things out, and let it partially finish in 2.6.
> > >
> > > Boyang
> > >
> > > On Fri, May 1, 2020 at 2:32 PM Sönke Liebau
> > >  wrote:
> > >
> > > > Hi Boyang,
> > > >
> > > > thanks for the update, sounds reasonable to me. Making it a breaking
> > > change
> > > > is definitely the safer route to go.
> > > >
> > > > Just one quick question regarding your mail, I didn't fully
> understand
> > > what
> > > > you mean by "I think this is the first time we need to introduce a
> KIP
> > > > without having it
> > > > fully accepted in next release."  - could you perhaps explain that
> some
> > > > more very briefly?
> > > >
> > > > Best regards,
> > > > Sönke
> > > >
> > > >
> > > >
> > > > On Fri, 1 May 2020 at 23:03, Boyang Chen  >
> > > > wrote:
> > > >
> > > > > Hey Tom,
> > > > >
> > > > > thanks for the suggestion. As long as we could correctly serialize
> > the
> > > > > principal and embed in the Envelope, I think we could still
> leverage
> > > the
> > > > > controller to do the client request authentication. Although this
> > pays
> > > an
> > > > > extra round trip if the authorization is doomed to fail on the
> > receiver
> > > > > side, having a centralized processing unit is more favorable such
> as
> > > > > ensuring the audit log is consistent instead of scattering between
> > > > > forwarder and receiver.
> > > > >
> > > > > Boyang
> > > > >
> > > > > On Wed, Apr 29, 2020 at 9:50 AM Tom Bentley 
> > > wrote:
> > > > >
> > > > > > Hi Boyang,
> > > > > >
> > > > > > Thanks for the update. In the EnvelopeRequest handling section of
> > the
> > > > KIP
> > > > > > it might be worth saying explicitly that authorization of the
> > request
> > > > > will
> > > > > > happen as normal. Otherwise what you're proposing makes sense to
> > me.
> > > > > >
> > > > > > Thanks again,
> > > > > >
> > > > > > Tom
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Wed, Apr 29, 2020 at 5:27 PM Boyang Chen <
> > > > reluctanthero...@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Thanks for the proposed idea Sonke. I reviewed it and had some
> > > > offline
> > > > > > > discussion with Colin, Rajini and Mathew.
> > > > > > >
> > > > > > > We do need to add 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-05-06 Thread David Jacot
Hi Boyang,

While re-reading the KIP, I've got few small questions/comments:

1. When auto topic creation is enabled, brokers will send a
CreateTopicRequest
to the controller instead of writing to ZK directly. It means that
creation of these
topics are subject to be rejected with an error if a CreateTopicPolicy is
used. Today,
it bypasses the policy entirely. I suppose that clusters allowing auto
topic creation
don't have a policy in place so it is not a big deal. I suggest to call
out explicitly the
limitation in the KIP though.

2. In the same vein as my first point. How do you plan to handle errors
when internal
topics are created by a broker? Do you plan to retry retryable errors
indefinitely?

3. Could you clarify which listener will be used for the internal requests?
Do you plan
to use the control plane listener or perhaps the inter-broker listener?

Thanks,
David

On Mon, May 4, 2020 at 9:37 AM Sönke Liebau
 wrote:

> Ah, I see, thanks for the clarification!
>
> Shouldn't be an issue I think. My understanding of KIPs was always that
> they are mostly intended as a place to discuss and agree changes up front,
> whereas tracking the actual releases that things go into should be handled
> in Jira.
> So maybe we just create new jiras for any subsequent work and either link
> those or make them subtasks (even though this jira is already a subtask
> itself), that should allow us to properly track all releases that work goes
> into.
>
> Thanks for your work on this!!
>
> Best,
> Sönke
>
>
> On Sat, 2 May 2020 at 00:31, Boyang Chen 
> wrote:
>
> > Sure thing Sonke, what I suggest is that usual KIPs get accepted to go
> into
> > next release. It could span for a couple of releases because of
> engineering
> > time, but no change has to be shipped in specific future releases, like
> the
> > backward incompatible change for KafkaPrincipal. But I guess it's not
> > really a blocker, as long as we stated clearly in the KIP how we are
> going
> > to roll things out, and let it partially finish in 2.6.
> >
> > Boyang
> >
> > On Fri, May 1, 2020 at 2:32 PM Sönke Liebau
> >  wrote:
> >
> > > Hi Boyang,
> > >
> > > thanks for the update, sounds reasonable to me. Making it a breaking
> > change
> > > is definitely the safer route to go.
> > >
> > > Just one quick question regarding your mail, I didn't fully understand
> > what
> > > you mean by "I think this is the first time we need to introduce a KIP
> > > without having it
> > > fully accepted in next release."  - could you perhaps explain that some
> > > more very briefly?
> > >
> > > Best regards,
> > > Sönke
> > >
> > >
> > >
> > > On Fri, 1 May 2020 at 23:03, Boyang Chen 
> > > wrote:
> > >
> > > > Hey Tom,
> > > >
> > > > thanks for the suggestion. As long as we could correctly serialize
> the
> > > > principal and embed in the Envelope, I think we could still leverage
> > the
> > > > controller to do the client request authentication. Although this
> pays
> > an
> > > > extra round trip if the authorization is doomed to fail on the
> receiver
> > > > side, having a centralized processing unit is more favorable such as
> > > > ensuring the audit log is consistent instead of scattering between
> > > > forwarder and receiver.
> > > >
> > > > Boyang
> > > >
> > > > On Wed, Apr 29, 2020 at 9:50 AM Tom Bentley 
> > wrote:
> > > >
> > > > > Hi Boyang,
> > > > >
> > > > > Thanks for the update. In the EnvelopeRequest handling section of
> the
> > > KIP
> > > > > it might be worth saying explicitly that authorization of the
> request
> > > > will
> > > > > happen as normal. Otherwise what you're proposing makes sense to
> me.
> > > > >
> > > > > Thanks again,
> > > > >
> > > > > Tom
> > > > >
> > > > >
> > > > >
> > > > > On Wed, Apr 29, 2020 at 5:27 PM Boyang Chen <
> > > reluctanthero...@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Thanks for the proposed idea Sonke. I reviewed it and had some
> > > offline
> > > > > > discussion with Colin, Rajini and Mathew.
> > > > > >
> > > > > > We do need to add serializability to the PrincipalBuilder
> > interface,
> > > > but
> > > > > we
> > > > > > should not make any default implementation which could go wrong
> and
> > > > messy
> > > > > > up with the security in a production environment if the user
> > neglects
> > > > it.
> > > > > > Instead we need to make it required and backward incompatible.
> So I
> > > > > > integrated your proposed methods and expand the Envelope RPC
> with a
> > > > > couple
> > > > > > of more fields for audit log purpose as well.
> > > > > >
> > > > > > Since the KafkaPrincipal builder serializability is a binary
> > > > incompatible
> > > > > > change, I propose (also stated in the KIP) the following
> > > implementation
> > > > > > plan:
> > > > > >
> > > > > >1. For next *2.x* release:
> > > > > >   1. Get new admin client forwarding changes
> > > > > >   2. Get the Envelope RPC implementation
> > > > > >   3. Get the forwarding path working and validate the
> 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-05-04 Thread Sönke Liebau
Ah, I see, thanks for the clarification!

Shouldn't be an issue I think. My understanding of KIPs was always that
they are mostly intended as a place to discuss and agree changes up front,
whereas tracking the actual releases that things go into should be handled
in Jira.
So maybe we just create new jiras for any subsequent work and either link
those or make them subtasks (even though this jira is already a subtask
itself), that should allow us to properly track all releases that work goes
into.

Thanks for your work on this!!

Best,
Sönke


On Sat, 2 May 2020 at 00:31, Boyang Chen  wrote:

> Sure thing Sonke, what I suggest is that usual KIPs get accepted to go into
> next release. It could span for a couple of releases because of engineering
> time, but no change has to be shipped in specific future releases, like the
> backward incompatible change for KafkaPrincipal. But I guess it's not
> really a blocker, as long as we stated clearly in the KIP how we are going
> to roll things out, and let it partially finish in 2.6.
>
> Boyang
>
> On Fri, May 1, 2020 at 2:32 PM Sönke Liebau
>  wrote:
>
> > Hi Boyang,
> >
> > thanks for the update, sounds reasonable to me. Making it a breaking
> change
> > is definitely the safer route to go.
> >
> > Just one quick question regarding your mail, I didn't fully understand
> what
> > you mean by "I think this is the first time we need to introduce a KIP
> > without having it
> > fully accepted in next release."  - could you perhaps explain that some
> > more very briefly?
> >
> > Best regards,
> > Sönke
> >
> >
> >
> > On Fri, 1 May 2020 at 23:03, Boyang Chen 
> > wrote:
> >
> > > Hey Tom,
> > >
> > > thanks for the suggestion. As long as we could correctly serialize the
> > > principal and embed in the Envelope, I think we could still leverage
> the
> > > controller to do the client request authentication. Although this pays
> an
> > > extra round trip if the authorization is doomed to fail on the receiver
> > > side, having a centralized processing unit is more favorable such as
> > > ensuring the audit log is consistent instead of scattering between
> > > forwarder and receiver.
> > >
> > > Boyang
> > >
> > > On Wed, Apr 29, 2020 at 9:50 AM Tom Bentley 
> wrote:
> > >
> > > > Hi Boyang,
> > > >
> > > > Thanks for the update. In the EnvelopeRequest handling section of the
> > KIP
> > > > it might be worth saying explicitly that authorization of the request
> > > will
> > > > happen as normal. Otherwise what you're proposing makes sense to me.
> > > >
> > > > Thanks again,
> > > >
> > > > Tom
> > > >
> > > >
> > > >
> > > > On Wed, Apr 29, 2020 at 5:27 PM Boyang Chen <
> > reluctanthero...@gmail.com>
> > > > wrote:
> > > >
> > > > > Thanks for the proposed idea Sonke. I reviewed it and had some
> > offline
> > > > > discussion with Colin, Rajini and Mathew.
> > > > >
> > > > > We do need to add serializability to the PrincipalBuilder
> interface,
> > > but
> > > > we
> > > > > should not make any default implementation which could go wrong and
> > > messy
> > > > > up with the security in a production environment if the user
> neglects
> > > it.
> > > > > Instead we need to make it required and backward incompatible. So I
> > > > > integrated your proposed methods and expand the Envelope RPC with a
> > > > couple
> > > > > of more fields for audit log purpose as well.
> > > > >
> > > > > Since the KafkaPrincipal builder serializability is a binary
> > > incompatible
> > > > > change, I propose (also stated in the KIP) the following
> > implementation
> > > > > plan:
> > > > >
> > > > >1. For next *2.x* release:
> > > > >   1. Get new admin client forwarding changes
> > > > >   2. Get the Envelope RPC implementation
> > > > >   3. Get the forwarding path working and validate the function
> > with
> > > > >   fake principals in testing environment, without actual
> > triggering
> > > > in
> > > > > the
> > > > >   production system
> > > > >2. For next *3.0 *release:
> > > > >   1. Introduce serializability to PrincipalBuilder
> > > > >   2. Turn on forwarding path in production and perform
> end-to-end
> > > > >   testing
> > > > >
> > > > >
> > > > > I think this is the first time we need to introduce a KIP without
> > > having
> > > > it
> > > > > fully accepted in next release. Let me know if this sounds
> > reasonable.
> > > > >
> > > > > On Fri, Apr 24, 2020 at 1:00 AM Sönke Liebau
> > > > >  wrote:
> > > > >
> > > > > > After thinking on this a little bit, maybe this would be an
> option:
> > > > > >
> > > > > > add default methods serialize and deserialize to the
> > > > > KafkaPrincipalBuilder
> > > > > > interface, these could be very short:
> > > > > >
> > > > > > default String serialize(KafkaPrincipal principal) {
> > > > > > return principal.toString();
> > > > > > }
> > > > > >
> > > > > > default KafkaPrincipal deserialize(String principalString) {
> > > > > > return 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-05-01 Thread Boyang Chen
Sure thing Sonke, what I suggest is that usual KIPs get accepted to go into
next release. It could span for a couple of releases because of engineering
time, but no change has to be shipped in specific future releases, like the
backward incompatible change for KafkaPrincipal. But I guess it's not
really a blocker, as long as we stated clearly in the KIP how we are going
to roll things out, and let it partially finish in 2.6.

Boyang

On Fri, May 1, 2020 at 2:32 PM Sönke Liebau
 wrote:

> Hi Boyang,
>
> thanks for the update, sounds reasonable to me. Making it a breaking change
> is definitely the safer route to go.
>
> Just one quick question regarding your mail, I didn't fully understand what
> you mean by "I think this is the first time we need to introduce a KIP
> without having it
> fully accepted in next release."  - could you perhaps explain that some
> more very briefly?
>
> Best regards,
> Sönke
>
>
>
> On Fri, 1 May 2020 at 23:03, Boyang Chen 
> wrote:
>
> > Hey Tom,
> >
> > thanks for the suggestion. As long as we could correctly serialize the
> > principal and embed in the Envelope, I think we could still leverage the
> > controller to do the client request authentication. Although this pays an
> > extra round trip if the authorization is doomed to fail on the receiver
> > side, having a centralized processing unit is more favorable such as
> > ensuring the audit log is consistent instead of scattering between
> > forwarder and receiver.
> >
> > Boyang
> >
> > On Wed, Apr 29, 2020 at 9:50 AM Tom Bentley  wrote:
> >
> > > Hi Boyang,
> > >
> > > Thanks for the update. In the EnvelopeRequest handling section of the
> KIP
> > > it might be worth saying explicitly that authorization of the request
> > will
> > > happen as normal. Otherwise what you're proposing makes sense to me.
> > >
> > > Thanks again,
> > >
> > > Tom
> > >
> > >
> > >
> > > On Wed, Apr 29, 2020 at 5:27 PM Boyang Chen <
> reluctanthero...@gmail.com>
> > > wrote:
> > >
> > > > Thanks for the proposed idea Sonke. I reviewed it and had some
> offline
> > > > discussion with Colin, Rajini and Mathew.
> > > >
> > > > We do need to add serializability to the PrincipalBuilder interface,
> > but
> > > we
> > > > should not make any default implementation which could go wrong and
> > messy
> > > > up with the security in a production environment if the user neglects
> > it.
> > > > Instead we need to make it required and backward incompatible. So I
> > > > integrated your proposed methods and expand the Envelope RPC with a
> > > couple
> > > > of more fields for audit log purpose as well.
> > > >
> > > > Since the KafkaPrincipal builder serializability is a binary
> > incompatible
> > > > change, I propose (also stated in the KIP) the following
> implementation
> > > > plan:
> > > >
> > > >1. For next *2.x* release:
> > > >   1. Get new admin client forwarding changes
> > > >   2. Get the Envelope RPC implementation
> > > >   3. Get the forwarding path working and validate the function
> with
> > > >   fake principals in testing environment, without actual
> triggering
> > > in
> > > > the
> > > >   production system
> > > >2. For next *3.0 *release:
> > > >   1. Introduce serializability to PrincipalBuilder
> > > >   2. Turn on forwarding path in production and perform end-to-end
> > > >   testing
> > > >
> > > >
> > > > I think this is the first time we need to introduce a KIP without
> > having
> > > it
> > > > fully accepted in next release. Let me know if this sounds
> reasonable.
> > > >
> > > > On Fri, Apr 24, 2020 at 1:00 AM Sönke Liebau
> > > >  wrote:
> > > >
> > > > > After thinking on this a little bit, maybe this would be an option:
> > > > >
> > > > > add default methods serialize and deserialize to the
> > > > KafkaPrincipalBuilder
> > > > > interface, these could be very short:
> > > > >
> > > > > default String serialize(KafkaPrincipal principal) {
> > > > > return principal.toString();
> > > > > }
> > > > >
> > > > > default KafkaPrincipal deserialize(String principalString) {
> > > > > return SecurityUtils.parseKafkaPrincipal(principalString);
> > > > > }
> > > > >
> > > > > This would mean that all existing implementations of that interface
> > > > > are unaffected, as this code is pretty much what is currently being
> > > > > used when their principals need to be serialized.
> > > > >
> > > > > But it offers people using custom principals the chance to override
> > > > > these methods and ensure that all information gets serialized for
> > > > > delegation tokens or request forwarding.
> > > > >
> > > > >
> > > > > Wherever we need to de/serialize principals (for example in the
> > > > > DelegationTokenManager [1]) we obtain an instance of the configured
> > > > > PrincipalBuilder class and use that to do the actual work.
> > > > >
> > > > > What do you think?
> > > > >
> > > > >
> > > > > Best regards,
> > > > >
> > > > > Sönke
> > > > >
> > > > >
> > > > > [1]
> > > > 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-05-01 Thread Sönke Liebau
Hi Boyang,

thanks for the update, sounds reasonable to me. Making it a breaking change
is definitely the safer route to go.

Just one quick question regarding your mail, I didn't fully understand what
you mean by "I think this is the first time we need to introduce a KIP
without having it
fully accepted in next release."  - could you perhaps explain that some
more very briefly?

Best regards,
Sönke



On Fri, 1 May 2020 at 23:03, Boyang Chen  wrote:

> Hey Tom,
>
> thanks for the suggestion. As long as we could correctly serialize the
> principal and embed in the Envelope, I think we could still leverage the
> controller to do the client request authentication. Although this pays an
> extra round trip if the authorization is doomed to fail on the receiver
> side, having a centralized processing unit is more favorable such as
> ensuring the audit log is consistent instead of scattering between
> forwarder and receiver.
>
> Boyang
>
> On Wed, Apr 29, 2020 at 9:50 AM Tom Bentley  wrote:
>
> > Hi Boyang,
> >
> > Thanks for the update. In the EnvelopeRequest handling section of the KIP
> > it might be worth saying explicitly that authorization of the request
> will
> > happen as normal. Otherwise what you're proposing makes sense to me.
> >
> > Thanks again,
> >
> > Tom
> >
> >
> >
> > On Wed, Apr 29, 2020 at 5:27 PM Boyang Chen 
> > wrote:
> >
> > > Thanks for the proposed idea Sonke. I reviewed it and had some offline
> > > discussion with Colin, Rajini and Mathew.
> > >
> > > We do need to add serializability to the PrincipalBuilder interface,
> but
> > we
> > > should not make any default implementation which could go wrong and
> messy
> > > up with the security in a production environment if the user neglects
> it.
> > > Instead we need to make it required and backward incompatible. So I
> > > integrated your proposed methods and expand the Envelope RPC with a
> > couple
> > > of more fields for audit log purpose as well.
> > >
> > > Since the KafkaPrincipal builder serializability is a binary
> incompatible
> > > change, I propose (also stated in the KIP) the following implementation
> > > plan:
> > >
> > >1. For next *2.x* release:
> > >   1. Get new admin client forwarding changes
> > >   2. Get the Envelope RPC implementation
> > >   3. Get the forwarding path working and validate the function with
> > >   fake principals in testing environment, without actual triggering
> > in
> > > the
> > >   production system
> > >2. For next *3.0 *release:
> > >   1. Introduce serializability to PrincipalBuilder
> > >   2. Turn on forwarding path in production and perform end-to-end
> > >   testing
> > >
> > >
> > > I think this is the first time we need to introduce a KIP without
> having
> > it
> > > fully accepted in next release. Let me know if this sounds reasonable.
> > >
> > > On Fri, Apr 24, 2020 at 1:00 AM Sönke Liebau
> > >  wrote:
> > >
> > > > After thinking on this a little bit, maybe this would be an option:
> > > >
> > > > add default methods serialize and deserialize to the
> > > KafkaPrincipalBuilder
> > > > interface, these could be very short:
> > > >
> > > > default String serialize(KafkaPrincipal principal) {
> > > > return principal.toString();
> > > > }
> > > >
> > > > default KafkaPrincipal deserialize(String principalString) {
> > > > return SecurityUtils.parseKafkaPrincipal(principalString);
> > > > }
> > > >
> > > > This would mean that all existing implementations of that interface
> > > > are unaffected, as this code is pretty much what is currently being
> > > > used when their principals need to be serialized.
> > > >
> > > > But it offers people using custom principals the chance to override
> > > > these methods and ensure that all information gets serialized for
> > > > delegation tokens or request forwarding.
> > > >
> > > >
> > > > Wherever we need to de/serialize principals (for example in the
> > > > DelegationTokenManager [1]) we obtain an instance of the configured
> > > > PrincipalBuilder class and use that to do the actual work.
> > > >
> > > > What do you think?
> > > >
> > > >
> > > > Best regards,
> > > >
> > > > Sönke
> > > >
> > > >
> > > > [1]
> > > >
> > >
> >
> https://github.com/apache/kafka/blob/33d06082117d971cdcddd4f01392006b543f3c01/core/src/main/scala/kafka/server/DelegationTokenManager.scala#L122
> > > >
> > > >
> > > > On Thu, 23 Apr 2020 at 17:42, Boyang Chen <
> reluctanthero...@gmail.com>
> > > > wrote:
> > > >
> > > > > Thanks all,
> > > > >
> > > > > IIUC, the necessity of doing the audit log on the controller side
> is
> > > > > because we need to make sure the authorized resource modifications
> > > > > eventually arrive on the target broker side, but is that really
> > > > necessary?
> > > > >
> > > > > I'm thinking the possibility of doing the audit log on the
> forwarding
> > > > > broker side, which could simplify the discussion of principal
> > > > serialization
> > > > > here. The other option I 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-05-01 Thread Boyang Chen
Hey Tom,

thanks for the suggestion. As long as we could correctly serialize the
principal and embed in the Envelope, I think we could still leverage the
controller to do the client request authentication. Although this pays an
extra round trip if the authorization is doomed to fail on the receiver
side, having a centralized processing unit is more favorable such as
ensuring the audit log is consistent instead of scattering between
forwarder and receiver.

Boyang

On Wed, Apr 29, 2020 at 9:50 AM Tom Bentley  wrote:

> Hi Boyang,
>
> Thanks for the update. In the EnvelopeRequest handling section of the KIP
> it might be worth saying explicitly that authorization of the request will
> happen as normal. Otherwise what you're proposing makes sense to me.
>
> Thanks again,
>
> Tom
>
>
>
> On Wed, Apr 29, 2020 at 5:27 PM Boyang Chen 
> wrote:
>
> > Thanks for the proposed idea Sonke. I reviewed it and had some offline
> > discussion with Colin, Rajini and Mathew.
> >
> > We do need to add serializability to the PrincipalBuilder interface, but
> we
> > should not make any default implementation which could go wrong and messy
> > up with the security in a production environment if the user neglects it.
> > Instead we need to make it required and backward incompatible. So I
> > integrated your proposed methods and expand the Envelope RPC with a
> couple
> > of more fields for audit log purpose as well.
> >
> > Since the KafkaPrincipal builder serializability is a binary incompatible
> > change, I propose (also stated in the KIP) the following implementation
> > plan:
> >
> >1. For next *2.x* release:
> >   1. Get new admin client forwarding changes
> >   2. Get the Envelope RPC implementation
> >   3. Get the forwarding path working and validate the function with
> >   fake principals in testing environment, without actual triggering
> in
> > the
> >   production system
> >2. For next *3.0 *release:
> >   1. Introduce serializability to PrincipalBuilder
> >   2. Turn on forwarding path in production and perform end-to-end
> >   testing
> >
> >
> > I think this is the first time we need to introduce a KIP without having
> it
> > fully accepted in next release. Let me know if this sounds reasonable.
> >
> > On Fri, Apr 24, 2020 at 1:00 AM Sönke Liebau
> >  wrote:
> >
> > > After thinking on this a little bit, maybe this would be an option:
> > >
> > > add default methods serialize and deserialize to the
> > KafkaPrincipalBuilder
> > > interface, these could be very short:
> > >
> > > default String serialize(KafkaPrincipal principal) {
> > > return principal.toString();
> > > }
> > >
> > > default KafkaPrincipal deserialize(String principalString) {
> > > return SecurityUtils.parseKafkaPrincipal(principalString);
> > > }
> > >
> > > This would mean that all existing implementations of that interface
> > > are unaffected, as this code is pretty much what is currently being
> > > used when their principals need to be serialized.
> > >
> > > But it offers people using custom principals the chance to override
> > > these methods and ensure that all information gets serialized for
> > > delegation tokens or request forwarding.
> > >
> > >
> > > Wherever we need to de/serialize principals (for example in the
> > > DelegationTokenManager [1]) we obtain an instance of the configured
> > > PrincipalBuilder class and use that to do the actual work.
> > >
> > > What do you think?
> > >
> > >
> > > Best regards,
> > >
> > > Sönke
> > >
> > >
> > > [1]
> > >
> >
> https://github.com/apache/kafka/blob/33d06082117d971cdcddd4f01392006b543f3c01/core/src/main/scala/kafka/server/DelegationTokenManager.scala#L122
> > >
> > >
> > > On Thu, 23 Apr 2020 at 17:42, Boyang Chen 
> > > wrote:
> > >
> > > > Thanks all,
> > > >
> > > > IIUC, the necessity of doing the audit log on the controller side is
> > > > because we need to make sure the authorized resource modifications
> > > > eventually arrive on the target broker side, but is that really
> > > necessary?
> > > >
> > > > I'm thinking the possibility of doing the audit log on the forwarding
> > > > broker side, which could simplify the discussion of principal
> > > serialization
> > > > here. The other option I could think of is to serialize the entire
> > audit
> > > > log message if we were supposed to approve, and pass it as part of
> the
> > > > Envelope.
> > > >
> > > > Let me know if you think either of these approaches would work.
> > > >
> > > > On Thu, Apr 23, 2020 at 7:01 AM Sönke Liebau
> > > >  wrote:
> > > >
> > > > > I agree that this would be useful to have and shouldn't create
> issues
> > > in
> > > > > 99% of all cases. But it would be a breaking change to a public
> API.
> > > > > I had a quick look at the two large projects that come to mind
> which
> > > > might
> > > > > be affected: Ranger and Sentry - both seem to operate directly with
> > > > > KafkaPrincipal instead of subclassing it. But anybody who
> > > > > 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-29 Thread Tom Bentley
Hi Boyang,

Thanks for the update. In the EnvelopeRequest handling section of the KIP
it might be worth saying explicitly that authorization of the request will
happen as normal. Otherwise what you're proposing makes sense to me.

Thanks again,

Tom



On Wed, Apr 29, 2020 at 5:27 PM Boyang Chen 
wrote:

> Thanks for the proposed idea Sonke. I reviewed it and had some offline
> discussion with Colin, Rajini and Mathew.
>
> We do need to add serializability to the PrincipalBuilder interface, but we
> should not make any default implementation which could go wrong and messy
> up with the security in a production environment if the user neglects it.
> Instead we need to make it required and backward incompatible. So I
> integrated your proposed methods and expand the Envelope RPC with a couple
> of more fields for audit log purpose as well.
>
> Since the KafkaPrincipal builder serializability is a binary incompatible
> change, I propose (also stated in the KIP) the following implementation
> plan:
>
>1. For next *2.x* release:
>   1. Get new admin client forwarding changes
>   2. Get the Envelope RPC implementation
>   3. Get the forwarding path working and validate the function with
>   fake principals in testing environment, without actual triggering in
> the
>   production system
>2. For next *3.0 *release:
>   1. Introduce serializability to PrincipalBuilder
>   2. Turn on forwarding path in production and perform end-to-end
>   testing
>
>
> I think this is the first time we need to introduce a KIP without having it
> fully accepted in next release. Let me know if this sounds reasonable.
>
> On Fri, Apr 24, 2020 at 1:00 AM Sönke Liebau
>  wrote:
>
> > After thinking on this a little bit, maybe this would be an option:
> >
> > add default methods serialize and deserialize to the
> KafkaPrincipalBuilder
> > interface, these could be very short:
> >
> > default String serialize(KafkaPrincipal principal) {
> > return principal.toString();
> > }
> >
> > default KafkaPrincipal deserialize(String principalString) {
> > return SecurityUtils.parseKafkaPrincipal(principalString);
> > }
> >
> > This would mean that all existing implementations of that interface
> > are unaffected, as this code is pretty much what is currently being
> > used when their principals need to be serialized.
> >
> > But it offers people using custom principals the chance to override
> > these methods and ensure that all information gets serialized for
> > delegation tokens or request forwarding.
> >
> >
> > Wherever we need to de/serialize principals (for example in the
> > DelegationTokenManager [1]) we obtain an instance of the configured
> > PrincipalBuilder class and use that to do the actual work.
> >
> > What do you think?
> >
> >
> > Best regards,
> >
> > Sönke
> >
> >
> > [1]
> >
> https://github.com/apache/kafka/blob/33d06082117d971cdcddd4f01392006b543f3c01/core/src/main/scala/kafka/server/DelegationTokenManager.scala#L122
> >
> >
> > On Thu, 23 Apr 2020 at 17:42, Boyang Chen 
> > wrote:
> >
> > > Thanks all,
> > >
> > > IIUC, the necessity of doing the audit log on the controller side is
> > > because we need to make sure the authorized resource modifications
> > > eventually arrive on the target broker side, but is that really
> > necessary?
> > >
> > > I'm thinking the possibility of doing the audit log on the forwarding
> > > broker side, which could simplify the discussion of principal
> > serialization
> > > here. The other option I could think of is to serialize the entire
> audit
> > > log message if we were supposed to approve, and pass it as part of the
> > > Envelope.
> > >
> > > Let me know if you think either of these approaches would work.
> > >
> > > On Thu, Apr 23, 2020 at 7:01 AM Sönke Liebau
> > >  wrote:
> > >
> > > > I agree that this would be useful to have and shouldn't create issues
> > in
> > > > 99% of all cases. But it would be a breaking change to a public API.
> > > > I had a quick look at the two large projects that come to mind which
> > > might
> > > > be affected: Ranger and Sentry - both seem to operate directly with
> > > > KafkaPrincipal instead of subclassing it. But anybody who
> > > > extended KafkaPrincipal would probably need to update their code..
> > > >
> > > > Writing this sparked the thought that this issue should also concern
> > > > delegation tokens, as Principals need to be stored/sent around for
> > those
> > > > too.
> > > > Had a brief look at the code and for Delegation Tokens we seem to use
> > > > SecurityUtils#parseKafkaPrincipal[1] which would ignore any
> additional
> > > > information from custom Principals.
> > > >
> > > > We'll probably want to at least add a note on that to the docs -
> unless
> > > it
> > > > is there already, I've only looked for about 30 seconds..
> > > >
> > > > Best regards,
> > > > Sönke
> > > >
> > > >
> > > > [1]
> > > >
> > > >
> > >
> >
> 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-29 Thread Boyang Chen
Thanks for the proposed idea Sonke. I reviewed it and had some offline
discussion with Colin, Rajini and Mathew.

We do need to add serializability to the PrincipalBuilder interface, but we
should not make any default implementation which could go wrong and messy
up with the security in a production environment if the user neglects it.
Instead we need to make it required and backward incompatible. So I
integrated your proposed methods and expand the Envelope RPC with a couple
of more fields for audit log purpose as well.

Since the KafkaPrincipal builder serializability is a binary incompatible
change, I propose (also stated in the KIP) the following implementation
plan:

   1. For next *2.x* release:
  1. Get new admin client forwarding changes
  2. Get the Envelope RPC implementation
  3. Get the forwarding path working and validate the function with
  fake principals in testing environment, without actual triggering in the
  production system
   2. For next *3.0 *release:
  1. Introduce serializability to PrincipalBuilder
  2. Turn on forwarding path in production and perform end-to-end
  testing


I think this is the first time we need to introduce a KIP without having it
fully accepted in next release. Let me know if this sounds reasonable.

On Fri, Apr 24, 2020 at 1:00 AM Sönke Liebau
 wrote:

> After thinking on this a little bit, maybe this would be an option:
>
> add default methods serialize and deserialize to the KafkaPrincipalBuilder
> interface, these could be very short:
>
> default String serialize(KafkaPrincipal principal) {
> return principal.toString();
> }
>
> default KafkaPrincipal deserialize(String principalString) {
> return SecurityUtils.parseKafkaPrincipal(principalString);
> }
>
> This would mean that all existing implementations of that interface
> are unaffected, as this code is pretty much what is currently being
> used when their principals need to be serialized.
>
> But it offers people using custom principals the chance to override
> these methods and ensure that all information gets serialized for
> delegation tokens or request forwarding.
>
>
> Wherever we need to de/serialize principals (for example in the
> DelegationTokenManager [1]) we obtain an instance of the configured
> PrincipalBuilder class and use that to do the actual work.
>
> What do you think?
>
>
> Best regards,
>
> Sönke
>
>
> [1]
> https://github.com/apache/kafka/blob/33d06082117d971cdcddd4f01392006b543f3c01/core/src/main/scala/kafka/server/DelegationTokenManager.scala#L122
>
>
> On Thu, 23 Apr 2020 at 17:42, Boyang Chen 
> wrote:
>
> > Thanks all,
> >
> > IIUC, the necessity of doing the audit log on the controller side is
> > because we need to make sure the authorized resource modifications
> > eventually arrive on the target broker side, but is that really
> necessary?
> >
> > I'm thinking the possibility of doing the audit log on the forwarding
> > broker side, which could simplify the discussion of principal
> serialization
> > here. The other option I could think of is to serialize the entire audit
> > log message if we were supposed to approve, and pass it as part of the
> > Envelope.
> >
> > Let me know if you think either of these approaches would work.
> >
> > On Thu, Apr 23, 2020 at 7:01 AM Sönke Liebau
> >  wrote:
> >
> > > I agree that this would be useful to have and shouldn't create issues
> in
> > > 99% of all cases. But it would be a breaking change to a public API.
> > > I had a quick look at the two large projects that come to mind which
> > might
> > > be affected: Ranger and Sentry - both seem to operate directly with
> > > KafkaPrincipal instead of subclassing it. But anybody who
> > > extended KafkaPrincipal would probably need to update their code..
> > >
> > > Writing this sparked the thought that this issue should also concern
> > > delegation tokens, as Principals need to be stored/sent around for
> those
> > > too.
> > > Had a brief look at the code and for Delegation Tokens we seem to use
> > > SecurityUtils#parseKafkaPrincipal[1] which would ignore any additional
> > > information from custom Principals.
> > >
> > > We'll probably want to at least add a note on that to the docs - unless
> > it
> > > is there already, I've only looked for about 30 seconds..
> > >
> > > Best regards,
> > > Sönke
> > >
> > >
> > > [1]
> > >
> > >
> >
> https://github.com/apache/kafka/blob/e9fcfe4fb7b9ae2f537ce355fe2ab51a58034c64/clients/src/main/java/org/apache/kafka/common/utils/SecurityUtils.java#L52
> > >
> > > On Thu, 23 Apr 2020 at 14:35, Colin McCabe  wrote:
> > >
> > > > Hmm... Maybe we need to add some way to serialize and deserialize
> > > > KafkaPrincipal subclasses to/from string.  We could add a method to
> > > > KafkaPrincipalBuilder#deserialize and a method
> > KafkaPrincipal#serialize,
> > > I
> > > > suppose.
> > > >
> > > > best,
> > > > Colin
> > > >
> > > >
> > > > On Thu, Apr 23, 2020, at 02:14, Tom Bentley wrote:
> > > > > Hi folks,

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-24 Thread Sönke Liebau
After thinking on this a little bit, maybe this would be an option:

add default methods serialize and deserialize to the KafkaPrincipalBuilder
interface, these could be very short:

default String serialize(KafkaPrincipal principal) {
return principal.toString();
}

default KafkaPrincipal deserialize(String principalString) {
return SecurityUtils.parseKafkaPrincipal(principalString);
}

This would mean that all existing implementations of that interface
are unaffected, as this code is pretty much what is currently being
used when their principals need to be serialized.

But it offers people using custom principals the chance to override
these methods and ensure that all information gets serialized for
delegation tokens or request forwarding.


Wherever we need to de/serialize principals (for example in the
DelegationTokenManager [1]) we obtain an instance of the configured
PrincipalBuilder class and use that to do the actual work.

What do you think?


Best regards,

Sönke


[1] 
https://github.com/apache/kafka/blob/33d06082117d971cdcddd4f01392006b543f3c01/core/src/main/scala/kafka/server/DelegationTokenManager.scala#L122


On Thu, 23 Apr 2020 at 17:42, Boyang Chen 
wrote:

> Thanks all,
>
> IIUC, the necessity of doing the audit log on the controller side is
> because we need to make sure the authorized resource modifications
> eventually arrive on the target broker side, but is that really necessary?
>
> I'm thinking the possibility of doing the audit log on the forwarding
> broker side, which could simplify the discussion of principal serialization
> here. The other option I could think of is to serialize the entire audit
> log message if we were supposed to approve, and pass it as part of the
> Envelope.
>
> Let me know if you think either of these approaches would work.
>
> On Thu, Apr 23, 2020 at 7:01 AM Sönke Liebau
>  wrote:
>
> > I agree that this would be useful to have and shouldn't create issues in
> > 99% of all cases. But it would be a breaking change to a public API.
> > I had a quick look at the two large projects that come to mind which
> might
> > be affected: Ranger and Sentry - both seem to operate directly with
> > KafkaPrincipal instead of subclassing it. But anybody who
> > extended KafkaPrincipal would probably need to update their code..
> >
> > Writing this sparked the thought that this issue should also concern
> > delegation tokens, as Principals need to be stored/sent around for those
> > too.
> > Had a brief look at the code and for Delegation Tokens we seem to use
> > SecurityUtils#parseKafkaPrincipal[1] which would ignore any additional
> > information from custom Principals.
> >
> > We'll probably want to at least add a note on that to the docs - unless
> it
> > is there already, I've only looked for about 30 seconds..
> >
> > Best regards,
> > Sönke
> >
> >
> > [1]
> >
> >
> https://github.com/apache/kafka/blob/e9fcfe4fb7b9ae2f537ce355fe2ab51a58034c64/clients/src/main/java/org/apache/kafka/common/utils/SecurityUtils.java#L52
> >
> > On Thu, 23 Apr 2020 at 14:35, Colin McCabe  wrote:
> >
> > > Hmm... Maybe we need to add some way to serialize and deserialize
> > > KafkaPrincipal subclasses to/from string.  We could add a method to
> > > KafkaPrincipalBuilder#deserialize and a method
> KafkaPrincipal#serialize,
> > I
> > > suppose.
> > >
> > > best,
> > > Colin
> > >
> > >
> > > On Thu, Apr 23, 2020, at 02:14, Tom Bentley wrote:
> > > > Hi folks,
> > > >
> > > > Colin wrote:
> > > >
> > > > > The final broker knows it can trust the principal name in the
> > envelope
> > > > > (since EnvelopeRequest requires CLUSTERACTION on CLUSTER).  So it
> can
> > > use
> > > > > that principal name for authorization (given who you are, what can
> > you
> > > > > do?)  The forwarded principal name will also be used for logging.
> > > > >
> > > >
> > > > My understanding (and I'm happy to be corrected) is that a custom
> > > > authoriser might rely on the KafkaPrincipal instance being a subclass
> > of
> > > > KafkaPrincipal (e.g. the subclass has extra fields with the
> principal's
> > > > "roles"). So you can't construct a KafkaPrinicpal on the controller
> > which
> > > > would be guaranteed to work for arbitrary authorizers. You have to
> > > perform
> > > > authorization on the first broker (rejecting some of the batched
> > > requests),
> > > > forward the authorized ones to the controller, which then has to
> trust
> > > that
> > > > the authorization has been done and make the ZK writes without
> > > > authorization. Because the controller cannot invoke the authorizer
> that
> > > > means that the authorizer audit logging (on the controller) would not
> > > > include these operations. But they would be in the audit logging from
> > the
> > > > original broker, so the information would not be lost.
> > > >
> > > > There's also a problem with using the constructed principal for other
> > > > logging (i.e. non authorizer logging) on the controller: There's
> > nothing
> > > 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-23 Thread Boyang Chen
Thanks all,

IIUC, the necessity of doing the audit log on the controller side is
because we need to make sure the authorized resource modifications
eventually arrive on the target broker side, but is that really necessary?

I'm thinking the possibility of doing the audit log on the forwarding
broker side, which could simplify the discussion of principal serialization
here. The other option I could think of is to serialize the entire audit
log message if we were supposed to approve, and pass it as part of the
Envelope.

Let me know if you think either of these approaches would work.

On Thu, Apr 23, 2020 at 7:01 AM Sönke Liebau
 wrote:

> I agree that this would be useful to have and shouldn't create issues in
> 99% of all cases. But it would be a breaking change to a public API.
> I had a quick look at the two large projects that come to mind which might
> be affected: Ranger and Sentry - both seem to operate directly with
> KafkaPrincipal instead of subclassing it. But anybody who
> extended KafkaPrincipal would probably need to update their code..
>
> Writing this sparked the thought that this issue should also concern
> delegation tokens, as Principals need to be stored/sent around for those
> too.
> Had a brief look at the code and for Delegation Tokens we seem to use
> SecurityUtils#parseKafkaPrincipal[1] which would ignore any additional
> information from custom Principals.
>
> We'll probably want to at least add a note on that to the docs - unless it
> is there already, I've only looked for about 30 seconds..
>
> Best regards,
> Sönke
>
>
> [1]
>
> https://github.com/apache/kafka/blob/e9fcfe4fb7b9ae2f537ce355fe2ab51a58034c64/clients/src/main/java/org/apache/kafka/common/utils/SecurityUtils.java#L52
>
> On Thu, 23 Apr 2020 at 14:35, Colin McCabe  wrote:
>
> > Hmm... Maybe we need to add some way to serialize and deserialize
> > KafkaPrincipal subclasses to/from string.  We could add a method to
> > KafkaPrincipalBuilder#deserialize and a method KafkaPrincipal#serialize,
> I
> > suppose.
> >
> > best,
> > Colin
> >
> >
> > On Thu, Apr 23, 2020, at 02:14, Tom Bentley wrote:
> > > Hi folks,
> > >
> > > Colin wrote:
> > >
> > > > The final broker knows it can trust the principal name in the
> envelope
> > > > (since EnvelopeRequest requires CLUSTERACTION on CLUSTER).  So it can
> > use
> > > > that principal name for authorization (given who you are, what can
> you
> > > > do?)  The forwarded principal name will also be used for logging.
> > > >
> > >
> > > My understanding (and I'm happy to be corrected) is that a custom
> > > authoriser might rely on the KafkaPrincipal instance being a subclass
> of
> > > KafkaPrincipal (e.g. the subclass has extra fields with the principal's
> > > "roles"). So you can't construct a KafkaPrinicpal on the controller
> which
> > > would be guaranteed to work for arbitrary authorizers. You have to
> > perform
> > > authorization on the first broker (rejecting some of the batched
> > requests),
> > > forward the authorized ones to the controller, which then has to trust
> > that
> > > the authorization has been done and make the ZK writes without
> > > authorization. Because the controller cannot invoke the authorizer that
> > > means that the authorizer audit logging (on the controller) would not
> > > include these operations. But they would be in the audit logging from
> the
> > > original broker, so the information would not be lost.
> > >
> > > There's also a problem with using the constructed principal for other
> > > logging (i.e. non authorizer logging) on the controller: There's
> nothing
> > > stopping a custom KafkaPrincipal subclass from overriding toString() to
> > > return something different from "${type}:${name}". If you're building a
> > > "fake" KafkaPrincipal on the controller from the type and name then its
> > > toString() would be "wrong". A solution to this would be to also
> > serialize
> > > the toString() into the envelope and have some ProxiedKafkaPrincipal
> > class
> > > which you instantiate on the controller which has overridden toString
> to
> > > return the right value. Obviously you could optimize this using an
> > optional
> > > field so you only serialize the toString() if it differed from the
> value
> > > you'd get from KafkaPrincipal.toString(). Maybe non-audit logging using
> > the
> > > wrong string value of a principal is sufficiently minor that this isn't
> > > even worth trying to solve.
> > >
> > > Kind regards,
> > >
> > > Tom
> > >
> > >
> > > On Wed, Apr 22, 2020 at 10:59 PM Sönke Liebau
> > >  wrote:
> > >
> > > > Hi Colin,
> > > >
> > > > thanks for your summary! Just one question - and I may be missing an
> > > > obvious point here..
> > > > You write:
> > > >
> > > > "The initial broker should do authentication (who are you?) and come
> up
> > > > with a principal name.  Then it creates an envelope request, which
> will
> > > > contain that principal name, and sends it along with the unmodified
> > > > original request 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-23 Thread Sönke Liebau
I agree that this would be useful to have and shouldn't create issues in
99% of all cases. But it would be a breaking change to a public API.
I had a quick look at the two large projects that come to mind which might
be affected: Ranger and Sentry - both seem to operate directly with
KafkaPrincipal instead of subclassing it. But anybody who
extended KafkaPrincipal would probably need to update their code..

Writing this sparked the thought that this issue should also concern
delegation tokens, as Principals need to be stored/sent around for those
too.
Had a brief look at the code and for Delegation Tokens we seem to use
SecurityUtils#parseKafkaPrincipal[1] which would ignore any additional
information from custom Principals.

We'll probably want to at least add a note on that to the docs - unless it
is there already, I've only looked for about 30 seconds..

Best regards,
Sönke


[1]
https://github.com/apache/kafka/blob/e9fcfe4fb7b9ae2f537ce355fe2ab51a58034c64/clients/src/main/java/org/apache/kafka/common/utils/SecurityUtils.java#L52

On Thu, 23 Apr 2020 at 14:35, Colin McCabe  wrote:

> Hmm... Maybe we need to add some way to serialize and deserialize
> KafkaPrincipal subclasses to/from string.  We could add a method to
> KafkaPrincipalBuilder#deserialize and a method KafkaPrincipal#serialize, I
> suppose.
>
> best,
> Colin
>
>
> On Thu, Apr 23, 2020, at 02:14, Tom Bentley wrote:
> > Hi folks,
> >
> > Colin wrote:
> >
> > > The final broker knows it can trust the principal name in the envelope
> > > (since EnvelopeRequest requires CLUSTERACTION on CLUSTER).  So it can
> use
> > > that principal name for authorization (given who you are, what can you
> > > do?)  The forwarded principal name will also be used for logging.
> > >
> >
> > My understanding (and I'm happy to be corrected) is that a custom
> > authoriser might rely on the KafkaPrincipal instance being a subclass of
> > KafkaPrincipal (e.g. the subclass has extra fields with the principal's
> > "roles"). So you can't construct a KafkaPrinicpal on the controller which
> > would be guaranteed to work for arbitrary authorizers. You have to
> perform
> > authorization on the first broker (rejecting some of the batched
> requests),
> > forward the authorized ones to the controller, which then has to trust
> that
> > the authorization has been done and make the ZK writes without
> > authorization. Because the controller cannot invoke the authorizer that
> > means that the authorizer audit logging (on the controller) would not
> > include these operations. But they would be in the audit logging from the
> > original broker, so the information would not be lost.
> >
> > There's also a problem with using the constructed principal for other
> > logging (i.e. non authorizer logging) on the controller: There's nothing
> > stopping a custom KafkaPrincipal subclass from overriding toString() to
> > return something different from "${type}:${name}". If you're building a
> > "fake" KafkaPrincipal on the controller from the type and name then its
> > toString() would be "wrong". A solution to this would be to also
> serialize
> > the toString() into the envelope and have some ProxiedKafkaPrincipal
> class
> > which you instantiate on the controller which has overridden toString to
> > return the right value. Obviously you could optimize this using an
> optional
> > field so you only serialize the toString() if it differed from the value
> > you'd get from KafkaPrincipal.toString(). Maybe non-audit logging using
> the
> > wrong string value of a principal is sufficiently minor that this isn't
> > even worth trying to solve.
> >
> > Kind regards,
> >
> > Tom
> >
> >
> > On Wed, Apr 22, 2020 at 10:59 PM Sönke Liebau
> >  wrote:
> >
> > > Hi Colin,
> > >
> > > thanks for your summary! Just one question - and I may be missing an
> > > obvious point here..
> > > You write:
> > >
> > > "The initial broker should do authentication (who are you?) and come up
> > > with a principal name.  Then it creates an envelope request, which will
> > > contain that principal name, and sends it along with the unmodified
> > > original request to the final broker.   [... ] The final broker knows
> it
> > > can trust the principal name in the envelope (since EnvelopeRequest
> > > requires CLUSTERACTION on CLUSTER).  So it can use that principal name
> for
> > > authorization (given who you are, what can you do?) "
> > >
> > > My understanding is, that you don't want to serialize the Principal
> (due to
> > > the discussed issues with custom principals) but reduce the principal
> down
> > > to a string representation that would be used for logging and
> > > authorization?
> > > If that understanding is correct then I don't think we could use the
> > > regular Authorizer on the target broker, because that would need the
> actual
> > > principal object to work on.
> > >
> > > Also, a thought that just occurred to me, we might actually need to log
> > > different principal strings for the 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-23 Thread Colin McCabe
Hmm... Maybe we need to add some way to serialize and deserialize 
KafkaPrincipal subclasses to/from string.  We could add a method to 
KafkaPrincipalBuilder#deserialize and a method KafkaPrincipal#serialize, I 
suppose.

best,
Colin


On Thu, Apr 23, 2020, at 02:14, Tom Bentley wrote:
> Hi folks,
> 
> Colin wrote:
> 
> > The final broker knows it can trust the principal name in the envelope
> > (since EnvelopeRequest requires CLUSTERACTION on CLUSTER).  So it can use
> > that principal name for authorization (given who you are, what can you
> > do?)  The forwarded principal name will also be used for logging.
> >
> 
> My understanding (and I'm happy to be corrected) is that a custom
> authoriser might rely on the KafkaPrincipal instance being a subclass of
> KafkaPrincipal (e.g. the subclass has extra fields with the principal's
> "roles"). So you can't construct a KafkaPrinicpal on the controller which
> would be guaranteed to work for arbitrary authorizers. You have to perform
> authorization on the first broker (rejecting some of the batched requests),
> forward the authorized ones to the controller, which then has to trust that
> the authorization has been done and make the ZK writes without
> authorization. Because the controller cannot invoke the authorizer that
> means that the authorizer audit logging (on the controller) would not
> include these operations. But they would be in the audit logging from the
> original broker, so the information would not be lost.
> 
> There's also a problem with using the constructed principal for other
> logging (i.e. non authorizer logging) on the controller: There's nothing
> stopping a custom KafkaPrincipal subclass from overriding toString() to
> return something different from "${type}:${name}". If you're building a
> "fake" KafkaPrincipal on the controller from the type and name then its
> toString() would be "wrong". A solution to this would be to also serialize
> the toString() into the envelope and have some ProxiedKafkaPrincipal class
> which you instantiate on the controller which has overridden toString to
> return the right value. Obviously you could optimize this using an optional
> field so you only serialize the toString() if it differed from the value
> you'd get from KafkaPrincipal.toString(). Maybe non-audit logging using the
> wrong string value of a principal is sufficiently minor that this isn't
> even worth trying to solve.
> 
> Kind regards,
> 
> Tom
> 
> 
> On Wed, Apr 22, 2020 at 10:59 PM Sönke Liebau
>  wrote:
> 
> > Hi Colin,
> >
> > thanks for your summary! Just one question - and I may be missing an
> > obvious point here..
> > You write:
> >
> > "The initial broker should do authentication (who are you?) and come up
> > with a principal name.  Then it creates an envelope request, which will
> > contain that principal name, and sends it along with the unmodified
> > original request to the final broker.   [... ] The final broker knows it
> > can trust the principal name in the envelope (since EnvelopeRequest
> > requires CLUSTERACTION on CLUSTER).  So it can use that principal name for
> > authorization (given who you are, what can you do?) "
> >
> > My understanding is, that you don't want to serialize the Principal (due to
> > the discussed issues with custom principals) but reduce the principal down
> > to a string representation that would be used for logging and
> > authorization?
> > If that understanding is correct then I don't think we could use the
> > regular Authorizer on the target broker, because that would need the actual
> > principal object to work on.
> >
> > Also, a thought that just occurred to me, we might actually need to log
> > different principal strings for the case of queries like AlterConfigs
> > (mentioned by Rajini) which may contain multiple resources. Take an LDAP
> > authorizer that grants access based on group membership - the same
> > alterconfig request may contain resources that are authorized based on
> > group1 as well as resources authorized based on membership in group 2 ..
> > And in all cases we'd need to log the specific reason I think..
> >
> > Basically I think that we might have a hard time properly authorizing and
> > logging without being able to forward the entire principal.. but again, I
> > might be heading down an entirely wrong path here :)
> >
> > Best regards,
> > Sönke
> >
> >
> >
> >
> >
> >
> >
> >
> > On Wed, 22 Apr 2020 at 23:13, Guozhang Wang  wrote:
> >
> > > Colin, Boyang: thanks for the updates, I agree that an EnvelopeRequest
> > > would be a less vulnerable approach than optional fields, and I'm just
> > > wondering if we would keep the EnvelopeRequest for a long time. I was
> > > thinking that, potentially if we require clients to be on newer version
> > > when talking to a 3.0+ (assuming 3.0 is the bridge release) brokers, then
> > > we do not need to keep this code for too long, but I think that would be
> > a
> > > very hasty compatibility breaking so maybe we 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-23 Thread Tom Bentley
Hi folks,

Colin wrote:

> The final broker knows it can trust the principal name in the envelope
> (since EnvelopeRequest requires CLUSTERACTION on CLUSTER).  So it can use
> that principal name for authorization (given who you are, what can you
> do?)  The forwarded principal name will also be used for logging.
>

My understanding (and I'm happy to be corrected) is that a custom
authoriser might rely on the KafkaPrincipal instance being a subclass of
KafkaPrincipal (e.g. the subclass has extra fields with the principal's
"roles"). So you can't construct a KafkaPrinicpal on the controller which
would be guaranteed to work for arbitrary authorizers. You have to perform
authorization on the first broker (rejecting some of the batched requests),
forward the authorized ones to the controller, which then has to trust that
the authorization has been done and make the ZK writes without
authorization. Because the controller cannot invoke the authorizer that
means that the authorizer audit logging (on the controller) would not
include these operations. But they would be in the audit logging from the
original broker, so the information would not be lost.

There's also a problem with using the constructed principal for other
logging (i.e. non authorizer logging) on the controller: There's nothing
stopping a custom KafkaPrincipal subclass from overriding toString() to
return something different from "${type}:${name}". If you're building a
"fake" KafkaPrincipal on the controller from the type and name then its
toString() would be "wrong". A solution to this would be to also serialize
the toString() into the envelope and have some ProxiedKafkaPrincipal class
which you instantiate on the controller which has overridden toString to
return the right value. Obviously you could optimize this using an optional
field so you only serialize the toString() if it differed from the value
you'd get from KafkaPrincipal.toString(). Maybe non-audit logging using the
wrong string value of a principal is sufficiently minor that this isn't
even worth trying to solve.

Kind regards,

Tom


On Wed, Apr 22, 2020 at 10:59 PM Sönke Liebau
 wrote:

> Hi Colin,
>
> thanks for your summary! Just one question - and I may be missing an
> obvious point here..
> You write:
>
> "The initial broker should do authentication (who are you?) and come up
> with a principal name.  Then it creates an envelope request, which will
> contain that principal name, and sends it along with the unmodified
> original request to the final broker.   [... ] The final broker knows it
> can trust the principal name in the envelope (since EnvelopeRequest
> requires CLUSTERACTION on CLUSTER).  So it can use that principal name for
> authorization (given who you are, what can you do?) "
>
> My understanding is, that you don't want to serialize the Principal (due to
> the discussed issues with custom principals) but reduce the principal down
> to a string representation that would be used for logging and
> authorization?
> If that understanding is correct then I don't think we could use the
> regular Authorizer on the target broker, because that would need the actual
> principal object to work on.
>
> Also, a thought that just occurred to me, we might actually need to log
> different principal strings for the case of queries like AlterConfigs
> (mentioned by Rajini) which may contain multiple resources. Take an LDAP
> authorizer that grants access based on group membership - the same
> alterconfig request may contain resources that are authorized based on
> group1 as well as resources authorized based on membership in group 2 ..
> And in all cases we'd need to log the specific reason I think..
>
> Basically I think that we might have a hard time properly authorizing and
> logging without being able to forward the entire principal.. but again, I
> might be heading down an entirely wrong path here :)
>
> Best regards,
> Sönke
>
>
>
>
>
>
>
>
> On Wed, 22 Apr 2020 at 23:13, Guozhang Wang  wrote:
>
> > Colin, Boyang: thanks for the updates, I agree that an EnvelopeRequest
> > would be a less vulnerable approach than optional fields, and I'm just
> > wondering if we would keep the EnvelopeRequest for a long time. I was
> > thinking that, potentially if we require clients to be on newer version
> > when talking to a 3.0+ (assuming 3.0 is the bridge release) brokers, then
> > we do not need to keep this code for too long, but I think that would be
> a
> > very hasty compatibility breaking so maybe we indeed need to keep this
> > forwarding mechanism many years.
> >
> > Regarding future use cases, I think the example that Boyang mentioned may
> > not be very practical honestly, because when there's a connectivity
> issue,
> > it is either a network partition between "controller, A | B", or
> > "controller | A, B". In other words, if the controller can talk to A,
> then
> > very likely A would not be able to talk to B either... anyways, since the
> > forwarding would be there for a 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-22 Thread Sönke Liebau
Hi Colin,

thanks for your summary! Just one question - and I may be missing an
obvious point here..
You write:

"The initial broker should do authentication (who are you?) and come up
with a principal name.  Then it creates an envelope request, which will
contain that principal name, and sends it along with the unmodified
original request to the final broker.   [... ] The final broker knows it
can trust the principal name in the envelope (since EnvelopeRequest
requires CLUSTERACTION on CLUSTER).  So it can use that principal name for
authorization (given who you are, what can you do?) "

My understanding is, that you don't want to serialize the Principal (due to
the discussed issues with custom principals) but reduce the principal down
to a string representation that would be used for logging and authorization?
If that understanding is correct then I don't think we could use the
regular Authorizer on the target broker, because that would need the actual
principal object to work on.

Also, a thought that just occurred to me, we might actually need to log
different principal strings for the case of queries like AlterConfigs
(mentioned by Rajini) which may contain multiple resources. Take an LDAP
authorizer that grants access based on group membership - the same
alterconfig request may contain resources that are authorized based on
group1 as well as resources authorized based on membership in group 2 ..
And in all cases we'd need to log the specific reason I think..

Basically I think that we might have a hard time properly authorizing and
logging without being able to forward the entire principal.. but again, I
might be heading down an entirely wrong path here :)

Best regards,
Sönke








On Wed, 22 Apr 2020 at 23:13, Guozhang Wang  wrote:

> Colin, Boyang: thanks for the updates, I agree that an EnvelopeRequest
> would be a less vulnerable approach than optional fields, and I'm just
> wondering if we would keep the EnvelopeRequest for a long time. I was
> thinking that, potentially if we require clients to be on newer version
> when talking to a 3.0+ (assuming 3.0 is the bridge release) brokers, then
> we do not need to keep this code for too long, but I think that would be a
> very hasty compatibility breaking so maybe we indeed need to keep this
> forwarding mechanism many years.
>
> Regarding future use cases, I think the example that Boyang mentioned may
> not be very practical honestly, because when there's a connectivity issue,
> it is either a network partition between "controller, A | B", or
> "controller | A, B". In other words, if the controller can talk to A, then
> very likely A would not be able to talk to B either... anyways, since the
> forwarding would be there for a sufficiently long time, I think keeping the
> additional envelope makes sense.
>
>
> Guozhang
>
> On Wed, Apr 22, 2020 at 1:47 PM Boyang Chen 
> wrote:
>
> > Thanks Colin for the summary! And Guozhang, regarding the future use
> cases,
> > consider a scenario where there are temporary connectivity issue between
> > controller to a fellow broker A, the controller could then leverage
> another
> > healthy broker B to do a forwarding request to A in order to maintain a
> > communication.
> >
> > Even for KIP-590 scope, the forwarding mechanism shall not be transit as
> we
> > do need to support older version of admin clients for a couple of years
> > IIUC.
> >
> > Boyang
> >
> > On Wed, Apr 22, 2020 at 1:29 PM Colin McCabe  wrote:
> >
> > > Hi all,
> > >
> > > I guess the way I see this working is that the request gets sent from
> the
> > > client, to the initial broker, and then forwarded to the final broker.
> > >
> > > The initial broker should do authentication (who are you?) and come up
> > > with a principal name.  Then it creates an envelope request, which will
> > > contain that principal name, and sends it along with the unmodified
> > > original request to the final broker.  (I agree with Tom and Rajini
> that
> > we
> > > can't forward the information needed to do authentication on the final
> > > broker, but I also think we don't need to, since we can do it on the
> > > initial broker.)
> > >
> > > The final broker knows it can trust the principal name in the envelope
> > > (since EnvelopeRequest requires CLUSTERACTION on CLUSTER).  So it can
> use
> > > that principal name for authorization (given who you are, what can you
> > > do?)  The forwarded principal name will also be used for logging.
> > >
> > > One question is why we need an EnvelopeRequest.  Well, if we don't have
> > an
> > > EnvelopeRequest, we need somewhere else to put the forwarded principal
> > > name.  I don't think overriding an existing field (like clientId) is a
> > good
> > > option for this.  It's messy, and loses information.  It also raises
> the
> > > question of how the final broker knows that the clientId in the
> received
> > > message is not "really" a clientId, but is a principal name.  Without
> an
> > > envelope, there's no way to 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-22 Thread Guozhang Wang
Colin, Boyang: thanks for the updates, I agree that an EnvelopeRequest
would be a less vulnerable approach than optional fields, and I'm just
wondering if we would keep the EnvelopeRequest for a long time. I was
thinking that, potentially if we require clients to be on newer version
when talking to a 3.0+ (assuming 3.0 is the bridge release) brokers, then
we do not need to keep this code for too long, but I think that would be a
very hasty compatibility breaking so maybe we indeed need to keep this
forwarding mechanism many years.

Regarding future use cases, I think the example that Boyang mentioned may
not be very practical honestly, because when there's a connectivity issue,
it is either a network partition between "controller, A | B", or
"controller | A, B". In other words, if the controller can talk to A, then
very likely A would not be able to talk to B either... anyways, since the
forwarding would be there for a sufficiently long time, I think keeping the
additional envelope makes sense.


Guozhang

On Wed, Apr 22, 2020 at 1:47 PM Boyang Chen 
wrote:

> Thanks Colin for the summary! And Guozhang, regarding the future use cases,
> consider a scenario where there are temporary connectivity issue between
> controller to a fellow broker A, the controller could then leverage another
> healthy broker B to do a forwarding request to A in order to maintain a
> communication.
>
> Even for KIP-590 scope, the forwarding mechanism shall not be transit as we
> do need to support older version of admin clients for a couple of years
> IIUC.
>
> Boyang
>
> On Wed, Apr 22, 2020 at 1:29 PM Colin McCabe  wrote:
>
> > Hi all,
> >
> > I guess the way I see this working is that the request gets sent from the
> > client, to the initial broker, and then forwarded to the final broker.
> >
> > The initial broker should do authentication (who are you?) and come up
> > with a principal name.  Then it creates an envelope request, which will
> > contain that principal name, and sends it along with the unmodified
> > original request to the final broker.  (I agree with Tom and Rajini that
> we
> > can't forward the information needed to do authentication on the final
> > broker, but I also think we don't need to, since we can do it on the
> > initial broker.)
> >
> > The final broker knows it can trust the principal name in the envelope
> > (since EnvelopeRequest requires CLUSTERACTION on CLUSTER).  So it can use
> > that principal name for authorization (given who you are, what can you
> > do?)  The forwarded principal name will also be used for logging.
> >
> > One question is why we need an EnvelopeRequest.  Well, if we don't have
> an
> > EnvelopeRequest, we need somewhere else to put the forwarded principal
> > name.  I don't think overriding an existing field (like clientId) is a
> good
> > option for this.  It's messy, and loses information.  It also raises the
> > question of how the final broker knows that the clientId in the received
> > message is not "really" a clientId, but is a principal name.  Without an
> > envelope, there's no way to clearly mark a request as forwarded, so
> there's
> > no reason for the final broker to treat this differently than a regular
> > clientId (or whatever).
> >
> > We talked about using optional fields to contain the forwarded principal
> > name, but this is also messy and potentially dangerous.  Older brokers
> will
> > simply ignore the optional fields, which could result in them executing
> > operations as the wrong principal.  Of course, this would require a
> > misconfiguration in order to happen, but it still seems better to set up
> > the system so that this misconfiguration is detected, rather than
> silently
> > ignored.
> >
> > It's true that the need for forwarding is "temporary" in some sense,
> since
> > we only need it for older clients.  However, we will want to support
> these
> > older clients for many years to come.
> >
> > I agree that the usefulness of EnvelopeRequest is limited by it being a
> > superuser-only request at the moment.  Perhaps there are some changes to
> > how custom principals work that would allow us to get around this in the
> > future.  We should think about that so that we have this functionality in
> > the future if it's needed.
> >
> > best,
> > Colin
> >
> >
> > On Wed, Apr 22, 2020, at 11:21, Guozhang Wang wrote:
> > > Hello Gwen,
> > >
> > > The purpose here is for maintaining compatibility old clients, who do
> not
> > > have functionality to do re-routing admin requests themselves. New
> > clients
> > > can of course do this themselves by detecting who's the controller.
> > >
> > >
> > > Hello Colin / Boyang,
> > >
> > > Regarding the usage of the envelope, I'm curious what are the potential
> > > future use cases that would require request forwarding and hence
> envelope
> > > would be useful? Originally I thought that the forwarding mechanism is
> > only
> > > temporary as we need it for the bridge release, and moving forward we
> > 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-22 Thread Boyang Chen
Thanks Colin for the summary! And Guozhang, regarding the future use cases,
consider a scenario where there are temporary connectivity issue between
controller to a fellow broker A, the controller could then leverage another
healthy broker B to do a forwarding request to A in order to maintain a
communication.

Even for KIP-590 scope, the forwarding mechanism shall not be transit as we
do need to support older version of admin clients for a couple of years
IIUC.

Boyang

On Wed, Apr 22, 2020 at 1:29 PM Colin McCabe  wrote:

> Hi all,
>
> I guess the way I see this working is that the request gets sent from the
> client, to the initial broker, and then forwarded to the final broker.
>
> The initial broker should do authentication (who are you?) and come up
> with a principal name.  Then it creates an envelope request, which will
> contain that principal name, and sends it along with the unmodified
> original request to the final broker.  (I agree with Tom and Rajini that we
> can't forward the information needed to do authentication on the final
> broker, but I also think we don't need to, since we can do it on the
> initial broker.)
>
> The final broker knows it can trust the principal name in the envelope
> (since EnvelopeRequest requires CLUSTERACTION on CLUSTER).  So it can use
> that principal name for authorization (given who you are, what can you
> do?)  The forwarded principal name will also be used for logging.
>
> One question is why we need an EnvelopeRequest.  Well, if we don't have an
> EnvelopeRequest, we need somewhere else to put the forwarded principal
> name.  I don't think overriding an existing field (like clientId) is a good
> option for this.  It's messy, and loses information.  It also raises the
> question of how the final broker knows that the clientId in the received
> message is not "really" a clientId, but is a principal name.  Without an
> envelope, there's no way to clearly mark a request as forwarded, so there's
> no reason for the final broker to treat this differently than a regular
> clientId (or whatever).
>
> We talked about using optional fields to contain the forwarded principal
> name, but this is also messy and potentially dangerous.  Older brokers will
> simply ignore the optional fields, which could result in them executing
> operations as the wrong principal.  Of course, this would require a
> misconfiguration in order to happen, but it still seems better to set up
> the system so that this misconfiguration is detected, rather than silently
> ignored.
>
> It's true that the need for forwarding is "temporary" in some sense, since
> we only need it for older clients.  However, we will want to support these
> older clients for many years to come.
>
> I agree that the usefulness of EnvelopeRequest is limited by it being a
> superuser-only request at the moment.  Perhaps there are some changes to
> how custom principals work that would allow us to get around this in the
> future.  We should think about that so that we have this functionality in
> the future if it's needed.
>
> best,
> Colin
>
>
> On Wed, Apr 22, 2020, at 11:21, Guozhang Wang wrote:
> > Hello Gwen,
> >
> > The purpose here is for maintaining compatibility old clients, who do not
> > have functionality to do re-routing admin requests themselves. New
> clients
> > can of course do this themselves by detecting who's the controller.
> >
> >
> > Hello Colin / Boyang,
> >
> > Regarding the usage of the envelope, I'm curious what are the potential
> > future use cases that would require request forwarding and hence envelope
> > would be useful? Originally I thought that the forwarding mechanism is
> only
> > temporary as we need it for the bridge release, and moving forward we
> will
> > get rid of this to simplify the code base. If we do have valid use cases
> in
> > the future which makes us believe that request forwarding would actually
> be
> > a permanent feature retained on the broker side, I'm on board with adding
> > the envelope request protocol.
> >
> >
> >
> > Guozhang
> >
> >
> >
> >
> > On Wed, Apr 22, 2020 at 8:22 AM Gwen Shapira  wrote:
> >
> > > Hey Boyang,
> > >
> > > Sorry if this was already discussed, but I didn't see this as rejected
> > > alternative:
> > >
> > > Until now, we always did client side routing - the client itself found
> the
> > > controller via metadata and directed requests accordingly. Brokers that
> > > were not the controller, rejected those requests.
> > >
> > > Why did we decide to move to broker side routing? Was the client-side
> > > option discussed and rejected somewhere and I missed it?
> > >
> > > Gwen
> > >
> > > On Fri, Apr 3, 2020, 4:45 PM Boyang Chen 
> > > wrote:
> > >
> > > > Hey all,
> > > >
> > > > I would like to start off the discussion for KIP-590, a follow-up
> > > > initiative after KIP-500:
> > > >
> > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-590%3A+Redirect+Zookeeper+Mutation+Protocols+to+The+Controller
> > > >
> > > > 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-22 Thread Colin McCabe
Hi all,

I guess the way I see this working is that the request gets sent from the 
client, to the initial broker, and then forwarded to the final broker.

The initial broker should do authentication (who are you?) and come up with a 
principal name.  Then it creates an envelope request, which will contain that 
principal name, and sends it along with the unmodified original request to the 
final broker.  (I agree with Tom and Rajini that we can't forward the 
information needed to do authentication on the final broker, but I also think 
we don't need to, since we can do it on the initial broker.)

The final broker knows it can trust the principal name in the envelope (since 
EnvelopeRequest requires CLUSTERACTION on CLUSTER).  So it can use that 
principal name for authorization (given who you are, what can you do?)  The 
forwarded principal name will also be used for logging.

One question is why we need an EnvelopeRequest.  Well, if we don't have an 
EnvelopeRequest, we need somewhere else to put the forwarded principal name.  I 
don't think overriding an existing field (like clientId) is a good option for 
this.  It's messy, and loses information.  It also raises the question of how 
the final broker knows that the clientId in the received message is not 
"really" a clientId, but is a principal name.  Without an envelope, there's no 
way to clearly mark a request as forwarded, so there's no reason for the final 
broker to treat this differently than a regular clientId (or whatever).

We talked about using optional fields to contain the forwarded principal name, 
but this is also messy and potentially dangerous.  Older brokers will simply 
ignore the optional fields, which could result in them executing operations as 
the wrong principal.  Of course, this would require a misconfiguration in order 
to happen, but it still seems better to set up the system so that this 
misconfiguration is detected, rather than silently ignored.

It's true that the need for forwarding is "temporary" in some sense, since we 
only need it for older clients.  However, we will want to support these older 
clients for many years to come.

I agree that the usefulness of EnvelopeRequest is limited by it being a 
superuser-only request at the moment.  Perhaps there are some changes to how 
custom principals work that would allow us to get around this in the future.  
We should think about that so that we have this functionality in the future if 
it's needed.

best,
Colin


On Wed, Apr 22, 2020, at 11:21, Guozhang Wang wrote:
> Hello Gwen,
> 
> The purpose here is for maintaining compatibility old clients, who do not
> have functionality to do re-routing admin requests themselves. New clients
> can of course do this themselves by detecting who's the controller.
> 
> 
> Hello Colin / Boyang,
> 
> Regarding the usage of the envelope, I'm curious what are the potential
> future use cases that would require request forwarding and hence envelope
> would be useful? Originally I thought that the forwarding mechanism is only
> temporary as we need it for the bridge release, and moving forward we will
> get rid of this to simplify the code base. If we do have valid use cases in
> the future which makes us believe that request forwarding would actually be
> a permanent feature retained on the broker side, I'm on board with adding
> the envelope request protocol.
> 
> 
> 
> Guozhang
> 
> 
> 
> 
> On Wed, Apr 22, 2020 at 8:22 AM Gwen Shapira  wrote:
> 
> > Hey Boyang,
> >
> > Sorry if this was already discussed, but I didn't see this as rejected
> > alternative:
> >
> > Until now, we always did client side routing - the client itself found the
> > controller via metadata and directed requests accordingly. Brokers that
> > were not the controller, rejected those requests.
> >
> > Why did we decide to move to broker side routing? Was the client-side
> > option discussed and rejected somewhere and I missed it?
> >
> > Gwen
> >
> > On Fri, Apr 3, 2020, 4:45 PM Boyang Chen 
> > wrote:
> >
> > > Hey all,
> > >
> > > I would like to start off the discussion for KIP-590, a follow-up
> > > initiative after KIP-500:
> > >
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-590%3A+Redirect+Zookeeper+Mutation+Protocols+to+The+Controller
> > >
> > > This KIP proposes to migrate existing Zookeeper mutation paths, including
> > > configuration, security and quota changes, to controller-only by always
> > > routing these alterations to the controller.
> > >
> > > Let me know your thoughts!
> > >
> > > Best,
> > > Boyang
> > >
> >
> 
> 
> -- 
> -- Guozhang
>


Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-22 Thread Guozhang Wang
Hello Gwen,

The purpose here is for maintaining compatibility old clients, who do not
have functionality to do re-routing admin requests themselves. New clients
can of course do this themselves by detecting who's the controller.


Hello Colin / Boyang,

Regarding the usage of the envelope, I'm curious what are the potential
future use cases that would require request forwarding and hence envelope
would be useful? Originally I thought that the forwarding mechanism is only
temporary as we need it for the bridge release, and moving forward we will
get rid of this to simplify the code base. If we do have valid use cases in
the future which makes us believe that request forwarding would actually be
a permanent feature retained on the broker side, I'm on board with adding
the envelope request protocol.



Guozhang




On Wed, Apr 22, 2020 at 8:22 AM Gwen Shapira  wrote:

> Hey Boyang,
>
> Sorry if this was already discussed, but I didn't see this as rejected
> alternative:
>
> Until now, we always did client side routing - the client itself found the
> controller via metadata and directed requests accordingly. Brokers that
> were not the controller, rejected those requests.
>
> Why did we decide to move to broker side routing? Was the client-side
> option discussed and rejected somewhere and I missed it?
>
> Gwen
>
> On Fri, Apr 3, 2020, 4:45 PM Boyang Chen 
> wrote:
>
> > Hey all,
> >
> > I would like to start off the discussion for KIP-590, a follow-up
> > initiative after KIP-500:
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-590%3A+Redirect+Zookeeper+Mutation+Protocols+to+The+Controller
> >
> > This KIP proposes to migrate existing Zookeeper mutation paths, including
> > configuration, security and quota changes, to controller-only by always
> > routing these alterations to the controller.
> >
> > Let me know your thoughts!
> >
> > Best,
> > Boyang
> >
>


-- 
-- Guozhang


Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-22 Thread Gwen Shapira
Hey Boyang,

Sorry if this was already discussed, but I didn't see this as rejected
alternative:

Until now, we always did client side routing - the client itself found the
controller via metadata and directed requests accordingly. Brokers that
were not the controller, rejected those requests.

Why did we decide to move to broker side routing? Was the client-side
option discussed and rejected somewhere and I missed it?

Gwen

On Fri, Apr 3, 2020, 4:45 PM Boyang Chen  wrote:

> Hey all,
>
> I would like to start off the discussion for KIP-590, a follow-up
> initiative after KIP-500:
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-590%3A+Redirect+Zookeeper+Mutation+Protocols+to+The+Controller
>
> This KIP proposes to migrate existing Zookeeper mutation paths, including
> configuration, security and quota changes, to controller-only by always
> routing these alterations to the controller.
>
> Let me know your thoughts!
>
> Best,
> Boyang
>


Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-22 Thread Rajini Sivaram
We do use custom principals that rely on information not contained in the
serialized principal and hence authorization based on serialized principals
can break existing production systems. Apart from the information contained
in the session principal, ACLs can also be based on host I, but I guess
that can be added easily to the envelope.

Several requests like AlterConfigs are compound requests containing
multiple resources, some of which may be authorized and some not. Given
that the controller cannot perform authorization of forwarded requests, the
forwarding broker needs to not only perform authorization, but also filter
out unauthorized resources from the request.

So assuming that the forwarding broker authorizes request from User:Alice
and audit logs the entry for User:Alice, what is Controller going to do
with the principal in the envelope? I guess we will just let the request go
through the normal request handlers in KafkaApis. Which means the request
will be authorized and audit logged again. The question is whether we would
use 1) principal User:Alice contained in the envelope or 2) the broker
context:
1) User:Alice may not have permissions since its permissions were based on
other fields in its authenticated context not forwarded to the Controller
2) User:Broker will need additional permissions if we use that since
brokers never needed AlterConfigs permissions before.

I think we should use 2). Which basically means that the principal in the
envelope is not particularly useful and we could just as well put that into
client-id (insecure and loggable, but not used in a security-critical
context).

Regards,

Rajini


On Wed, Apr 22, 2020 at 8:39 AM Tom Bentley  wrote:

> Hi Boyang and Sönke,
>
> Regarding custom Principals, I don't think too many people do this in
> > practice, but in theory you can provide you own PrincipalBuilder and use
> > your own Principal objects that contain as much additional information as
> > you wish. And since these can basically be any Java object that makes
> them
> > very tough to serialize.
> > Currently these Principals don't need to be serialized, because they are
> > created and used within the same JVM, there is no need to forward them
> to a
> > different broker.
> >
>
> This is exactly the point I was trying to make (clearly not so
> successfully).
>
> You can't (in general) serialize the principal itself. (There's nothing
> stopping a custom principal builder from returning an instance of some
> subclass of KafkaPrincipal with extra fields which are then used by some
> custom authorizer). And I don't think it's possible to serialize the
> AuthenticationContext (which would allow you to obtain a principal instance
> on the controller using the principal builder). For example,
> SslAuthenticationContext contains an SSLSession.
>
> I think Sönke is correct that this isn't likely to be functionality which
> is used frequently, but these contracts existed already, so we can't just
> decide that only KafkaPrincipal can be used.
>
> Kind regards,
>
> Tom
>
> On Tue, Apr 21, 2020 at 10:43 PM Sönke Liebau
>  wrote:
>
> > Hi Boyang,
> >
> > I think what Tom is referring to is that it is very hard to forward
> enough
> > information to the controller to put it into a position to properly
> > authenticate any request.
> >
> > While the Default KafkaPrincipal can easily be serialized and sent to the
> > controller, as previously seen, those are just strings. For the
> Controller
> > to properly authenticate a request we'd need to forward the
> > AuthenticationContext (from which the Principal is built [1]) containing
> > the SSL/SASL details to the controller, in order for the controller to
> then
> > check certificates for validity etc.
> > And those checks will be very difficult, because what we are effectively
> > doing here is a man-in-the-middle attack (broadly speaking), as we are
> > forwarding a request "in the name of" someone else. And most
> authentication
> > methods have been built to prevent exactly that.
> > As soon as we have only the Principal we are trusting someone else to
> have
> > properly authenticated that principal, because we do not have all the
> > information to do that verification ourselves. And if we do that, then I
> > don't see why we should a
> >
> > Regarding custom Principals, I don't think too many people do this in
> > practice, but in theory you can provide you own PrincipalBuilder and use
> > your own Principal objects that contain as much additional information as
> > you wish. And since these can basically be any Java object that makes
> them
> > very tough to serialize.
> > Currently these Principals don't need to be serialized, because they are
> > created and used within the same JVM, there is no need to forward them
> to a
> > different broker.
> > I wrote a blog post [2] about a scenario that uses a custom Principal a
> > little while ago, that shows a possible scenario, maybe that helps a
> > little.
> >
> > Feel free to 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-22 Thread Tom Bentley
Hi Boyang and Sönke,

Regarding custom Principals, I don't think too many people do this in
> practice, but in theory you can provide you own PrincipalBuilder and use
> your own Principal objects that contain as much additional information as
> you wish. And since these can basically be any Java object that makes them
> very tough to serialize.
> Currently these Principals don't need to be serialized, because they are
> created and used within the same JVM, there is no need to forward them to a
> different broker.
>

This is exactly the point I was trying to make (clearly not so
successfully).

You can't (in general) serialize the principal itself. (There's nothing
stopping a custom principal builder from returning an instance of some
subclass of KafkaPrincipal with extra fields which are then used by some
custom authorizer). And I don't think it's possible to serialize the
AuthenticationContext (which would allow you to obtain a principal instance
on the controller using the principal builder). For example,
SslAuthenticationContext contains an SSLSession.

I think Sönke is correct that this isn't likely to be functionality which
is used frequently, but these contracts existed already, so we can't just
decide that only KafkaPrincipal can be used.

Kind regards,

Tom

On Tue, Apr 21, 2020 at 10:43 PM Sönke Liebau
 wrote:

> Hi Boyang,
>
> I think what Tom is referring to is that it is very hard to forward enough
> information to the controller to put it into a position to properly
> authenticate any request.
>
> While the Default KafkaPrincipal can easily be serialized and sent to the
> controller, as previously seen, those are just strings. For the Controller
> to properly authenticate a request we'd need to forward the
> AuthenticationContext (from which the Principal is built [1]) containing
> the SSL/SASL details to the controller, in order for the controller to then
> check certificates for validity etc.
> And those checks will be very difficult, because what we are effectively
> doing here is a man-in-the-middle attack (broadly speaking), as we are
> forwarding a request "in the name of" someone else. And most authentication
> methods have been built to prevent exactly that.
> As soon as we have only the Principal we are trusting someone else to have
> properly authenticated that principal, because we do not have all the
> information to do that verification ourselves. And if we do that, then I
> don't see why we should a
>
> Regarding custom Principals, I don't think too many people do this in
> practice, but in theory you can provide you own PrincipalBuilder and use
> your own Principal objects that contain as much additional information as
> you wish. And since these can basically be any Java object that makes them
> very tough to serialize.
> Currently these Principals don't need to be serialized, because they are
> created and used within the same JVM, there is no need to forward them to a
> different broker.
> I wrote a blog post [2] about a scenario that uses a custom Principal a
> little while ago, that shows a possible scenario, maybe that helps a
> little.
>
> Feel free to correct me if I misinterpreted your meaning Tom :)
>
> Best regards,
> Sönke
>
> [1] https://imgur.com/a/Gi0cFNH
> [2]
> https://www.opencore.com/de/blog/2018/3/group-based-authorization-in-kafka/
>
> On Tue, 21 Apr 2020 at 20:33, Boyang Chen 
> wrote:
>
> > Hey Tom,
> >
> > I agree with the claim here. All the brokers should have the same
> > authentication power, which means getting the forwarding broker verify
> the
> > client request first is more favorable. This approach avoids sending one
> > unnecessary forwarding request if it couldn't pass the authorization in
> the
> > first place.
> >
> > In the meantime, could you give more context on the custom Kafka
> principal
> > you are referring to? How does that get encoded today, and how server and
> > client could both agree on the serialization? As the plain principal is
> > only a String, I would like to know more about the security strategy
> people
> > are using, thanks!
> >
> > Boyang
> >
> > On Tue, Apr 21, 2020 at 2:24 AM Tom Bentley  wrote:
> >
> > > Hi Boyang,
> > >
> > > The answer to my original question about the request principal was that
> > the
> > > forwarding broker would authorize the request and the controller would
> > > trust the request since it was from another broker. AFAIU you added the
> > > principal purely for logging purposes. In the "EnvelopeRequest
> Handling"
> > > section the KIP now says "Once that part is done, we shall replace the
> > > request context with Principal information embedded inside the
> > > EnvelopeRequest to complete the inner request permission check.", which
> > > sounds to me like the controller is now authorizing the request (maybe
> in
> > > addition to the forwarding broker) using a principal it's deserialized
> > from
> > > the EnvelopeRequest. I don't think that works if a custom principal
> > builder
> > > is 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-21 Thread Sönke Liebau
Hi Boyang,

I think what Tom is referring to is that it is very hard to forward enough
information to the controller to put it into a position to properly
authenticate any request.

While the Default KafkaPrincipal can easily be serialized and sent to the
controller, as previously seen, those are just strings. For the Controller
to properly authenticate a request we'd need to forward the
AuthenticationContext (from which the Principal is built [1]) containing
the SSL/SASL details to the controller, in order for the controller to then
check certificates for validity etc.
And those checks will be very difficult, because what we are effectively
doing here is a man-in-the-middle attack (broadly speaking), as we are
forwarding a request "in the name of" someone else. And most authentication
methods have been built to prevent exactly that.
As soon as we have only the Principal we are trusting someone else to have
properly authenticated that principal, because we do not have all the
information to do that verification ourselves. And if we do that, then I
don't see why we should a

Regarding custom Principals, I don't think too many people do this in
practice, but in theory you can provide you own PrincipalBuilder and use
your own Principal objects that contain as much additional information as
you wish. And since these can basically be any Java object that makes them
very tough to serialize.
Currently these Principals don't need to be serialized, because they are
created and used within the same JVM, there is no need to forward them to a
different broker.
I wrote a blog post [2] about a scenario that uses a custom Principal a
little while ago, that shows a possible scenario, maybe that helps a little.

Feel free to correct me if I misinterpreted your meaning Tom :)

Best regards,
Sönke

[1] https://imgur.com/a/Gi0cFNH
[2]
https://www.opencore.com/de/blog/2018/3/group-based-authorization-in-kafka/

On Tue, 21 Apr 2020 at 20:33, Boyang Chen 
wrote:

> Hey Tom,
>
> I agree with the claim here. All the brokers should have the same
> authentication power, which means getting the forwarding broker verify the
> client request first is more favorable. This approach avoids sending one
> unnecessary forwarding request if it couldn't pass the authorization in the
> first place.
>
> In the meantime, could you give more context on the custom Kafka principal
> you are referring to? How does that get encoded today, and how server and
> client could both agree on the serialization? As the plain principal is
> only a String, I would like to know more about the security strategy people
> are using, thanks!
>
> Boyang
>
> On Tue, Apr 21, 2020 at 2:24 AM Tom Bentley  wrote:
>
> > Hi Boyang,
> >
> > The answer to my original question about the request principal was that
> the
> > forwarding broker would authorize the request and the controller would
> > trust the request since it was from another broker. AFAIU you added the
> > principal purely for logging purposes. In the "EnvelopeRequest Handling"
> > section the KIP now says "Once that part is done, we shall replace the
> > request context with Principal information embedded inside the
> > EnvelopeRequest to complete the inner request permission check.", which
> > sounds to me like the controller is now authorizing the request (maybe in
> > addition to the forwarding broker) using a principal it's deserialized
> from
> > the EnvelopeRequest. I don't think that works if a custom principal
> builder
> > is returning a subclass of KafkaPrincipal (the Javadoc for KafkaPrincipal
> > describes the contract I'm talking about). Basically the controller would
> > not be able to instantiate the subclass (even if that was included in the
> > envelope) because it wouldn't necessarily know the signature of the
> > constructor. Nor can it use the principal builder itself because it
> doesn't
> > have access to the original AuthenticationContext. Maybe you figure out
> > some way to make it work, otherwise I think the best you can do is to
> > revert to the model you had before where the controller trusts the
> embedded
> > request because it's been received from a broker.
> >
> > Cheers,
> >
> > Tom
> >
> > On Sat, Apr 18, 2020 at 8:56 PM Colin McCabe  wrote:
> >
> > > On Fri, Apr 17, 2020, at 13:11, Ismael Juma wrote:
> > > > Hi Colin,
> > > >
> > > > The read/modify/write is protected by the zk version, right?
> > > >
> > > > Ismael
> > >
> > > No, we don't use the ZK version when doing the write to the config
> > > znodes.  We do for ACLs, I think.
> > >
> > > This is something that we could fix just by using the ZK version, but
> > > there are other race conditions like what if we're deleting a topic
> while
> > > setting this config, etc.  A single writer is a lot easier to reason
> > about.
> > >
> > > best,
> > > Colin
> > >
> > >
> > > >
> > > > On Fri, Apr 17, 2020 at 12:53 PM Colin McCabe 
> > > wrote:
> > > >
> > > > > On Thu, Apr 16, 2020, at 08:51, Ismael Juma wrote:
> > > > > > I 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-21 Thread Boyang Chen
Hey Tom,

I agree with the claim here. All the brokers should have the same
authentication power, which means getting the forwarding broker verify the
client request first is more favorable. This approach avoids sending one
unnecessary forwarding request if it couldn't pass the authorization in the
first place.

In the meantime, could you give more context on the custom Kafka principal
you are referring to? How does that get encoded today, and how server and
client could both agree on the serialization? As the plain principal is
only a String, I would like to know more about the security strategy people
are using, thanks!

Boyang

On Tue, Apr 21, 2020 at 2:24 AM Tom Bentley  wrote:

> Hi Boyang,
>
> The answer to my original question about the request principal was that the
> forwarding broker would authorize the request and the controller would
> trust the request since it was from another broker. AFAIU you added the
> principal purely for logging purposes. In the "EnvelopeRequest Handling"
> section the KIP now says "Once that part is done, we shall replace the
> request context with Principal information embedded inside the
> EnvelopeRequest to complete the inner request permission check.", which
> sounds to me like the controller is now authorizing the request (maybe in
> addition to the forwarding broker) using a principal it's deserialized from
> the EnvelopeRequest. I don't think that works if a custom principal builder
> is returning a subclass of KafkaPrincipal (the Javadoc for KafkaPrincipal
> describes the contract I'm talking about). Basically the controller would
> not be able to instantiate the subclass (even if that was included in the
> envelope) because it wouldn't necessarily know the signature of the
> constructor. Nor can it use the principal builder itself because it doesn't
> have access to the original AuthenticationContext. Maybe you figure out
> some way to make it work, otherwise I think the best you can do is to
> revert to the model you had before where the controller trusts the embedded
> request because it's been received from a broker.
>
> Cheers,
>
> Tom
>
> On Sat, Apr 18, 2020 at 8:56 PM Colin McCabe  wrote:
>
> > On Fri, Apr 17, 2020, at 13:11, Ismael Juma wrote:
> > > Hi Colin,
> > >
> > > The read/modify/write is protected by the zk version, right?
> > >
> > > Ismael
> >
> > No, we don't use the ZK version when doing the write to the config
> > znodes.  We do for ACLs, I think.
> >
> > This is something that we could fix just by using the ZK version, but
> > there are other race conditions like what if we're deleting a topic while
> > setting this config, etc.  A single writer is a lot easier to reason
> about.
> >
> > best,
> > Colin
> >
> >
> > >
> > > On Fri, Apr 17, 2020 at 12:53 PM Colin McCabe 
> > wrote:
> > >
> > > > On Thu, Apr 16, 2020, at 08:51, Ismael Juma wrote:
> > > > > I don't think these requests are necessarily infrequent under multi
> > > > tenant
> > > > > environments though. I've seen Controller availability being an
> > issue for
> > > > > describe topics for example (before it was changed to go to any
> > broker).
> > > >
> > > > Hi Ismael,
> > > >
> > > > I don't think DescribeTopics is a good comparison.  That RPC is
> > available
> > > > to regular users and is used many orders of magnitude more frequently
> > than
> > > > administrative operations like changing ACLs or setting quotas.
> > > >
> > > > The operations we're talking about redirecting here all require the
> > > > highest possible permissions and will not be frequent in any
> real-world
> > > > cluster... unless someone is running a stress-test or a benchmark.
> We
> > > > didn't even notice some of the serious bugs in setting dynamic
> configs
> > > > until recently because the alterConfigs / incrementalAlterConfigs
> RPCs
> > are
> > > > so infrequently called.
> > > >
> > > > Additionally, this KIP fixes some existing bugs.  The current
> approach
> > of
> > > > having random writers do a read-write-modify cycle on a configuration
> > znode
> > > > is buggy since it could be interleaved with another node's
> read-modify
> > > > write cycle.  It has a "lost updates" problem.
> > > >
> > > > For example, node 1 reads a config znode.  Node 2 reads the same
> config
> > > > znode.  Node 1 writes back a modified version of the znode.  Node 2
> > writes
> > > > back its (differently) modified version, overwriting the changes from
> > node
> > > > 1.
> > > >
> > > > I don't think anyone ever noticed this problem since, again, these
> > > > operations are very infrequent, making the chance of such a collision
> > low.
> > > > But it is a serious bug that is fixed by having a single writer.  (We
> > > > should add this to the KIP...)
> > > >
> > > > >
> > > > > Would it be better to redirect once the controller quorum is there?
> > > >
> > > > This KIP is needed for the bridge release.  The bridge release
> upgrade
> > > > process relies on the old nodes sending their administrative
> > 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-21 Thread Tom Bentley
Hi Boyang,

The answer to my original question about the request principal was that the
forwarding broker would authorize the request and the controller would
trust the request since it was from another broker. AFAIU you added the
principal purely for logging purposes. In the "EnvelopeRequest Handling"
section the KIP now says "Once that part is done, we shall replace the
request context with Principal information embedded inside the
EnvelopeRequest to complete the inner request permission check.", which
sounds to me like the controller is now authorizing the request (maybe in
addition to the forwarding broker) using a principal it's deserialized from
the EnvelopeRequest. I don't think that works if a custom principal builder
is returning a subclass of KafkaPrincipal (the Javadoc for KafkaPrincipal
describes the contract I'm talking about). Basically the controller would
not be able to instantiate the subclass (even if that was included in the
envelope) because it wouldn't necessarily know the signature of the
constructor. Nor can it use the principal builder itself because it doesn't
have access to the original AuthenticationContext. Maybe you figure out
some way to make it work, otherwise I think the best you can do is to
revert to the model you had before where the controller trusts the embedded
request because it's been received from a broker.

Cheers,

Tom

On Sat, Apr 18, 2020 at 8:56 PM Colin McCabe  wrote:

> On Fri, Apr 17, 2020, at 13:11, Ismael Juma wrote:
> > Hi Colin,
> >
> > The read/modify/write is protected by the zk version, right?
> >
> > Ismael
>
> No, we don't use the ZK version when doing the write to the config
> znodes.  We do for ACLs, I think.
>
> This is something that we could fix just by using the ZK version, but
> there are other race conditions like what if we're deleting a topic while
> setting this config, etc.  A single writer is a lot easier to reason about.
>
> best,
> Colin
>
>
> >
> > On Fri, Apr 17, 2020 at 12:53 PM Colin McCabe 
> wrote:
> >
> > > On Thu, Apr 16, 2020, at 08:51, Ismael Juma wrote:
> > > > I don't think these requests are necessarily infrequent under multi
> > > tenant
> > > > environments though. I've seen Controller availability being an
> issue for
> > > > describe topics for example (before it was changed to go to any
> broker).
> > >
> > > Hi Ismael,
> > >
> > > I don't think DescribeTopics is a good comparison.  That RPC is
> available
> > > to regular users and is used many orders of magnitude more frequently
> than
> > > administrative operations like changing ACLs or setting quotas.
> > >
> > > The operations we're talking about redirecting here all require the
> > > highest possible permissions and will not be frequent in any real-world
> > > cluster... unless someone is running a stress-test or a benchmark.  We
> > > didn't even notice some of the serious bugs in setting dynamic configs
> > > until recently because the alterConfigs / incrementalAlterConfigs RPCs
> are
> > > so infrequently called.
> > >
> > > Additionally, this KIP fixes some existing bugs.  The current approach
> of
> > > having random writers do a read-write-modify cycle on a configuration
> znode
> > > is buggy since it could be interleaved with another node's read-modify
> > > write cycle.  It has a "lost updates" problem.
> > >
> > > For example, node 1 reads a config znode.  Node 2 reads the same config
> > > znode.  Node 1 writes back a modified version of the znode.  Node 2
> writes
> > > back its (differently) modified version, overwriting the changes from
> node
> > > 1.
> > >
> > > I don't think anyone ever noticed this problem since, again, these
> > > operations are very infrequent, making the chance of such a collision
> low.
> > > But it is a serious bug that is fixed by having a single writer.  (We
> > > should add this to the KIP...)
> > >
> > > >
> > > > Would it be better to redirect once the controller quorum is there?
> > >
> > > This KIP is needed for the bridge release.  The bridge release upgrade
> > > process relies on the old nodes sending their administrative
> operations to
> > > the controller quorum, not directly to zookeeper.
> > >
> > > best,
> > > Colin
> > >
> > >
> > > >
> > > > Note that this is different from things like AlterIsr since these
> calls
> > > are
> > > > coming from clients versus other brokers.
> > > >
> > > > Ismael
> > > >
> > > > On Wed, Apr 15, 2020, 5:10 PM Colin McCabe 
> wrote:
> > > >
> > > > > Hi Ismael,
> > > > >
> > > > > I agree that sending these requests through the controller will not
> > > work
> > > > > during the periods when there is no controller.  However, those
> periods
> > > > > should be short-- otherwise we have bigger problems in the cluster.
> > > > >
> > > > > These requests are very infrequent because they are administrative
> > > > > operations.  Basically the affected operations are changing ACLs,
> > > changing
> > > > > dynamic configurations, and changing quotas.
> > > > >
> > > > > best,

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-18 Thread Colin McCabe
On Fri, Apr 17, 2020, at 13:11, Ismael Juma wrote:
> Hi Colin,
> 
> The read/modify/write is protected by the zk version, right?
> 
> Ismael

No, we don't use the ZK version when doing the write to the config znodes.  We 
do for ACLs, I think.

This is something that we could fix just by using the ZK version, but there are 
other race conditions like what if we're deleting a topic while setting this 
config, etc.  A single writer is a lot easier to reason about.

best,
Colin


> 
> On Fri, Apr 17, 2020 at 12:53 PM Colin McCabe  wrote:
> 
> > On Thu, Apr 16, 2020, at 08:51, Ismael Juma wrote:
> > > I don't think these requests are necessarily infrequent under multi
> > tenant
> > > environments though. I've seen Controller availability being an issue for
> > > describe topics for example (before it was changed to go to any broker).
> >
> > Hi Ismael,
> >
> > I don't think DescribeTopics is a good comparison.  That RPC is available
> > to regular users and is used many orders of magnitude more frequently than
> > administrative operations like changing ACLs or setting quotas.
> >
> > The operations we're talking about redirecting here all require the
> > highest possible permissions and will not be frequent in any real-world
> > cluster... unless someone is running a stress-test or a benchmark.  We
> > didn't even notice some of the serious bugs in setting dynamic configs
> > until recently because the alterConfigs / incrementalAlterConfigs RPCs are
> > so infrequently called.
> >
> > Additionally, this KIP fixes some existing bugs.  The current approach of
> > having random writers do a read-write-modify cycle on a configuration znode
> > is buggy since it could be interleaved with another node's read-modify
> > write cycle.  It has a "lost updates" problem.
> >
> > For example, node 1 reads a config znode.  Node 2 reads the same config
> > znode.  Node 1 writes back a modified version of the znode.  Node 2 writes
> > back its (differently) modified version, overwriting the changes from node
> > 1.
> >
> > I don't think anyone ever noticed this problem since, again, these
> > operations are very infrequent, making the chance of such a collision low.
> > But it is a serious bug that is fixed by having a single writer.  (We
> > should add this to the KIP...)
> >
> > >
> > > Would it be better to redirect once the controller quorum is there?
> >
> > This KIP is needed for the bridge release.  The bridge release upgrade
> > process relies on the old nodes sending their administrative operations to
> > the controller quorum, not directly to zookeeper.
> >
> > best,
> > Colin
> >
> >
> > >
> > > Note that this is different from things like AlterIsr since these calls
> > are
> > > coming from clients versus other brokers.
> > >
> > > Ismael
> > >
> > > On Wed, Apr 15, 2020, 5:10 PM Colin McCabe  wrote:
> > >
> > > > Hi Ismael,
> > > >
> > > > I agree that sending these requests through the controller will not
> > work
> > > > during the periods when there is no controller.  However, those periods
> > > > should be short-- otherwise we have bigger problems in the cluster.
> > > >
> > > > These requests are very infrequent because they are administrative
> > > > operations.  Basically the affected operations are changing ACLs,
> > changing
> > > > dynamic configurations, and changing quotas.
> > > >
> > > > best,
> > > > Colin
> > > >
> > > >
> > > > On Wed, Apr 15, 2020, at 15:25, Ismael Juma wrote:
> > > > > Hi Boyang,
> > > > >
> > > > > Thanks for the KIP. Have we considered that this reduces
> > availability for
> > > > > these operations since we have a single Controller instead of the ZK
> > > > quorum?
> > > > >
> > > > > Ismael
> > > > >
> > > > > On Fri, Apr 3, 2020 at 4:45 PM Boyang Chen <
> > reluctanthero...@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hey all,
> > > > > >
> > > > > > I would like to start off the discussion for KIP-590, a follow-up
> > > > > > initiative after KIP-500:
> > > > > >
> > > > > >
> > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-590%3A+Redirect+Zookeeper+Mutation+Protocols+to+The+Controller
> > > > > >
> > > > > > This KIP proposes to migrate existing Zookeeper mutation paths,
> > > > including
> > > > > > configuration, security and quota changes, to controller-only by
> > always
> > > > > > routing these alterations to the controller.
> > > > > >
> > > > > > Let me know your thoughts!
> > > > > >
> > > > > > Best,
> > > > > > Boyang
> > > > > >
> > > > >
> > > >
> > >
> >
>


Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-17 Thread Boyang Chen
Thanks Colin for the suggestions! I have added metrics to track the number
of total messages being forwarded as a more generic monitoring. Right now
the list of metrics are:


   - num-client-forwarding-to-controller-rate
   - num-client-fowarding-to-controller-count
   - num-messages-redirected-rate
   - num-messages-redirected-count
   - request-forwarding-authorization-fail-count (Optional)

Let me know if we need to add more monitoring tags.

Boyang

On Fri, Apr 17, 2020 at 1:16 PM Colin McCabe  wrote:

> On Thu, Apr 16, 2020, at 12:33, Jose Garcia Sancio wrote:
> > Hi Boyang,
> >
> > Thanks for the KIP. The KIP looks good. I have a few questions and
> comments.
> >
> > > As part of the KIP-500
> > <
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-500%3A+Replace+ZooKeeper+with+a+Self-Managed+Metadata+Quorum
> >
> > initiative, we need to build a bridge release version of Kafka that
> > could
> > isolate the direct Zookeeper write access only to the controller.
> >
> > This may be outside the scope of this KIP but I am trying to understand
> how
> > this is going to be used to implement KIP-500. My understanding is that
> the
> > Admin client discovers the controller by performing a
> > Metadata{Request,Response} round trip. The response of this request
> > includes the id of the controller. Based on my understanding of the
> > KIP-500, this architecture will need to change.
>
> Hi Jose,
>
> Thanks for the questions.
>
> We're not proposing to change the MetadataRequest / MetadataResponse.  It
> will still work the same way that it always has from the perspective of
> clients.  We have to do this to maintain compatibility, I think.
>
> This does mean that controllers and brokers will occupy the same ID
> namespace, even when running in standalone controller mode.  I think that's
> OK, though.
>
> > For example the controller
> > will not necessarily be a broker in which case the id may not correlate
> to
> > a broker id. Is the expectation that the Kafka Controller Quorum (as
> > defined in KIP-500) will push this new connection information to all of
> the
> > brokers? Will the Kafka Controller Quorum expose and implement all of the
> > RPCs being redirected in this KIP and the ones that are currently routed
> to
> > the controller? These include:
> >
> > ListPartitionReassignment
> > AlterPartitionReassignment
> > ElectLeaders
> > CreatePartitions
> > DeleteTopics
> > CreateTopics
>
> Yes, the KIP-500 controller (quorum) will implement all of those RPCs.
>
> >
> > > AUTHORIZATION_FAILED if the inter-broker verification failed.
> >
> > The rest of the document mentions CLUSTER_AUTHORIZATION_FAILED.
> >
> > > For CLUSTER_AUTHORIZATION_FAILED, this indicates an internal error for
> > broker security setup which has nothing to do with the client, so we have
> > no other way but returning an UNKNOWN_SERVER_ERROR to the admin client.
> >
> > I don't understand this. I think I don't understand this because it is
> not
> > clear to me who, how and when authorization is going to work when using
> > Envelopre{Request,Response}. Can you please add a section that explains
> how
> > authorization works when envelopes are involved?
>
> The only way a broker would fail to authorize itself to another broker is
> if the ACLs were incorrectly set up for the cluster.  This would cause
> other problems beyond just making forwarding not work, though.
>
> This isn't really related to envelopes -- any time a broker makes an RPC
> to another broker, in theory the ACLs could be totally screwed up and we
> could get denied.  It clearly means that administrator made a mistake, but
> we still have to handle the case somehow.
>
> best,
> Colin
>


Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-17 Thread Colin McCabe
On Thu, Apr 16, 2020, at 12:33, Jose Garcia Sancio wrote:
> Hi Boyang,
> 
> Thanks for the KIP. The KIP looks good. I have a few questions and comments.
> 
> > As part of the KIP-500
> 
> initiative, we need to build a bridge release version of Kafka that 
> could
> isolate the direct Zookeeper write access only to the controller.
> 
> This may be outside the scope of this KIP but I am trying to understand how
> this is going to be used to implement KIP-500. My understanding is that the
> Admin client discovers the controller by performing a
> Metadata{Request,Response} round trip. The response of this request
> includes the id of the controller. Based on my understanding of the
> KIP-500, this architecture will need to change.

Hi Jose,

Thanks for the questions.

We're not proposing to change the MetadataRequest / MetadataResponse.  It will 
still work the same way that it always has from the perspective of clients.  We 
have to do this to maintain compatibility, I think.

This does mean that controllers and brokers will occupy the same ID namespace, 
even when running in standalone controller mode.  I think that's OK, though.

> For example the controller
> will not necessarily be a broker in which case the id may not correlate to
> a broker id. Is the expectation that the Kafka Controller Quorum (as
> defined in KIP-500) will push this new connection information to all of the
> brokers? Will the Kafka Controller Quorum expose and implement all of the
> RPCs being redirected in this KIP and the ones that are currently routed to
> the controller? These include:
> 
> ListPartitionReassignment
> AlterPartitionReassignment
> ElectLeaders
> CreatePartitions
> DeleteTopics
> CreateTopics

Yes, the KIP-500 controller (quorum) will implement all of those RPCs.

> 
> > AUTHORIZATION_FAILED if the inter-broker verification failed.
> 
> The rest of the document mentions CLUSTER_AUTHORIZATION_FAILED.
> 
> > For CLUSTER_AUTHORIZATION_FAILED, this indicates an internal error for
> broker security setup which has nothing to do with the client, so we have
> no other way but returning an UNKNOWN_SERVER_ERROR to the admin client.
> 
> I don't understand this. I think I don't understand this because it is not
> clear to me who, how and when authorization is going to work when using
> Envelopre{Request,Response}. Can you please add a section that explains how
> authorization works when envelopes are involved?

The only way a broker would fail to authorize itself to another broker is if 
the ACLs were incorrectly set up for the cluster.  This would cause other 
problems beyond just making forwarding not work, though.

This isn't really related to envelopes -- any time a broker makes an RPC to 
another broker, in theory the ACLs could be totally screwed up and we could get 
denied.  It clearly means that administrator made a mistake, but we still have 
to handle the case somehow.

best,
Colin


Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-17 Thread Ismael Juma
Hi Colin,

The read/modify/write is protected by the zk version, right?

Ismael

On Fri, Apr 17, 2020 at 12:53 PM Colin McCabe  wrote:

> On Thu, Apr 16, 2020, at 08:51, Ismael Juma wrote:
> > I don't think these requests are necessarily infrequent under multi
> tenant
> > environments though. I've seen Controller availability being an issue for
> > describe topics for example (before it was changed to go to any broker).
>
> Hi Ismael,
>
> I don't think DescribeTopics is a good comparison.  That RPC is available
> to regular users and is used many orders of magnitude more frequently than
> administrative operations like changing ACLs or setting quotas.
>
> The operations we're talking about redirecting here all require the
> highest possible permissions and will not be frequent in any real-world
> cluster... unless someone is running a stress-test or a benchmark.  We
> didn't even notice some of the serious bugs in setting dynamic configs
> until recently because the alterConfigs / incrementalAlterConfigs RPCs are
> so infrequently called.
>
> Additionally, this KIP fixes some existing bugs.  The current approach of
> having random writers do a read-write-modify cycle on a configuration znode
> is buggy since it could be interleaved with another node's read-modify
> write cycle.  It has a "lost updates" problem.
>
> For example, node 1 reads a config znode.  Node 2 reads the same config
> znode.  Node 1 writes back a modified version of the znode.  Node 2 writes
> back its (differently) modified version, overwriting the changes from node
> 1.
>
> I don't think anyone ever noticed this problem since, again, these
> operations are very infrequent, making the chance of such a collision low.
> But it is a serious bug that is fixed by having a single writer.  (We
> should add this to the KIP...)
>
> >
> > Would it be better to redirect once the controller quorum is there?
>
> This KIP is needed for the bridge release.  The bridge release upgrade
> process relies on the old nodes sending their administrative operations to
> the controller quorum, not directly to zookeeper.
>
> best,
> Colin
>
>
> >
> > Note that this is different from things like AlterIsr since these calls
> are
> > coming from clients versus other brokers.
> >
> > Ismael
> >
> > On Wed, Apr 15, 2020, 5:10 PM Colin McCabe  wrote:
> >
> > > Hi Ismael,
> > >
> > > I agree that sending these requests through the controller will not
> work
> > > during the periods when there is no controller.  However, those periods
> > > should be short-- otherwise we have bigger problems in the cluster.
> > >
> > > These requests are very infrequent because they are administrative
> > > operations.  Basically the affected operations are changing ACLs,
> changing
> > > dynamic configurations, and changing quotas.
> > >
> > > best,
> > > Colin
> > >
> > >
> > > On Wed, Apr 15, 2020, at 15:25, Ismael Juma wrote:
> > > > Hi Boyang,
> > > >
> > > > Thanks for the KIP. Have we considered that this reduces
> availability for
> > > > these operations since we have a single Controller instead of the ZK
> > > quorum?
> > > >
> > > > Ismael
> > > >
> > > > On Fri, Apr 3, 2020 at 4:45 PM Boyang Chen <
> reluctanthero...@gmail.com>
> > > > wrote:
> > > >
> > > > > Hey all,
> > > > >
> > > > > I would like to start off the discussion for KIP-590, a follow-up
> > > > > initiative after KIP-500:
> > > > >
> > > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-590%3A+Redirect+Zookeeper+Mutation+Protocols+to+The+Controller
> > > > >
> > > > > This KIP proposes to migrate existing Zookeeper mutation paths,
> > > including
> > > > > configuration, security and quota changes, to controller-only by
> always
> > > > > routing these alterations to the controller.
> > > > >
> > > > > Let me know your thoughts!
> > > > >
> > > > > Best,
> > > > > Boyang
> > > > >
> > > >
> > >
> >
>


Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-17 Thread Colin McCabe
On Thu, Apr 16, 2020, at 08:51, Ismael Juma wrote:
> I don't think these requests are necessarily infrequent under multi tenant
> environments though. I've seen Controller availability being an issue for
> describe topics for example (before it was changed to go to any broker).

Hi Ismael,

I don't think DescribeTopics is a good comparison.  That RPC is available to 
regular users and is used many orders of magnitude more frequently than 
administrative operations like changing ACLs or setting quotas.

The operations we're talking about redirecting here all require the highest 
possible permissions and will not be frequent in any real-world cluster... 
unless someone is running a stress-test or a benchmark.  We didn't even notice 
some of the serious bugs in setting dynamic configs until recently because the 
alterConfigs / incrementalAlterConfigs RPCs are so infrequently called.

Additionally, this KIP fixes some existing bugs.  The current approach of 
having random writers do a read-write-modify cycle on a configuration znode is 
buggy since it could be interleaved with another node's read-modify write 
cycle.  It has a "lost updates" problem.

For example, node 1 reads a config znode.  Node 2 reads the same config znode.  
Node 1 writes back a modified version of the znode.  Node 2 writes back its 
(differently) modified version, overwriting the changes from node 1.

I don't think anyone ever noticed this problem since, again, these operations 
are very infrequent, making the chance of such a collision low.  But it is a 
serious bug that is fixed by having a single writer.  (We should add this to 
the KIP...)

> 
> Would it be better to redirect once the controller quorum is there?

This KIP is needed for the bridge release.  The bridge release upgrade process 
relies on the old nodes sending their administrative operations to the 
controller quorum, not directly to zookeeper.

best,
Colin


> 
> Note that this is different from things like AlterIsr since these calls are
> coming from clients versus other brokers.
> 
> Ismael
> 
> On Wed, Apr 15, 2020, 5:10 PM Colin McCabe  wrote:
> 
> > Hi Ismael,
> >
> > I agree that sending these requests through the controller will not work
> > during the periods when there is no controller.  However, those periods
> > should be short-- otherwise we have bigger problems in the cluster.
> >
> > These requests are very infrequent because they are administrative
> > operations.  Basically the affected operations are changing ACLs, changing
> > dynamic configurations, and changing quotas.
> >
> > best,
> > Colin
> >
> >
> > On Wed, Apr 15, 2020, at 15:25, Ismael Juma wrote:
> > > Hi Boyang,
> > >
> > > Thanks for the KIP. Have we considered that this reduces availability for
> > > these operations since we have a single Controller instead of the ZK
> > quorum?
> > >
> > > Ismael
> > >
> > > On Fri, Apr 3, 2020 at 4:45 PM Boyang Chen 
> > > wrote:
> > >
> > > > Hey all,
> > > >
> > > > I would like to start off the discussion for KIP-590, a follow-up
> > > > initiative after KIP-500:
> > > >
> > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-590%3A+Redirect+Zookeeper+Mutation+Protocols+to+The+Controller
> > > >
> > > > This KIP proposes to migrate existing Zookeeper mutation paths,
> > including
> > > > configuration, security and quota changes, to controller-only by always
> > > > routing these alterations to the controller.
> > > >
> > > > Let me know your thoughts!
> > > >
> > > > Best,
> > > > Boyang
> > > >
> > >
> >
>


Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-17 Thread Colin McCabe
Hi David & Boyang,

I thought the intention of the "old-client-connections-count" metric was to 
give some information about how many redirections were going on in the cluster. 
 This is different from the "unknown" software version metric.  After all, many 
versions that support KIP-511 will still need redirection.

I think the problem with the metric is that it assumes an entire connection is 
used only for old client redirection.  This won't be the case-- we'll use those 
connections for other RPCs to the controller as well.  So I'd suggest a metric 
more like num-messages-redirected or something like that which operates at the 
message level, and can be monitored on each broker.

best,
Colin

On Thu, Apr 16, 2020, at 07:40, David Jacot wrote:
> Hi Boyang,
> 
> Thanks for the KIP. Overall, it looks good to me. I really like the
> envelope RPC!
> 
> One minor comment regarding the `old-client-connections-count` metric. Is
> it really necessary? The number of connected clients whose version is not
> known (prior to KIP-511) is already reported but with an "unknown" software
> name and an "unknown" software version, which is, I suppose, similar to what
> you intend to expose with this new metric, isn't it?
> 
> Regards,
> David
> 
> On Thu, Apr 16, 2020 at 7:24 AM Boyang Chen 
> wrote:
> 
> > Thanks Raymond and Colin for the detailed discussions! I totally agree
> > with the rational here. The new `Envelope` RPC has been added to the KIP
> > and the forwarding section logic has been revised, feel free to take
> > another look.
> >
> > On Wed, Apr 15, 2020 at 5:19 PM Colin McCabe  wrote:
> >
> > > Hi Boyang,
> > >
> > > I agree that we need a version bump on the request types we are going to
> > > forward.  The new versions will be able to return the NOT_CONTROLLER
> > error,
> > > and let the client do the retrying, which is what we typically prefer.
> > > The  existing versions can't ever return NOT_CONTROLLER.
> > >
> > > Since we have to have a new version for all these requests, we could
> > > technically do everything with just optional fields, like we originally
> > > discussed.  However, there is probably some value in having a real
> > > EnvelopeRequest (or whatever) that makes it clearer what is going on.
> > > Optional fields don't come with "guard rails" to prevent us from
> > > accidentally ignoring them on older versions of the broker.  A new ApiKey
> > > certainly does.
> > >
> > > Another issue is that it's nice to avoid changing the version of the
> > > request when forwarding it.  Sometimes different versions have slightly
> > > different semantics, and it simplifies things to avoid worrying about
> > that.
> > >
> > > We should restrict the use of forwarding to just principals that have
> > > CLUSTERACTION on CLUSTER for now, so that only the brokers and superusers
> > > can do it.
> > >
> > > best,
> > > Colin
> > >
> > >
> > > On Tue, Apr 14, 2020, at 13:15, Boyang Chen wrote:
> > > > Thanks Raymond for the proposal! Yea, adding a unified forwarding
> > > envelope
> > > > request is a good idea, but it doesn't really solve our problem in this
> > > KIP.
> > > >
> > > > On Mon, Apr 13, 2020 at 11:14 PM Raymond Ng  wrote:
> > > >
> > > > > Hi Boyang,
> > > > >
> > > > > Thanks for the KIP. Overall looks great.
> > > > >
> > > > > One suggestion: instead of bumping the version and adding an optional
> > > field
> > > > > (PrincipalName) for a number of requests, can we consider adding a
> > > general
> > > > > ProxyRequest that acts as an "envelope" for the forwarded requests?
> > > > >
> > > > > A few advantages to this approach come to mind:
> > > > >
> > > > >1. Add one (new) Request API instead of modifying a number of them
> > > > >2. Make the forwarded nature of the request explicit instead of
> > > > >implicitly relying on an optional field and a specific version
> > that
> > > > > varies
> > > > >by type.
> > > > >3. This approach is flexible enough to be potentially useful
> > beyond
> > > the
> > > > >current use case (e.g. federated, inter-cluster scenarios)
> > > > >
> > > > > As a bonus, the combination of 1. and 2. should also simplify
> > > > > implementation & validation.
> > > > >
> > > > >
> > > > Firstly the broker has to differentiate old and new admin clients as it
> > > > should only support forwarding for old ones. Without a version bump,
> > > broker
> > > > couldn't differentiate both. Besides the bumping of the existing
> > > > protocol is not a big overhead comparing with adding a new RPC, so I
> > > don't
> > > > worry too much about the complexity here.
> > > >
> > > >
> > > > > On the other hand, it's not clear if the underlying RPC request
> > > > > encoding/decode machinery supports embedded requests. Hopefully, even
> > > if it
> > > > > doesn't it would not be too difficult to extend.
> > > > >
> > > >
> > > > Making the forwarding behavior more general is great, but could also
> > come
> > > > with problems we couldn't anticipate 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-16 Thread Boyang Chen
Thanks Jose for the questions!

On Thu, Apr 16, 2020 at 12:33 PM Jose Garcia Sancio 
wrote:

> Hi Boyang,
>
> Thanks for the KIP. The KIP looks good. I have a few questions and
> comments.
>
> > As part of the KIP-500
> <
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-500%3A+Replace+ZooKeeper+with+a+Self-Managed+Metadata+Quorum
> >
> initiative, we need to build a bridge release version of Kafka that could
> isolate the direct Zookeeper write access only to the controller.
>
> This may be outside the scope of this KIP but I am trying to understand how
> this is going to be used to implement KIP-500. My understanding is that the
> Admin client discovers the controller by performing a
> Metadata{Request,Response} round trip. The response of this request
> includes the id of the controller. Based on my understanding of the
> KIP-500, this architecture will need to change. For example the controller
> will not necessarily be a broker in which case the id may not correlate to
> a broker id. Is the expectation that the Kafka Controller Quorum (as
> defined in KIP-500) will push this new connection information to all of the
> brokers? Will the Kafka Controller Quorum expose and implement all of the
> RPCs being redirected in this KIP and the ones that are currently routed to
> the controller? These include:
>
> ListPartitionReassignment
> AlterPartitionReassignment
> ElectLeaders
> CreatePartitions
> DeleteTopics
> CreateTopics
>
>
According to the draft Raft quorum KIP
,
we are going to reuse the broker.id as the id for each broker participating
inside the
metadata quorum, either it is a voter or observer. So the controller.id
embedded inside metadata response should still be valid for controller
communication IIUC. As for Kafka Controller Quorum expose and implement all
of the RPCs I think I also need a bit more context from you, as I think the
controller logic is agnostic to the metadata log replication, which means
the handling for these mutation RPCs shall be the same as the existing
implementations for ZK based controller, not sure if this answers your
question?


> > AUTHORIZATION_FAILED if the inter-broker verification failed.
>
> The rest of the document mentions CLUSTER_AUTHORIZATION_FAILED.
>
> > For CLUSTER_AUTHORIZATION_FAILED, this indicates an internal error for
> broker security setup which has nothing to do with the client, so we have
> no other way but returning an UNKNOWN_SERVER_ERROR to the admin client.
>
> I don't understand this. I think I don't understand this because it is not
> clear to me who, how and when authorization is going to work when using
> Envelopre{Request,Response}. Can you please add a section that explains how
> authorization works when envelopes are involved?
>

Addressed the inconsistent error code. The authorization for
EnvelopeRequest shall be the same as other requests which require CLUSTER
permission, so break down as:
1. Who: the controller broker
2. How: the same as verifying other RPCs requiring CLUSTER permission, for
example LeaderAndIsrRequest
3. When: during the EnvelopeRequest processing

Will add more details for that context.


> --
> -Jose
>


Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-16 Thread Jose Garcia Sancio
Hi Boyang,

Thanks for the KIP. The KIP looks good. I have a few questions and comments.

> As part of the KIP-500

initiative, we need to build a bridge release version of Kafka that could
isolate the direct Zookeeper write access only to the controller.

This may be outside the scope of this KIP but I am trying to understand how
this is going to be used to implement KIP-500. My understanding is that the
Admin client discovers the controller by performing a
Metadata{Request,Response} round trip. The response of this request
includes the id of the controller. Based on my understanding of the
KIP-500, this architecture will need to change. For example the controller
will not necessarily be a broker in which case the id may not correlate to
a broker id. Is the expectation that the Kafka Controller Quorum (as
defined in KIP-500) will push this new connection information to all of the
brokers? Will the Kafka Controller Quorum expose and implement all of the
RPCs being redirected in this KIP and the ones that are currently routed to
the controller? These include:

ListPartitionReassignment
AlterPartitionReassignment
ElectLeaders
CreatePartitions
DeleteTopics
CreateTopics

> AUTHORIZATION_FAILED if the inter-broker verification failed.

The rest of the document mentions CLUSTER_AUTHORIZATION_FAILED.

> For CLUSTER_AUTHORIZATION_FAILED, this indicates an internal error for
broker security setup which has nothing to do with the client, so we have
no other way but returning an UNKNOWN_SERVER_ERROR to the admin client.

I don't understand this. I think I don't understand this because it is not
clear to me who, how and when authorization is going to work when using
Envelopre{Request,Response}. Can you please add a section that explains how
authorization works when envelopes are involved?

-- 
-Jose


Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-16 Thread Boyang Chen
Hey Ismael,

my understanding is that we are inevitably making the controller single
point, even when the metadata quorum work is done. Redirecting won't make
things easier as the controller will still be the role to perform
alterations.

Just for the sake of argument, DescribeTopic is not migrated as it is read
access. Even I don't have concrete numbers to justify any production use
case for Kafka, mostly people do metadata lookup more frequent than
mutation if you agree. Also I don't fully understand why the load of
AlterISR is necessary better than admin requests, which could also be
harming the cluster availability when there are growing URPs?

The performance is definitely a good concern for all these single-point
access changes, we would definitely build benchmarks in KIP-500 scope to
understand any potential regression, which may go beyond the scope of the
current KIP and build a separate interesting topic to discuss.

Boyang

On Thu, Apr 16, 2020 at 8:52 AM Ismael Juma  wrote:

> I don't think these requests are necessarily infrequent under multi tenant
> environments though. I've seen Controller availability being an issue for
> describe topics for example (before it was changed to go to any broker).
>
> Would it be better to redirect once the controller quorum is there?
>
> Note that this is different from things like AlterIsr since these calls are
> coming from clients versus other brokers.
>
> Ismael
>
> On Wed, Apr 15, 2020, 5:10 PM Colin McCabe  wrote:
>
> > Hi Ismael,
> >
> > I agree that sending these requests through the controller will not work
> > during the periods when there is no controller.  However, those periods
> > should be short-- otherwise we have bigger problems in the cluster.
> >
> > These requests are very infrequent because they are administrative
> > operations.  Basically the affected operations are changing ACLs,
> changing
> > dynamic configurations, and changing quotas.
> >
> > best,
> > Colin
> >
> >
> > On Wed, Apr 15, 2020, at 15:25, Ismael Juma wrote:
> > > Hi Boyang,
> > >
> > > Thanks for the KIP. Have we considered that this reduces availability
> for
> > > these operations since we have a single Controller instead of the ZK
> > quorum?
> > >
> > > Ismael
> > >
> > > On Fri, Apr 3, 2020 at 4:45 PM Boyang Chen  >
> > > wrote:
> > >
> > > > Hey all,
> > > >
> > > > I would like to start off the discussion for KIP-590, a follow-up
> > > > initiative after KIP-500:
> > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-590%3A+Redirect+Zookeeper+Mutation+Protocols+to+The+Controller
> > > >
> > > > This KIP proposes to migrate existing Zookeeper mutation paths,
> > including
> > > > configuration, security and quota changes, to controller-only by
> always
> > > > routing these alterations to the controller.
> > > >
> > > > Let me know your thoughts!
> > > >
> > > > Best,
> > > > Boyang
> > > >
> > >
> >
>


Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-16 Thread Ismael Juma
I don't think these requests are necessarily infrequent under multi tenant
environments though. I've seen Controller availability being an issue for
describe topics for example (before it was changed to go to any broker).

Would it be better to redirect once the controller quorum is there?

Note that this is different from things like AlterIsr since these calls are
coming from clients versus other brokers.

Ismael

On Wed, Apr 15, 2020, 5:10 PM Colin McCabe  wrote:

> Hi Ismael,
>
> I agree that sending these requests through the controller will not work
> during the periods when there is no controller.  However, those periods
> should be short-- otherwise we have bigger problems in the cluster.
>
> These requests are very infrequent because they are administrative
> operations.  Basically the affected operations are changing ACLs, changing
> dynamic configurations, and changing quotas.
>
> best,
> Colin
>
>
> On Wed, Apr 15, 2020, at 15:25, Ismael Juma wrote:
> > Hi Boyang,
> >
> > Thanks for the KIP. Have we considered that this reduces availability for
> > these operations since we have a single Controller instead of the ZK
> quorum?
> >
> > Ismael
> >
> > On Fri, Apr 3, 2020 at 4:45 PM Boyang Chen 
> > wrote:
> >
> > > Hey all,
> > >
> > > I would like to start off the discussion for KIP-590, a follow-up
> > > initiative after KIP-500:
> > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-590%3A+Redirect+Zookeeper+Mutation+Protocols+to+The+Controller
> > >
> > > This KIP proposes to migrate existing Zookeeper mutation paths,
> including
> > > configuration, security and quota changes, to controller-only by always
> > > routing these alterations to the controller.
> > >
> > > Let me know your thoughts!
> > >
> > > Best,
> > > Boyang
> > >
> >
>


Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-16 Thread Boyang Chen
Thanks David for the review! Great suggestion, addressed in the KIP.

On Thu, Apr 16, 2020 at 7:40 AM David Jacot  wrote:

> Hi Boyang,
>
> Thanks for the KIP. Overall, it looks good to me. I really like the
> envelope RPC!
>
> One minor comment regarding the `old-client-connections-count` metric. Is
> it really necessary? The number of connected clients whose version is not
> known (prior to KIP-511) is already reported but with an "unknown" software
> name and an "unknown" software version, which is, I suppose, similar to
> what
> you intend to expose with this new metric, isn't it?
>
> Regards,
> David
>
> On Thu, Apr 16, 2020 at 7:24 AM Boyang Chen 
> wrote:
>
> > Thanks Raymond and Colin for the detailed discussions! I totally agree
> > with the rational here. The new `Envelope` RPC has been added to the KIP
> > and the forwarding section logic has been revised, feel free to take
> > another look.
> >
> > On Wed, Apr 15, 2020 at 5:19 PM Colin McCabe  wrote:
> >
> > > Hi Boyang,
> > >
> > > I agree that we need a version bump on the request types we are going
> to
> > > forward.  The new versions will be able to return the NOT_CONTROLLER
> > error,
> > > and let the client do the retrying, which is what we typically prefer.
> > > The  existing versions can't ever return NOT_CONTROLLER.
> > >
> > > Since we have to have a new version for all these requests, we could
> > > technically do everything with just optional fields, like we originally
> > > discussed.  However, there is probably some value in having a real
> > > EnvelopeRequest (or whatever) that makes it clearer what is going on.
> > > Optional fields don't come with "guard rails" to prevent us from
> > > accidentally ignoring them on older versions of the broker.  A new
> ApiKey
> > > certainly does.
> > >
> > > Another issue is that it's nice to avoid changing the version of the
> > > request when forwarding it.  Sometimes different versions have slightly
> > > different semantics, and it simplifies things to avoid worrying about
> > that.
> > >
> > > We should restrict the use of forwarding to just principals that have
> > > CLUSTERACTION on CLUSTER for now, so that only the brokers and
> superusers
> > > can do it.
> > >
> > > best,
> > > Colin
> > >
> > >
> > > On Tue, Apr 14, 2020, at 13:15, Boyang Chen wrote:
> > > > Thanks Raymond for the proposal! Yea, adding a unified forwarding
> > > envelope
> > > > request is a good idea, but it doesn't really solve our problem in
> this
> > > KIP.
> > > >
> > > > On Mon, Apr 13, 2020 at 11:14 PM Raymond Ng 
> wrote:
> > > >
> > > > > Hi Boyang,
> > > > >
> > > > > Thanks for the KIP. Overall looks great.
> > > > >
> > > > > One suggestion: instead of bumping the version and adding an
> optional
> > > field
> > > > > (PrincipalName) for a number of requests, can we consider adding a
> > > general
> > > > > ProxyRequest that acts as an "envelope" for the forwarded requests?
> > > > >
> > > > > A few advantages to this approach come to mind:
> > > > >
> > > > >1. Add one (new) Request API instead of modifying a number of
> them
> > > > >2. Make the forwarded nature of the request explicit instead of
> > > > >implicitly relying on an optional field and a specific version
> > that
> > > > > varies
> > > > >by type.
> > > > >3. This approach is flexible enough to be potentially useful
> > beyond
> > > the
> > > > >current use case (e.g. federated, inter-cluster scenarios)
> > > > >
> > > > > As a bonus, the combination of 1. and 2. should also simplify
> > > > > implementation & validation.
> > > > >
> > > > >
> > > > Firstly the broker has to differentiate old and new admin clients as
> it
> > > > should only support forwarding for old ones. Without a version bump,
> > > broker
> > > > couldn't differentiate both. Besides the bumping of the existing
> > > > protocol is not a big overhead comparing with adding a new RPC, so I
> > > don't
> > > > worry too much about the complexity here.
> > > >
> > > >
> > > > > On the other hand, it's not clear if the underlying RPC request
> > > > > encoding/decode machinery supports embedded requests. Hopefully,
> even
> > > if it
> > > > > doesn't it would not be too difficult to extend.
> > > > >
> > > >
> > > > Making the forwarding behavior more general is great, but could also
> > come
> > > > with problems we couldn't anticipate such as usage abusiveness, more
> > > > changes to auto generation framework and increased metadata overhead.
> > At
> > > > the moment, we don't expect the direct forwarding would be a
> > bottleneck,
> > > so
> > > > I'm more inclined to make this proposal as simple as possible for
> now.
> > If
> > > > we do have a strong need in the future, getting the ProxyRequest is
> > > > definitely worth the effort.
> > > >
> > > >
> > > > > What do you think?
> > > > >
> > > > > Regards,
> > > > > Ray
> > > > >
> > > > >
> > > > > On Wed, Apr 8, 2020 at 4:36 PM Boyang Chen <
> > reluctanthero...@gmail.com
> > > >
> > > 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-16 Thread David Jacot
Hi Boyang,

Thanks for the KIP. Overall, it looks good to me. I really like the
envelope RPC!

One minor comment regarding the `old-client-connections-count` metric. Is
it really necessary? The number of connected clients whose version is not
known (prior to KIP-511) is already reported but with an "unknown" software
name and an "unknown" software version, which is, I suppose, similar to what
you intend to expose with this new metric, isn't it?

Regards,
David

On Thu, Apr 16, 2020 at 7:24 AM Boyang Chen 
wrote:

> Thanks Raymond and Colin for the detailed discussions! I totally agree
> with the rational here. The new `Envelope` RPC has been added to the KIP
> and the forwarding section logic has been revised, feel free to take
> another look.
>
> On Wed, Apr 15, 2020 at 5:19 PM Colin McCabe  wrote:
>
> > Hi Boyang,
> >
> > I agree that we need a version bump on the request types we are going to
> > forward.  The new versions will be able to return the NOT_CONTROLLER
> error,
> > and let the client do the retrying, which is what we typically prefer.
> > The  existing versions can't ever return NOT_CONTROLLER.
> >
> > Since we have to have a new version for all these requests, we could
> > technically do everything with just optional fields, like we originally
> > discussed.  However, there is probably some value in having a real
> > EnvelopeRequest (or whatever) that makes it clearer what is going on.
> > Optional fields don't come with "guard rails" to prevent us from
> > accidentally ignoring them on older versions of the broker.  A new ApiKey
> > certainly does.
> >
> > Another issue is that it's nice to avoid changing the version of the
> > request when forwarding it.  Sometimes different versions have slightly
> > different semantics, and it simplifies things to avoid worrying about
> that.
> >
> > We should restrict the use of forwarding to just principals that have
> > CLUSTERACTION on CLUSTER for now, so that only the brokers and superusers
> > can do it.
> >
> > best,
> > Colin
> >
> >
> > On Tue, Apr 14, 2020, at 13:15, Boyang Chen wrote:
> > > Thanks Raymond for the proposal! Yea, adding a unified forwarding
> > envelope
> > > request is a good idea, but it doesn't really solve our problem in this
> > KIP.
> > >
> > > On Mon, Apr 13, 2020 at 11:14 PM Raymond Ng  wrote:
> > >
> > > > Hi Boyang,
> > > >
> > > > Thanks for the KIP. Overall looks great.
> > > >
> > > > One suggestion: instead of bumping the version and adding an optional
> > field
> > > > (PrincipalName) for a number of requests, can we consider adding a
> > general
> > > > ProxyRequest that acts as an "envelope" for the forwarded requests?
> > > >
> > > > A few advantages to this approach come to mind:
> > > >
> > > >1. Add one (new) Request API instead of modifying a number of them
> > > >2. Make the forwarded nature of the request explicit instead of
> > > >implicitly relying on an optional field and a specific version
> that
> > > > varies
> > > >by type.
> > > >3. This approach is flexible enough to be potentially useful
> beyond
> > the
> > > >current use case (e.g. federated, inter-cluster scenarios)
> > > >
> > > > As a bonus, the combination of 1. and 2. should also simplify
> > > > implementation & validation.
> > > >
> > > >
> > > Firstly the broker has to differentiate old and new admin clients as it
> > > should only support forwarding for old ones. Without a version bump,
> > broker
> > > couldn't differentiate both. Besides the bumping of the existing
> > > protocol is not a big overhead comparing with adding a new RPC, so I
> > don't
> > > worry too much about the complexity here.
> > >
> > >
> > > > On the other hand, it's not clear if the underlying RPC request
> > > > encoding/decode machinery supports embedded requests. Hopefully, even
> > if it
> > > > doesn't it would not be too difficult to extend.
> > > >
> > >
> > > Making the forwarding behavior more general is great, but could also
> come
> > > with problems we couldn't anticipate such as usage abusiveness, more
> > > changes to auto generation framework and increased metadata overhead.
> At
> > > the moment, we don't expect the direct forwarding would be a
> bottleneck,
> > so
> > > I'm more inclined to make this proposal as simple as possible for now.
> If
> > > we do have a strong need in the future, getting the ProxyRequest is
> > > definitely worth the effort.
> > >
> > >
> > > > What do you think?
> > > >
> > > > Regards,
> > > > Ray
> > > >
> > > >
> > > > On Wed, Apr 8, 2020 at 4:36 PM Boyang Chen <
> reluctanthero...@gmail.com
> > >
> > > > wrote:
> > > >
> > > > > Thanks for the info Agam! Will add to the KIP.
> > > > >
> > > > > On Wed, Apr 8, 2020 at 4:26 PM Agam Brahma 
> > wrote:
> > > > >
> > > > > > Hi Boyang,
> > > > > >
> > > > > > The KIP already talks about incorporating changes for
> > FindCoordinator
> > > > > > request routing, wanted to point out one additional case where
> > internal
> > > > > > topics are 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-15 Thread Boyang Chen
Thanks Raymond and Colin for the detailed discussions! I totally agree
with the rational here. The new `Envelope` RPC has been added to the KIP
and the forwarding section logic has been revised, feel free to take
another look.

On Wed, Apr 15, 2020 at 5:19 PM Colin McCabe  wrote:

> Hi Boyang,
>
> I agree that we need a version bump on the request types we are going to
> forward.  The new versions will be able to return the NOT_CONTROLLER error,
> and let the client do the retrying, which is what we typically prefer.
> The  existing versions can't ever return NOT_CONTROLLER.
>
> Since we have to have a new version for all these requests, we could
> technically do everything with just optional fields, like we originally
> discussed.  However, there is probably some value in having a real
> EnvelopeRequest (or whatever) that makes it clearer what is going on.
> Optional fields don't come with "guard rails" to prevent us from
> accidentally ignoring them on older versions of the broker.  A new ApiKey
> certainly does.
>
> Another issue is that it's nice to avoid changing the version of the
> request when forwarding it.  Sometimes different versions have slightly
> different semantics, and it simplifies things to avoid worrying about that.
>
> We should restrict the use of forwarding to just principals that have
> CLUSTERACTION on CLUSTER for now, so that only the brokers and superusers
> can do it.
>
> best,
> Colin
>
>
> On Tue, Apr 14, 2020, at 13:15, Boyang Chen wrote:
> > Thanks Raymond for the proposal! Yea, adding a unified forwarding
> envelope
> > request is a good idea, but it doesn't really solve our problem in this
> KIP.
> >
> > On Mon, Apr 13, 2020 at 11:14 PM Raymond Ng  wrote:
> >
> > > Hi Boyang,
> > >
> > > Thanks for the KIP. Overall looks great.
> > >
> > > One suggestion: instead of bumping the version and adding an optional
> field
> > > (PrincipalName) for a number of requests, can we consider adding a
> general
> > > ProxyRequest that acts as an "envelope" for the forwarded requests?
> > >
> > > A few advantages to this approach come to mind:
> > >
> > >1. Add one (new) Request API instead of modifying a number of them
> > >2. Make the forwarded nature of the request explicit instead of
> > >implicitly relying on an optional field and a specific version that
> > > varies
> > >by type.
> > >3. This approach is flexible enough to be potentially useful beyond
> the
> > >current use case (e.g. federated, inter-cluster scenarios)
> > >
> > > As a bonus, the combination of 1. and 2. should also simplify
> > > implementation & validation.
> > >
> > >
> > Firstly the broker has to differentiate old and new admin clients as it
> > should only support forwarding for old ones. Without a version bump,
> broker
> > couldn't differentiate both. Besides the bumping of the existing
> > protocol is not a big overhead comparing with adding a new RPC, so I
> don't
> > worry too much about the complexity here.
> >
> >
> > > On the other hand, it's not clear if the underlying RPC request
> > > encoding/decode machinery supports embedded requests. Hopefully, even
> if it
> > > doesn't it would not be too difficult to extend.
> > >
> >
> > Making the forwarding behavior more general is great, but could also come
> > with problems we couldn't anticipate such as usage abusiveness, more
> > changes to auto generation framework and increased metadata overhead. At
> > the moment, we don't expect the direct forwarding would be a bottleneck,
> so
> > I'm more inclined to make this proposal as simple as possible for now. If
> > we do have a strong need in the future, getting the ProxyRequest is
> > definitely worth the effort.
> >
> >
> > > What do you think?
> > >
> > > Regards,
> > > Ray
> > >
> > >
> > > On Wed, Apr 8, 2020 at 4:36 PM Boyang Chen  >
> > > wrote:
> > >
> > > > Thanks for the info Agam! Will add to the KIP.
> > > >
> > > > On Wed, Apr 8, 2020 at 4:26 PM Agam Brahma 
> wrote:
> > > >
> > > > > Hi Boyang,
> > > > >
> > > > > The KIP already talks about incorporating changes for
> FindCoordinator
> > > > > request routing, wanted to point out one additional case where
> internal
> > > > > topics are created "as a side effect":
> > > > >
> > > > > As part of handling metadata requests, if we are looking for
> metadata
> > > for
> > > > > an internal topic and auto-topic-creation is enabled [1], the
> broker
> > > > > currently goes ahead and creates the internal topic in the same
> way [2]
> > > > as
> > > > > it would for the FindCoordinator request.
> > > > >
> > > > > -Agam
> > > > >
> > > > > [1]
> > > > >
> > > > >
> > > >
> > >
> https://github.com/apache/kafka/blob/cd1e46c8bb46f1e5303c51f476c74e33b522fce8/core/src/main/scala/kafka/server/KafkaApis.scala#L1096
> > > > > [2]
> > > > >
> > > > >
> > > >
> > >
> https://github.com/apache/kafka/blob/cd1e46c8bb46f1e5303c51f476c74e33b522fce8/core/src/main/scala/kafka/server/KafkaApis.scala#L1041
> > > > >
> > > > >
> > > > >
> > 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-15 Thread Colin McCabe
Hi Boyang,

I agree that we need a version bump on the request types we are going to 
forward.  The new versions will be able to return the NOT_CONTROLLER error, and 
let the client do the retrying, which is what we typically prefer.  The  
existing versions can't ever return NOT_CONTROLLER.

Since we have to have a new version for all these requests, we could 
technically do everything with just optional fields, like we originally 
discussed.  However, there is probably some value in having a real 
EnvelopeRequest (or whatever) that makes it clearer what is going on.  Optional 
fields don't come with "guard rails" to prevent us from accidentally ignoring 
them on older versions of the broker.  A new ApiKey certainly does.

Another issue is that it's nice to avoid changing the version of the request 
when forwarding it.  Sometimes different versions have slightly different 
semantics, and it simplifies things to avoid worrying about that.

We should restrict the use of forwarding to just principals that have 
CLUSTERACTION on CLUSTER for now, so that only the brokers and superusers can 
do it.

best,
Colin


On Tue, Apr 14, 2020, at 13:15, Boyang Chen wrote:
> Thanks Raymond for the proposal! Yea, adding a unified forwarding envelope
> request is a good idea, but it doesn't really solve our problem in this KIP.
> 
> On Mon, Apr 13, 2020 at 11:14 PM Raymond Ng  wrote:
> 
> > Hi Boyang,
> >
> > Thanks for the KIP. Overall looks great.
> >
> > One suggestion: instead of bumping the version and adding an optional field
> > (PrincipalName) for a number of requests, can we consider adding a general
> > ProxyRequest that acts as an "envelope" for the forwarded requests?
> >
> > A few advantages to this approach come to mind:
> >
> >1. Add one (new) Request API instead of modifying a number of them
> >2. Make the forwarded nature of the request explicit instead of
> >implicitly relying on an optional field and a specific version that
> > varies
> >by type.
> >3. This approach is flexible enough to be potentially useful beyond the
> >current use case (e.g. federated, inter-cluster scenarios)
> >
> > As a bonus, the combination of 1. and 2. should also simplify
> > implementation & validation.
> >
> >
> Firstly the broker has to differentiate old and new admin clients as it
> should only support forwarding for old ones. Without a version bump, broker
> couldn't differentiate both. Besides the bumping of the existing
> protocol is not a big overhead comparing with adding a new RPC, so I don't
> worry too much about the complexity here.
> 
> 
> > On the other hand, it's not clear if the underlying RPC request
> > encoding/decode machinery supports embedded requests. Hopefully, even if it
> > doesn't it would not be too difficult to extend.
> >
> 
> Making the forwarding behavior more general is great, but could also come
> with problems we couldn't anticipate such as usage abusiveness, more
> changes to auto generation framework and increased metadata overhead. At
> the moment, we don't expect the direct forwarding would be a bottleneck, so
> I'm more inclined to make this proposal as simple as possible for now. If
> we do have a strong need in the future, getting the ProxyRequest is
> definitely worth the effort.
> 
> 
> > What do you think?
> >
> > Regards,
> > Ray
> >
> >
> > On Wed, Apr 8, 2020 at 4:36 PM Boyang Chen 
> > wrote:
> >
> > > Thanks for the info Agam! Will add to the KIP.
> > >
> > > On Wed, Apr 8, 2020 at 4:26 PM Agam Brahma  wrote:
> > >
> > > > Hi Boyang,
> > > >
> > > > The KIP already talks about incorporating changes for FindCoordinator
> > > > request routing, wanted to point out one additional case where internal
> > > > topics are created "as a side effect":
> > > >
> > > > As part of handling metadata requests, if we are looking for metadata
> > for
> > > > an internal topic and auto-topic-creation is enabled [1], the broker
> > > > currently goes ahead and creates the internal topic in the same way [2]
> > > as
> > > > it would for the FindCoordinator request.
> > > >
> > > > -Agam
> > > >
> > > > [1]
> > > >
> > > >
> > >
> > https://github.com/apache/kafka/blob/cd1e46c8bb46f1e5303c51f476c74e33b522fce8/core/src/main/scala/kafka/server/KafkaApis.scala#L1096
> > > > [2]
> > > >
> > > >
> > >
> > https://github.com/apache/kafka/blob/cd1e46c8bb46f1e5303c51f476c74e33b522fce8/core/src/main/scala/kafka/server/KafkaApis.scala#L1041
> > > >
> > > >
> > > >
> > > > On Mon, Apr 6, 2020 at 8:25 PM Boyang Chen  > >
> > > > wrote:
> > > >
> > > > > Thanks for the various inputs everyone!
> > > > >
> > > > > I think Sonke and Colin's suggestions make sense. The tagged field
> > also
> > > > > avoids the unnecessary protocol changes for affected requests. Will
> > add
> > > > it
> > > > > to the header. As for the verification, I'm not sure whether it's
> > > > necessary
> > > > > to require a higher permission level, as it is just an ignorable
> > field?
> > > > >
> > > > > 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-15 Thread Colin McCabe
Hi Ismael,

I agree that sending these requests through the controller will not work during 
the periods when there is no controller.  However, those periods should be 
short-- otherwise we have bigger problems in the cluster.

These requests are very infrequent because they are administrative operations.  
Basically the affected operations are changing ACLs, changing dynamic 
configurations, and changing quotas.

best,
Colin


On Wed, Apr 15, 2020, at 15:25, Ismael Juma wrote:
> Hi Boyang,
> 
> Thanks for the KIP. Have we considered that this reduces availability for
> these operations since we have a single Controller instead of the ZK quorum?
> 
> Ismael
> 
> On Fri, Apr 3, 2020 at 4:45 PM Boyang Chen 
> wrote:
> 
> > Hey all,
> >
> > I would like to start off the discussion for KIP-590, a follow-up
> > initiative after KIP-500:
> >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-590%3A+Redirect+Zookeeper+Mutation+Protocols+to+The+Controller
> >
> > This KIP proposes to migrate existing Zookeeper mutation paths, including
> > configuration, security and quota changes, to controller-only by always
> > routing these alterations to the controller.
> >
> > Let me know your thoughts!
> >
> > Best,
> > Boyang
> >
>


Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-15 Thread Ismael Juma
Hi Boyang,

Thanks for the KIP. Have we considered that this reduces availability for
these operations since we have a single Controller instead of the ZK quorum?

Ismael

On Fri, Apr 3, 2020 at 4:45 PM Boyang Chen 
wrote:

> Hey all,
>
> I would like to start off the discussion for KIP-590, a follow-up
> initiative after KIP-500:
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-590%3A+Redirect+Zookeeper+Mutation+Protocols+to+The+Controller
>
> This KIP proposes to migrate existing Zookeeper mutation paths, including
> configuration, security and quota changes, to controller-only by always
> routing these alterations to the controller.
>
> Let me know your thoughts!
>
> Best,
> Boyang
>


Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-14 Thread Boyang Chen
Thanks Raymond for the proposal! Yea, adding a unified forwarding envelope
request is a good idea, but it doesn't really solve our problem in this KIP.

On Mon, Apr 13, 2020 at 11:14 PM Raymond Ng  wrote:

> Hi Boyang,
>
> Thanks for the KIP. Overall looks great.
>
> One suggestion: instead of bumping the version and adding an optional field
> (PrincipalName) for a number of requests, can we consider adding a general
> ProxyRequest that acts as an "envelope" for the forwarded requests?
>
> A few advantages to this approach come to mind:
>
>1. Add one (new) Request API instead of modifying a number of them
>2. Make the forwarded nature of the request explicit instead of
>implicitly relying on an optional field and a specific version that
> varies
>by type.
>3. This approach is flexible enough to be potentially useful beyond the
>current use case (e.g. federated, inter-cluster scenarios)
>
> As a bonus, the combination of 1. and 2. should also simplify
> implementation & validation.
>
>
Firstly the broker has to differentiate old and new admin clients as it
should only support forwarding for old ones. Without a version bump, broker
couldn't differentiate both. Besides the bumping of the existing
protocol is not a big overhead comparing with adding a new RPC, so I don't
worry too much about the complexity here.


> On the other hand, it's not clear if the underlying RPC request
> encoding/decode machinery supports embedded requests. Hopefully, even if it
> doesn't it would not be too difficult to extend.
>

Making the forwarding behavior more general is great, but could also come
with problems we couldn't anticipate such as usage abusiveness, more
changes to auto generation framework and increased metadata overhead. At
the moment, we don't expect the direct forwarding would be a bottleneck, so
I'm more inclined to make this proposal as simple as possible for now. If
we do have a strong need in the future, getting the ProxyRequest is
definitely worth the effort.


> What do you think?
>
> Regards,
> Ray
>
>
> On Wed, Apr 8, 2020 at 4:36 PM Boyang Chen 
> wrote:
>
> > Thanks for the info Agam! Will add to the KIP.
> >
> > On Wed, Apr 8, 2020 at 4:26 PM Agam Brahma  wrote:
> >
> > > Hi Boyang,
> > >
> > > The KIP already talks about incorporating changes for FindCoordinator
> > > request routing, wanted to point out one additional case where internal
> > > topics are created "as a side effect":
> > >
> > > As part of handling metadata requests, if we are looking for metadata
> for
> > > an internal topic and auto-topic-creation is enabled [1], the broker
> > > currently goes ahead and creates the internal topic in the same way [2]
> > as
> > > it would for the FindCoordinator request.
> > >
> > > -Agam
> > >
> > > [1]
> > >
> > >
> >
> https://github.com/apache/kafka/blob/cd1e46c8bb46f1e5303c51f476c74e33b522fce8/core/src/main/scala/kafka/server/KafkaApis.scala#L1096
> > > [2]
> > >
> > >
> >
> https://github.com/apache/kafka/blob/cd1e46c8bb46f1e5303c51f476c74e33b522fce8/core/src/main/scala/kafka/server/KafkaApis.scala#L1041
> > >
> > >
> > >
> > > On Mon, Apr 6, 2020 at 8:25 PM Boyang Chen  >
> > > wrote:
> > >
> > > > Thanks for the various inputs everyone!
> > > >
> > > > I think Sonke and Colin's suggestions make sense. The tagged field
> also
> > > > avoids the unnecessary protocol changes for affected requests. Will
> add
> > > it
> > > > to the header. As for the verification, I'm not sure whether it's
> > > necessary
> > > > to require a higher permission level, as it is just an ignorable
> field?
> > > >
> > > > Guozhang's suggestions about metrics also sound great, I will think
> > > through
> > > > the use cases and make some changes to the KIP.
> > > >
> > > > Best,
> > > > Boyang
> > > >
> > > > On Mon, Apr 6, 2020 at 4:28 PM Guozhang Wang 
> > wrote:
> > > >
> > > > > Thanks for the KIP Boyang, this looks good to me. Some minor
> > comments:
> > > > >
> > > > > 1) I think in order to implement the forwarding mechanism the
> brokers
> > > > needs
> > > > > some purgatory to keep the forwarded requests; if that's true,
> should
> > > we
> > > > > add some broker-side metrics for those purgatories for debugging
> > > > purposes?
> > > > >
> > > > > 2) Should we also consider adding some extra metric counting old
> > > > versioned
> > > > > admin client request rates (this goes beyond
> > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-511%3A+Collect+and+Expose+Client%27s+Name+and+Version+in+the+Brokers
> > > > > since
> > > > > old versioned client would not report its Kafka version anyways);
> one
> > > use
> > > > > case I can think of besides debugging purposes, is that if we ever
> > > > decides
> > > > > to break compatibility in future versions way after the bridge
> > > releases,
> > > > to
> > > > > reject any v1 requests and hence can totally remove this forwarding
> > > logic
> > > > > on brokers, we can leverage on this 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-14 Thread Raymond Ng
Hi Boyang,

Thanks for the KIP. Overall looks great.

One suggestion: instead of bumping the version and adding an optional field
(PrincipalName) for a number of requests, can we consider adding a general
ProxyRequest that acts as an "envelope" for the forwarded requests?

A few advantages to this approach come to mind:

   1. Add one (new) Request API instead of modifying a number of them
   2. Make the forwarded nature of the request explicit instead of
   implicitly relying on an optional field and a specific version that varies
   by type.
   3. This approach is flexible enough to be potentially useful beyond the
   current use case (e.g. federated, inter-cluster scenarios)

As a bonus, the combination of 1. and 2. should also simplify
implementation & validation.

On the other hand, it's not clear if the underlying RPC request
encoding/decode machinery supports embedded requests. Hopefully, even if it
doesn't it would not be too difficult to extend.

What do you think?

Regards,
Ray


On Wed, Apr 8, 2020 at 4:36 PM Boyang Chen 
wrote:

> Thanks for the info Agam! Will add to the KIP.
>
> On Wed, Apr 8, 2020 at 4:26 PM Agam Brahma  wrote:
>
> > Hi Boyang,
> >
> > The KIP already talks about incorporating changes for FindCoordinator
> > request routing, wanted to point out one additional case where internal
> > topics are created "as a side effect":
> >
> > As part of handling metadata requests, if we are looking for metadata for
> > an internal topic and auto-topic-creation is enabled [1], the broker
> > currently goes ahead and creates the internal topic in the same way [2]
> as
> > it would for the FindCoordinator request.
> >
> > -Agam
> >
> > [1]
> >
> >
> https://github.com/apache/kafka/blob/cd1e46c8bb46f1e5303c51f476c74e33b522fce8/core/src/main/scala/kafka/server/KafkaApis.scala#L1096
> > [2]
> >
> >
> https://github.com/apache/kafka/blob/cd1e46c8bb46f1e5303c51f476c74e33b522fce8/core/src/main/scala/kafka/server/KafkaApis.scala#L1041
> >
> >
> >
> > On Mon, Apr 6, 2020 at 8:25 PM Boyang Chen 
> > wrote:
> >
> > > Thanks for the various inputs everyone!
> > >
> > > I think Sonke and Colin's suggestions make sense. The tagged field also
> > > avoids the unnecessary protocol changes for affected requests. Will add
> > it
> > > to the header. As for the verification, I'm not sure whether it's
> > necessary
> > > to require a higher permission level, as it is just an ignorable field?
> > >
> > > Guozhang's suggestions about metrics also sound great, I will think
> > through
> > > the use cases and make some changes to the KIP.
> > >
> > > Best,
> > > Boyang
> > >
> > > On Mon, Apr 6, 2020 at 4:28 PM Guozhang Wang 
> wrote:
> > >
> > > > Thanks for the KIP Boyang, this looks good to me. Some minor
> comments:
> > > >
> > > > 1) I think in order to implement the forwarding mechanism the brokers
> > > needs
> > > > some purgatory to keep the forwarded requests; if that's true, should
> > we
> > > > add some broker-side metrics for those purgatories for debugging
> > > purposes?
> > > >
> > > > 2) Should we also consider adding some extra metric counting old
> > > versioned
> > > > admin client request rates (this goes beyond
> > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-511%3A+Collect+and+Expose+Client%27s+Name+and+Version+in+the+Brokers
> > > > since
> > > > old versioned client would not report its Kafka version anyways); one
> > use
> > > > case I can think of besides debugging purposes, is that if we ever
> > > decides
> > > > to break compatibility in future versions way after the bridge
> > releases,
> > > to
> > > > reject any v1 requests and hence can totally remove this forwarding
> > logic
> > > > on brokers, we can leverage on this metric to find a safe time to
> > > upgrade.
> > > >
> > > >
> > > > Guozhang
> > > >
> > > >
> > > >
> > > > On Mon, Apr 6, 2020 at 3:50 PM Colin McCabe 
> > wrote:
> > > >
> > > > > Hi Sönke,
> > > > >
> > > > > Yeah, that was my thought too.  The request has already been
> > validated
> > > on
> > > > > the forwarding broker, so we don't need to validate it again.
> > However,
> > > > you
> > > > > make a good point that it's unfortunate that the audit log would
> lose
> > > the
> > > > > principal information if we didn't forward it as well.
> > > > >
> > > > > Perhaps we could add a tagged field to the request header for all
> > > > > messages.  This field would contain the principal name.  Of course,
> > > this
> > > > > field should only be allowed if the request arrives with the
> highest
> > > > > permission levels (Probably ClusterAction on Cluster, since that's
> > what
> > > > all
> > > > > the brokers have.)
> > > > >
> > > > > regards,
> > > > > Colin
> > > > >
> > > > >
> > > > > On Mon, Apr 6, 2020, at 14:37, Sönke Liebau wrote:
> > > > > > Hi Boyang,
> > > > > >
> > > > > > thanks for the KIP. Sounds good overall.
> > > > > >
> > > > > > @Tom: I thought about your remark a little and think that in
> > > principle

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-08 Thread Boyang Chen
Thanks for the info Agam! Will add to the KIP.

On Wed, Apr 8, 2020 at 4:26 PM Agam Brahma  wrote:

> Hi Boyang,
>
> The KIP already talks about incorporating changes for FindCoordinator
> request routing, wanted to point out one additional case where internal
> topics are created "as a side effect":
>
> As part of handling metadata requests, if we are looking for metadata for
> an internal topic and auto-topic-creation is enabled [1], the broker
> currently goes ahead and creates the internal topic in the same way [2] as
> it would for the FindCoordinator request.
>
> -Agam
>
> [1]
>
> https://github.com/apache/kafka/blob/cd1e46c8bb46f1e5303c51f476c74e33b522fce8/core/src/main/scala/kafka/server/KafkaApis.scala#L1096
> [2]
>
> https://github.com/apache/kafka/blob/cd1e46c8bb46f1e5303c51f476c74e33b522fce8/core/src/main/scala/kafka/server/KafkaApis.scala#L1041
>
>
>
> On Mon, Apr 6, 2020 at 8:25 PM Boyang Chen 
> wrote:
>
> > Thanks for the various inputs everyone!
> >
> > I think Sonke and Colin's suggestions make sense. The tagged field also
> > avoids the unnecessary protocol changes for affected requests. Will add
> it
> > to the header. As for the verification, I'm not sure whether it's
> necessary
> > to require a higher permission level, as it is just an ignorable field?
> >
> > Guozhang's suggestions about metrics also sound great, I will think
> through
> > the use cases and make some changes to the KIP.
> >
> > Best,
> > Boyang
> >
> > On Mon, Apr 6, 2020 at 4:28 PM Guozhang Wang  wrote:
> >
> > > Thanks for the KIP Boyang, this looks good to me. Some minor comments:
> > >
> > > 1) I think in order to implement the forwarding mechanism the brokers
> > needs
> > > some purgatory to keep the forwarded requests; if that's true, should
> we
> > > add some broker-side metrics for those purgatories for debugging
> > purposes?
> > >
> > > 2) Should we also consider adding some extra metric counting old
> > versioned
> > > admin client request rates (this goes beyond
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-511%3A+Collect+and+Expose+Client%27s+Name+and+Version+in+the+Brokers
> > > since
> > > old versioned client would not report its Kafka version anyways); one
> use
> > > case I can think of besides debugging purposes, is that if we ever
> > decides
> > > to break compatibility in future versions way after the bridge
> releases,
> > to
> > > reject any v1 requests and hence can totally remove this forwarding
> logic
> > > on brokers, we can leverage on this metric to find a safe time to
> > upgrade.
> > >
> > >
> > > Guozhang
> > >
> > >
> > >
> > > On Mon, Apr 6, 2020 at 3:50 PM Colin McCabe 
> wrote:
> > >
> > > > Hi Sönke,
> > > >
> > > > Yeah, that was my thought too.  The request has already been
> validated
> > on
> > > > the forwarding broker, so we don't need to validate it again.
> However,
> > > you
> > > > make a good point that it's unfortunate that the audit log would lose
> > the
> > > > principal information if we didn't forward it as well.
> > > >
> > > > Perhaps we could add a tagged field to the request header for all
> > > > messages.  This field would contain the principal name.  Of course,
> > this
> > > > field should only be allowed if the request arrives with the highest
> > > > permission levels (Probably ClusterAction on Cluster, since that's
> what
> > > all
> > > > the brokers have.)
> > > >
> > > > regards,
> > > > Colin
> > > >
> > > >
> > > > On Mon, Apr 6, 2020, at 14:37, Sönke Liebau wrote:
> > > > > Hi Boyang,
> > > > >
> > > > > thanks for the KIP. Sounds good overall.
> > > > >
> > > > > @Tom: I thought about your remark a little and think that in
> > principle
> > > we
> > > > > can get away without forwarding the principal at all. Brokers
> > currently
> > > > > authenticate and authorize requests before performing writes to
> > > > Zookeeper -
> > > > > as long as we don't change that it shouldn't matter, whether the
> > write
> > > > goes
> > > > > to ZK or the controller, as long as that request is properly
> > > > authenticated.
> > > > > So the broker would simply authorize and authenticate the original
> > > > request
> > > > > and then forward it to the controller using its own credentials.
> And
> > > the
> > > > > controller could simply trust that this is a bona-fide request,
> > because
> > > > it
> > > > > came from a trusted peer.
> > > > >
> > > > > I can see two issues here, one is a bit academic I think..
> > > > >
> > > > > 1. The controller would be unable to write a proper audit log,
> > because
> > > it
> > > > > cannot know who sent the original request.
> > > > >
> > > > > 2. In theory, clusters could use Plaintext Listeners for inter
> broker
> > > > > traffic because that is on a separate, secure network or similar
> > > reasons.
> > > > > In that case, the forwarded request would be unauthenticated - then
> > > > again,
> > > > > so are all other requests between brokers, so nothing lost really.
> > > > >
> > > > 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-08 Thread Agam Brahma
Hi Boyang,

The KIP already talks about incorporating changes for FindCoordinator
request routing, wanted to point out one additional case where internal
topics are created "as a side effect":

As part of handling metadata requests, if we are looking for metadata for
an internal topic and auto-topic-creation is enabled [1], the broker
currently goes ahead and creates the internal topic in the same way [2] as
it would for the FindCoordinator request.

-Agam

[1]
https://github.com/apache/kafka/blob/cd1e46c8bb46f1e5303c51f476c74e33b522fce8/core/src/main/scala/kafka/server/KafkaApis.scala#L1096
[2]
https://github.com/apache/kafka/blob/cd1e46c8bb46f1e5303c51f476c74e33b522fce8/core/src/main/scala/kafka/server/KafkaApis.scala#L1041



On Mon, Apr 6, 2020 at 8:25 PM Boyang Chen 
wrote:

> Thanks for the various inputs everyone!
>
> I think Sonke and Colin's suggestions make sense. The tagged field also
> avoids the unnecessary protocol changes for affected requests. Will add it
> to the header. As for the verification, I'm not sure whether it's necessary
> to require a higher permission level, as it is just an ignorable field?
>
> Guozhang's suggestions about metrics also sound great, I will think through
> the use cases and make some changes to the KIP.
>
> Best,
> Boyang
>
> On Mon, Apr 6, 2020 at 4:28 PM Guozhang Wang  wrote:
>
> > Thanks for the KIP Boyang, this looks good to me. Some minor comments:
> >
> > 1) I think in order to implement the forwarding mechanism the brokers
> needs
> > some purgatory to keep the forwarded requests; if that's true, should we
> > add some broker-side metrics for those purgatories for debugging
> purposes?
> >
> > 2) Should we also consider adding some extra metric counting old
> versioned
> > admin client request rates (this goes beyond
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-511%3A+Collect+and+Expose+Client%27s+Name+and+Version+in+the+Brokers
> > since
> > old versioned client would not report its Kafka version anyways); one use
> > case I can think of besides debugging purposes, is that if we ever
> decides
> > to break compatibility in future versions way after the bridge releases,
> to
> > reject any v1 requests and hence can totally remove this forwarding logic
> > on brokers, we can leverage on this metric to find a safe time to
> upgrade.
> >
> >
> > Guozhang
> >
> >
> >
> > On Mon, Apr 6, 2020 at 3:50 PM Colin McCabe  wrote:
> >
> > > Hi Sönke,
> > >
> > > Yeah, that was my thought too.  The request has already been validated
> on
> > > the forwarding broker, so we don't need to validate it again.  However,
> > you
> > > make a good point that it's unfortunate that the audit log would lose
> the
> > > principal information if we didn't forward it as well.
> > >
> > > Perhaps we could add a tagged field to the request header for all
> > > messages.  This field would contain the principal name.  Of course,
> this
> > > field should only be allowed if the request arrives with the highest
> > > permission levels (Probably ClusterAction on Cluster, since that's what
> > all
> > > the brokers have.)
> > >
> > > regards,
> > > Colin
> > >
> > >
> > > On Mon, Apr 6, 2020, at 14:37, Sönke Liebau wrote:
> > > > Hi Boyang,
> > > >
> > > > thanks for the KIP. Sounds good overall.
> > > >
> > > > @Tom: I thought about your remark a little and think that in
> principle
> > we
> > > > can get away without forwarding the principal at all. Brokers
> currently
> > > > authenticate and authorize requests before performing writes to
> > > Zookeeper -
> > > > as long as we don't change that it shouldn't matter, whether the
> write
> > > goes
> > > > to ZK or the controller, as long as that request is properly
> > > authenticated.
> > > > So the broker would simply authorize and authenticate the original
> > > request
> > > > and then forward it to the controller using its own credentials. And
> > the
> > > > controller could simply trust that this is a bona-fide request,
> because
> > > it
> > > > came from a trusted peer.
> > > >
> > > > I can see two issues here, one is a bit academic I think..
> > > >
> > > > 1. The controller would be unable to write a proper audit log,
> because
> > it
> > > > cannot know who sent the original request.
> > > >
> > > > 2. In theory, clusters could use Plaintext Listeners for inter broker
> > > > traffic because that is on a separate, secure network or similar
> > reasons.
> > > > In that case, the forwarded request would be unauthenticated - then
> > > again,
> > > > so are all other requests between brokers, so nothing lost really.
> > > >
> > > > Overall though, I think that sending the principal along with the
> > request
> > > > shouldn't be a large issue though, it is just two Strings and a
> > boolean.
> > > > And the controller could bypass the PrincipalBuilder and just pass
> the
> > > > Principal that was built and sent by the remote broker straight to
> the
> > > > Authorizer. Since PrincipalBuilders are the same on 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-06 Thread Boyang Chen
Thanks for the various inputs everyone!

I think Sonke and Colin's suggestions make sense. The tagged field also
avoids the unnecessary protocol changes for affected requests. Will add it
to the header. As for the verification, I'm not sure whether it's necessary
to require a higher permission level, as it is just an ignorable field?

Guozhang's suggestions about metrics also sound great, I will think through
the use cases and make some changes to the KIP.

Best,
Boyang

On Mon, Apr 6, 2020 at 4:28 PM Guozhang Wang  wrote:

> Thanks for the KIP Boyang, this looks good to me. Some minor comments:
>
> 1) I think in order to implement the forwarding mechanism the brokers needs
> some purgatory to keep the forwarded requests; if that's true, should we
> add some broker-side metrics for those purgatories for debugging purposes?
>
> 2) Should we also consider adding some extra metric counting old versioned
> admin client request rates (this goes beyond
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-511%3A+Collect+and+Expose+Client%27s+Name+and+Version+in+the+Brokers
> since
> old versioned client would not report its Kafka version anyways); one use
> case I can think of besides debugging purposes, is that if we ever decides
> to break compatibility in future versions way after the bridge releases, to
> reject any v1 requests and hence can totally remove this forwarding logic
> on brokers, we can leverage on this metric to find a safe time to upgrade.
>
>
> Guozhang
>
>
>
> On Mon, Apr 6, 2020 at 3:50 PM Colin McCabe  wrote:
>
> > Hi Sönke,
> >
> > Yeah, that was my thought too.  The request has already been validated on
> > the forwarding broker, so we don't need to validate it again.  However,
> you
> > make a good point that it's unfortunate that the audit log would lose the
> > principal information if we didn't forward it as well.
> >
> > Perhaps we could add a tagged field to the request header for all
> > messages.  This field would contain the principal name.  Of course, this
> > field should only be allowed if the request arrives with the highest
> > permission levels (Probably ClusterAction on Cluster, since that's what
> all
> > the brokers have.)
> >
> > regards,
> > Colin
> >
> >
> > On Mon, Apr 6, 2020, at 14:37, Sönke Liebau wrote:
> > > Hi Boyang,
> > >
> > > thanks for the KIP. Sounds good overall.
> > >
> > > @Tom: I thought about your remark a little and think that in principle
> we
> > > can get away without forwarding the principal at all. Brokers currently
> > > authenticate and authorize requests before performing writes to
> > Zookeeper -
> > > as long as we don't change that it shouldn't matter, whether the write
> > goes
> > > to ZK or the controller, as long as that request is properly
> > authenticated.
> > > So the broker would simply authorize and authenticate the original
> > request
> > > and then forward it to the controller using its own credentials. And
> the
> > > controller could simply trust that this is a bona-fide request, because
> > it
> > > came from a trusted peer.
> > >
> > > I can see two issues here, one is a bit academic I think..
> > >
> > > 1. The controller would be unable to write a proper audit log, because
> it
> > > cannot know who sent the original request.
> > >
> > > 2. In theory, clusters could use Plaintext Listeners for inter broker
> > > traffic because that is on a separate, secure network or similar
> reasons.
> > > In that case, the forwarded request would be unauthenticated - then
> > again,
> > > so are all other requests between brokers, so nothing lost really.
> > >
> > > Overall though, I think that sending the principal along with the
> request
> > > shouldn't be a large issue though, it is just two Strings and a
> boolean.
> > > And the controller could bypass the PrincipalBuilder and just pass the
> > > Principal that was built and sent by the remote broker straight to the
> > > Authorizer. Since PrincipalBuilders are the same on all brokers it
> > > shouldn't matter who does the processing I think.
> > >
> > > Best regards,
> > > Sönke
> > >
> > >
> > > On Mon, 6 Apr 2020 at 22:30, Boyang Chen 
> > wrote:
> > >
> > > > Thanks Tom for the question! I'm not super familiar with the
> Principal
> > > > stuff, could you elaborate more on the two points you proposed here?
> > > >
> > > > I looked up Admin client and just take `createDelegationToken` API
> for
> > an
> > > > example, the request data encodes the principal information already,
> so
> > > > broker should also leverage that information to proxy the request
> IMHO.
> > > >
> > > > Boyang
> > > >
> > > > On Mon, Apr 6, 2020 at 9:21 AM Tom Bentley 
> > wrote:
> > > >
> > > > > Hi Boyang,
> > > > >
> > > > > Thanks for the KIP!
> > > > >
> > > > > When a broker proxies a request to the controller how does the
> > > > > authenticated principal get propagated? I think a couple of things
> > might
> > > > > complicate this:
> > > > >
> > > > > 1. A PrincipalBuilder might be in 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-06 Thread Guozhang Wang
Thanks for the KIP Boyang, this looks good to me. Some minor comments:

1) I think in order to implement the forwarding mechanism the brokers needs
some purgatory to keep the forwarded requests; if that's true, should we
add some broker-side metrics for those purgatories for debugging purposes?

2) Should we also consider adding some extra metric counting old versioned
admin client request rates (this goes beyond
https://cwiki.apache.org/confluence/display/KAFKA/KIP-511%3A+Collect+and+Expose+Client%27s+Name+and+Version+in+the+Brokers
since
old versioned client would not report its Kafka version anyways); one use
case I can think of besides debugging purposes, is that if we ever decides
to break compatibility in future versions way after the bridge releases, to
reject any v1 requests and hence can totally remove this forwarding logic
on brokers, we can leverage on this metric to find a safe time to upgrade.


Guozhang



On Mon, Apr 6, 2020 at 3:50 PM Colin McCabe  wrote:

> Hi Sönke,
>
> Yeah, that was my thought too.  The request has already been validated on
> the forwarding broker, so we don't need to validate it again.  However, you
> make a good point that it's unfortunate that the audit log would lose the
> principal information if we didn't forward it as well.
>
> Perhaps we could add a tagged field to the request header for all
> messages.  This field would contain the principal name.  Of course, this
> field should only be allowed if the request arrives with the highest
> permission levels (Probably ClusterAction on Cluster, since that's what all
> the brokers have.)
>
> regards,
> Colin
>
>
> On Mon, Apr 6, 2020, at 14:37, Sönke Liebau wrote:
> > Hi Boyang,
> >
> > thanks for the KIP. Sounds good overall.
> >
> > @Tom: I thought about your remark a little and think that in principle we
> > can get away without forwarding the principal at all. Brokers currently
> > authenticate and authorize requests before performing writes to
> Zookeeper -
> > as long as we don't change that it shouldn't matter, whether the write
> goes
> > to ZK or the controller, as long as that request is properly
> authenticated.
> > So the broker would simply authorize and authenticate the original
> request
> > and then forward it to the controller using its own credentials. And the
> > controller could simply trust that this is a bona-fide request, because
> it
> > came from a trusted peer.
> >
> > I can see two issues here, one is a bit academic I think..
> >
> > 1. The controller would be unable to write a proper audit log, because it
> > cannot know who sent the original request.
> >
> > 2. In theory, clusters could use Plaintext Listeners for inter broker
> > traffic because that is on a separate, secure network or similar reasons.
> > In that case, the forwarded request would be unauthenticated - then
> again,
> > so are all other requests between brokers, so nothing lost really.
> >
> > Overall though, I think that sending the principal along with the request
> > shouldn't be a large issue though, it is just two Strings and a boolean.
> > And the controller could bypass the PrincipalBuilder and just pass the
> > Principal that was built and sent by the remote broker straight to the
> > Authorizer. Since PrincipalBuilders are the same on all brokers it
> > shouldn't matter who does the processing I think.
> >
> > Best regards,
> > Sönke
> >
> >
> > On Mon, 6 Apr 2020 at 22:30, Boyang Chen 
> wrote:
> >
> > > Thanks Tom for the question! I'm not super familiar with the Principal
> > > stuff, could you elaborate more on the two points you proposed here?
> > >
> > > I looked up Admin client and just take `createDelegationToken` API for
> an
> > > example, the request data encodes the principal information already, so
> > > broker should also leverage that information to proxy the request IMHO.
> > >
> > > Boyang
> > >
> > > On Mon, Apr 6, 2020 at 9:21 AM Tom Bentley 
> wrote:
> > >
> > > > Hi Boyang,
> > > >
> > > > Thanks for the KIP!
> > > >
> > > > When a broker proxies a request to the controller how does the
> > > > authenticated principal get propagated? I think a couple of things
> might
> > > > complicate this:
> > > >
> > > > 1. A PrincipalBuilder might be in use,
> > > > 2. A Principal does not have to be serializable.
> > > >
> > > >
> > > > Kind regards,
> > > >
> > > > Tom
> > > >
> > > > On Sat, Apr 4, 2020 at 12:52 AM Boyang Chen <
> reluctanthero...@gmail.com>
> > > > wrote:
> > > >
> > > > > Hey all,
> > > > >
> > > > > I would like to start off the discussion for KIP-590, a follow-up
> > > > > initiative after KIP-500:
> > > > >
> > > > >
> > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-590%3A+Redirect+Zookeeper+Mutation+Protocols+to+The+Controller
> > > > >
> > > > > This KIP proposes to migrate existing Zookeeper mutation paths,
> > > including
> > > > > configuration, security and quota changes, to controller-only by
> always
> > > > > routing these alterations to the 

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-06 Thread Colin McCabe
Hi Sönke,

Yeah, that was my thought too.  The request has already been validated on the 
forwarding broker, so we don't need to validate it again.  However, you make a 
good point that it's unfortunate that the audit log would lose the principal 
information if we didn't forward it as well.

Perhaps we could add a tagged field to the request header for all messages.  
This field would contain the principal name.  Of course, this field should only 
be allowed if the request arrives with the highest permission levels (Probably 
ClusterAction on Cluster, since that's what all the brokers have.)

regards,
Colin


On Mon, Apr 6, 2020, at 14:37, Sönke Liebau wrote:
> Hi Boyang,
> 
> thanks for the KIP. Sounds good overall.
> 
> @Tom: I thought about your remark a little and think that in principle we
> can get away without forwarding the principal at all. Brokers currently
> authenticate and authorize requests before performing writes to Zookeeper -
> as long as we don't change that it shouldn't matter, whether the write goes
> to ZK or the controller, as long as that request is properly authenticated.
> So the broker would simply authorize and authenticate the original request
> and then forward it to the controller using its own credentials. And the
> controller could simply trust that this is a bona-fide request, because it
> came from a trusted peer.
> 
> I can see two issues here, one is a bit academic I think..
> 
> 1. The controller would be unable to write a proper audit log, because it
> cannot know who sent the original request.
> 
> 2. In theory, clusters could use Plaintext Listeners for inter broker
> traffic because that is on a separate, secure network or similar reasons.
> In that case, the forwarded request would be unauthenticated - then again,
> so are all other requests between brokers, so nothing lost really.
> 
> Overall though, I think that sending the principal along with the request
> shouldn't be a large issue though, it is just two Strings and a boolean.
> And the controller could bypass the PrincipalBuilder and just pass the
> Principal that was built and sent by the remote broker straight to the
> Authorizer. Since PrincipalBuilders are the same on all brokers it
> shouldn't matter who does the processing I think.
> 
> Best regards,
> Sönke
> 
> 
> On Mon, 6 Apr 2020 at 22:30, Boyang Chen  wrote:
> 
> > Thanks Tom for the question! I'm not super familiar with the Principal
> > stuff, could you elaborate more on the two points you proposed here?
> >
> > I looked up Admin client and just take `createDelegationToken` API for an
> > example, the request data encodes the principal information already, so
> > broker should also leverage that information to proxy the request IMHO.
> >
> > Boyang
> >
> > On Mon, Apr 6, 2020 at 9:21 AM Tom Bentley  wrote:
> >
> > > Hi Boyang,
> > >
> > > Thanks for the KIP!
> > >
> > > When a broker proxies a request to the controller how does the
> > > authenticated principal get propagated? I think a couple of things might
> > > complicate this:
> > >
> > > 1. A PrincipalBuilder might be in use,
> > > 2. A Principal does not have to be serializable.
> > >
> > >
> > > Kind regards,
> > >
> > > Tom
> > >
> > > On Sat, Apr 4, 2020 at 12:52 AM Boyang Chen 
> > > wrote:
> > >
> > > > Hey all,
> > > >
> > > > I would like to start off the discussion for KIP-590, a follow-up
> > > > initiative after KIP-500:
> > > >
> > > >
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-590%3A+Redirect+Zookeeper+Mutation+Protocols+to+The+Controller
> > > >
> > > > This KIP proposes to migrate existing Zookeeper mutation paths,
> > including
> > > > configuration, security and quota changes, to controller-only by always
> > > > routing these alterations to the controller.
> > > >
> > > > Let me know your thoughts!
> > > >
> > > > Best,
> > > > Boyang
> > > >
> > >
> >
> 
> 
> -- 
> Sönke Liebau
> Partner
> Tel. +49 179 7940878
> OpenCore GmbH & Co. KG - Thomas-Mann-Straße 8 - 22880 Wedel - Germany
>


Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-06 Thread Sönke Liebau
Hi Boyang,

thanks for the KIP. Sounds good overall.

@Tom: I thought about your remark a little and think that in principle we
can get away without forwarding the principal at all. Brokers currently
authenticate and authorize requests before performing writes to Zookeeper -
as long as we don't change that it shouldn't matter, whether the write goes
to ZK or the controller, as long as that request is properly authenticated.
So the broker would simply authorize and authenticate the original request
and then forward it to the controller using its own credentials. And the
controller could simply trust that this is a bona-fide request, because it
came from a trusted peer.

I can see two issues here, one is a bit academic I think..

1. The controller would be unable to write a proper audit log, because it
cannot know who sent the original request.

2. In theory, clusters could use Plaintext Listeners for inter broker
traffic because that is on a separate, secure network or similar reasons.
In that case, the forwarded request would be unauthenticated - then again,
so are all other requests between brokers, so nothing lost really.

Overall though, I think that sending the principal along with the request
shouldn't be a large issue though, it is just two Strings and a boolean.
And the controller could bypass the PrincipalBuilder and just pass the
Principal that was built and sent by the remote broker straight to the
Authorizer. Since PrincipalBuilders are the same on all brokers it
shouldn't matter who does the processing I think.

Best regards,
Sönke


On Mon, 6 Apr 2020 at 22:30, Boyang Chen  wrote:

> Thanks Tom for the question! I'm not super familiar with the Principal
> stuff, could you elaborate more on the two points you proposed here?
>
> I looked up Admin client and just take `createDelegationToken` API for an
> example, the request data encodes the principal information already, so
> broker should also leverage that information to proxy the request IMHO.
>
> Boyang
>
> On Mon, Apr 6, 2020 at 9:21 AM Tom Bentley  wrote:
>
> > Hi Boyang,
> >
> > Thanks for the KIP!
> >
> > When a broker proxies a request to the controller how does the
> > authenticated principal get propagated? I think a couple of things might
> > complicate this:
> >
> > 1. A PrincipalBuilder might be in use,
> > 2. A Principal does not have to be serializable.
> >
> >
> > Kind regards,
> >
> > Tom
> >
> > On Sat, Apr 4, 2020 at 12:52 AM Boyang Chen 
> > wrote:
> >
> > > Hey all,
> > >
> > > I would like to start off the discussion for KIP-590, a follow-up
> > > initiative after KIP-500:
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-590%3A+Redirect+Zookeeper+Mutation+Protocols+to+The+Controller
> > >
> > > This KIP proposes to migrate existing Zookeeper mutation paths,
> including
> > > configuration, security and quota changes, to controller-only by always
> > > routing these alterations to the controller.
> > >
> > > Let me know your thoughts!
> > >
> > > Best,
> > > Boyang
> > >
> >
>


-- 
Sönke Liebau
Partner
Tel. +49 179 7940878
OpenCore GmbH & Co. KG - Thomas-Mann-Straße 8 - 22880 Wedel - Germany


Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-06 Thread Boyang Chen
Thanks Tom for the question! I'm not super familiar with the Principal
stuff, could you elaborate more on the two points you proposed here?

I looked up Admin client and just take `createDelegationToken` API for an
example, the request data encodes the principal information already, so
broker should also leverage that information to proxy the request IMHO.

Boyang

On Mon, Apr 6, 2020 at 9:21 AM Tom Bentley  wrote:

> Hi Boyang,
>
> Thanks for the KIP!
>
> When a broker proxies a request to the controller how does the
> authenticated principal get propagated? I think a couple of things might
> complicate this:
>
> 1. A PrincipalBuilder might be in use,
> 2. A Principal does not have to be serializable.
>
>
> Kind regards,
>
> Tom
>
> On Sat, Apr 4, 2020 at 12:52 AM Boyang Chen 
> wrote:
>
> > Hey all,
> >
> > I would like to start off the discussion for KIP-590, a follow-up
> > initiative after KIP-500:
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-590%3A+Redirect+Zookeeper+Mutation+Protocols+to+The+Controller
> >
> > This KIP proposes to migrate existing Zookeeper mutation paths, including
> > configuration, security and quota changes, to controller-only by always
> > routing these alterations to the controller.
> >
> > Let me know your thoughts!
> >
> > Best,
> > Boyang
> >
>


Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-06 Thread Tom Bentley
Hi Boyang,

Thanks for the KIP!

When a broker proxies a request to the controller how does the
authenticated principal get propagated? I think a couple of things might
complicate this:

1. A PrincipalBuilder might be in use,
2. A Principal does not have to be serializable.


Kind regards,

Tom

On Sat, Apr 4, 2020 at 12:52 AM Boyang Chen 
wrote:

> Hey all,
>
> I would like to start off the discussion for KIP-590, a follow-up
> initiative after KIP-500:
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-590%3A+Redirect+Zookeeper+Mutation+Protocols+to+The+Controller
>
> This KIP proposes to migrate existing Zookeeper mutation paths, including
> configuration, security and quota changes, to controller-only by always
> routing these alterations to the controller.
>
> Let me know your thoughts!
>
> Best,
> Boyang
>