You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Ron Dagostino <rn...@gmail.com> on 2021/07/13 16:48:06 UTC

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

Hi everyone.  I know it has been 9 months since the last message appeared
on this vote thread, but a potential oversight exists in the implementation
of DefaultKafkaPrincipalBuilder.KafkaPrincipalSerde from
https://github.com/apache/kafka/pull/9103.  Specifically, there is no magic
number at the top of the wire format, and this causes obscure parsing
errors if incompatible principal implementations are mixed.  A magic number
at the top would allow deserialization code to provide an intuitive error
message.  The current implementation (without a magic number) was released
in 2.8, but it presumably has never been used since forwarding is disabled (
https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/Kafka.scala#L73).
We would like to change the implementation to include a magic number at the
top for the 3.0 release.  This would be a breaking change, but again this
presumably has never been used anywhere in production and would therefore
break nothing.

Note that forwarding is always enabled for KRaft-based clusters, but such
clusters are not supported in any production sense and there is no upgrade
path from a 2.8 KRaft cluster to a 3.0 KRaft cluster (from
config/kraft/README.md: "KRaft mode in Kafka 2.8 is provided for testing
only, NOT for production. We do not yet support upgrading existing
ZooKeeper-based Kafka clusters into this mode. In fact, when Kafka 3.0 is
released, it will not be possible to upgrade your KRaft clusters from 2.8
to 3.0").

A PR to add the magic number appears at
https://github.com/apache/kafka/pull/11038.

Please respond to this thread if you have any concerns or objections.

Thanks,

Ron

On Fri, Oct 9, 2020 at 1:21 PM Boyang Chen <re...@gmail.com>
wrote:

> Thanks Jason for the great thoughts, and we basically decided to shift the
> gear for a limited impersonation approach offline.
>
> The goal here is to simplify the handling logic by relying on the active
> controller to do the actual authorization for resources in the original
> client request. We are also adding the `KafkaPrincipalSerde` type to
> provide the functionality for principal serialization/deserialization so
> that it could embed in the Envelope and send to the active controller.
> Before 3.0, customized principal builders could optionally extend the serde
> type, which is required after 3.0 is released. Either way having the
> capability to serde KafkaPrincipal becomes a prerequisite to enable
> redirection besides IBP. Additionally, we add a forwardingPrincipal field
> to the Authorizer context for authorization and audit logging purposes,
> instead of going tagged fields in header.
>
> The KIP is updated to reflect the current approach, thanks.
>
>
>
> On Fri, Sep 25, 2020 at 5:55 PM Jason Gustafson <ja...@confluent.io>
> wrote:
>
> > Hey All,
> >
> > So the main thing the EnvelopeRequest gives us is a way to avoid
> converting
> > older API versions in order to attach the initial principal name and the
> > clientId. It also saves the need to add the initial principal and client
> id
> > as a tagged field to all of the forwarded protocols, which is nice. We
> > still have the challenge of advertising API versions which are compatible
> > with both the broker receiving the request and the controller that the
> > request is ultimately forwarded to, but not sure I see a way around that.
> >
> > I realize I might be walking into a minefield here, but since the
> envelope
> > is being revisited, it seems useful to compare the approach suggested
> above
> > with the option relying on impersonation. I favor the use of
> impersonation
> > because it makes forwarding simpler. As the proposal stands, we will have
> > to maintain logic for each forwarded API to unpack, authorize, and repack
> > any forwarded requests which flow through the broker. This is probably
> not
> > a huge concern from an efficiency perspective as long as we are talking
> > about just the Admin APIs, but it does have a big maintenance cost since
> > we'll need to ensure that every new field gets properly carried through.
> It
> > would be nice if we just didn't have to think about that. We also might
> > eventually come up with reasons to extend forwarding to non-admin APIs,
> so
> > it would be nice to start with an efficient approach.
> >
> > It seems like the main difference comes down to where the authorization
> is
> > done. Suppose that broker B receives an AlterConfig request from the
> client
> > in order to change topic configs and wants to forward to controller C.
> >
> > Option 1 (no impersonation): B authorizes AlterConfigs for the included
> > topics with the client principal. Rejected topics are stripped out of the
> > request.  Authorized topics are repackaged into a new request and sent in
> > an envelope to C, which verifies ClusterAction permission with the broker
> > principal and assumes authorization for the underlying request
> > Option 2 (with impersonation): B authenticates the client, but does no
> > authorization and forwards the request in an envelope to C containing the
> > authenticated principal. C checks ClusterAction for the envelope request
> > using the broker principal and AlterConfigs for the underlying request
> > using the forwarded client principal.
> >
> > In either case, broker B implicitly gets AlterConfigs permission for the
> > topic. This is true even without the envelope and seems like a reasonable
> > requirement. The broker should itself be authorized to perform any action
> > that it might have to forward requests for. As far as I know, all the
> > proposals we have considered require this. The main question from a
> > security perspective is whether any of these proposals require additional
> > unnecessary access, which is probably the main doubt about impersonation.
> > However, there are a couple ways we can restrict it:
> >
> > 1. We can restrict the principals that are allowed to be impersonated
> > 2. We can restrict the actions that are possible through impersonation.
> >
> > Considering the first point, there's probably no reason to allow
> > impersonation of superusers. Additionally, a custom authorizer could
> forbid
> > impersonation outside of a particular group. To support this, it would be
> > helpful to extend `KafkaPrincipal` or `AuthorizableRequestContext` so
> that
> > it indicates whether a request is an impersonated request.
> >
> > Considering the second point, it doesn't make sense to allow arbitrary
> > requests to be forwarded. We know exactly the set of forwardable APIs and
> > we can reject any other APIs without even looking at the principal. This
> is
> > the nice thing that the Envelope request gives us. I don't know if we
> would
> > ever have finer-grained restrictions, but in principle I don't see why we
> > couldn't.
> >
> > In the future, I guess we could go even further so that the broker itself
> > wouldn't need the same permissions as the client. If the client and the
> > controller shared some secret or if the client had a public key that we
> > could rely on, then the client could send along a MAC or token of some
> > sort, which could then be forwarded through the envelope. Then the broker
> > would not be allowed to do anything except exactly what the client
> > requested. I'm not suggesting we do this, just that we will have the
> > flexibility for it.
> >
> > From the discussion thread, it looks like the main problem here is that
> > `KafkaPrincipal` does not currently have a defined serialization
> mechanism.
> > We can add this, but it is a breaking change, so we have to wait for a
> > major release version before we make it a requirement. This is fine from
> > the perspective of KIP-500 since the bridge release will require a major
> > release bump anyway. What I imagine we could do is something like this:
> >
> > 1. In versions 2.7/2.8/.., we can add the new API in a mixin interface,
> say
> > `KafkaPrincipalSerde`. We can implement this interface for
> > `DefaultKafkaPrincipalBuilder` trivially. On startup, we can check if
> this
> > interface is implemented by the provided `KafkaPrincipalBuilder`. If it
> is
> > not, we can log a warning and revert to non-forwarding mode.
> > 2. Starting in version 3.0, we can make `KafkaPrincipalSerde` a
> requirement
> > and fail startup if it is not implemented.
> >
> > I think this approach would give users enough time to add the required
> > support. In any case, I don't think compatibility should be a blocker
> here
> > if we think it's the best way forward. This seems like a decision which
> > will affect things for a long time and we need to be allowed to change
> > plugin APIs.
> >
> > Anyway, let me know what you think.
> >
> > Best,
> > Jason
> >
> >
> >
> > On Fri, Sep 25, 2020 at 2:21 PM Colin McCabe <cm...@apache.org> wrote:
> >
> > > On Fri, Sep 25, 2020, at 10:49, Boyang Chen wrote:
> > > > Hey Jun,
> > > >
> > > > On Fri, Sep 25, 2020 at 10:19 AM Jun Rao <ju...@confluent.io> wrote:
> > > >
> > > > > Hi, Boyang,
> > > > >
> > > > > Does EnvelopeRequest avoid the need for IBP? How do we know if the
> > > > > controller supports EnvelopeRequest or not?
> > > > >
> > > > > Unfortunately, the EnvelopeRequest is solving the inter-broker
> > > > > communication problem only. Admin clients still need to learn the
> > > proper
> > > > > ApiVersion from the broker, which means we need to bump IBP to
> limit
> > > the
> > > > > version range.
> > > >
> > >
> > > Right-- the purpose of EnvelopeRequest is to avoid downconversion /
> > > upconversion on the forwarding broker.  It unfortunately doesn't avoid
> > the
> > > need to tie ApiVersionsResponse to IBP.
> > >
> > > > > > On Thu, Sep 24, 2020 at 4:53 PM Jun Rao <ju...@confluent.io>
> wrote:
> > > > > >
> > > > > > > Hi, Jason,
> > > > > > >
> > > > > > > Yes, the most important thing is to be able to avoid two
> rolling
> > > > > > > restarts
> > > > > > > in the future. If we have a path to achieve that down the road,
> > the
> > > > > > > changes here are fine.
> > > > > > >
> > >
> > > Yeah.  I think it would be good to make IBP a feature flag, as long as
> it
> > > could be changed without doing a second rolling restart.  We actually
> > don't
> > > want to have too many feature flags, since it blows up the test matrix.
> > >
> > > best,
> > > Colin
> > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > Jun
> > > > > > >
> > > > > > > On Thu, Sep 24, 2020 at 3:20 PM Jason Gustafson <
> > > jason@confluent.io>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > > One of the goals of KIP-584 (feature versioning) is that we
> > > can get
> > > > > > rid
> > > > > > > > of
> > > > > > > > IBP in the future. So does this change prevent us from
> removing
> > > IBP
> > > > > in
> > > > > > > the
> > > > > > > > future?
> > > > > > > >
> > > > > > > > That is a good question. I think the problem here is that
> > request
> > > > > > > > forwarding puts an expectation on api version support which
> > > covers
> > > > > more
> > > > > > > > than one broker. This is why the normal ApiVersions behavior
> > > doesn't
> > > > > > > work.
> > > > > > > > I thought about this a bit and haven't come up with a good
> > > > > alternative.
> > > > > > > One
> > > > > > > > thought I've been considering is letting the controller in
> the
> > > > > > > post-kip-500
> > > > > > > > world set the maximum range of api support for the cluster.
> > > However,
> > > > > > even
> > > > > > > > then we would need some way to tell when the controller
> quorum
> > > itself
> > > > > > is
> > > > > > > > ready to enable support for a new api version. My feeling is
> > > that we
> > > > > > will
> > > > > > > > probably always need something like the IBP to control when
> it
> > is
> > > > > safe
> > > > > > to
> > > > > > > > expose versions of APIs which have a cross-broker dependence.
> > > > > However,
> > > > > > > > KIP-584 would still allow us to manage the IBP at the level
> of
> > a
> > > > > > feature
> > > > > > > so
> > > > > > > > that we don't need two rolling restarts anymore.
> > > > > > > >
> > > > > > > > Best,
> > > > > > > > Jason
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > On Thu, Sep 24, 2020 at 1:59 PM Jun Rao <ju...@confluent.io>
> > > wrote:
> > > > > > > >
> > > > > > > > > Hi, Boyang,
> > > > > > > > >
> > > > > > > > > One of the goals of KIP-584 (feature versioning) is that we
> > > can get
> > > > > > rid
> > > > > > > > of
> > > > > > > > > IBP in the future. So does this change prevent us from
> > > removing IBP
> > > > > > in
> > > > > > > > the
> > > > > > > > > future?
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > >
> > > > > > > > > Jun
> > > > > > > > >
> > > > > > > > > On Thu, Sep 24, 2020 at 12:46 PM Jason Gustafson <
> > > > > jason@confluent.io
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hey Boyang,
> > > > > > > > > >
> > > > > > > > > > Thanks for the update. This seems like the best thing we
> > can
> > > do.
> > > > > > The
> > > > > > > > > > alternative would be to always ensure that the forwarded
> > > APIs are
> > > > > > > safe
> > > > > > > > > for
> > > > > > > > > > conversion between versions, but that would restrict the
> > > > > > flexibility
> > > > > > > > that
> > > > > > > > > > the versioning is providing. It would also be a large
> > effort
> > > to
> > > > > > avoid
> > > > > > > > > > introducing regressions through conversion. Sadly this
> > > broadens
> > > > > the
> > > > > > > > scope
> > > > > > > > > > of the IBP, but in fact forwarded APIs are inter-broker
> > APIs.
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > > Jason
> > > > > > > > > >
> > > > > > > > > > On Thu, Sep 24, 2020 at 9:23 AM Boyang Chen <
> > > > > > > > reluctanthero104@gmail.com>
> > > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hey there,
> > > > > > > > > > >
> > > > > > > > > > > we spotted a necessary case to handle the redirect
> > request
> > > > > > > > versioning,
> > > > > > > > > > and
> > > > > > > > > > > proposed the following changes:
> > > > > > > > > > >
> > > > > > > > > > > 1. For redirection RPCs (AlterConfig, Acl, Token etc),
> > the
> > > > > > > > > corresponding
> > > > > > > > > > > allowed versions in the ApiVersionResponse will be
> > > affected by
> > > > > > the
> > > > > > > > > entire
> > > > > > > > > > > cluster's versioning, not just the receiving broker,
> > since
> > > we
> > > > > > need
> > > > > > > to
> > > > > > > > > > > ensure the chosen version get properly handled by all
> > > parties.
> > > > > > Thus
> > > > > > > > > from
> > > > > > > > > > > now on, RPC with redirection will be treated as
> > > inter-broker
> > > > > RPC,
> > > > > > > and
> > > > > > > > > any
> > > > > > > > > > > version bump for these RPCs has to go through IBP bump
> as
> > > well.
> > > > > > > > > > > ApiVersionResponse will take IBP into considerations
> for
> > > the
> > > > > > > > > redirection
> > > > > > > > > > > RPCs allowable versions.
> > > > > > > > > > >
> > > > > > > > > > > 2. We would do the best effort to maintain the same
> > request
> > > > > > version
> > > > > > > > for
> > > > > > > > > > > the entire admin client -> receiving broker ->
> controller
> > > > > broker
> > > > > > > > path,
> > > > > > > > > > but
> > > > > > > > > > > for old RPC versions, they may not have flexible fields
> > > > > > introduced
> > > > > > > > yet.
> > > > > > > > > > > Thus, we would have to upgrade the RPC to the minimum
> > > version
> > > > > > which
> > > > > > > > > > > supports flexible fields
> > > > > > > > > > > and add another tagged field in the header called
> > > > > > > > > > `OriginalRequestVersion`
> > > > > > > > > > > to help the controller broker correctly deserialize the
> > > request
> > > > > > > with
> > > > > > > > > the
> > > > > > > > > > > original admin client sent out version. We would not
> > > downgrade
> > > > > > the
> > > > > > > > > > original
> > > > > > > > > > > request in any circumstance, since the flexible field
> > > support
> > > > > is
> > > > > > > > > required
> > > > > > > > > > > to be open-ended on the high side.
> > > > > > > > > > >
> > > > > > > > > > > Let me know if you have any questions.
> > > > > > > > > > >
> > > > > > > > > > > Best,
> > > > > > > > > > > Boyang
> > > > > > > > > > >
> > > > > > > > > > > On Thu, Aug 6, 2020 at 6:11 PM Boyang Chen <
> > > > > > > > reluctanthero104@gmail.com
> > > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Hey there,
> > > > > > > > > > > >
> > > > > > > > > > > > we are going to introduce a minor change to bump the
> > > version
> > > > > of
> > > > > > > > > several
> > > > > > > > > > > > RPCs which are currently not supporting flexible
> > > versions. It
> > > > > > is
> > > > > > > > > > > necessary
> > > > > > > > > > > > because they need to be able to construct request
> > header
> > > with
> > > > > > > > initial
> > > > > > > > > > > > principal name and client id as optional fields for
> > > > > > redirection.
> > > > > > > > The
> > > > > > > > > > are
> > > > > > > > > > > > only two of them:
> > > > > > > > > > > >
> > > > > > > > > > > > 1. AlterConfig
> > > > > > > > > > > > 2. AlterClientQuotas
> > > > > > > > > > > >
> > > > > > > > > > > > Let me know if you have any questions.
> > > > > > > > > > > >
> > > > > > > > > > > > Boyang
> > > > > > > > > > > >
> > > > > > > > > > > > On Fri, Jul 31, 2020 at 11:42 AM Boyang Chen <
> > > > > > > > > > reluctanthero104@gmail.com
> > > > > > > > > > > >
> > > > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > >> Hey David,
> > > > > > > > > > > >>
> > > > > > > > > > > >> After discussing with Colin offline, I would like to
> > > correct
> > > > > > one
> > > > > > > > > case
> > > > > > > > > > in
> > > > > > > > > > > >> the described workflow, where the CLUSTER_ACTION
> > > > > authorization
> > > > > > > > would
> > > > > > > > > > > not be
> > > > > > > > > > > >> based on the initial principal field check, because
> it
> > > is
> > > > > not
> > > > > > a
> > > > > > > > > > secured
> > > > > > > > > > > >> condition which anyone could forge. The revised
> > workflow
> > > > > shall
> > > > > > > be:
> > > > > > > > > > > >>
> > > > > > > > > > > >> Step 1. Filter out resources that are authorized
> > > > > > > > > > > >>          1.1 Use traditional principals to verify
> > > first. If
> > > > > > > > > > authorized,
> > > > > > > > > > > >> continue
> > > > > > > > > > > >>          1.2 If not authorized, check whether the
> > > request is
> > > > > > > from
> > > > > > > > > the
> > > > > > > > > > > >> control plane. Note that this is a best-effort to
> > verify
> > > > > > whether
> > > > > > > > the
> > > > > > > > > > > >> request is internal.
> > > > > > > > > > > >>          1.3 If the request is not from the control
> > > plane,
> > > > > > > return
> > > > > > > > > > > >> authorization failure
> > > > > > > > > > > >>          1.4 If the request is from the control
> plane,
> > > use
> > > > > > > > > > > CLUSTER_ACTION
> > > > > > > > > > > >> to verify and determine the result
> > > > > > > > > > > >>
> > > > > > > > > > > >> Step 2. Check the request context to see if this is
> a
> > > > > > forwarding
> > > > > > > > > > > request,
> > > > > > > > > > > >> by checking whether it is from control plane and
> uses
> > > extra
> > > > > > > header
> > > > > > > > > > > fields
> > > > > > > > > > > >>         2.1 if the resource is authorized, and if
> this
> > > is
> > > > > the
> > > > > > > > active
> > > > > > > > > > > >> controller, process it
> > > > > > > > > > > >>         2.2 if the resource is authorized but this
> is
> > > not
> > > > > the
> > > > > > > > active
> > > > > > > > > > > >> controller, return NOT_CONTROLLER to the sender
> > > (forwarding
> > > > > > > > broker)
> > > > > > > > > > for
> > > > > > > > > > > >> retry
> > > > > > > > > > > >>         2.3 if the resource is not authorized,
> return
> > > > > > > > > > > >> CLUSTER_AUTHORIZATION_FAILURE to propagate back to
> the
> > > > > > original
> > > > > > > > > client
> > > > > > > > > > > >> through forwarding broker
> > > > > > > > > > > >> Step 3. If the request is not a forwarding request
> > > > > > > > > > > >>         3.1 If the resource is authorized, and this
> is
> > > the
> > > > > > > active
> > > > > > > > > > > >> controller, process it
> > > > > > > > > > > >>         3.2 If the resource is authorized, but this
> is
> > > not
> > > > > > > active
> > > > > > > > > > > >> controller, put the resource into the preparation
> for
> > a
> > > new
> > > > > > > > > > AlterConfig
> > > > > > > > > > > >> request for forwarding
> > > > > > > > > > > >>         3.3 If the resource is not authorized, reply
> > the
> > > > > > > original
> > > > > > > > > > client
> > > > > > > > > > > >> AUTHORIZATION_FAILURE when the forwarding request is
> > > > > returned
> > > > > > > > > > > >>
> > > > > > > > > > > >> On Thu, Jul 30, 2020 at 3:47 PM Boyang Chen <
> > > > > > > > > > reluctanthero104@gmail.com
> > > > > > > > > > > >
> > > > > > > > > > > >> wrote:
> > > > > > > > > > > >>
> > > > > > > > > > > >>>
> > > > > > > > > > > >>>
> > > > > > > > > > > >>> On Thu, Jul 30, 2020 at 7:18 AM David Jacot <
> > > > > > > djacot@confluent.io
> > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > > >>>
> > > > > > > > > > > >>>> Hi Boyang,
> > > > > > > > > > > >>>>
> > > > > > > > > > > >>>> Thanks for your answers.
> > > > > > > > > > > >>>>
> > > > > > > > > > > >>>> > The point for using the listener name is more
> of a
> > > > > > security
> > > > > > > > > > purpose,
> > > > > > > > > > > >>>> to
> > > > > > > > > > > >>>> > detect any forged request to our best effort.
> > > > > > > > > > > >>>> > For throttling I think we could just check the
> > > request
> > > > > > > header
> > > > > > > > > for
> > > > > > > > > > > >>>> > *InitialClientId* existence, to distinguish
> > whether
> > > to
> > > > > > apply
> > > > > > > > > > > >>>> > throttling strategy as forwarded request or
> direct
> > > > > > request.
> > > > > > > > > > > >>>>
> > > > > > > > > > > >>>> Reading "security" and "best effort" in the same
> > > sentence
> > > > > > > makes
> > > > > > > > > me a
> > > > > > > > > > > >>>> little nervous :).
> > > > > > > > > > > >>>>
> > > > > > > > > > > >>>> The identification issue is also valid for quota
> as
> > we
> > > > > don't
> > > > > > > > want
> > > > > > > > > > one
> > > > > > > > > > > >>>> to be
> > > > > > > > > > > >>>> able to bypass the quota by forging a request as
> > well,
> > > > > isn't
> > > > > > > it?
> > > > > > > > > > > >>>> Otherwise,
> > > > > > > > > > > >>>> anyone could just set the InitialPrincipal to
> bypass
> > > it. I
> > > > > > > think
> > > > > > > > > > that
> > > > > > > > > > > we
> > > > > > > > > > > >>>> should
> > > > > > > > > > > >>>> only use InitialPrincipal and/or InitialClientId
> > when
> > > we
> > > > > > know
> > > > > > > > that
> > > > > > > > > > > they
> > > > > > > > > > > >>>> come
> > > > > > > > > > > >>>> from another broker. Based on what I read in the
> > KIP,
> > > it
> > > > > > looks
> > > > > > > > > like
> > > > > > > > > > we
> > > > > > > > > > > >>>> could
> > > > > > > > > > > >>>> only use them when the principal has
> CLUSTER_ACTION
> > > > > > privilege.
> > > > > > > > Do
> > > > > > > > > I
> > > > > > > > > > > >>>> understand it correctly?
> > > > > > > > > > > >>>>
> > > > > > > > > > > >>>> There is no 100% safe way to distinguish between
> raw
> > > > > request
> > > > > > > and
> > > > > > > > > > > >>> forward request if you take malicious request into
> > > > > > > consideration,
> > > > > > > > > > which
> > > > > > > > > > > >>> could happen
> > > > > > > > > > > >>> anyway if the attacker prepares those requests to
> > pass
> > > the
> > > > > > > checks
> > > > > > > > > > for a
> > > > > > > > > > > >>> pre-KIP-500 cluster.
> > > > > > > > > > > >>> We could at most know whether it is sent to the
> > control
> > > > > > plane,
> > > > > > > or
> > > > > > > > > the
> > > > > > > > > > > >>> data plane, and whether it has extra header fields
> > > such as
> > > > > > > > initial
> > > > > > > > > > > >>> principal and client id defined. For a
> non-malicious
> > > > > request
> > > > > > > > > > > >>> going into the control plane, it must be sent from
> a
> > > valid
> > > > > > > > broker,
> > > > > > > > > > > which
> > > > > > > > > > > >>> is a prerequisite to check its
> > > > > > > > > > > >>> CLUSTER_ACTION principal. Take AlterConfig for an
> > > example,
> > > > > > the
> > > > > > > > > > intended
> > > > > > > > > > > >>> workflow for a *KIP-590 broker* would be:
> > > > > > > > > > > >>>
> > > > > > > > > > > >>> Step 1. Check the request context to see if this
> is a
> > > > > > > forwarding
> > > > > > > > > > > >>> request, by checking whether it is from control
> plane
> > > and
> > > > > > uses
> > > > > > > > > extra
> > > > > > > > > > > header
> > > > > > > > > > > >>> fields
> > > > > > > > > > > >>>         1.1 if it is a forwarding request, use
> > > > > CLUSTER_ACTION
> > > > > > > to
> > > > > > > > > > verify
> > > > > > > > > > > >>> the given resource
> > > > > > > > > > > >>>         1.2 if the resource is authorized, and if
> > this
> > > is
> > > > > the
> > > > > > > > > active
> > > > > > > > > > > >>> controller, process it
> > > > > > > > > > > >>>         1.3 if the resource is authorized but this
> is
> > > not
> > > > > the
> > > > > > > > > active
> > > > > > > > > > > >>> controller, return NOT_CONTROLLER to the sender
> > > (forwarding
> > > > > > > > broker)
> > > > > > > > > > for
> > > > > > > > > > > >>> retry
> > > > > > > > > > > >>>         1.4 if the resource is not authorized,
> return
> > > > > > > > > > > >>> CLUSTER_AUTHORIZATION_FAILURE to propagate back to
> > the
> > > > > > original
> > > > > > > > > > client
> > > > > > > > > > > >>> through forwarding broker
> > > > > > > > > > > >>> Step 2. If the request is not a forwarding request
> > > > > > > > > > > >>>         2.1 Verify with normal principal as ALTER
> on
> > > the
> > > > > > given
> > > > > > > > > > > resources
> > > > > > > > > > > >>>         2.2 If the resource is authorized, and this
> > is
> > > the
> > > > > > > active
> > > > > > > > > > > >>> controller, process it
> > > > > > > > > > > >>>         2.3 If the resource is authorized, but this
> > is
> > > not
> > > > > > > active
> > > > > > > > > > > >>> controller, put the resource into the preparation
> for
> > > a new
> > > > > > > > > > AlterConfig
> > > > > > > > > > > >>> request for forwarding
> > > > > > > > > > > >>>         2.4 If the resource is not authorized,
> reply
> > > the
> > > > > > > original
> > > > > > > > > > > client
> > > > > > > > > > > >>> AUTHORIZATION_FAILURE when the forwarding request
> is
> > > > > returned
> > > > > > > > > > > >>>
> > > > > > > > > > > >>> When the control plane and data plane are using the
> > > same
> > > > > > > > listener,
> > > > > > > > > we
> > > > > > > > > > > >>> couldn't distinguish whether a request is forwarded
> > or
> > > not
> > > > > > for
> > > > > > > > > sure,
> > > > > > > > > > > so in
> > > > > > > > > > > >>> terms of the forward request checking, we have to
> > > require
> > > > > the
> > > > > > > > extra
> > > > > > > > > > > header
> > > > > > > > > > > >>> fields to present. A stronger checking mechanism
> > could
> > > only
> > > > > > be
> > > > > > > > > > enforced
> > > > > > > > > > > >>> when we upgrade to post-KIP-500 with a separate
> > > controller
> > > > > > > > network.
> > > > > > > > > > > >>>
> > > > > > > > > > > >>>
> > > > > > > > > > > >>>> I have made another pass on the whole KIP, I have
> > few
> > > > > nits:
> > > > > > > > > > > >>>>
> > > > > > > > > > > >>>> - The sentence "Take AlterConfig as an example to
> > > > > understand
> > > > > > > the
> > > > > > > > > > > changes
> > > > > > > > > > > >>>> we are making." does not make much sense anymore
> in
> > > the
> > > > > > > > beginning
> > > > > > > > > of
> > > > > > > > > > > the
> > > > > > > > > > > >>>> "Proposed Changes" chapter.
> > > > > > > > > > > >>>>
> > > > > > > > > > > >>>> Sure, deleted.
> > > > > > > > > > > >>>
> > > > > > > > > > > >>>
> > > > > > > > > > > >>>> - When you say "Existing RPCs which are sending
> > > directly
> > > > > to
> > > > > > > the
> > > > > > > > > > > >>>> controller
> > > > > > > > > > > >>>> will
> > > > > > > > > > > >>>> rely on forwarding as well.". I suggest to
> > explicitly
> > > > > > mention
> > > > > > > > how
> > > > > > > > > > "old
> > > > > > > > > > > >>>> admin clients"
> > > > > > > > > > > >>>> will work here to complement the sentence.
> Something
> > > like:
> > > > > > > They
> > > > > > > > > will
> > > > > > > > > > > >>>> get a
> > > > > > > > > > > >>>> random
> > > > > > > > > > > >>>> broker id as the controller id in the metadata
> > > response
> > > > > and
> > > > > > > > stick
> > > > > > > > > to
> > > > > > > > > > > it
> > > > > > > > > > > >>>> as
> > > > > > > > > > > >>>> you explained.
> > > > > > > > > > > >>>>
> > > > > > > > > > > >>>> Sg, changed.
> > > > > > > > > > > >>>
> > > > > > > > > > > >>>
> > > > > > > > > > > >>>> - "The purpose of adding principal name is for the
> > > audit
> > > > > > > > logging,
> > > > > > > > > > and
> > > > > > > > > > > >>>> the
> > > > > > > > > > > >>>> client id is
> > > > > > > > > > > >>>> being used to throttling according to KIP-599
> > > > > requirement."
> > > > > > > > > > Actually,
> > > > > > > > > > > >>>> KIP-599 needs
> > > > > > > > > > > >>>> both the principal and the clientId.
> > > > > > > > > > > >>>>
> > > > > > > > > > > >>>> Makes sense.
> > > > > > > > > > > >>>
> > > > > > > > > > > >>>
> > > > > > > > > > > >>>> - In the "Routing Request Security" chapter. It is
> > > written
> > > > > > > that
> > > > > > > > > the
> > > > > > > > > > > >>>> forwarding broker
> > > > > > > > > > > >>>> will verify the request with its own authorizer
> and
> > > will
> > > > > > just
> > > > > > > > > > forward
> > > > > > > > > > > >>>> it if
> > > > > > > > > > > >>>> the request
> > > > > > > > > > > >>>> looks good. When a request contains for instance
> > > multiple
> > > > > > > > topics,
> > > > > > > > > I
> > > > > > > > > > > >>>> suppose
> > > > > > > > > > > >>>> that
> > > > > > > > > > > >>>> we will forward only the authorized ones and not
> the
> > > whole
> > > > > > > > > original
> > > > > > > > > > > >>>> request
> > > > > > > > > > > >>>> as is.
> > > > > > > > > > > >>>> We may want to reword the sentence to make this
> > clear.
> > > > > > > > > > > >>>>
> > > > > > > > > > > >>>> That makes sense, I will put this more detailed
> > > version of
> > > > > > > > routing
> > > > > > > > > > > into
> > > > > > > > > > > >>> the design.
> > > > > > > > > > > >>>
> > > > > > > > > > > >>>
> > > > > > > > > > > >>>> - For the record, should we put the previous
> > proposal
> > > in
> > > > > the
> > > > > > > > > > rejected
> > > > > > > > > > > >>>> alternatives as
> > > > > > > > > > > >>>> well?
> > > > > > > > > > > >>>>
> > > > > > > > > > > >>>> We do have big changes in this KIP, our current
> > > strategy
> > > > > is
> > > > > > to
> > > > > > > > > rely
> > > > > > > > > > on
> > > > > > > > > > > >>> wiki revisions if people
> > > > > > > > > > > >>> are interested to figure out our previous design
> > > strategy.
> > > > > > > > Putting
> > > > > > > > > > the
> > > > > > > > > > > >>> full design in current KIP proposal
> > > > > > > > > > > >>> would distract too much for readers as we discussed
> > > during
> > > > > > the
> > > > > > > > last
> > > > > > > > > > > time
> > > > > > > > > > > >>> of big refactoring.
> > > > > > > > > > > >>>
> > > > > > > > > > > >>> Best,
> > > > > > > > > > > >>>> David
> > > > > > > > > > > >>>>
> > > > > > > > > > > >>>> On Thu, Jul 30, 2020 at 3:51 AM Boyang Chen <
> > > > > > > > > > > reluctanthero104@gmail.com
> > > > > > > > > > > >>>> >
> > > > > > > > > > > >>>> wrote:
> > > > > > > > > > > >>>>
> > > > > > > > > > > >>>> > Thanks David for the feedback!
> > > > > > > > > > > >>>> >
> > > > > > > > > > > >>>> > On Wed, Jul 29, 2020 at 7:53 AM David Jacot <
> > > > > > > > > djacot@confluent.io>
> > > > > > > > > > > >>>> wrote:
> > > > > > > > > > > >>>> >
> > > > > > > > > > > >>>> > > Hi, Colin, Boyang,
> > > > > > > > > > > >>>> > >
> > > > > > > > > > > >>>> > > Colin, thanks for the clarification. Somehow,
> I
> > > > > thought
> > > > > > > that
> > > > > > > > > > even
> > > > > > > > > > > >>>> if the
> > > > > > > > > > > >>>> > > controller is ran independently, it
> > > > > > > > > > > >>>> > > would still run the listeners of the broker
> and
> > > thus
> > > > > > would
> > > > > > > > be
> > > > > > > > > > > >>>> accessible
> > > > > > > > > > > >>>> > by
> > > > > > > > > > > >>>> > > redirecting on the loopback
> > > > > > > > > > > >>>> > > interface. My mistake.
> > > > > > > > > > > >>>> > >
> > > > > > > > > > > >>>> > > Boyang, I have few questions/comments
> regarding
> > > the
> > > > > > > updated
> > > > > > > > > KIP:
> > > > > > > > > > > >>>> > >
> > > > > > > > > > > >>>> > > 1. I think that it would be great if we could
> > > clarify
> > > > > > how
> > > > > > > > old
> > > > > > > > > > > admin
> > > > > > > > > > > >>>> > clients
> > > > > > > > > > > >>>> > > which are directly talking to the
> > > > > > > > > > > >>>> > > controller will work with this KIP. I read
> > > between the
> > > > > > > lines
> > > > > > > > > > that,
> > > > > > > > > > > >>>> as we
> > > > > > > > > > > >>>> > > propose to provide a random
> > > > > > > > > > > >>>> > > broker Id as the controller Id in the metadata
> > > > > response,
> > > > > > > > they
> > > > > > > > > > will
> > > > > > > > > > > >>>> use a
> > > > > > > > > > > >>>> > > single node as a proxy. Is that
> > > > > > > > > > > >>>> > > correct? This deserves to be called out more
> > > > > explicitly
> > > > > > in
> > > > > > > > the
> > > > > > > > > > > >>>> design
> > > > > > > > > > > >>>> > > section instead of being hidden
> > > > > > > > > > > >>>> > > in the protocol bump of the metadata RPC.
> > > > > > > > > > > >>>> > >
> > > > > > > > > > > >>>> > > Makes sense, I stress this point in the
> > > compatibility
> > > > > > > > section.
> > > > > > > > > > > >>>> >
> > > > > > > > > > > >>>> >
> > > > > > > > > > > >>>> > > 1.1 If I understand correctly, could we assume
> > > that
> > > > > old
> > > > > > > > admin
> > > > > > > > > > > >>>> clients
> > > > > > > > > > > >>>> > will
> > > > > > > > > > > >>>> > > stick to the same "fake controller"
> > > > > > > > > > > >>>> > > until they refresh their metadata? Refreshing
> > the
> > > > > > metadata
> > > > > > > > > > usually
> > > > > > > > > > > >>>> > happens
> > > > > > > > > > > >>>> > > when NOT_CONTROLLER
> > > > > > > > > > > >>>> > > is received but this won't happen anymore so
> > they
> > > > > should
> > > > > > > > > change
> > > > > > > > > > > >>>> > > infrequently.
> > > > > > > > > > > >>>> > >
> > > > > > > > > > > >>>> > > That is correct, old admin clients would not
> try
> > > to
> > > > > > > refresh
> > > > > > > > > > their
> > > > > > > > > > > >>>> > metadata
> > > > > > > > > > > >>>> > due to NOT_CONTROLLER,
> > > > > > > > > > > >>>> > which is impossible to happen with the new
> broker
> > > > > cluster.
> > > > > > > > > > > >>>> >
> > > > > > > > > > > >>>> >
> > > > > > > > > > > >>>> > > 2. For the new admin client, I suppose that we
> > > plan on
> > > > > > > using
> > > > > > > > > > > >>>> > > LeastLoadedNodeProvider for the
> > > > > > > > > > > >>>> > > requests that are using
> ControllerNodeProvider.
> > We
> > > > > could
> > > > > > > > > perhaps
> > > > > > > > > > > >>>> mention
> > > > > > > > > > > >>>> > > it.
> > > > > > > > > > > >>>> > >
> > > > > > > > > > > >>>> > > Sure, added.
> > > > > > > > > > > >>>> >
> > > > > > > > > > > >>>> >
> > > > > > > > > > > >>>> > > 3. Pre KIP-500, will we have a way to
> > distinguish
> > > if a
> > > > > > > > request
> > > > > > > > > > > that
> > > > > > > > > > > >>>> is
> > > > > > > > > > > >>>> > > received by the controller is
> > > > > > > > > > > >>>> > > coming directly from a client or from a
> broker?
> > > You
> > > > > > > mention
> > > > > > > > > that
> > > > > > > > > > > the
> > > > > > > > > > > >>>> > > listener can be used to do
> > > > > > > > > > > >>>> > > this but as you pointed out, it is not
> > mandatory.
> > > Do
> > > > > we
> > > > > > > have
> > > > > > > > > > > another
> > > > > > > > > > > >>>> > > reliable method? I am asking
> > > > > > > > > > > >>>> > > in the context of KIP-599 with the current
> > > controller,
> > > > > > we
> > > > > > > > may
> > > > > > > > > > need
> > > > > > > > > > > >>>> to
> > > > > > > > > > > >>>> > > throttle differently if the
> > > > > > > > > > > >>>> > > request comes from a client or from a broker.
> > > > > > > > > > > >>>> > >
> > > > > > > > > > > >>>> > > The point for using the listener name is more
> > of a
> > > > > > > security
> > > > > > > > > > > >>>> purpose, to
> > > > > > > > > > > >>>> > detect any forged request to our best effort.
> > > > > > > > > > > >>>> > For throttling I think we could just check the
> > > request
> > > > > > > header
> > > > > > > > > for
> > > > > > > > > > > >>>> > *InitialClientId* existence, to distinguish
> > whether
> > > to
> > > > > > apply
> > > > > > > > > > > >>>> > throttling strategy as forwarded request or
> direct
> > > > > > request.
> > > > > > > > > > > >>>> >
> > > > > > > > > > > >>>> >
> > > > > > > > > > > >>>> > > 4. Could we add `InitialClientId` as well?
> This
> > > will
> > > > > be
> > > > > > > > > required
> > > > > > > > > > > >>>> for the
> > > > > > > > > > > >>>> > > quota as we can apply them
> > > > > > > > > > > >>>> > > by principal and/or clientId.
> > > > > > > > > > > >>>> > >
> > > > > > > > > > > >>>> > > Sounds good, added.
> > > > > > > > > > > >>>> >
> > > > > > > > > > > >>>> >
> > > > > > > > > > > >>>> > > 5. A small remark regarding the structure of
> the
> > > KIP.
> > > > > It
> > > > > > > is
> > > > > > > > a
> > > > > > > > > > bit
> > > > > > > > > > > >>>> weird
> > > > > > > > > > > >>>> > > that requests that do not go
> > > > > > > > > > > >>>> > > to the controller are mentioned in the
> Proposed
> > > Design
> > > > > > > > section
> > > > > > > > > > and
> > > > > > > > > > > >>>> the
> > > > > > > > > > > >>>> > > requests that go to the
> > > > > > > > > > > >>>> > > controller are mentioned in the Public
> > Interfaces.
> > > > > When
> > > > > > > one
> > > > > > > > > read
> > > > > > > > > > > the
> > > > > > > > > > > >>>> > > Proposed Design, it does not
> > > > > > > > > > > >>>> > > get a full picture of the whole new routing
> > > proposal
> > > > > for
> > > > > > > old
> > > > > > > > > and
> > > > > > > > > > > new
> > > > > > > > > > > >>>> > > clients. It would be great if we
> > > > > > > > > > > >>>> > > could have a full overview in that section.
> > > > > > > > > > > >>>> > >
> > > > > > > > > > > >>>> > > Good point, I will move the pieces around.
> > > > > > > > > > > >>>> >
> > > > > > > > > > > >>>> >
> > > > > > > > > > > >>>> > > Overall the change makes sense to me. I will
> > work
> > > on
> > > > > > > > drafting
> > > > > > > > > an
> > > > > > > > > > > >>>> addendum
> > > > > > > > > > > >>>> > > to KIP-599 to
> > > > > > > > > > > >>>> > > alter the design to cope with these changes.
> At
> > a
> > > > > first
> > > > > > > > > glance,
> > > > > > > > > > > that
> > > > > > > > > > > >>>> > seems
> > > > > > > > > > > >>>> > > doable if 1.1, 3
> > > > > > > > > > > >>>> > > and 4 are OK.
> > > > > > > > > > > >>>> > >
> > > > > > > > > > > >>>> > > Thank you for the help!
> > > > > > > > > > > >>>> >
> > > > > > > > > > > >>>> >
> > > > > > > > > > > >>>> > > Thanks,
> > > > > > > > > > > >>>> > > David
> > > > > > > > > > > >>>> > >
> > > > > > > > > > > >>>> > > On Wed, Jul 29, 2020 at 5:29 AM Boyang Chen <
> > > > > > > > > > > >>>> reluctanthero104@gmail.com>
> > > > > > > > > > > >>>> > > wrote:
> > > > > > > > > > > >>>> > >
> > > > > > > > > > > >>>> > > > Thanks for the feedback Colin!
> > > > > > > > > > > >>>> > > >
> > > > > > > > > > > >>>> > > > On Tue, Jul 28, 2020 at 2:11 PM Colin
> McCabe <
> > > > > > > > > > > cmccabe@apache.org>
> > > > > > > > > > > >>>> > wrote:
> > > > > > > > > > > >>>> > > >
> > > > > > > > > > > >>>> > > > > Hi Boyang,
> > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > >>>> > > > > Thanks for updating this.  A few comments
> > > below:
> > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > >>>> > > > > In the "Routing Request Security" section,
> > > there
> > > > > is
> > > > > > a
> > > > > > > > > > > reference
> > > > > > > > > > > >>>> to
> > > > > > > > > > > >>>> > > "older
> > > > > > > > > > > >>>> > > > > requests that need redirection."  But
> after
> > > these
> > > > > > new
> > > > > > > > > > > >>>> revisions, both
> > > > > > > > > > > >>>> > > new
> > > > > > > > > > > >>>> > > > > and old requests need redirection.  So we
> > > should
> > > > > > > > rephrase
> > > > > > > > > > > this.
> > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > >>>> > > > > > In addition, to avoid exposing this
> > > forwarding
> > > > > > power
> > > > > > > > to
> > > > > > > > > > the
> > > > > > > > > > > >>>> admin
> > > > > > > > > > > >>>> > > > > clients,
> > > > > > > > > > > >>>> > > > > > the routing request shall be forwarded
> > > towards
> > > > > the
> > > > > > > > > > > controller
> > > > > > > > > > > >>>> > broker
> > > > > > > > > > > >>>> > > > > internal
> > > > > > > > > > > >>>> > > > > > endpoint which should be only visible to
> > > other
> > > > > > > brokers
> > > > > > > > > > > inside
> > > > > > > > > > > >>>> the
> > > > > > > > > > > >>>> > > > > cluster
> > > > > > > > > > > >>>> > > > > > in the KIP-500 controller. Any admin
> > > > > configuration
> > > > > > > > > request
> > > > > > > > > > > >>>> with
> > > > > > > > > > > >>>> > > broker
> > > > > > > > > > > >>>> > > > > > principal should not be going through
> the
> > > public
> > > > > > > > > endpoint
> > > > > > > > > > > and
> > > > > > > > > > > >>>> will
> > > > > > > > > > > >>>> > be
> > > > > > > > > > > >>>> > > > > > rejected for security purpose.
> > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > >>>> > > > > We should also describe how this will work
> > in
> > > the
> > > > > > > > > > pre-KIP-500
> > > > > > > > > > > >>>> case.
> > > > > > > > > > > >>>> > In
> > > > > > > > > > > >>>> > > > > that case, CLUSTER_ACTION gets the extra
> > > > > permissions
> > > > > > > > > > described
> > > > > > > > > > > >>>> here
> > > > > > > > > > > >>>> > > only
> > > > > > > > > > > >>>> > > > > when the message comes in on the
> > inter-broker
> > > > > > > listener.
> > > > > > > > > We
> > > > > > > > > > > >>>> should
> > > > > > > > > > > >>>> > > state
> > > > > > > > > > > >>>> > > > > that here.
> > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > >>>> > > > > (I can see that you have this information
> > > later on
> > > > > > in
> > > > > > > > the
> > > > > > > > > > > >>>> "Security
> > > > > > > > > > > >>>> > > > Access
> > > > > > > > > > > >>>> > > > > Changes" section, but it would be good to
> > > have it
> > > > > > here
> > > > > > > > as
> > > > > > > > > > > well,
> > > > > > > > > > > >>>> to
> > > > > > > > > > > >>>> > > avoid
> > > > > > > > > > > >>>> > > > > confusion.)
> > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > >>>> > > > > > To be more strict of protecting
> controller
> > > > > > > > information,
> > > > > > > > > > the
> > > > > > > > > > > >>>> > > > > "ControllerId"
> > > > > > > > > > > >>>> > > > > > field in new MetadataResponse shall be
> set
> > > to -1
> > > > > > > when
> > > > > > > > > the
> > > > > > > > > > > >>>> original
> > > > > > > > > > > >>>> > > > > request
> > > > > > > > > > > >>>> > > > > > comes from a non-broker client and it is
> > > already
> > > > > > on
> > > > > > > > v10.
> > > > > > > > > > We
> > > > > > > > > > > >>>> shall
> > > > > > > > > > > >>>> > use
> > > > > > > > > > > >>>> > > > the
> > > > > > > > > > > >>>> > > > > > request listener name to distinguish
> > > whether a
> > > > > > given
> > > > > > > > > > request
> > > > > > > > > > > >>>> is
> > > > > > > > > > > >>>> > > > > inter-broker,
> > > > > > > > > > > >>>> > > > > > or from the client.
> > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > >>>> > > > > I'm not sure why we would need to
> > distinguish
> > > > > > between
> > > > > > > > > broker
> > > > > > > > > > > >>>> clients
> > > > > > > > > > > >>>> > > and
> > > > > > > > > > > >>>> > > > > non-broker clients.  Brokers don't
> generally
> > > send
> > > > > > > > > > > >>>> MetadataRequests to
> > > > > > > > > > > >>>> > > > other
> > > > > > > > > > > >>>> > > > > brokers, do they?  Brokers learn about
> > > metadata
> > > > > from
> > > > > > > > > > > >>>> > > > UpdateMetadataRequest
> > > > > > > > > > > >>>> > > > > and LeaderAndIsrRequest, not by sending
> > > > > > > MetadataRequests
> > > > > > > > > to
> > > > > > > > > > > >>>> other
> > > > > > > > > > > >>>> > > > brokers.
> > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > >>>> > > > > We do have one use case where the
> > > MetadataRequest
> > > > > > gets
> > > > > > > > > sent
> > > > > > > > > > > >>>> between
> > > > > > > > > > > >>>> > the
> > > > > > > > > > > >>>> > > > brokers, which is the InterBrokerSendThread.
> > > > > Currently
> > > > > > > we
> > > > > > > > > > don't
> > > > > > > > > > > >>>> rely on
> > > > > > > > > > > >>>> > > it
> > > > > > > > > > > >>>> > > > to get the controller id, so I guess your
> > > suggestion
> > > > > > > > should
> > > > > > > > > be
> > > > > > > > > > > >>>> good to
> > > > > > > > > > > >>>> > > > enforce. We could use some meta comment on
> the
> > > > > > > > NetworkClient
> > > > > > > > > > > that
> > > > > > > > > > > >>>> it
> > > > > > > > > > > >>>> > > should
> > > > > > > > > > > >>>> > > > not be used to get the controller location.
> > > > > > > > > > > >>>> > > >
> > > > > > > > > > > >>>> > > > Probably what we want here is: v0-v9:
> return a
> > > > > random
> > > > > > > > broker
> > > > > > > > > > in
> > > > > > > > > > > >>>> the
> > > > > > > > > > > >>>> > > cluster
> > > > > > > > > > > >>>> > > > > as the controller ID.  v10: no
> controllerID
> > > > > present
> > > > > > in
> > > > > > > > the
> > > > > > > > > > > >>>> > > > > MetadataResponse.  We should also
> deprecate
> > > the
> > > > > > > > > adminClient
> > > > > > > > > > > >>>> method
> > > > > > > > > > > >>>> > > which
> > > > > > > > > > > >>>> > > > > gets the controllerId.
> > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > >>>> > > > > > BROKER_AUTHORIZATION_FAILURE(92,
> > > "Authorization
> > > > > > > failed
> > > > > > > > > for
> > > > > > > > > > > the
> > > > > > > > > > > >>>> > > > > > request during forwarding, this
> indicates
> > an
> > > > > > > internal
> > > > > > > > > > error
> > > > > > > > > > > >>>> on the
> > > > > > > > > > > >>>> > > > broker
> > > > > > > > > > > >>>> > > > > > cluster security setup.",
> > > > > > > > > > > >>>> > BrokerAuthorizationFailureException::new);
> > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > >>>> > > > > Grammar nitpick: It would be good to have
> a
> > > period
> > > > > > > > between
> > > > > > > > > > > >>>> > "forwarding"
> > > > > > > > > > > >>>> > > > > and "this" to avoid a run-on sentence :)
> > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > >>>> > > > > best,
> > > > > > > > > > > >>>> > > > > Colin
> > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > >>>> > > > > On Mon, Jul 27, 2020, at 21:47, Boyang
> Chen
> > > wrote:
> > > > > > > > > > > >>>> > > > > > Hey there,
> > > > > > > > > > > >>>> > > > > >
> > > > > > > > > > > >>>> > > > > > I'm re-opening this thread because after
> > > some
> > > > > > > initial
> > > > > > > > > > > >>>> > implementations
> > > > > > > > > > > >>>> > > > > > started, we spotted some gaps in the
> > > approved
> > > > > KIP
> > > > > > as
> > > > > > > > > well
> > > > > > > > > > as
> > > > > > > > > > > >>>> some
> > > > > > > > > > > >>>> > > > > > inconsistencies with KIP-631 controller.
> > > There
> > > > > > are a
> > > > > > > > > > couple
> > > > > > > > > > > of
> > > > > > > > > > > >>>> > > > addendums
> > > > > > > > > > > >>>> > > > > to
> > > > > > > > > > > >>>> > > > > > the existing KIP, specifically:
> > > > > > > > > > > >>>> > > > > >
> > > > > > > > > > > >>>> > > > > > 1. As the controller is foreseen to be
> > only
> > > > > > > accessible
> > > > > > > > > to
> > > > > > > > > > > the
> > > > > > > > > > > >>>> > > brokers,
> > > > > > > > > > > >>>> > > > > the
> > > > > > > > > > > >>>> > > > > > new admin client would not have direct
> > > access to
> > > > > > the
> > > > > > > > > > > >>>> controller. It
> > > > > > > > > > > >>>> > > is
> > > > > > > > > > > >>>> > > > > > guaranteed on the MetadataResponse level
> > > which
> > > > > no
> > > > > > > > longer
> > > > > > > > > > > >>>> provides
> > > > > > > > > > > >>>> > > > > > `ControllerId` to client side requests.
> > > > > > > > > > > >>>> > > > > >
> > > > > > > > > > > >>>> > > > > > 2. The broker would forward any direct
> ZK
> > > path
> > > > > > > > mutation
> > > > > > > > > > > >>>> requests,
> > > > > > > > > > > >>>> > > > > including
> > > > > > > > > > > >>>> > > > > > topic creation/deletion, reassignment,
> etc
> > > since
> > > > > > we
> > > > > > > > > > > deprecate
> > > > > > > > > > > >>>> the
> > > > > > > > > > > >>>> > > > direct
> > > > > > > > > > > >>>> > > > > > controller access on the client side. No
> > > more
> > > > > > > protocol
> > > > > > > > > > > >>>> version bump
> > > > > > > > > > > >>>> > > is
> > > > > > > > > > > >>>> > > > > > necessary for the configuration
> requests.
> > > > > > > > > > > >>>> > > > > >
> > > > > > > > > > > >>>> > > > > > 3. To make sure forwarding requests pass
> > the
> > > > > > > > > > authorization,
> > > > > > > > > > > >>>> broker
> > > > > > > > > > > >>>> > > > > > principal CLUSTER_ACTION would be
> allowed
> > > to be
> > > > > > used
> > > > > > > > as
> > > > > > > > > an
> > > > > > > > > > > >>>> > > alternative
> > > > > > > > > > > >>>> > > > > > authentication method for a variety of
> > > principal
> > > > > > > > > > operations,
> > > > > > > > > > > >>>> > > including
> > > > > > > > > > > >>>> > > > > > ALTER, ALTER_CONFIG, DELETE, etc. It is
> > > because
> > > > > > the
> > > > > > > > > > > forwarding
> > > > > > > > > > > >>>> > > request
> > > > > > > > > > > >>>> > > > > > needs to use the proxy broker's own
> > > principal,
> > > > > > which
> > > > > > > > is
> > > > > > > > > > > >>>> currently
> > > > > > > > > > > >>>> > not
> > > > > > > > > > > >>>> > > > > > supported to be used for many
> > configuration
> > > > > change
> > > > > > > > > > > >>>> authentication
> > > > > > > > > > > >>>> > > > listed
> > > > > > > > > > > >>>> > > > > > above. The full list could be found in
> the
> > > KIP.
> > > > > > > > > > > >>>> > > > > >
> > > > > > > > > > > >>>> > > > > > 4. Add a new
> BROKER_AUTHORIZATION_FAILURE
> > > error
> > > > > > code
> > > > > > > > to
> > > > > > > > > > > >>>> indicate
> > > > > > > > > > > >>>> > any
> > > > > > > > > > > >>>> > > > > > internal security configuration failure,
> > > when
> > > > > the
> > > > > > > > > > forwarded
> > > > > > > > > > > >>>> request
> > > > > > > > > > > >>>> > > > > failed
> > > > > > > > > > > >>>> > > > > > authentication on the controller side.
> > > > > > > > > > > >>>> > > > > >
> > > > > > > > > > > >>>> > > > > > Let me know what you think. With such a
> > > major
> > > > > > > > refinement
> > > > > > > > > > of
> > > > > > > > > > > >>>> the
> > > > > > > > > > > >>>> > KIP,
> > > > > > > > > > > >>>> > > > I'm
> > > > > > > > > > > >>>> > > > > > open for re-vote after discussions
> > converge.
> > > > > > > > > > > >>>> > > > > >
> > > > > > > > > > > >>>> > > > > > Boyang
> > > > > > > > > > > >>>> > > > > >
> > > > > > > > > > > >>>> > > > > > On Wed, Jul 1, 2020 at 2:17 PM Boyang
> > Chen <
> > > > > > > > > > > >>>> > > reluctanthero104@gmail.com
> > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > >>>> > > > > > wrote:
> > > > > > > > > > > >>>> > > > > >
> > > > > > > > > > > >>>> > > > > > > Hey folks,
> > > > > > > > > > > >>>> > > > > > >
> > > > > > > > > > > >>>> > > > > > > I have also synced on the KIP-578
> which
> > > was
> > > > > > doing
> > > > > > > > the
> > > > > > > > > > > >>>> partition
> > > > > > > > > > > >>>> > > > limit,
> > > > > > > > > > > >>>> > > > > to
> > > > > > > > > > > >>>> > > > > > > make sure the partition limit error
> code
> > > would
> > > > > > be
> > > > > > > > > > properly
> > > > > > > > > > > >>>> > > propagated
> > > > > > > > > > > >>>> > > > > once
> > > > > > > > > > > >>>> > > > > > > it is done on top of KIP-590. Let me
> > know
> > > if
> > > > > you
> > > > > > > > have
> > > > > > > > > > > >>>> further
> > > > > > > > > > > >>>> > > > > questions or
> > > > > > > > > > > >>>> > > > > > > concerns.
> > > > > > > > > > > >>>> > > > > > >
> > > > > > > > > > > >>>> > > > > > > Boyang
> > > > > > > > > > > >>>> > > > > > >
> > > > > > > > > > > >>>> > > > > > > On Tue, Jun 23, 2020 at 5:08 PM Boyang
> > > Chen <
> > > > > > > > > > > >>>> > > > > reluctanthero104@gmail.com>
> > > > > > > > > > > >>>> > > > > > > wrote:
> > > > > > > > > > > >>>> > > > > > >
> > > > > > > > > > > >>>> > > > > > >> Thanks for the clarification, Colin
> and
> > > > > Ismael.
> > > > > > > > > > > Personally
> > > > > > > > > > > >>>> I
> > > > > > > > > > > >>>> > also
> > > > > > > > > > > >>>> > > > feel
> > > > > > > > > > > >>>> > > > > > >> Option A is better to prioritize
> fixing
> > > the
> > > > > > gap.
> > > > > > > > Just
> > > > > > > > > > to
> > > > > > > > > > > be
> > > > > > > > > > > >>>> > clear,
> > > > > > > > > > > >>>> > > > the
> > > > > > > > > > > >>>> > > > > > >> proposed solution would be:
> > > > > > > > > > > >>>> > > > > > >>
> > > > > > > > > > > >>>> > > > > > >> 1. Bump the Metadata RPC version to
> > > return
> > > > > > > > > > > >>>> POLICY_VIOLATION. In
> > > > > > > > > > > >>>> > > the
> > > > > > > > > > > >>>> > > > > > >> application level, we should swap the
> > > error
> > > > > > > message
> > > > > > > > > > with
> > > > > > > > > > > >>>> the
> > > > > > > > > > > >>>> > > actual
> > > > > > > > > > > >>>> > > > > failure
> > > > > > > > > > > >>>> > > > > > >> reason such as "violation of topic
> > > creation
> > > > > > > policy
> > > > > > > > > when
> > > > > > > > > > > >>>> > attempting
> > > > > > > > > > > >>>> > > > to
> > > > > > > > > > > >>>> > > > > auto
> > > > > > > > > > > >>>> > > > > > >> create internal topic through
> > > > > MetadataRequest."
> > > > > > > > > > > >>>> > > > > > >>
> > > > > > > > > > > >>>> > > > > > >> 2. For older Metadata RPC, return
> > > > > > > > > AUTHORIZATION_FAILED
> > > > > > > > > > to
> > > > > > > > > > > >>>> fail
> > > > > > > > > > > >>>> > > fast.
> > > > > > > > > > > >>>> > > > > > >>
> > > > > > > > > > > >>>> > > > > > >> Will address our other discussed
> points
> > > as
> > > > > well
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > > >>>> KIP, let
> > > > > > > > > > > >>>> > me
> > > > > > > > > > > >>>> > > > > know
> > > > > > > > > > > >>>> > > > > > >> if you have further questions.
> > > > > > > > > > > >>>> > > > > > >>
> > > > > > > > > > > >>>> > > > > > >> Thanks,
> > > > > > > > > > > >>>> > > > > > >> Boyang
> > > > > > > > > > > >>>> > > > > > >>
> > > > > > > > > > > >>>> > > > > > >> On Tue, Jun 23, 2020 at 10:41 AM
> Ismael
> > > Juma
> > > > > <
> > > > > > > > > > > >>>> ismael@juma.me.uk
> > > > > > > > > > > >>>> > >
> > > > > > > > > > > >>>> > > > > wrote:
> > > > > > > > > > > >>>> > > > > > >>
> > > > > > > > > > > >>>> > > > > > >>> Option A is basically what I was
> > > thinking.
> > > > > But
> > > > > > > > with
> > > > > > > > > a
> > > > > > > > > > > >>>> slight
> > > > > > > > > > > >>>> > > > > adjustment:
> > > > > > > > > > > >>>> > > > > > >>>
> > > > > > > > > > > >>>> > > > > > >>> New versions of MetadataResponse
> > return
> > > > > > > > > > > POLICY_VIOLATION,
> > > > > > > > > > > >>>> old
> > > > > > > > > > > >>>> > > > > versions
> > > > > > > > > > > >>>> > > > > > >>> return AUTHORIZATION_FAILED. The
> > latter
> > > > > works
> > > > > > > > > > correctly
> > > > > > > > > > > >>>> with
> > > > > > > > > > > >>>> > old
> > > > > > > > > > > >>>> > > > Java
> > > > > > > > > > > >>>> > > > > > >>> clients (i.e. the client fails fast
> > and
> > > > > > > propagates
> > > > > > > > > the
> > > > > > > > > > > >>>> error),
> > > > > > > > > > > >>>> > > I've
> > > > > > > > > > > >>>> > > > > > >>> tested
> > > > > > > > > > > >>>> > > > > > >>> it. Adjust new clients to treat
> > > > > > POLICY_VIOLATION
> > > > > > > > > like
> > > > > > > > > > > >>>> > > > > > >>> AUTHORIZATION_FAILED,
> > > > > > > > > > > >>>> > > > > > >>> but propagate the custom error
> > message.
> > > > > > > > > > > >>>> > > > > > >>>
> > > > > > > > > > > >>>> > > > > > >>> Ismael
> > > > > > > > > > > >>>> > > > > > >>>
> > > > > > > > > > > >>>> > > > > > >>> On Mon, Jun 22, 2020 at 11:00 PM
> Colin
> > > > > McCabe
> > > > > > <
> > > > > > > > > > > >>>> > > cmccabe@apache.org>
> > > > > > > > > > > >>>> > > > > > >>> wrote:
> > > > > > > > > > > >>>> > > > > > >>>
> > > > > > > > > > > >>>> > > > > > >>> > > > > On Fri, Jun 19, 2020 at 3:18
> > PM
> > > > > Ismael
> > > > > > > > Juma
> > > > > > > > > <
> > > > > > > > > > > >>>> > > > > ismael@juma.me.uk>
> > > > > > > > > > > >>>> > > > > > >>> > wrote:
> > > > > > > > > > > >>>> > > > > > >>> > > > >
> > > > > > > > > > > >>>> > > > > > >>> > > > > > Hi Colin,
> > > > > > > > > > > >>>> > > > > > >>> > > > > >
> > > > > > > > > > > >>>> > > > > > >>> > > > > > The KIP states in the
> > > > > Compatibility
> > > > > > > > > section
> > > > > > > > > > > (not
> > > > > > > > > > > >>>> > Future
> > > > > > > > > > > >>>> > > > > work):
> > > > > > > > > > > >>>> > > > > > >>> > > > > >
> > > > > > > > > > > >>>> > > > > > >>> > > > > > "To support the proxy of
> > > requests,
> > > > > > we
> > > > > > > > need
> > > > > > > > > > to
> > > > > > > > > > > >>>> build a
> > > > > > > > > > > >>>> > > > > channel
> > > > > > > > > > > >>>> > > > > > >>> for
> > > > > > > > > > > >>>> > > > > > >>> > > > > > brokers to talk directly
> to
> > > the
> > > > > > > > > controller.
> > > > > > > > > > > >>>> This part
> > > > > > > > > > > >>>> > > of
> > > > > > > > > > > >>>> > > > > the
> > > > > > > > > > > >>>> > > > > > >>> design
> > > > > > > > > > > >>>> > > > > > >>> > > > > > is internal change only
> and
> > > won’t
> > > > > > > block
> > > > > > > > > the
> > > > > > > > > > > KIP
> > > > > > > > > > > >>>> > > > progress."
> > > > > > > > > > > >>>> > > > > > >>> > > > > >
> > > > > > > > > > > >>>> > > > > > >>> > > > > > I am clarifying that this
> is
> > > not
> > > > > > > > internal
> > > > > > > > > > only
> > > > > > > > > > > >>>> due to
> > > > > > > > > > > >>>> > > the
> > > > > > > > > > > >>>> > > > > > >>> config.
> > > > > > > > > > > >>>> > > > > > >>> > If we
> > > > > > > > > > > >>>> > > > > > >>> > > > > > say that this KIP depends
> on
> > > > > another
> > > > > > > KIP
> > > > > > > > > > > before
> > > > > > > > > > > >>>> we
> > > > > > > > > > > >>>> > can
> > > > > > > > > > > >>>> > > > > merge
> > > > > > > > > > > >>>> > > > > > >>> > > > > > it, that's fine although
> it
> > > feels
> > > > > a
> > > > > > > bit
> > > > > > > > > > > >>>> unnecessary.
> > > > > > > > > > > >>>> > > > > > >>> > > > > >
> > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > >>>> > > > > > >>> > Hi Ismael,
> > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > >>>> > > > > > >>> > I didn't realize there was still a
> > > > > reference
> > > > > > > to
> > > > > > > > > the
> > > > > > > > > > > >>>> separate
> > > > > > > > > > > >>>> > > > > controller
> > > > > > > > > > > >>>> > > > > > >>> > channel in the "Compatibility,
> > > > > Deprecation,
> > > > > > > and
> > > > > > > > > > > >>>> Migration
> > > > > > > > > > > >>>> > Plan"
> > > > > > > > > > > >>>> > > > > > >>> section.  I
> > > > > > > > > > > >>>> > > > > > >>> > agree that it doesn't really
> belong
> > > there.
> > > > > > > > Given
> > > > > > > > > > that
> > > > > > > > > > > >>>> this
> > > > > > > > > > > >>>> > is
> > > > > > > > > > > >>>> > > > > creating
> > > > > > > > > > > >>>> > > > > > >>> > confusion, I would suggest that we
> > > just
> > > > > drop
> > > > > > > > this
> > > > > > > > > > from
> > > > > > > > > > > >>>> the
> > > > > > > > > > > >>>> > KIP
> > > > > > > > > > > >>>> > > > > > >>> entirely.
> > > > > > > > > > > >>>> > > > > > >>> > It really is orthogonal to what
> this
> > > KIP
> > > > > is
> > > > > > > > > about--
> > > > > > > > > > we
> > > > > > > > > > > >>>> don't
> > > > > > > > > > > >>>> > > > need a
> > > > > > > > > > > >>>> > > > > > >>> > separate channel to implement
> > > redirection.
> > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > >>>> > > > > > >>> > Boyang wrote:
> > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > >>>> > > > > > >>> > >
> > > > > > > > > > > >>>> > > > > > >>> > > We are only opening the doors
> for
> > > > > specific
> > > > > > > > > > internal
> > > > > > > > > > > >>>> topics
> > > > > > > > > > > >>>> > > > > (offsets,
> > > > > > > > > > > >>>> > > > > > >>> txn
> > > > > > > > > > > >>>> > > > > > >>> > > log), which I assume the client
> > > should
> > > > > > have
> > > > > > > no
> > > > > > > > > > > >>>> possibility
> > > > > > > > > > > >>>> > to
> > > > > > > > > > > >>>> > > > > mutate
> > > > > > > > > > > >>>> > > > > > >>> the
> > > > > > > > > > > >>>> > > > > > >>> > > topic policy?
> > > > > > > > > > > >>>> > > > > > >>> > >
> > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > >>>> > > > > > >>> > Hi Boyang,
> > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > >>>> > > > > > >>> > I think you and Ismael are talking
> > > about
> > > > > > > > different
> > > > > > > > > > > >>>> scenarios.
> > > > > > > > > > > >>>> > > > You
> > > > > > > > > > > >>>> > > > > are
> > > > > > > > > > > >>>> > > > > > >>> > describing the scenario where the
> > > broker
> > > > > is
> > > > > > > > > > > >>>> auto-creating the
> > > > > > > > > > > >>>> > > > > > >>> transaction
> > > > > > > > > > > >>>> > > > > > >>> > log topic or consumer offset
> topic.
> > > This
> > > > > > > > scenario
> > > > > > > > > > > >>>> indeed
> > > > > > > > > > > >>>> > > should
> > > > > > > > > > > >>>> > > > > not
> > > > > > > > > > > >>>> > > > > > >>> happen
> > > > > > > > > > > >>>> > > > > > >>> > in a properly-configured cluster.
> > > > > However,
> > > > > > > > Ismael
> > > > > > > > > > is
> > > > > > > > > > > >>>> > > describing
> > > > > > > > > > > >>>> > > > a
> > > > > > > > > > > >>>> > > > > > >>> scenario
> > > > > > > > > > > >>>> > > > > > >>> > where the client is auto-creating
> > some
> > > > > > > arbitrary
> > > > > > > > > > > >>>> non-internal
> > > > > > > > > > > >>>> > > > topic
> > > > > > > > > > > >>>> > > > > > >>> just by
> > > > > > > > > > > >>>> > > > > > >>> > sending a metadata request.
> > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > >>>> > > > > > >>> > As far as I can see, there are two
> > > > > solutions
> > > > > > > > here:
> > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > >>>> > > > > > >>> > A. Close the hole in
> > > CreateTopicsPolicy
> > > > > > > > > immediately.
> > > > > > > > > > > >>>> In new
> > > > > > > > > > > >>>> > > > > versions,
> > > > > > > > > > > >>>> > > > > > >>> > allow MetadataResponse to return
> > > > > > > > > > AUTHORIZATION_FAILED
> > > > > > > > > > > >>>> if we
> > > > > > > > > > > >>>> > > tried
> > > > > > > > > > > >>>> > > > > to
> > > > > > > > > > > >>>> > > > > > >>> > auto-create a topic and failed.
> > Find
> > > some
> > > > > > > other
> > > > > > > > > > error
> > > > > > > > > > > >>>> code
> > > > > > > > > > > >>>> > to
> > > > > > > > > > > >>>> > > > > return
> > > > > > > > > > > >>>> > > > > > >>> for
> > > > > > > > > > > >>>> > > > > > >>> > existing versions.
> > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > >>>> > > > > > >>> > B. Keep the hole in
> > > CreateTopicsPolicy and
> > > > > > add
> > > > > > > > > some
> > > > > > > > > > > >>>> > > configuration
> > > > > > > > > > > >>>> > > > > to
> > > > > > > > > > > >>>> > > > > > >>> allow
> > > > > > > > > > > >>>> > > > > > >>> > admins to gradually migrate to
> > > closing it.
> > > > > > In
> > > > > > > > > > > >>>> practice, this
> > > > > > > > > > > >>>> > > > > probably
> > > > > > > > > > > >>>> > > > > > >>> > means a configuration toggle that
> > > enables
> > > > > > > direct
> > > > > > > > > ZK
> > > > > > > > > > > >>>> access,
> > > > > > > > > > > >>>> > > that
> > > > > > > > > > > >>>> > > > > > >>> starts off
> > > > > > > > > > > >>>> > > > > > >>> > as enabled.  Then we can
> eventually
> > > > > default
> > > > > > it
> > > > > > > > to
> > > > > > > > > > > false
> > > > > > > > > > > >>>> and
> > > > > > > > > > > >>>> > > then
> > > > > > > > > > > >>>> > > > > > >>> remove it
> > > > > > > > > > > >>>> > > > > > >>> > entirely over time.
> > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > >>>> > > > > > >>> > best,
> > > > > > > > > > > >>>> > > > > > >>> > Colin
> > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > >>>> > > > > > >>>
> > > > > > > > > > > >>>> > > > > > >>
> > > > > > > > > > > >>>> > > > > >
> > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > >>>> > > >
> > > > > > > > > > > >>>> > >
> > > > > > > > > > > >>>> >
> > > > > > > > > > > >>>>
> > > > > > > > > > > >>>
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

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

Posted by Colin McCabe <cm...@apache.org>.
Agreed. Flexible fields are the way to go here, I think.

Thanks for the discussion, all.

best,
Colin


On Tue, Jul 20, 2021, at 07:18, Ismael Juma wrote:
> Hi Ron,
> 
> Thanks for bringing this up. Thinking about it, the combination of flexible
> fields and the principal type field gives us enough flexibility that we
> don't need a magic number.
> 
> Ismael
> 
> P.S. For a magic number to be useful for third party implementations, we
> would need a mechanism to coordinate what each number means, so it's a bit
> complicated to do it well.
> 
> On Tue, Jul 13, 2021 at 9:48 AM Ron Dagostino <rn...@gmail.com> wrote:
> 
> > Hi everyone.  I know it has been 9 months since the last message appeared
> > on this vote thread, but a potential oversight exists in the implementation
> > of DefaultKafkaPrincipalBuilder.KafkaPrincipalSerde from
> > https://github.com/apache/kafka/pull/9103.  Specifically, there is no
> > magic
> > number at the top of the wire format, and this causes obscure parsing
> > errors if incompatible principal implementations are mixed.  A magic number
> > at the top would allow deserialization code to provide an intuitive error
> > message.  The current implementation (without a magic number) was released
> > in 2.8, but it presumably has never been used since forwarding is disabled
> > (
> >
> > https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/Kafka.scala#L73
> > ).
> > We would like to change the implementation to include a magic number at the
> > top for the 3.0 release.  This would be a breaking change, but again this
> > presumably has never been used anywhere in production and would therefore
> > break nothing.
> >
> > Note that forwarding is always enabled for KRaft-based clusters, but such
> > clusters are not supported in any production sense and there is no upgrade
> > path from a 2.8 KRaft cluster to a 3.0 KRaft cluster (from
> > config/kraft/README.md: "KRaft mode in Kafka 2.8 is provided for testing
> > only, NOT for production. We do not yet support upgrading existing
> > ZooKeeper-based Kafka clusters into this mode. In fact, when Kafka 3.0 is
> > released, it will not be possible to upgrade your KRaft clusters from 2.8
> > to 3.0").
> >
> > A PR to add the magic number appears at
> > https://github.com/apache/kafka/pull/11038.
> >
> > Please respond to this thread if you have any concerns or objections.
> >
> > Thanks,
> >
> > Ron
> >
> > On Fri, Oct 9, 2020 at 1:21 PM Boyang Chen <re...@gmail.com>
> > wrote:
> >
> > > Thanks Jason for the great thoughts, and we basically decided to shift
> > the
> > > gear for a limited impersonation approach offline.
> > >
> > > The goal here is to simplify the handling logic by relying on the active
> > > controller to do the actual authorization for resources in the original
> > > client request. We are also adding the `KafkaPrincipalSerde` type to
> > > provide the functionality for principal serialization/deserialization so
> > > that it could embed in the Envelope and send to the active controller.
> > > Before 3.0, customized principal builders could optionally extend the
> > serde
> > > type, which is required after 3.0 is released. Either way having the
> > > capability to serde KafkaPrincipal becomes a prerequisite to enable
> > > redirection besides IBP. Additionally, we add a forwardingPrincipal field
> > > to the Authorizer context for authorization and audit logging purposes,
> > > instead of going tagged fields in header.
> > >
> > > The KIP is updated to reflect the current approach, thanks.
> > >
> > >
> > >
> > > On Fri, Sep 25, 2020 at 5:55 PM Jason Gustafson <ja...@confluent.io>
> > > wrote:
> > >
> > > > Hey All,
> > > >
> > > > So the main thing the EnvelopeRequest gives us is a way to avoid
> > > converting
> > > > older API versions in order to attach the initial principal name and
> > the
> > > > clientId. It also saves the need to add the initial principal and
> > client
> > > id
> > > > as a tagged field to all of the forwarded protocols, which is nice. We
> > > > still have the challenge of advertising API versions which are
> > compatible
> > > > with both the broker receiving the request and the controller that the
> > > > request is ultimately forwarded to, but not sure I see a way around
> > that.
> > > >
> > > > I realize I might be walking into a minefield here, but since the
> > > envelope
> > > > is being revisited, it seems useful to compare the approach suggested
> > > above
> > > > with the option relying on impersonation. I favor the use of
> > > impersonation
> > > > because it makes forwarding simpler. As the proposal stands, we will
> > have
> > > > to maintain logic for each forwarded API to unpack, authorize, and
> > repack
> > > > any forwarded requests which flow through the broker. This is probably
> > > not
> > > > a huge concern from an efficiency perspective as long as we are talking
> > > > about just the Admin APIs, but it does have a big maintenance cost
> > since
> > > > we'll need to ensure that every new field gets properly carried
> > through.
> > > It
> > > > would be nice if we just didn't have to think about that. We also might
> > > > eventually come up with reasons to extend forwarding to non-admin APIs,
> > > so
> > > > it would be nice to start with an efficient approach.
> > > >
> > > > It seems like the main difference comes down to where the authorization
> > > is
> > > > done. Suppose that broker B receives an AlterConfig request from the
> > > client
> > > > in order to change topic configs and wants to forward to controller C.
> > > >
> > > > Option 1 (no impersonation): B authorizes AlterConfigs for the included
> > > > topics with the client principal. Rejected topics are stripped out of
> > the
> > > > request.  Authorized topics are repackaged into a new request and sent
> > in
> > > > an envelope to C, which verifies ClusterAction permission with the
> > broker
> > > > principal and assumes authorization for the underlying request
> > > > Option 2 (with impersonation): B authenticates the client, but does no
> > > > authorization and forwards the request in an envelope to C containing
> > the
> > > > authenticated principal. C checks ClusterAction for the envelope
> > request
> > > > using the broker principal and AlterConfigs for the underlying request
> > > > using the forwarded client principal.
> > > >
> > > > In either case, broker B implicitly gets AlterConfigs permission for
> > the
> > > > topic. This is true even without the envelope and seems like a
> > reasonable
> > > > requirement. The broker should itself be authorized to perform any
> > action
> > > > that it might have to forward requests for. As far as I know, all the
> > > > proposals we have considered require this. The main question from a
> > > > security perspective is whether any of these proposals require
> > additional
> > > > unnecessary access, which is probably the main doubt about
> > impersonation.
> > > > However, there are a couple ways we can restrict it:
> > > >
> > > > 1. We can restrict the principals that are allowed to be impersonated
> > > > 2. We can restrict the actions that are possible through impersonation.
> > > >
> > > > Considering the first point, there's probably no reason to allow
> > > > impersonation of superusers. Additionally, a custom authorizer could
> > > forbid
> > > > impersonation outside of a particular group. To support this, it would
> > be
> > > > helpful to extend `KafkaPrincipal` or `AuthorizableRequestContext` so
> > > that
> > > > it indicates whether a request is an impersonated request.
> > > >
> > > > Considering the second point, it doesn't make sense to allow arbitrary
> > > > requests to be forwarded. We know exactly the set of forwardable APIs
> > and
> > > > we can reject any other APIs without even looking at the principal.
> > This
> > > is
> > > > the nice thing that the Envelope request gives us. I don't know if we
> > > would
> > > > ever have finer-grained restrictions, but in principle I don't see why
> > we
> > > > couldn't.
> > > >
> > > > In the future, I guess we could go even further so that the broker
> > itself
> > > > wouldn't need the same permissions as the client. If the client and the
> > > > controller shared some secret or if the client had a public key that we
> > > > could rely on, then the client could send along a MAC or token of some
> > > > sort, which could then be forwarded through the envelope. Then the
> > broker
> > > > would not be allowed to do anything except exactly what the client
> > > > requested. I'm not suggesting we do this, just that we will have the
> > > > flexibility for it.
> > > >
> > > > From the discussion thread, it looks like the main problem here is that
> > > > `KafkaPrincipal` does not currently have a defined serialization
> > > mechanism.
> > > > We can add this, but it is a breaking change, so we have to wait for a
> > > > major release version before we make it a requirement. This is fine
> > from
> > > > the perspective of KIP-500 since the bridge release will require a
> > major
> > > > release bump anyway. What I imagine we could do is something like this:
> > > >
> > > > 1. In versions 2.7/2.8/.., we can add the new API in a mixin interface,
> > > say
> > > > `KafkaPrincipalSerde`. We can implement this interface for
> > > > `DefaultKafkaPrincipalBuilder` trivially. On startup, we can check if
> > > this
> > > > interface is implemented by the provided `KafkaPrincipalBuilder`. If it
> > > is
> > > > not, we can log a warning and revert to non-forwarding mode.
> > > > 2. Starting in version 3.0, we can make `KafkaPrincipalSerde` a
> > > requirement
> > > > and fail startup if it is not implemented.
> > > >
> > > > I think this approach would give users enough time to add the required
> > > > support. In any case, I don't think compatibility should be a blocker
> > > here
> > > > if we think it's the best way forward. This seems like a decision which
> > > > will affect things for a long time and we need to be allowed to change
> > > > plugin APIs.
> > > >
> > > > Anyway, let me know what you think.
> > > >
> > > > Best,
> > > > Jason
> > > >
> > > >
> > > >
> > > > On Fri, Sep 25, 2020 at 2:21 PM Colin McCabe <cm...@apache.org>
> > wrote:
> > > >
> > > > > On Fri, Sep 25, 2020, at 10:49, Boyang Chen wrote:
> > > > > > Hey Jun,
> > > > > >
> > > > > > On Fri, Sep 25, 2020 at 10:19 AM Jun Rao <ju...@confluent.io> wrote:
> > > > > >
> > > > > > > Hi, Boyang,
> > > > > > >
> > > > > > > Does EnvelopeRequest avoid the need for IBP? How do we know if
> > the
> > > > > > > controller supports EnvelopeRequest or not?
> > > > > > >
> > > > > > > Unfortunately, the EnvelopeRequest is solving the inter-broker
> > > > > > > communication problem only. Admin clients still need to learn the
> > > > > proper
> > > > > > > ApiVersion from the broker, which means we need to bump IBP to
> > > limit
> > > > > the
> > > > > > > version range.
> > > > > >
> > > > >
> > > > > Right-- the purpose of EnvelopeRequest is to avoid downconversion /
> > > > > upconversion on the forwarding broker.  It unfortunately doesn't
> > avoid
> > > > the
> > > > > need to tie ApiVersionsResponse to IBP.
> > > > >
> > > > > > > > On Thu, Sep 24, 2020 at 4:53 PM Jun Rao <ju...@confluent.io>
> > > wrote:
> > > > > > > >
> > > > > > > > > Hi, Jason,
> > > > > > > > >
> > > > > > > > > Yes, the most important thing is to be able to avoid two
> > > rolling
> > > > > > > > > restarts
> > > > > > > > > in the future. If we have a path to achieve that down the
> > road,
> > > > the
> > > > > > > > > changes here are fine.
> > > > > > > > >
> > > > >
> > > > > Yeah.  I think it would be good to make IBP a feature flag, as long
> > as
> > > it
> > > > > could be changed without doing a second rolling restart.  We actually
> > > > don't
> > > > > want to have too many feature flags, since it blows up the test
> > matrix.
> > > > >
> > > > > best,
> > > > > Colin
> > > > >
> > > > > > > > > Thanks,
> > > > > > > > >
> > > > > > > > > Jun
> > > > > > > > >
> > > > > > > > > On Thu, Sep 24, 2020 at 3:20 PM Jason Gustafson <
> > > > > jason@confluent.io>
> > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > > One of the goals of KIP-584 (feature versioning) is that
> > we
> > > > > can get
> > > > > > > > rid
> > > > > > > > > > of
> > > > > > > > > > IBP in the future. So does this change prevent us from
> > > removing
> > > > > IBP
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > future?
> > > > > > > > > >
> > > > > > > > > > That is a good question. I think the problem here is that
> > > > request
> > > > > > > > > > forwarding puts an expectation on api version support which
> > > > > covers
> > > > > > > more
> > > > > > > > > > than one broker. This is why the normal ApiVersions
> > behavior
> > > > > doesn't
> > > > > > > > > work.
> > > > > > > > > > I thought about this a bit and haven't come up with a good
> > > > > > > alternative.
> > > > > > > > > One
> > > > > > > > > > thought I've been considering is letting the controller in
> > > the
> > > > > > > > > post-kip-500
> > > > > > > > > > world set the maximum range of api support for the cluster.
> > > > > However,
> > > > > > > > even
> > > > > > > > > > then we would need some way to tell when the controller
> > > quorum
> > > > > itself
> > > > > > > > is
> > > > > > > > > > ready to enable support for a new api version. My feeling
> > is
> > > > > that we
> > > > > > > > will
> > > > > > > > > > probably always need something like the IBP to control when
> > > it
> > > > is
> > > > > > > safe
> > > > > > > > to
> > > > > > > > > > expose versions of APIs which have a cross-broker
> > dependence.
> > > > > > > However,
> > > > > > > > > > KIP-584 would still allow us to manage the IBP at the level
> > > of
> > > > a
> > > > > > > > feature
> > > > > > > > > so
> > > > > > > > > > that we don't need two rolling restarts anymore.
> > > > > > > > > >
> > > > > > > > > > Best,
> > > > > > > > > > Jason
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Thu, Sep 24, 2020 at 1:59 PM Jun Rao <ju...@confluent.io>
> > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hi, Boyang,
> > > > > > > > > > >
> > > > > > > > > > > One of the goals of KIP-584 (feature versioning) is that
> > we
> > > > > can get
> > > > > > > > rid
> > > > > > > > > > of
> > > > > > > > > > > IBP in the future. So does this change prevent us from
> > > > > removing IBP
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > future?
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > >
> > > > > > > > > > > Jun
> > > > > > > > > > >
> > > > > > > > > > > On Thu, Sep 24, 2020 at 12:46 PM Jason Gustafson <
> > > > > > > jason@confluent.io
> > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Hey Boyang,
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks for the update. This seems like the best thing
> > we
> > > > can
> > > > > do.
> > > > > > > > The
> > > > > > > > > > > > alternative would be to always ensure that the
> > forwarded
> > > > > APIs are
> > > > > > > > > safe
> > > > > > > > > > > for
> > > > > > > > > > > > conversion between versions, but that would restrict
> > the
> > > > > > > > flexibility
> > > > > > > > > > that
> > > > > > > > > > > > the versioning is providing. It would also be a large
> > > > effort
> > > > > to
> > > > > > > > avoid
> > > > > > > > > > > > introducing regressions through conversion. Sadly this
> > > > > broadens
> > > > > > > the
> > > > > > > > > > scope
> > > > > > > > > > > > of the IBP, but in fact forwarded APIs are inter-broker
> > > > APIs.
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > > Jason
> > > > > > > > > > > >
> > > > > > > > > > > > On Thu, Sep 24, 2020 at 9:23 AM Boyang Chen <
> > > > > > > > > > reluctanthero104@gmail.com>
> > > > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Hey there,
> > > > > > > > > > > > >
> > > > > > > > > > > > > we spotted a necessary case to handle the redirect
> > > > request
> > > > > > > > > > versioning,
> > > > > > > > > > > > and
> > > > > > > > > > > > > proposed the following changes:
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1. For redirection RPCs (AlterConfig, Acl, Token
> > etc),
> > > > the
> > > > > > > > > > > corresponding
> > > > > > > > > > > > > allowed versions in the ApiVersionResponse will be
> > > > > affected by
> > > > > > > > the
> > > > > > > > > > > entire
> > > > > > > > > > > > > cluster's versioning, not just the receiving broker,
> > > > since
> > > > > we
> > > > > > > > need
> > > > > > > > > to
> > > > > > > > > > > > > ensure the chosen version get properly handled by all
> > > > > parties.
> > > > > > > > Thus
> > > > > > > > > > > from
> > > > > > > > > > > > > now on, RPC with redirection will be treated as
> > > > > inter-broker
> > > > > > > RPC,
> > > > > > > > > and
> > > > > > > > > > > any
> > > > > > > > > > > > > version bump for these RPCs has to go through IBP
> > bump
> > > as
> > > > > well.
> > > > > > > > > > > > > ApiVersionResponse will take IBP into considerations
> > > for
> > > > > the
> > > > > > > > > > > redirection
> > > > > > > > > > > > > RPCs allowable versions.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 2. We would do the best effort to maintain the same
> > > > request
> > > > > > > > version
> > > > > > > > > > for
> > > > > > > > > > > > > the entire admin client -> receiving broker ->
> > > controller
> > > > > > > broker
> > > > > > > > > > path,
> > > > > > > > > > > > but
> > > > > > > > > > > > > for old RPC versions, they may not have flexible
> > fields
> > > > > > > > introduced
> > > > > > > > > > yet.
> > > > > > > > > > > > > Thus, we would have to upgrade the RPC to the minimum
> > > > > version
> > > > > > > > which
> > > > > > > > > > > > > supports flexible fields
> > > > > > > > > > > > > and add another tagged field in the header called
> > > > > > > > > > > > `OriginalRequestVersion`
> > > > > > > > > > > > > to help the controller broker correctly deserialize
> > the
> > > > > request
> > > > > > > > > with
> > > > > > > > > > > the
> > > > > > > > > > > > > original admin client sent out version. We would not
> > > > > downgrade
> > > > > > > > the
> > > > > > > > > > > > original
> > > > > > > > > > > > > request in any circumstance, since the flexible field
> > > > > support
> > > > > > > is
> > > > > > > > > > > required
> > > > > > > > > > > > > to be open-ended on the high side.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Let me know if you have any questions.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Best,
> > > > > > > > > > > > > Boyang
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Thu, Aug 6, 2020 at 6:11 PM Boyang Chen <
> > > > > > > > > > reluctanthero104@gmail.com
> > > > > > > > > > > >
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Hey there,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > we are going to introduce a minor change to bump
> > the
> > > > > version
> > > > > > > of
> > > > > > > > > > > several
> > > > > > > > > > > > > > RPCs which are currently not supporting flexible
> > > > > versions. It
> > > > > > > > is
> > > > > > > > > > > > > necessary
> > > > > > > > > > > > > > because they need to be able to construct request
> > > > header
> > > > > with
> > > > > > > > > > initial
> > > > > > > > > > > > > > principal name and client id as optional fields for
> > > > > > > > redirection.
> > > > > > > > > > The
> > > > > > > > > > > > are
> > > > > > > > > > > > > > only two of them:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1. AlterConfig
> > > > > > > > > > > > > > 2. AlterClientQuotas
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Let me know if you have any questions.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Boyang
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Fri, Jul 31, 2020 at 11:42 AM Boyang Chen <
> > > > > > > > > > > > reluctanthero104@gmail.com
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >> Hey David,
> > > > > > > > > > > > > >>
> > > > > > > > > > > > > >> After discussing with Colin offline, I would like
> > to
> > > > > correct
> > > > > > > > one
> > > > > > > > > > > case
> > > > > > > > > > > > in
> > > > > > > > > > > > > >> the described workflow, where the CLUSTER_ACTION
> > > > > > > authorization
> > > > > > > > > > would
> > > > > > > > > > > > > not be
> > > > > > > > > > > > > >> based on the initial principal field check,
> > because
> > > it
> > > > > is
> > > > > > > not
> > > > > > > > a
> > > > > > > > > > > > secured
> > > > > > > > > > > > > >> condition which anyone could forge. The revised
> > > > workflow
> > > > > > > shall
> > > > > > > > > be:
> > > > > > > > > > > > > >>
> > > > > > > > > > > > > >> Step 1. Filter out resources that are authorized
> > > > > > > > > > > > > >>          1.1 Use traditional principals to verify
> > > > > first. If
> > > > > > > > > > > > authorized,
> > > > > > > > > > > > > >> continue
> > > > > > > > > > > > > >>          1.2 If not authorized, check whether the
> > > > > request is
> > > > > > > > > from
> > > > > > > > > > > the
> > > > > > > > > > > > > >> control plane. Note that this is a best-effort to
> > > > verify
> > > > > > > > whether
> > > > > > > > > > the
> > > > > > > > > > > > > >> request is internal.
> > > > > > > > > > > > > >>          1.3 If the request is not from the
> > control
> > > > > plane,
> > > > > > > > > return
> > > > > > > > > > > > > >> authorization failure
> > > > > > > > > > > > > >>          1.4 If the request is from the control
> > > plane,
> > > > > use
> > > > > > > > > > > > > CLUSTER_ACTION
> > > > > > > > > > > > > >> to verify and determine the result
> > > > > > > > > > > > > >>
> > > > > > > > > > > > > >> Step 2. Check the request context to see if this
> > is
> > > a
> > > > > > > > forwarding
> > > > > > > > > > > > > request,
> > > > > > > > > > > > > >> by checking whether it is from control plane and
> > > uses
> > > > > extra
> > > > > > > > > header
> > > > > > > > > > > > > fields
> > > > > > > > > > > > > >>         2.1 if the resource is authorized, and if
> > > this
> > > > > is
> > > > > > > the
> > > > > > > > > > active
> > > > > > > > > > > > > >> controller, process it
> > > > > > > > > > > > > >>         2.2 if the resource is authorized but this
> > > is
> > > > > not
> > > > > > > the
> > > > > > > > > > active
> > > > > > > > > > > > > >> controller, return NOT_CONTROLLER to the sender
> > > > > (forwarding
> > > > > > > > > > broker)
> > > > > > > > > > > > for
> > > > > > > > > > > > > >> retry
> > > > > > > > > > > > > >>         2.3 if the resource is not authorized,
> > > return
> > > > > > > > > > > > > >> CLUSTER_AUTHORIZATION_FAILURE to propagate back to
> > > the
> > > > > > > > original
> > > > > > > > > > > client
> > > > > > > > > > > > > >> through forwarding broker
> > > > > > > > > > > > > >> Step 3. If the request is not a forwarding request
> > > > > > > > > > > > > >>         3.1 If the resource is authorized, and
> > this
> > > is
> > > > > the
> > > > > > > > > active
> > > > > > > > > > > > > >> controller, process it
> > > > > > > > > > > > > >>         3.2 If the resource is authorized, but
> > this
> > > is
> > > > > not
> > > > > > > > > active
> > > > > > > > > > > > > >> controller, put the resource into the preparation
> > > for
> > > > a
> > > > > new
> > > > > > > > > > > > AlterConfig
> > > > > > > > > > > > > >> request for forwarding
> > > > > > > > > > > > > >>         3.3 If the resource is not authorized,
> > reply
> > > > the
> > > > > > > > > original
> > > > > > > > > > > > client
> > > > > > > > > > > > > >> AUTHORIZATION_FAILURE when the forwarding request
> > is
> > > > > > > returned
> > > > > > > > > > > > > >>
> > > > > > > > > > > > > >> On Thu, Jul 30, 2020 at 3:47 PM Boyang Chen <
> > > > > > > > > > > > reluctanthero104@gmail.com
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >> wrote:
> > > > > > > > > > > > > >>
> > > > > > > > > > > > > >>>
> > > > > > > > > > > > > >>>
> > > > > > > > > > > > > >>> On Thu, Jul 30, 2020 at 7:18 AM David Jacot <
> > > > > > > > > djacot@confluent.io
> > > > > > > > > > >
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > >>>
> > > > > > > > > > > > > >>>> Hi Boyang,
> > > > > > > > > > > > > >>>>
> > > > > > > > > > > > > >>>> Thanks for your answers.
> > > > > > > > > > > > > >>>>
> > > > > > > > > > > > > >>>> > The point for using the listener name is more
> > > of a
> > > > > > > > security
> > > > > > > > > > > > purpose,
> > > > > > > > > > > > > >>>> to
> > > > > > > > > > > > > >>>> > detect any forged request to our best effort.
> > > > > > > > > > > > > >>>> > For throttling I think we could just check the
> > > > > request
> > > > > > > > > header
> > > > > > > > > > > for
> > > > > > > > > > > > > >>>> > *InitialClientId* existence, to distinguish
> > > > whether
> > > > > to
> > > > > > > > apply
> > > > > > > > > > > > > >>>> > throttling strategy as forwarded request or
> > > direct
> > > > > > > > request.
> > > > > > > > > > > > > >>>>
> > > > > > > > > > > > > >>>> Reading "security" and "best effort" in the same
> > > > > sentence
> > > > > > > > > makes
> > > > > > > > > > > me a
> > > > > > > > > > > > > >>>> little nervous :).
> > > > > > > > > > > > > >>>>
> > > > > > > > > > > > > >>>> The identification issue is also valid for quota
> > > as
> > > > we
> > > > > > > don't
> > > > > > > > > > want
> > > > > > > > > > > > one
> > > > > > > > > > > > > >>>> to be
> > > > > > > > > > > > > >>>> able to bypass the quota by forging a request as
> > > > well,
> > > > > > > isn't
> > > > > > > > > it?
> > > > > > > > > > > > > >>>> Otherwise,
> > > > > > > > > > > > > >>>> anyone could just set the InitialPrincipal to
> > > bypass
> > > > > it. I
> > > > > > > > > think
> > > > > > > > > > > > that
> > > > > > > > > > > > > we
> > > > > > > > > > > > > >>>> should
> > > > > > > > > > > > > >>>> only use InitialPrincipal and/or InitialClientId
> > > > when
> > > > > we
> > > > > > > > know
> > > > > > > > > > that
> > > > > > > > > > > > > they
> > > > > > > > > > > > > >>>> come
> > > > > > > > > > > > > >>>> from another broker. Based on what I read in the
> > > > KIP,
> > > > > it
> > > > > > > > looks
> > > > > > > > > > > like
> > > > > > > > > > > > we
> > > > > > > > > > > > > >>>> could
> > > > > > > > > > > > > >>>> only use them when the principal has
> > > CLUSTER_ACTION
> > > > > > > > privilege.
> > > > > > > > > > Do
> > > > > > > > > > > I
> > > > > > > > > > > > > >>>> understand it correctly?
> > > > > > > > > > > > > >>>>
> > > > > > > > > > > > > >>>> There is no 100% safe way to distinguish between
> > > raw
> > > > > > > request
> > > > > > > > > and
> > > > > > > > > > > > > >>> forward request if you take malicious request
> > into
> > > > > > > > > consideration,
> > > > > > > > > > > > which
> > > > > > > > > > > > > >>> could happen
> > > > > > > > > > > > > >>> anyway if the attacker prepares those requests to
> > > > pass
> > > > > the
> > > > > > > > > checks
> > > > > > > > > > > > for a
> > > > > > > > > > > > > >>> pre-KIP-500 cluster.
> > > > > > > > > > > > > >>> We could at most know whether it is sent to the
> > > > control
> > > > > > > > plane,
> > > > > > > > > or
> > > > > > > > > > > the
> > > > > > > > > > > > > >>> data plane, and whether it has extra header
> > fields
> > > > > such as
> > > > > > > > > > initial
> > > > > > > > > > > > > >>> principal and client id defined. For a
> > > non-malicious
> > > > > > > request
> > > > > > > > > > > > > >>> going into the control plane, it must be sent
> > from
> > > a
> > > > > valid
> > > > > > > > > > broker,
> > > > > > > > > > > > > which
> > > > > > > > > > > > > >>> is a prerequisite to check its
> > > > > > > > > > > > > >>> CLUSTER_ACTION principal. Take AlterConfig for an
> > > > > example,
> > > > > > > > the
> > > > > > > > > > > > intended
> > > > > > > > > > > > > >>> workflow for a *KIP-590 broker* would be:
> > > > > > > > > > > > > >>>
> > > > > > > > > > > > > >>> Step 1. Check the request context to see if this
> > > is a
> > > > > > > > > forwarding
> > > > > > > > > > > > > >>> request, by checking whether it is from control
> > > plane
> > > > > and
> > > > > > > > uses
> > > > > > > > > > > extra
> > > > > > > > > > > > > header
> > > > > > > > > > > > > >>> fields
> > > > > > > > > > > > > >>>         1.1 if it is a forwarding request, use
> > > > > > > CLUSTER_ACTION
> > > > > > > > > to
> > > > > > > > > > > > verify
> > > > > > > > > > > > > >>> the given resource
> > > > > > > > > > > > > >>>         1.2 if the resource is authorized, and if
> > > > this
> > > > > is
> > > > > > > the
> > > > > > > > > > > active
> > > > > > > > > > > > > >>> controller, process it
> > > > > > > > > > > > > >>>         1.3 if the resource is authorized but
> > this
> > > is
> > > > > not
> > > > > > > the
> > > > > > > > > > > active
> > > > > > > > > > > > > >>> controller, return NOT_CONTROLLER to the sender
> > > > > (forwarding
> > > > > > > > > > broker)
> > > > > > > > > > > > for
> > > > > > > > > > > > > >>> retry
> > > > > > > > > > > > > >>>         1.4 if the resource is not authorized,
> > > return
> > > > > > > > > > > > > >>> CLUSTER_AUTHORIZATION_FAILURE to propagate back
> > to
> > > > the
> > > > > > > > original
> > > > > > > > > > > > client
> > > > > > > > > > > > > >>> through forwarding broker
> > > > > > > > > > > > > >>> Step 2. If the request is not a forwarding
> > request
> > > > > > > > > > > > > >>>         2.1 Verify with normal principal as ALTER
> > > on
> > > > > the
> > > > > > > > given
> > > > > > > > > > > > > resources
> > > > > > > > > > > > > >>>         2.2 If the resource is authorized, and
> > this
> > > > is
> > > > > the
> > > > > > > > > active
> > > > > > > > > > > > > >>> controller, process it
> > > > > > > > > > > > > >>>         2.3 If the resource is authorized, but
> > this
> > > > is
> > > > > not
> > > > > > > > > active
> > > > > > > > > > > > > >>> controller, put the resource into the preparation
> > > for
> > > > > a new
> > > > > > > > > > > > AlterConfig
> > > > > > > > > > > > > >>> request for forwarding
> > > > > > > > > > > > > >>>         2.4 If the resource is not authorized,
> > > reply
> > > > > the
> > > > > > > > > original
> > > > > > > > > > > > > client
> > > > > > > > > > > > > >>> AUTHORIZATION_FAILURE when the forwarding request
> > > is
> > > > > > > returned
> > > > > > > > > > > > > >>>
> > > > > > > > > > > > > >>> When the control plane and data plane are using
> > the
> > > > > same
> > > > > > > > > > listener,
> > > > > > > > > > > we
> > > > > > > > > > > > > >>> couldn't distinguish whether a request is
> > forwarded
> > > > or
> > > > > not
> > > > > > > > for
> > > > > > > > > > > sure,
> > > > > > > > > > > > > so in
> > > > > > > > > > > > > >>> terms of the forward request checking, we have to
> > > > > require
> > > > > > > the
> > > > > > > > > > extra
> > > > > > > > > > > > > header
> > > > > > > > > > > > > >>> fields to present. A stronger checking mechanism
> > > > could
> > > > > only
> > > > > > > > be
> > > > > > > > > > > > enforced
> > > > > > > > > > > > > >>> when we upgrade to post-KIP-500 with a separate
> > > > > controller
> > > > > > > > > > network.
> > > > > > > > > > > > > >>>
> > > > > > > > > > > > > >>>
> > > > > > > > > > > > > >>>> I have made another pass on the whole KIP, I
> > have
> > > > few
> > > > > > > nits:
> > > > > > > > > > > > > >>>>
> > > > > > > > > > > > > >>>> - The sentence "Take AlterConfig as an example
> > to
> > > > > > > understand
> > > > > > > > > the
> > > > > > > > > > > > > changes
> > > > > > > > > > > > > >>>> we are making." does not make much sense anymore
> > > in
> > > > > the
> > > > > > > > > > beginning
> > > > > > > > > > > of
> > > > > > > > > > > > > the
> > > > > > > > > > > > > >>>> "Proposed Changes" chapter.
> > > > > > > > > > > > > >>>>
> > > > > > > > > > > > > >>>> Sure, deleted.
> > > > > > > > > > > > > >>>
> > > > > > > > > > > > > >>>
> > > > > > > > > > > > > >>>> - When you say "Existing RPCs which are sending
> > > > > directly
> > > > > > > to
> > > > > > > > > the
> > > > > > > > > > > > > >>>> controller
> > > > > > > > > > > > > >>>> will
> > > > > > > > > > > > > >>>> rely on forwarding as well.". I suggest to
> > > > explicitly
> > > > > > > > mention
> > > > > > > > > > how
> > > > > > > > > > > > "old
> > > > > > > > > > > > > >>>> admin clients"
> > > > > > > > > > > > > >>>> will work here to complement the sentence.
> > > Something
> > > > > like:
> > > > > > > > > They
> > > > > > > > > > > will
> > > > > > > > > > > > > >>>> get a
> > > > > > > > > > > > > >>>> random
> > > > > > > > > > > > > >>>> broker id as the controller id in the metadata
> > > > > response
> > > > > > > and
> > > > > > > > > > stick
> > > > > > > > > > > to
> > > > > > > > > > > > > it
> > > > > > > > > > > > > >>>> as
> > > > > > > > > > > > > >>>> you explained.
> > > > > > > > > > > > > >>>>
> > > > > > > > > > > > > >>>> Sg, changed.
> > > > > > > > > > > > > >>>
> > > > > > > > > > > > > >>>
> > > > > > > > > > > > > >>>> - "The purpose of adding principal name is for
> > the
> > > > > audit
> > > > > > > > > > logging,
> > > > > > > > > > > > and
> > > > > > > > > > > > > >>>> the
> > > > > > > > > > > > > >>>> client id is
> > > > > > > > > > > > > >>>> being used to throttling according to KIP-599
> > > > > > > requirement."
> > > > > > > > > > > > Actually,
> > > > > > > > > > > > > >>>> KIP-599 needs
> > > > > > > > > > > > > >>>> both the principal and the clientId.
> > > > > > > > > > > > > >>>>
> > > > > > > > > > > > > >>>> Makes sense.
> > > > > > > > > > > > > >>>
> > > > > > > > > > > > > >>>
> > > > > > > > > > > > > >>>> - In the "Routing Request Security" chapter. It
> > is
> > > > > written
> > > > > > > > > that
> > > > > > > > > > > the
> > > > > > > > > > > > > >>>> forwarding broker
> > > > > > > > > > > > > >>>> will verify the request with its own authorizer
> > > and
> > > > > will
> > > > > > > > just
> > > > > > > > > > > > forward
> > > > > > > > > > > > > >>>> it if
> > > > > > > > > > > > > >>>> the request
> > > > > > > > > > > > > >>>> looks good. When a request contains for instance
> > > > > multiple
> > > > > > > > > > topics,
> > > > > > > > > > > I
> > > > > > > > > > > > > >>>> suppose
> > > > > > > > > > > > > >>>> that
> > > > > > > > > > > > > >>>> we will forward only the authorized ones and not
> > > the
> > > > > whole
> > > > > > > > > > > original
> > > > > > > > > > > > > >>>> request
> > > > > > > > > > > > > >>>> as is.
> > > > > > > > > > > > > >>>> We may want to reword the sentence to make this
> > > > clear.
> > > > > > > > > > > > > >>>>
> > > > > > > > > > > > > >>>> That makes sense, I will put this more detailed
> > > > > version of
> > > > > > > > > > routing
> > > > > > > > > > > > > into
> > > > > > > > > > > > > >>> the design.
> > > > > > > > > > > > > >>>
> > > > > > > > > > > > > >>>
> > > > > > > > > > > > > >>>> - For the record, should we put the previous
> > > > proposal
> > > > > in
> > > > > > > the
> > > > > > > > > > > > rejected
> > > > > > > > > > > > > >>>> alternatives as
> > > > > > > > > > > > > >>>> well?
> > > > > > > > > > > > > >>>>
> > > > > > > > > > > > > >>>> We do have big changes in this KIP, our current
> > > > > strategy
> > > > > > > is
> > > > > > > > to
> > > > > > > > > > > rely
> > > > > > > > > > > > on
> > > > > > > > > > > > > >>> wiki revisions if people
> > > > > > > > > > > > > >>> are interested to figure out our previous design
> > > > > strategy.
> > > > > > > > > > Putting
> > > > > > > > > > > > the
> > > > > > > > > > > > > >>> full design in current KIP proposal
> > > > > > > > > > > > > >>> would distract too much for readers as we
> > discussed
> > > > > during
> > > > > > > > the
> > > > > > > > > > last
> > > > > > > > > > > > > time
> > > > > > > > > > > > > >>> of big refactoring.
> > > > > > > > > > > > > >>>
> > > > > > > > > > > > > >>> Best,
> > > > > > > > > > > > > >>>> David
> > > > > > > > > > > > > >>>>
> > > > > > > > > > > > > >>>> On Thu, Jul 30, 2020 at 3:51 AM Boyang Chen <
> > > > > > > > > > > > > reluctanthero104@gmail.com
> > > > > > > > > > > > > >>>> >
> > > > > > > > > > > > > >>>> wrote:
> > > > > > > > > > > > > >>>>
> > > > > > > > > > > > > >>>> > Thanks David for the feedback!
> > > > > > > > > > > > > >>>> >
> > > > > > > > > > > > > >>>> > On Wed, Jul 29, 2020 at 7:53 AM David Jacot <
> > > > > > > > > > > djacot@confluent.io>
> > > > > > > > > > > > > >>>> wrote:
> > > > > > > > > > > > > >>>> >
> > > > > > > > > > > > > >>>> > > Hi, Colin, Boyang,
> > > > > > > > > > > > > >>>> > >
> > > > > > > > > > > > > >>>> > > Colin, thanks for the clarification.
> > Somehow,
> > > I
> > > > > > > thought
> > > > > > > > > that
> > > > > > > > > > > > even
> > > > > > > > > > > > > >>>> if the
> > > > > > > > > > > > > >>>> > > controller is ran independently, it
> > > > > > > > > > > > > >>>> > > would still run the listeners of the broker
> > > and
> > > > > thus
> > > > > > > > would
> > > > > > > > > > be
> > > > > > > > > > > > > >>>> accessible
> > > > > > > > > > > > > >>>> > by
> > > > > > > > > > > > > >>>> > > redirecting on the loopback
> > > > > > > > > > > > > >>>> > > interface. My mistake.
> > > > > > > > > > > > > >>>> > >
> > > > > > > > > > > > > >>>> > > Boyang, I have few questions/comments
> > > regarding
> > > > > the
> > > > > > > > > updated
> > > > > > > > > > > KIP:
> > > > > > > > > > > > > >>>> > >
> > > > > > > > > > > > > >>>> > > 1. I think that it would be great if we
> > could
> > > > > clarify
> > > > > > > > how
> > > > > > > > > > old
> > > > > > > > > > > > > admin
> > > > > > > > > > > > > >>>> > clients
> > > > > > > > > > > > > >>>> > > which are directly talking to the
> > > > > > > > > > > > > >>>> > > controller will work with this KIP. I read
> > > > > between the
> > > > > > > > > lines
> > > > > > > > > > > > that,
> > > > > > > > > > > > > >>>> as we
> > > > > > > > > > > > > >>>> > > propose to provide a random
> > > > > > > > > > > > > >>>> > > broker Id as the controller Id in the
> > metadata
> > > > > > > response,
> > > > > > > > > > they
> > > > > > > > > > > > will
> > > > > > > > > > > > > >>>> use a
> > > > > > > > > > > > > >>>> > > single node as a proxy. Is that
> > > > > > > > > > > > > >>>> > > correct? This deserves to be called out more
> > > > > > > explicitly
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > > >>>> design
> > > > > > > > > > > > > >>>> > > section instead of being hidden
> > > > > > > > > > > > > >>>> > > in the protocol bump of the metadata RPC.
> > > > > > > > > > > > > >>>> > >
> > > > > > > > > > > > > >>>> > > Makes sense, I stress this point in the
> > > > > compatibility
> > > > > > > > > > section.
> > > > > > > > > > > > > >>>> >
> > > > > > > > > > > > > >>>> >
> > > > > > > > > > > > > >>>> > > 1.1 If I understand correctly, could we
> > assume
> > > > > that
> > > > > > > old
> > > > > > > > > > admin
> > > > > > > > > > > > > >>>> clients
> > > > > > > > > > > > > >>>> > will
> > > > > > > > > > > > > >>>> > > stick to the same "fake controller"
> > > > > > > > > > > > > >>>> > > until they refresh their metadata?
> > Refreshing
> > > > the
> > > > > > > > metadata
> > > > > > > > > > > > usually
> > > > > > > > > > > > > >>>> > happens
> > > > > > > > > > > > > >>>> > > when NOT_CONTROLLER
> > > > > > > > > > > > > >>>> > > is received but this won't happen anymore so
> > > > they
> > > > > > > should
> > > > > > > > > > > change
> > > > > > > > > > > > > >>>> > > infrequently.
> > > > > > > > > > > > > >>>> > >
> > > > > > > > > > > > > >>>> > > That is correct, old admin clients would not
> > > try
> > > > > to
> > > > > > > > > refresh
> > > > > > > > > > > > their
> > > > > > > > > > > > > >>>> > metadata
> > > > > > > > > > > > > >>>> > due to NOT_CONTROLLER,
> > > > > > > > > > > > > >>>> > which is impossible to happen with the new
> > > broker
> > > > > > > cluster.
> > > > > > > > > > > > > >>>> >
> > > > > > > > > > > > > >>>> >
> > > > > > > > > > > > > >>>> > > 2. For the new admin client, I suppose that
> > we
> > > > > plan on
> > > > > > > > > using
> > > > > > > > > > > > > >>>> > > LeastLoadedNodeProvider for the
> > > > > > > > > > > > > >>>> > > requests that are using
> > > ControllerNodeProvider.
> > > > We
> > > > > > > could
> > > > > > > > > > > perhaps
> > > > > > > > > > > > > >>>> mention
> > > > > > > > > > > > > >>>> > > it.
> > > > > > > > > > > > > >>>> > >
> > > > > > > > > > > > > >>>> > > Sure, added.
> > > > > > > > > > > > > >>>> >
> > > > > > > > > > > > > >>>> >
> > > > > > > > > > > > > >>>> > > 3. Pre KIP-500, will we have a way to
> > > > distinguish
> > > > > if a
> > > > > > > > > > request
> > > > > > > > > > > > > that
> > > > > > > > > > > > > >>>> is
> > > > > > > > > > > > > >>>> > > received by the controller is
> > > > > > > > > > > > > >>>> > > coming directly from a client or from a
> > > broker?
> > > > > You
> > > > > > > > > mention
> > > > > > > > > > > that
> > > > > > > > > > > > > the
> > > > > > > > > > > > > >>>> > > listener can be used to do
> > > > > > > > > > > > > >>>> > > this but as you pointed out, it is not
> > > > mandatory.
> > > > > Do
> > > > > > > we
> > > > > > > > > have
> > > > > > > > > > > > > another
> > > > > > > > > > > > > >>>> > > reliable method? I am asking
> > > > > > > > > > > > > >>>> > > in the context of KIP-599 with the current
> > > > > controller,
> > > > > > > > we
> > > > > > > > > > may
> > > > > > > > > > > > need
> > > > > > > > > > > > > >>>> to
> > > > > > > > > > > > > >>>> > > throttle differently if the
> > > > > > > > > > > > > >>>> > > request comes from a client or from a
> > broker.
> > > > > > > > > > > > > >>>> > >
> > > > > > > > > > > > > >>>> > > The point for using the listener name is
> > more
> > > > of a
> > > > > > > > > security
> > > > > > > > > > > > > >>>> purpose, to
> > > > > > > > > > > > > >>>> > detect any forged request to our best effort.
> > > > > > > > > > > > > >>>> > For throttling I think we could just check the
> > > > > request
> > > > > > > > > header
> > > > > > > > > > > for
> > > > > > > > > > > > > >>>> > *InitialClientId* existence, to distinguish
> > > > whether
> > > > > to
> > > > > > > > apply
> > > > > > > > > > > > > >>>> > throttling strategy as forwarded request or
> > > direct
> > > > > > > > request.
> > > > > > > > > > > > > >>>> >
> > > > > > > > > > > > > >>>> >
> > > > > > > > > > > > > >>>> > > 4. Could we add `InitialClientId` as well?
> > > This
> > > > > will
> > > > > > > be
> > > > > > > > > > > required
> > > > > > > > > > > > > >>>> for the
> > > > > > > > > > > > > >>>> > > quota as we can apply them
> > > > > > > > > > > > > >>>> > > by principal and/or clientId.
> > > > > > > > > > > > > >>>> > >
> > > > > > > > > > > > > >>>> > > Sounds good, added.
> > > > > > > > > > > > > >>>> >
> > > > > > > > > > > > > >>>> >
> > > > > > > > > > > > > >>>> > > 5. A small remark regarding the structure of
> > > the
> > > > > KIP.
> > > > > > > It
> > > > > > > > > is
> > > > > > > > > > a
> > > > > > > > > > > > bit
> > > > > > > > > > > > > >>>> weird
> > > > > > > > > > > > > >>>> > > that requests that do not go
> > > > > > > > > > > > > >>>> > > to the controller are mentioned in the
> > > Proposed
> > > > > Design
> > > > > > > > > > section
> > > > > > > > > > > > and
> > > > > > > > > > > > > >>>> the
> > > > > > > > > > > > > >>>> > > requests that go to the
> > > > > > > > > > > > > >>>> > > controller are mentioned in the Public
> > > > Interfaces.
> > > > > > > When
> > > > > > > > > one
> > > > > > > > > > > read
> > > > > > > > > > > > > the
> > > > > > > > > > > > > >>>> > > Proposed Design, it does not
> > > > > > > > > > > > > >>>> > > get a full picture of the whole new routing
> > > > > proposal
> > > > > > > for
> > > > > > > > > old
> > > > > > > > > > > and
> > > > > > > > > > > > > new
> > > > > > > > > > > > > >>>> > > clients. It would be great if we
> > > > > > > > > > > > > >>>> > > could have a full overview in that section.
> > > > > > > > > > > > > >>>> > >
> > > > > > > > > > > > > >>>> > > Good point, I will move the pieces around.
> > > > > > > > > > > > > >>>> >
> > > > > > > > > > > > > >>>> >
> > > > > > > > > > > > > >>>> > > Overall the change makes sense to me. I will
> > > > work
> > > > > on
> > > > > > > > > > drafting
> > > > > > > > > > > an
> > > > > > > > > > > > > >>>> addendum
> > > > > > > > > > > > > >>>> > > to KIP-599 to
> > > > > > > > > > > > > >>>> > > alter the design to cope with these changes.
> > > At
> > > > a
> > > > > > > first
> > > > > > > > > > > glance,
> > > > > > > > > > > > > that
> > > > > > > > > > > > > >>>> > seems
> > > > > > > > > > > > > >>>> > > doable if 1.1, 3
> > > > > > > > > > > > > >>>> > > and 4 are OK.
> > > > > > > > > > > > > >>>> > >
> > > > > > > > > > > > > >>>> > > Thank you for the help!
> > > > > > > > > > > > > >>>> >
> > > > > > > > > > > > > >>>> >
> > > > > > > > > > > > > >>>> > > Thanks,
> > > > > > > > > > > > > >>>> > > David
> > > > > > > > > > > > > >>>> > >
> > > > > > > > > > > > > >>>> > > On Wed, Jul 29, 2020 at 5:29 AM Boyang Chen
> > <
> > > > > > > > > > > > > >>>> reluctanthero104@gmail.com>
> > > > > > > > > > > > > >>>> > > wrote:
> > > > > > > > > > > > > >>>> > >
> > > > > > > > > > > > > >>>> > > > Thanks for the feedback Colin!
> > > > > > > > > > > > > >>>> > > >
> > > > > > > > > > > > > >>>> > > > On Tue, Jul 28, 2020 at 2:11 PM Colin
> > > McCabe <
> > > > > > > > > > > > > cmccabe@apache.org>
> > > > > > > > > > > > > >>>> > wrote:
> > > > > > > > > > > > > >>>> > > >
> > > > > > > > > > > > > >>>> > > > > Hi Boyang,
> > > > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > > > >>>> > > > > Thanks for updating this.  A few
> > comments
> > > > > below:
> > > > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > > > >>>> > > > > In the "Routing Request Security"
> > section,
> > > > > there
> > > > > > > is
> > > > > > > > a
> > > > > > > > > > > > > reference
> > > > > > > > > > > > > >>>> to
> > > > > > > > > > > > > >>>> > > "older
> > > > > > > > > > > > > >>>> > > > > requests that need redirection."  But
> > > after
> > > > > these
> > > > > > > > new
> > > > > > > > > > > > > >>>> revisions, both
> > > > > > > > > > > > > >>>> > > new
> > > > > > > > > > > > > >>>> > > > > and old requests need redirection.  So
> > we
> > > > > should
> > > > > > > > > > rephrase
> > > > > > > > > > > > > this.
> > > > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > > > >>>> > > > > > In addition, to avoid exposing this
> > > > > forwarding
> > > > > > > > power
> > > > > > > > > > to
> > > > > > > > > > > > the
> > > > > > > > > > > > > >>>> admin
> > > > > > > > > > > > > >>>> > > > > clients,
> > > > > > > > > > > > > >>>> > > > > > the routing request shall be forwarded
> > > > > towards
> > > > > > > the
> > > > > > > > > > > > > controller
> > > > > > > > > > > > > >>>> > broker
> > > > > > > > > > > > > >>>> > > > > internal
> > > > > > > > > > > > > >>>> > > > > > endpoint which should be only visible
> > to
> > > > > other
> > > > > > > > > brokers
> > > > > > > > > > > > > inside
> > > > > > > > > > > > > >>>> the
> > > > > > > > > > > > > >>>> > > > > cluster
> > > > > > > > > > > > > >>>> > > > > > in the KIP-500 controller. Any admin
> > > > > > > configuration
> > > > > > > > > > > request
> > > > > > > > > > > > > >>>> with
> > > > > > > > > > > > > >>>> > > broker
> > > > > > > > > > > > > >>>> > > > > > principal should not be going through
> > > the
> > > > > public
> > > > > > > > > > > endpoint
> > > > > > > > > > > > > and
> > > > > > > > > > > > > >>>> will
> > > > > > > > > > > > > >>>> > be
> > > > > > > > > > > > > >>>> > > > > > rejected for security purpose.
> > > > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > > > >>>> > > > > We should also describe how this will
> > work
> > > > in
> > > > > the
> > > > > > > > > > > > pre-KIP-500
> > > > > > > > > > > > > >>>> case.
> > > > > > > > > > > > > >>>> > In
> > > > > > > > > > > > > >>>> > > > > that case, CLUSTER_ACTION gets the extra
> > > > > > > permissions
> > > > > > > > > > > > described
> > > > > > > > > > > > > >>>> here
> > > > > > > > > > > > > >>>> > > only
> > > > > > > > > > > > > >>>> > > > > when the message comes in on the
> > > > inter-broker
> > > > > > > > > listener.
> > > > > > > > > > > We
> > > > > > > > > > > > > >>>> should
> > > > > > > > > > > > > >>>> > > state
> > > > > > > > > > > > > >>>> > > > > that here.
> > > > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > > > >>>> > > > > (I can see that you have this
> > information
> > > > > later on
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > > >>>> "Security
> > > > > > > > > > > > > >>>> > > > Access
> > > > > > > > > > > > > >>>> > > > > Changes" section, but it would be good
> > to
> > > > > have it
> > > > > > > > here
> > > > > > > > > > as
> > > > > > > > > > > > > well,
> > > > > > > > > > > > > >>>> to
> > > > > > > > > > > > > >>>> > > avoid
> > > > > > > > > > > > > >>>> > > > > confusion.)
> > > > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > > > >>>> > > > > > To be more strict of protecting
> > > controller
> > > > > > > > > > information,
> > > > > > > > > > > > the
> > > > > > > > > > > > > >>>> > > > > "ControllerId"
> > > > > > > > > > > > > >>>> > > > > > field in new MetadataResponse shall be
> > > set
> > > > > to -1
> > > > > > > > > when
> > > > > > > > > > > the
> > > > > > > > > > > > > >>>> original
> > > > > > > > > > > > > >>>> > > > > request
> > > > > > > > > > > > > >>>> > > > > > comes from a non-broker client and it
> > is
> > > > > already
> > > > > > > > on
> > > > > > > > > > v10.
> > > > > > > > > > > > We
> > > > > > > > > > > > > >>>> shall
> > > > > > > > > > > > > >>>> > use
> > > > > > > > > > > > > >>>> > > > the
> > > > > > > > > > > > > >>>> > > > > > request listener name to distinguish
> > > > > whether a
> > > > > > > > given
> > > > > > > > > > > > request
> > > > > > > > > > > > > >>>> is
> > > > > > > > > > > > > >>>> > > > > inter-broker,
> > > > > > > > > > > > > >>>> > > > > > or from the client.
> > > > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > > > >>>> > > > > I'm not sure why we would need to
> > > > distinguish
> > > > > > > > between
> > > > > > > > > > > broker
> > > > > > > > > > > > > >>>> clients
> > > > > > > > > > > > > >>>> > > and
> > > > > > > > > > > > > >>>> > > > > non-broker clients.  Brokers don't
> > > generally
> > > > > send
> > > > > > > > > > > > > >>>> MetadataRequests to
> > > > > > > > > > > > > >>>> > > > other
> > > > > > > > > > > > > >>>> > > > > brokers, do they?  Brokers learn about
> > > > > metadata
> > > > > > > from
> > > > > > > > > > > > > >>>> > > > UpdateMetadataRequest
> > > > > > > > > > > > > >>>> > > > > and LeaderAndIsrRequest, not by sending
> > > > > > > > > MetadataRequests
> > > > > > > > > > > to
> > > > > > > > > > > > > >>>> other
> > > > > > > > > > > > > >>>> > > > brokers.
> > > > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > > > >>>> > > > > We do have one use case where the
> > > > > MetadataRequest
> > > > > > > > gets
> > > > > > > > > > > sent
> > > > > > > > > > > > > >>>> between
> > > > > > > > > > > > > >>>> > the
> > > > > > > > > > > > > >>>> > > > brokers, which is the
> > InterBrokerSendThread.
> > > > > > > Currently
> > > > > > > > > we
> > > > > > > > > > > > don't
> > > > > > > > > > > > > >>>> rely on
> > > > > > > > > > > > > >>>> > > it
> > > > > > > > > > > > > >>>> > > > to get the controller id, so I guess your
> > > > > suggestion
> > > > > > > > > > should
> > > > > > > > > > > be
> > > > > > > > > > > > > >>>> good to
> > > > > > > > > > > > > >>>> > > > enforce. We could use some meta comment on
> > > the
> > > > > > > > > > NetworkClient
> > > > > > > > > > > > > that
> > > > > > > > > > > > > >>>> it
> > > > > > > > > > > > > >>>> > > should
> > > > > > > > > > > > > >>>> > > > not be used to get the controller
> > location.
> > > > > > > > > > > > > >>>> > > >
> > > > > > > > > > > > > >>>> > > > Probably what we want here is: v0-v9:
> > > return a
> > > > > > > random
> > > > > > > > > > broker
> > > > > > > > > > > > in
> > > > > > > > > > > > > >>>> the
> > > > > > > > > > > > > >>>> > > cluster
> > > > > > > > > > > > > >>>> > > > > as the controller ID.  v10: no
> > > controllerID
> > > > > > > present
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > > >>>> > > > > MetadataResponse.  We should also
> > > deprecate
> > > > > the
> > > > > > > > > > > adminClient
> > > > > > > > > > > > > >>>> method
> > > > > > > > > > > > > >>>> > > which
> > > > > > > > > > > > > >>>> > > > > gets the controllerId.
> > > > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > > > >>>> > > > > > BROKER_AUTHORIZATION_FAILURE(92,
> > > > > "Authorization
> > > > > > > > > failed
> > > > > > > > > > > for
> > > > > > > > > > > > > the
> > > > > > > > > > > > > >>>> > > > > > request during forwarding, this
> > > indicates
> > > > an
> > > > > > > > > internal
> > > > > > > > > > > > error
> > > > > > > > > > > > > >>>> on the
> > > > > > > > > > > > > >>>> > > > broker
> > > > > > > > > > > > > >>>> > > > > > cluster security setup.",
> > > > > > > > > > > > > >>>> > BrokerAuthorizationFailureException::new);
> > > > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > > > >>>> > > > > Grammar nitpick: It would be good to
> > have
> > > a
> > > > > period
> > > > > > > > > > between
> > > > > > > > > > > > > >>>> > "forwarding"
> > > > > > > > > > > > > >>>> > > > > and "this" to avoid a run-on sentence :)
> > > > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > > > >>>> > > > > best,
> > > > > > > > > > > > > >>>> > > > > Colin
> > > > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > > > >>>> > > > > On Mon, Jul 27, 2020, at 21:47, Boyang
> > > Chen
> > > > > wrote:
> > > > > > > > > > > > > >>>> > > > > > Hey there,
> > > > > > > > > > > > > >>>> > > > > >
> > > > > > > > > > > > > >>>> > > > > > I'm re-opening this thread because
> > after
> > > > > some
> > > > > > > > > initial
> > > > > > > > > > > > > >>>> > implementations
> > > > > > > > > > > > > >>>> > > > > > started, we spotted some gaps in the
> > > > > approved
> > > > > > > KIP
> > > > > > > > as
> > > > > > > > > > > well
> > > > > > > > > > > > as
> > > > > > > > > > > > > >>>> some
> > > > > > > > > > > > > >>>> > > > > > inconsistencies with KIP-631
> > controller.
> > > > > There
> > > > > > > > are a
> > > > > > > > > > > > couple
> > > > > > > > > > > > > of
> > > > > > > > > > > > > >>>> > > > addendums
> > > > > > > > > > > > > >>>> > > > > to
> > > > > > > > > > > > > >>>> > > > > > the existing KIP, specifically:
> > > > > > > > > > > > > >>>> > > > > >
> > > > > > > > > > > > > >>>> > > > > > 1. As the controller is foreseen to be
> > > > only
> > > > > > > > > accessible
> > > > > > > > > > > to
> > > > > > > > > > > > > the
> > > > > > > > > > > > > >>>> > > brokers,
> > > > > > > > > > > > > >>>> > > > > the
> > > > > > > > > > > > > >>>> > > > > > new admin client would not have direct
> > > > > access to
> > > > > > > > the
> > > > > > > > > > > > > >>>> controller. It
> > > > > > > > > > > > > >>>> > > is
> > > > > > > > > > > > > >>>> > > > > > guaranteed on the MetadataResponse
> > level
> > > > > which
> > > > > > > no
> > > > > > > > > > longer
> > > > > > > > > > > > > >>>> provides
> > > > > > > > > > > > > >>>> > > > > > `ControllerId` to client side
> > requests.
> > > > > > > > > > > > > >>>> > > > > >
> > > > > > > > > > > > > >>>> > > > > > 2. The broker would forward any direct
> > > ZK
> > > > > path
> > > > > > > > > > mutation
> > > > > > > > > > > > > >>>> requests,
> > > > > > > > > > > > > >>>> > > > > including
> > > > > > > > > > > > > >>>> > > > > > topic creation/deletion, reassignment,
> > > etc
> > > > > since
> > > > > > > > we
> > > > > > > > > > > > > deprecate
> > > > > > > > > > > > > >>>> the
> > > > > > > > > > > > > >>>> > > > direct
> > > > > > > > > > > > > >>>> > > > > > controller access on the client side.
> > No
> > > > > more
> > > > > > > > > protocol
> > > > > > > > > > > > > >>>> version bump
> > > > > > > > > > > > > >>>> > > is
> > > > > > > > > > > > > >>>> > > > > > necessary for the configuration
> > > requests.
> > > > > > > > > > > > > >>>> > > > > >
> > > > > > > > > > > > > >>>> > > > > > 3. To make sure forwarding requests
> > pass
> > > > the
> > > > > > > > > > > > authorization,
> > > > > > > > > > > > > >>>> broker
> > > > > > > > > > > > > >>>> > > > > > principal CLUSTER_ACTION would be
> > > allowed
> > > > > to be
> > > > > > > > used
> > > > > > > > > > as
> > > > > > > > > > > an
> > > > > > > > > > > > > >>>> > > alternative
> > > > > > > > > > > > > >>>> > > > > > authentication method for a variety of
> > > > > principal
> > > > > > > > > > > > operations,
> > > > > > > > > > > > > >>>> > > including
> > > > > > > > > > > > > >>>> > > > > > ALTER, ALTER_CONFIG, DELETE, etc. It
> > is
> > > > > because
> > > > > > > > the
> > > > > > > > > > > > > forwarding
> > > > > > > > > > > > > >>>> > > request
> > > > > > > > > > > > > >>>> > > > > > needs to use the proxy broker's own
> > > > > principal,
> > > > > > > > which
> > > > > > > > > > is
> > > > > > > > > > > > > >>>> currently
> > > > > > > > > > > > > >>>> > not
> > > > > > > > > > > > > >>>> > > > > > supported to be used for many
> > > > configuration
> > > > > > > change
> > > > > > > > > > > > > >>>> authentication
> > > > > > > > > > > > > >>>> > > > listed
> > > > > > > > > > > > > >>>> > > > > > above. The full list could be found in
> > > the
> > > > > KIP.
> > > > > > > > > > > > > >>>> > > > > >
> > > > > > > > > > > > > >>>> > > > > > 4. Add a new
> > > BROKER_AUTHORIZATION_FAILURE
> > > > > error
> > > > > > > > code
> > > > > > > > > > to
> > > > > > > > > > > > > >>>> indicate
> > > > > > > > > > > > > >>>> > any
> > > > > > > > > > > > > >>>> > > > > > internal security configuration
> > failure,
> > > > > when
> > > > > > > the
> > > > > > > > > > > > forwarded
> > > > > > > > > > > > > >>>> request
> > > > > > > > > > > > > >>>> > > > > failed
> > > > > > > > > > > > > >>>> > > > > > authentication on the controller side.
> > > > > > > > > > > > > >>>> > > > > >
> > > > > > > > > > > > > >>>> > > > > > Let me know what you think. With such
> > a
> > > > > major
> > > > > > > > > > refinement
> > > > > > > > > > > > of
> > > > > > > > > > > > > >>>> the
> > > > > > > > > > > > > >>>> > KIP,
> > > > > > > > > > > > > >>>> > > > I'm
> > > > > > > > > > > > > >>>> > > > > > open for re-vote after discussions
> > > > converge.
> > > > > > > > > > > > > >>>> > > > > >
> > > > > > > > > > > > > >>>> > > > > > Boyang
> > > > > > > > > > > > > >>>> > > > > >
> > > > > > > > > > > > > >>>> > > > > > On Wed, Jul 1, 2020 at 2:17 PM Boyang
> > > > Chen <
> > > > > > > > > > > > > >>>> > > reluctanthero104@gmail.com
> > > > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > > > >>>> > > > > > wrote:
> > > > > > > > > > > > > >>>> > > > > >
> > > > > > > > > > > > > >>>> > > > > > > Hey folks,
> > > > > > > > > > > > > >>>> > > > > > >
> > > > > > > > > > > > > >>>> > > > > > > I have also synced on the KIP-578
> > > which
> > > > > was
> > > > > > > > doing
> > > > > > > > > > the
> > > > > > > > > > > > > >>>> partition
> > > > > > > > > > > > > >>>> > > > limit,
> > > > > > > > > > > > > >>>> > > > > to
> > > > > > > > > > > > > >>>> > > > > > > make sure the partition limit error
> > > code
> > > > > would
> > > > > > > > be
> > > > > > > > > > > > properly
> > > > > > > > > > > > > >>>> > > propagated
> > > > > > > > > > > > > >>>> > > > > once
> > > > > > > > > > > > > >>>> > > > > > > it is done on top of KIP-590. Let me
> > > > know
> > > > > if
> > > > > > > you
> > > > > > > > > > have
> > > > > > > > > > > > > >>>> further
> > > > > > > > > > > > > >>>> > > > > questions or
> > > > > > > > > > > > > >>>> > > > > > > concerns.
> > > > > > > > > > > > > >>>> > > > > > >
> > > > > > > > > > > > > >>>> > > > > > > Boyang
> > > > > > > > > > > > > >>>> > > > > > >
> > > > > > > > > > > > > >>>> > > > > > > On Tue, Jun 23, 2020 at 5:08 PM
> > Boyang
> > > > > Chen <
> > > > > > > > > > > > > >>>> > > > > reluctanthero104@gmail.com>
> > > > > > > > > > > > > >>>> > > > > > > wrote:
> > > > > > > > > > > > > >>>> > > > > > >
> > > > > > > > > > > > > >>>> > > > > > >> Thanks for the clarification, Colin
> > > and
> > > > > > > Ismael.
> > > > > > > > > > > > > Personally
> > > > > > > > > > > > > >>>> I
> > > > > > > > > > > > > >>>> > also
> > > > > > > > > > > > > >>>> > > > feel
> > > > > > > > > > > > > >>>> > > > > > >> Option A is better to prioritize
> > > fixing
> > > > > the
> > > > > > > > gap.
> > > > > > > > > > Just
> > > > > > > > > > > > to
> > > > > > > > > > > > > be
> > > > > > > > > > > > > >>>> > clear,
> > > > > > > > > > > > > >>>> > > > the
> > > > > > > > > > > > > >>>> > > > > > >> proposed solution would be:
> > > > > > > > > > > > > >>>> > > > > > >>
> > > > > > > > > > > > > >>>> > > > > > >> 1. Bump the Metadata RPC version to
> > > > > return
> > > > > > > > > > > > > >>>> POLICY_VIOLATION. In
> > > > > > > > > > > > > >>>> > > the
> > > > > > > > > > > > > >>>> > > > > > >> application level, we should swap
> > the
> > > > > error
> > > > > > > > > message
> > > > > > > > > > > > with
> > > > > > > > > > > > > >>>> the
> > > > > > > > > > > > > >>>> > > actual
> > > > > > > > > > > > > >>>> > > > > failure
> > > > > > > > > > > > > >>>> > > > > > >> reason such as "violation of topic
> > > > > creation
> > > > > > > > > policy
> > > > > > > > > > > when
> > > > > > > > > > > > > >>>> > attempting
> > > > > > > > > > > > > >>>> > > > to
> > > > > > > > > > > > > >>>> > > > > auto
> > > > > > > > > > > > > >>>> > > > > > >> create internal topic through
> > > > > > > MetadataRequest."
> > > > > > > > > > > > > >>>> > > > > > >>
> > > > > > > > > > > > > >>>> > > > > > >> 2. For older Metadata RPC, return
> > > > > > > > > > > AUTHORIZATION_FAILED
> > > > > > > > > > > > to
> > > > > > > > > > > > > >>>> fail
> > > > > > > > > > > > > >>>> > > fast.
> > > > > > > > > > > > > >>>> > > > > > >>
> > > > > > > > > > > > > >>>> > > > > > >> Will address our other discussed
> > > points
> > > > > as
> > > > > > > well
> > > > > > > > > in
> > > > > > > > > > > the
> > > > > > > > > > > > > >>>> KIP, let
> > > > > > > > > > > > > >>>> > me
> > > > > > > > > > > > > >>>> > > > > know
> > > > > > > > > > > > > >>>> > > > > > >> if you have further questions.
> > > > > > > > > > > > > >>>> > > > > > >>
> > > > > > > > > > > > > >>>> > > > > > >> Thanks,
> > > > > > > > > > > > > >>>> > > > > > >> Boyang
> > > > > > > > > > > > > >>>> > > > > > >>
> > > > > > > > > > > > > >>>> > > > > > >> On Tue, Jun 23, 2020 at 10:41 AM
> > > Ismael
> > > > > Juma
> > > > > > > <
> > > > > > > > > > > > > >>>> ismael@juma.me.uk
> > > > > > > > > > > > > >>>> > >
> > > > > > > > > > > > > >>>> > > > > wrote:
> > > > > > > > > > > > > >>>> > > > > > >>
> > > > > > > > > > > > > >>>> > > > > > >>> Option A is basically what I was
> > > > > thinking.
> > > > > > > But
> > > > > > > > > > with
> > > > > > > > > > > a
> > > > > > > > > > > > > >>>> slight
> > > > > > > > > > > > > >>>> > > > > adjustment:
> > > > > > > > > > > > > >>>> > > > > > >>>
> > > > > > > > > > > > > >>>> > > > > > >>> New versions of MetadataResponse
> > > > return
> > > > > > > > > > > > > POLICY_VIOLATION,
> > > > > > > > > > > > > >>>> old
> > > > > > > > > > > > > >>>> > > > > versions
> > > > > > > > > > > > > >>>> > > > > > >>> return AUTHORIZATION_FAILED. The
> > > > latter
> > > > > > > works
> > > > > > > > > > > > correctly
> > > > > > > > > > > > > >>>> with
> > > > > > > > > > > > > >>>> > old
> > > > > > > > > > > > > >>>> > > > Java
> > > > > > > > > > > > > >>>> > > > > > >>> clients (i.e. the client fails
> > fast
> > > > and
> > > > > > > > > propagates
> > > > > > > > > > > the
> > > > > > > > > > > > > >>>> error),
> > > > > > > > > > > > > >>>> > > I've
> > > > > > > > > > > > > >>>> > > > > > >>> tested
> > > > > > > > > > > > > >>>> > > > > > >>> it. Adjust new clients to treat
> > > > > > > > POLICY_VIOLATION
> > > > > > > > > > > like
> > > > > > > > > > > > > >>>> > > > > > >>> AUTHORIZATION_FAILED,
> > > > > > > > > > > > > >>>> > > > > > >>> but propagate the custom error
> > > > message.
> > > > > > > > > > > > > >>>> > > > > > >>>
> > > > > > > > > > > > > >>>> > > > > > >>> Ismael
> > > > > > > > > > > > > >>>> > > > > > >>>
> > > > > > > > > > > > > >>>> > > > > > >>> On Mon, Jun 22, 2020 at 11:00 PM
> > > Colin
> > > > > > > McCabe
> > > > > > > > <
> > > > > > > > > > > > > >>>> > > cmccabe@apache.org>
> > > > > > > > > > > > > >>>> > > > > > >>> wrote:
> > > > > > > > > > > > > >>>> > > > > > >>>
> > > > > > > > > > > > > >>>> > > > > > >>> > > > > On Fri, Jun 19, 2020 at
> > 3:18
> > > > PM
> > > > > > > Ismael
> > > > > > > > > > Juma
> > > > > > > > > > > <
> > > > > > > > > > > > > >>>> > > > > ismael@juma.me.uk>
> > > > > > > > > > > > > >>>> > > > > > >>> > wrote:
> > > > > > > > > > > > > >>>> > > > > > >>> > > > >
> > > > > > > > > > > > > >>>> > > > > > >>> > > > > > Hi Colin,
> > > > > > > > > > > > > >>>> > > > > > >>> > > > > >
> > > > > > > > > > > > > >>>> > > > > > >>> > > > > > The KIP states in the
> > > > > > > Compatibility
> > > > > > > > > > > section
> > > > > > > > > > > > > (not
> > > > > > > > > > > > > >>>> > Future
> > > > > > > > > > > > > >>>> > > > > work):
> > > > > > > > > > > > > >>>> > > > > > >>> > > > > >
> > > > > > > > > > > > > >>>> > > > > > >>> > > > > > "To support the proxy of
> > > > > requests,
> > > > > > > > we
> > > > > > > > > > need
> > > > > > > > > > > > to
> > > > > > > > > > > > > >>>> build a
> > > > > > > > > > > > > >>>> > > > > channel
> > > > > > > > > > > > > >>>> > > > > > >>> for
> > > > > > > > > > > > > >>>> > > > > > >>> > > > > > brokers to talk directly
> > > to
> > > > > the
> > > > > > > > > > > controller.
> > > > > > > > > > > > > >>>> This part
> > > > > > > > > > > > > >>>> > > of
> > > > > > > > > > > > > >>>> > > > > the
> > > > > > > > > > > > > >>>> > > > > > >>> design
> > > > > > > > > > > > > >>>> > > > > > >>> > > > > > is internal change only
> > > and
> > > > > won’t
> > > > > > > > > block
> > > > > > > > > > > the
> > > > > > > > > > > > > KIP
> > > > > > > > > > > > > >>>> > > > progress."
> > > > > > > > > > > > > >>>> > > > > > >>> > > > > >
> > > > > > > > > > > > > >>>> > > > > > >>> > > > > > I am clarifying that
> > this
> > > is
> > > > > not
> > > > > > > > > > internal
> > > > > > > > > > > > only
> > > > > > > > > > > > > >>>> due to
> > > > > > > > > > > > > >>>> > > the
> > > > > > > > > > > > > >>>> > > > > > >>> config.
> > > > > > > > > > > > > >>>> > > > > > >>> > If we
> > > > > > > > > > > > > >>>> > > > > > >>> > > > > > say that this KIP
> > depends
> > > on
> > > > > > > another
> > > > > > > > > KIP
> > > > > > > > > > > > > before
> > > > > > > > > > > > > >>>> we
> > > > > > > > > > > > > >>>> > can
> > > > > > > > > > > > > >>>> > > > > merge
> > > > > > > > > > > > > >>>> > > > > > >>> > > > > > it, that's fine although
> > > it
> > > > > feels
> > > > > > > a
> > > > > > > > > bit
> > > > > > > > > > > > > >>>> unnecessary.
> > > > > > > > > > > > > >>>> > > > > > >>> > > > > >
> > > > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > > > >>>> > > > > > >>> > Hi Ismael,
> > > > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > > > >>>> > > > > > >>> > I didn't realize there was
> > still a
> > > > > > > reference
> > > > > > > > > to
> > > > > > > > > > > the
> > > > > > > > > > > > > >>>> separate
> > > > > > > > > > > > > >>>> > > > > controller
> > > > > > > > > > > > > >>>> > > > > > >>> > channel in the "Compatibility,
> > > > > > > Deprecation,
> > > > > > > > > and
> > > > > > > > > > > > > >>>> Migration
> > > > > > > > > > > > > >>>> > Plan"
> > > > > > > > > > > > > >>>> > > > > > >>> section.  I
> > > > > > > > > > > > > >>>> > > > > > >>> > agree that it doesn't really
> > > belong
> > > > > there.
> > > > > > > > > > Given
> > > > > > > > > > > > that
> > > > > > > > > > > > > >>>> this
> > > > > > > > > > > > > >>>> > is
> > > > > > > > > > > > > >>>> > > > > creating
> > > > > > > > > > > > > >>>> > > > > > >>> > confusion, I would suggest that
> > we
> > > > > just
> > > > > > > drop
> > > > > > > > > > this
> > > > > > > > > > > > from
> > > > > > > > > > > > > >>>> the
> > > > > > > > > > > > > >>>> > KIP
> > > > > > > > > > > > > >>>> > > > > > >>> entirely.
> > > > > > > > > > > > > >>>> > > > > > >>> > It really is orthogonal to what
> > > this
> > > > > KIP
> > > > > > > is
> > > > > > > > > > > about--
> > > > > > > > > > > > we
> > > > > > > > > > > > > >>>> don't
> > > > > > > > > > > > > >>>> > > > need a
> > > > > > > > > > > > > >>>> > > > > > >>> > separate channel to implement
> > > > > redirection.
> > > > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > > > >>>> > > > > > >>> > Boyang wrote:
> > > > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > > > >>>> > > > > > >>> > >
> > > > > > > > > > > > > >>>> > > > > > >>> > > We are only opening the doors
> > > for
> > > > > > > specific
> > > > > > > > > > > > internal
> > > > > > > > > > > > > >>>> topics
> > > > > > > > > > > > > >>>> > > > > (offsets,
> > > > > > > > > > > > > >>>> > > > > > >>> txn
> > > > > > > > > > > > > >>>> > > > > > >>> > > log), which I assume the
> > client
> > > > > should
> > > > > > > > have
> > > > > > > > > no
> > > > > > > > > > > > > >>>> possibility
> > > > > > > > > > > > > >>>> > to
> > > > > > > > > > > > > >>>> > > > > mutate
> > > > > > > > > > > > > >>>> > > > > > >>> the
> > > > > > > > > > > > > >>>> > > > > > >>> > > topic policy?
> > > > > > > > > > > > > >>>> > > > > > >>> > >
> > > > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > > > >>>> > > > > > >>> > Hi Boyang,
> > > > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > > > >>>> > > > > > >>> > I think you and Ismael are
> > talking
> > > > > about
> > > > > > > > > > different
> > > > > > > > > > > > > >>>> scenarios.
> > > > > > > > > > > > > >>>> > > > You
> > > > > > > > > > > > > >>>> > > > > are
> > > > > > > > > > > > > >>>> > > > > > >>> > describing the scenario where
> > the
> > > > > broker
> > > > > > > is
> > > > > > > > > > > > > >>>> auto-creating the
> > > > > > > > > > > > > >>>> > > > > > >>> transaction
> > > > > > > > > > > > > >>>> > > > > > >>> > log topic or consumer offset
> > > topic.
> > > > > This
> > > > > > > > > > scenario
> > > > > > > > > > > > > >>>> indeed
> > > > > > > > > > > > > >>>> > > should
> > > > > > > > > > > > > >>>> > > > > not
> > > > > > > > > > > > > >>>> > > > > > >>> happen
> > > > > > > > > > > > > >>>> > > > > > >>> > in a properly-configured
> > cluster.
> > > > > > > However,
> > > > > > > > > > Ismael
> > > > > > > > > > > > is
> > > > > > > > > > > > > >>>> > > describing
> > > > > > > > > > > > > >>>> > > > a
> > > > > > > > > > > > > >>>> > > > > > >>> scenario
> > > > > > > > > > > > > >>>> > > > > > >>> > where the client is
> > auto-creating
> > > > some
> > > > > > > > > arbitrary
> > > > > > > > > > > > > >>>> non-internal
> > > > > > > > > > > > > >>>> > > > topic
> > > > > > > > > > > > > >>>> > > > > > >>> just by
> > > > > > > > > > > > > >>>> > > > > > >>> > sending a metadata request.
> > > > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > > > >>>> > > > > > >>> > As far as I can see, there are
> > two
> > > > > > > solutions
> > > > > > > > > > here:
> > > > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > > > >>>> > > > > > >>> > A. Close the hole in
> > > > > CreateTopicsPolicy
> > > > > > > > > > > immediately.
> > > > > > > > > > > > > >>>> In new
> > > > > > > > > > > > > >>>> > > > > versions,
> > > > > > > > > > > > > >>>> > > > > > >>> > allow MetadataResponse to return
> > > > > > > > > > > > AUTHORIZATION_FAILED
> > > > > > > > > > > > > >>>> if we
> > > > > > > > > > > > > >>>> > > tried
> > > > > > > > > > > > > >>>> > > > > to
> > > > > > > > > > > > > >>>> > > > > > >>> > auto-create a topic and failed.
> > > > Find
> > > > > some
> > > > > > > > > other
> > > > > > > > > > > > error
> > > > > > > > > > > > > >>>> code
> > > > > > > > > > > > > >>>> > to
> > > > > > > > > > > > > >>>> > > > > return
> > > > > > > > > > > > > >>>> > > > > > >>> for
> > > > > > > > > > > > > >>>> > > > > > >>> > existing versions.
> > > > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > > > >>>> > > > > > >>> > B. Keep the hole in
> > > > > CreateTopicsPolicy and
> > > > > > > > add
> > > > > > > > > > > some
> > > > > > > > > > > > > >>>> > > configuration
> > > > > > > > > > > > > >>>> > > > > to
> > > > > > > > > > > > > >>>> > > > > > >>> allow
> > > > > > > > > > > > > >>>> > > > > > >>> > admins to gradually migrate to
> > > > > closing it.
> > > > > > > > In
> > > > > > > > > > > > > >>>> practice, this
> > > > > > > > > > > > > >>>> > > > > probably
> > > > > > > > > > > > > >>>> > > > > > >>> > means a configuration toggle
> > that
> > > > > enables
> > > > > > > > > direct
> > > > > > > > > > > ZK
> > > > > > > > > > > > > >>>> access,
> > > > > > > > > > > > > >>>> > > that
> > > > > > > > > > > > > >>>> > > > > > >>> starts off
> > > > > > > > > > > > > >>>> > > > > > >>> > as enabled.  Then we can
> > > eventually
> > > > > > > default
> > > > > > > > it
> > > > > > > > > > to
> > > > > > > > > > > > > false
> > > > > > > > > > > > > >>>> and
> > > > > > > > > > > > > >>>> > > then
> > > > > > > > > > > > > >>>> > > > > > >>> remove it
> > > > > > > > > > > > > >>>> > > > > > >>> > entirely over time.
> > > > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > > > >>>> > > > > > >>> > best,
> > > > > > > > > > > > > >>>> > > > > > >>> > Colin
> > > > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > > > >>>> > > > > > >>>
> > > > > > > > > > > > > >>>> > > > > > >>
> > > > > > > > > > > > > >>>> > > > > >
> > > > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > > > >>>> > > >
> > > > > > > > > > > > > >>>> > >
> > > > > > > > > > > > > >>>> >
> > > > > > > > > > > > > >>>>
> > > > > > > > > > > > > >>>
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> 

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

Posted by Ismael Juma <is...@juma.me.uk>.
Hi Ron,

Thanks for bringing this up. Thinking about it, the combination of flexible
fields and the principal type field gives us enough flexibility that we
don't need a magic number.

Ismael

P.S. For a magic number to be useful for third party implementations, we
would need a mechanism to coordinate what each number means, so it's a bit
complicated to do it well.

On Tue, Jul 13, 2021 at 9:48 AM Ron Dagostino <rn...@gmail.com> wrote:

> Hi everyone.  I know it has been 9 months since the last message appeared
> on this vote thread, but a potential oversight exists in the implementation
> of DefaultKafkaPrincipalBuilder.KafkaPrincipalSerde from
> https://github.com/apache/kafka/pull/9103.  Specifically, there is no
> magic
> number at the top of the wire format, and this causes obscure parsing
> errors if incompatible principal implementations are mixed.  A magic number
> at the top would allow deserialization code to provide an intuitive error
> message.  The current implementation (without a magic number) was released
> in 2.8, but it presumably has never been used since forwarding is disabled
> (
>
> https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/Kafka.scala#L73
> ).
> We would like to change the implementation to include a magic number at the
> top for the 3.0 release.  This would be a breaking change, but again this
> presumably has never been used anywhere in production and would therefore
> break nothing.
>
> Note that forwarding is always enabled for KRaft-based clusters, but such
> clusters are not supported in any production sense and there is no upgrade
> path from a 2.8 KRaft cluster to a 3.0 KRaft cluster (from
> config/kraft/README.md: "KRaft mode in Kafka 2.8 is provided for testing
> only, NOT for production. We do not yet support upgrading existing
> ZooKeeper-based Kafka clusters into this mode. In fact, when Kafka 3.0 is
> released, it will not be possible to upgrade your KRaft clusters from 2.8
> to 3.0").
>
> A PR to add the magic number appears at
> https://github.com/apache/kafka/pull/11038.
>
> Please respond to this thread if you have any concerns or objections.
>
> Thanks,
>
> Ron
>
> On Fri, Oct 9, 2020 at 1:21 PM Boyang Chen <re...@gmail.com>
> wrote:
>
> > Thanks Jason for the great thoughts, and we basically decided to shift
> the
> > gear for a limited impersonation approach offline.
> >
> > The goal here is to simplify the handling logic by relying on the active
> > controller to do the actual authorization for resources in the original
> > client request. We are also adding the `KafkaPrincipalSerde` type to
> > provide the functionality for principal serialization/deserialization so
> > that it could embed in the Envelope and send to the active controller.
> > Before 3.0, customized principal builders could optionally extend the
> serde
> > type, which is required after 3.0 is released. Either way having the
> > capability to serde KafkaPrincipal becomes a prerequisite to enable
> > redirection besides IBP. Additionally, we add a forwardingPrincipal field
> > to the Authorizer context for authorization and audit logging purposes,
> > instead of going tagged fields in header.
> >
> > The KIP is updated to reflect the current approach, thanks.
> >
> >
> >
> > On Fri, Sep 25, 2020 at 5:55 PM Jason Gustafson <ja...@confluent.io>
> > wrote:
> >
> > > Hey All,
> > >
> > > So the main thing the EnvelopeRequest gives us is a way to avoid
> > converting
> > > older API versions in order to attach the initial principal name and
> the
> > > clientId. It also saves the need to add the initial principal and
> client
> > id
> > > as a tagged field to all of the forwarded protocols, which is nice. We
> > > still have the challenge of advertising API versions which are
> compatible
> > > with both the broker receiving the request and the controller that the
> > > request is ultimately forwarded to, but not sure I see a way around
> that.
> > >
> > > I realize I might be walking into a minefield here, but since the
> > envelope
> > > is being revisited, it seems useful to compare the approach suggested
> > above
> > > with the option relying on impersonation. I favor the use of
> > impersonation
> > > because it makes forwarding simpler. As the proposal stands, we will
> have
> > > to maintain logic for each forwarded API to unpack, authorize, and
> repack
> > > any forwarded requests which flow through the broker. This is probably
> > not
> > > a huge concern from an efficiency perspective as long as we are talking
> > > about just the Admin APIs, but it does have a big maintenance cost
> since
> > > we'll need to ensure that every new field gets properly carried
> through.
> > It
> > > would be nice if we just didn't have to think about that. We also might
> > > eventually come up with reasons to extend forwarding to non-admin APIs,
> > so
> > > it would be nice to start with an efficient approach.
> > >
> > > It seems like the main difference comes down to where the authorization
> > is
> > > done. Suppose that broker B receives an AlterConfig request from the
> > client
> > > in order to change topic configs and wants to forward to controller C.
> > >
> > > Option 1 (no impersonation): B authorizes AlterConfigs for the included
> > > topics with the client principal. Rejected topics are stripped out of
> the
> > > request.  Authorized topics are repackaged into a new request and sent
> in
> > > an envelope to C, which verifies ClusterAction permission with the
> broker
> > > principal and assumes authorization for the underlying request
> > > Option 2 (with impersonation): B authenticates the client, but does no
> > > authorization and forwards the request in an envelope to C containing
> the
> > > authenticated principal. C checks ClusterAction for the envelope
> request
> > > using the broker principal and AlterConfigs for the underlying request
> > > using the forwarded client principal.
> > >
> > > In either case, broker B implicitly gets AlterConfigs permission for
> the
> > > topic. This is true even without the envelope and seems like a
> reasonable
> > > requirement. The broker should itself be authorized to perform any
> action
> > > that it might have to forward requests for. As far as I know, all the
> > > proposals we have considered require this. The main question from a
> > > security perspective is whether any of these proposals require
> additional
> > > unnecessary access, which is probably the main doubt about
> impersonation.
> > > However, there are a couple ways we can restrict it:
> > >
> > > 1. We can restrict the principals that are allowed to be impersonated
> > > 2. We can restrict the actions that are possible through impersonation.
> > >
> > > Considering the first point, there's probably no reason to allow
> > > impersonation of superusers. Additionally, a custom authorizer could
> > forbid
> > > impersonation outside of a particular group. To support this, it would
> be
> > > helpful to extend `KafkaPrincipal` or `AuthorizableRequestContext` so
> > that
> > > it indicates whether a request is an impersonated request.
> > >
> > > Considering the second point, it doesn't make sense to allow arbitrary
> > > requests to be forwarded. We know exactly the set of forwardable APIs
> and
> > > we can reject any other APIs without even looking at the principal.
> This
> > is
> > > the nice thing that the Envelope request gives us. I don't know if we
> > would
> > > ever have finer-grained restrictions, but in principle I don't see why
> we
> > > couldn't.
> > >
> > > In the future, I guess we could go even further so that the broker
> itself
> > > wouldn't need the same permissions as the client. If the client and the
> > > controller shared some secret or if the client had a public key that we
> > > could rely on, then the client could send along a MAC or token of some
> > > sort, which could then be forwarded through the envelope. Then the
> broker
> > > would not be allowed to do anything except exactly what the client
> > > requested. I'm not suggesting we do this, just that we will have the
> > > flexibility for it.
> > >
> > > From the discussion thread, it looks like the main problem here is that
> > > `KafkaPrincipal` does not currently have a defined serialization
> > mechanism.
> > > We can add this, but it is a breaking change, so we have to wait for a
> > > major release version before we make it a requirement. This is fine
> from
> > > the perspective of KIP-500 since the bridge release will require a
> major
> > > release bump anyway. What I imagine we could do is something like this:
> > >
> > > 1. In versions 2.7/2.8/.., we can add the new API in a mixin interface,
> > say
> > > `KafkaPrincipalSerde`. We can implement this interface for
> > > `DefaultKafkaPrincipalBuilder` trivially. On startup, we can check if
> > this
> > > interface is implemented by the provided `KafkaPrincipalBuilder`. If it
> > is
> > > not, we can log a warning and revert to non-forwarding mode.
> > > 2. Starting in version 3.0, we can make `KafkaPrincipalSerde` a
> > requirement
> > > and fail startup if it is not implemented.
> > >
> > > I think this approach would give users enough time to add the required
> > > support. In any case, I don't think compatibility should be a blocker
> > here
> > > if we think it's the best way forward. This seems like a decision which
> > > will affect things for a long time and we need to be allowed to change
> > > plugin APIs.
> > >
> > > Anyway, let me know what you think.
> > >
> > > Best,
> > > Jason
> > >
> > >
> > >
> > > On Fri, Sep 25, 2020 at 2:21 PM Colin McCabe <cm...@apache.org>
> wrote:
> > >
> > > > On Fri, Sep 25, 2020, at 10:49, Boyang Chen wrote:
> > > > > Hey Jun,
> > > > >
> > > > > On Fri, Sep 25, 2020 at 10:19 AM Jun Rao <ju...@confluent.io> wrote:
> > > > >
> > > > > > Hi, Boyang,
> > > > > >
> > > > > > Does EnvelopeRequest avoid the need for IBP? How do we know if
> the
> > > > > > controller supports EnvelopeRequest or not?
> > > > > >
> > > > > > Unfortunately, the EnvelopeRequest is solving the inter-broker
> > > > > > communication problem only. Admin clients still need to learn the
> > > > proper
> > > > > > ApiVersion from the broker, which means we need to bump IBP to
> > limit
> > > > the
> > > > > > version range.
> > > > >
> > > >
> > > > Right-- the purpose of EnvelopeRequest is to avoid downconversion /
> > > > upconversion on the forwarding broker.  It unfortunately doesn't
> avoid
> > > the
> > > > need to tie ApiVersionsResponse to IBP.
> > > >
> > > > > > > On Thu, Sep 24, 2020 at 4:53 PM Jun Rao <ju...@confluent.io>
> > wrote:
> > > > > > >
> > > > > > > > Hi, Jason,
> > > > > > > >
> > > > > > > > Yes, the most important thing is to be able to avoid two
> > rolling
> > > > > > > > restarts
> > > > > > > > in the future. If we have a path to achieve that down the
> road,
> > > the
> > > > > > > > changes here are fine.
> > > > > > > >
> > > >
> > > > Yeah.  I think it would be good to make IBP a feature flag, as long
> as
> > it
> > > > could be changed without doing a second rolling restart.  We actually
> > > don't
> > > > want to have too many feature flags, since it blows up the test
> matrix.
> > > >
> > > > best,
> > > > Colin
> > > >
> > > > > > > > Thanks,
> > > > > > > >
> > > > > > > > Jun
> > > > > > > >
> > > > > > > > On Thu, Sep 24, 2020 at 3:20 PM Jason Gustafson <
> > > > jason@confluent.io>
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > > One of the goals of KIP-584 (feature versioning) is that
> we
> > > > can get
> > > > > > > rid
> > > > > > > > > of
> > > > > > > > > IBP in the future. So does this change prevent us from
> > removing
> > > > IBP
> > > > > > in
> > > > > > > > the
> > > > > > > > > future?
> > > > > > > > >
> > > > > > > > > That is a good question. I think the problem here is that
> > > request
> > > > > > > > > forwarding puts an expectation on api version support which
> > > > covers
> > > > > > more
> > > > > > > > > than one broker. This is why the normal ApiVersions
> behavior
> > > > doesn't
> > > > > > > > work.
> > > > > > > > > I thought about this a bit and haven't come up with a good
> > > > > > alternative.
> > > > > > > > One
> > > > > > > > > thought I've been considering is letting the controller in
> > the
> > > > > > > > post-kip-500
> > > > > > > > > world set the maximum range of api support for the cluster.
> > > > However,
> > > > > > > even
> > > > > > > > > then we would need some way to tell when the controller
> > quorum
> > > > itself
> > > > > > > is
> > > > > > > > > ready to enable support for a new api version. My feeling
> is
> > > > that we
> > > > > > > will
> > > > > > > > > probably always need something like the IBP to control when
> > it
> > > is
> > > > > > safe
> > > > > > > to
> > > > > > > > > expose versions of APIs which have a cross-broker
> dependence.
> > > > > > However,
> > > > > > > > > KIP-584 would still allow us to manage the IBP at the level
> > of
> > > a
> > > > > > > feature
> > > > > > > > so
> > > > > > > > > that we don't need two rolling restarts anymore.
> > > > > > > > >
> > > > > > > > > Best,
> > > > > > > > > Jason
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Thu, Sep 24, 2020 at 1:59 PM Jun Rao <ju...@confluent.io>
> > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hi, Boyang,
> > > > > > > > > >
> > > > > > > > > > One of the goals of KIP-584 (feature versioning) is that
> we
> > > > can get
> > > > > > > rid
> > > > > > > > > of
> > > > > > > > > > IBP in the future. So does this change prevent us from
> > > > removing IBP
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > future?
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > >
> > > > > > > > > > Jun
> > > > > > > > > >
> > > > > > > > > > On Thu, Sep 24, 2020 at 12:46 PM Jason Gustafson <
> > > > > > jason@confluent.io
> > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hey Boyang,
> > > > > > > > > > >
> > > > > > > > > > > Thanks for the update. This seems like the best thing
> we
> > > can
> > > > do.
> > > > > > > The
> > > > > > > > > > > alternative would be to always ensure that the
> forwarded
> > > > APIs are
> > > > > > > > safe
> > > > > > > > > > for
> > > > > > > > > > > conversion between versions, but that would restrict
> the
> > > > > > > flexibility
> > > > > > > > > that
> > > > > > > > > > > the versioning is providing. It would also be a large
> > > effort
> > > > to
> > > > > > > avoid
> > > > > > > > > > > introducing regressions through conversion. Sadly this
> > > > broadens
> > > > > > the
> > > > > > > > > scope
> > > > > > > > > > > of the IBP, but in fact forwarded APIs are inter-broker
> > > APIs.
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > > Jason
> > > > > > > > > > >
> > > > > > > > > > > On Thu, Sep 24, 2020 at 9:23 AM Boyang Chen <
> > > > > > > > > reluctanthero104@gmail.com>
> > > > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Hey there,
> > > > > > > > > > > >
> > > > > > > > > > > > we spotted a necessary case to handle the redirect
> > > request
> > > > > > > > > versioning,
> > > > > > > > > > > and
> > > > > > > > > > > > proposed the following changes:
> > > > > > > > > > > >
> > > > > > > > > > > > 1. For redirection RPCs (AlterConfig, Acl, Token
> etc),
> > > the
> > > > > > > > > > corresponding
> > > > > > > > > > > > allowed versions in the ApiVersionResponse will be
> > > > affected by
> > > > > > > the
> > > > > > > > > > entire
> > > > > > > > > > > > cluster's versioning, not just the receiving broker,
> > > since
> > > > we
> > > > > > > need
> > > > > > > > to
> > > > > > > > > > > > ensure the chosen version get properly handled by all
> > > > parties.
> > > > > > > Thus
> > > > > > > > > > from
> > > > > > > > > > > > now on, RPC with redirection will be treated as
> > > > inter-broker
> > > > > > RPC,
> > > > > > > > and
> > > > > > > > > > any
> > > > > > > > > > > > version bump for these RPCs has to go through IBP
> bump
> > as
> > > > well.
> > > > > > > > > > > > ApiVersionResponse will take IBP into considerations
> > for
> > > > the
> > > > > > > > > > redirection
> > > > > > > > > > > > RPCs allowable versions.
> > > > > > > > > > > >
> > > > > > > > > > > > 2. We would do the best effort to maintain the same
> > > request
> > > > > > > version
> > > > > > > > > for
> > > > > > > > > > > > the entire admin client -> receiving broker ->
> > controller
> > > > > > broker
> > > > > > > > > path,
> > > > > > > > > > > but
> > > > > > > > > > > > for old RPC versions, they may not have flexible
> fields
> > > > > > > introduced
> > > > > > > > > yet.
> > > > > > > > > > > > Thus, we would have to upgrade the RPC to the minimum
> > > > version
> > > > > > > which
> > > > > > > > > > > > supports flexible fields
> > > > > > > > > > > > and add another tagged field in the header called
> > > > > > > > > > > `OriginalRequestVersion`
> > > > > > > > > > > > to help the controller broker correctly deserialize
> the
> > > > request
> > > > > > > > with
> > > > > > > > > > the
> > > > > > > > > > > > original admin client sent out version. We would not
> > > > downgrade
> > > > > > > the
> > > > > > > > > > > original
> > > > > > > > > > > > request in any circumstance, since the flexible field
> > > > support
> > > > > > is
> > > > > > > > > > required
> > > > > > > > > > > > to be open-ended on the high side.
> > > > > > > > > > > >
> > > > > > > > > > > > Let me know if you have any questions.
> > > > > > > > > > > >
> > > > > > > > > > > > Best,
> > > > > > > > > > > > Boyang
> > > > > > > > > > > >
> > > > > > > > > > > > On Thu, Aug 6, 2020 at 6:11 PM Boyang Chen <
> > > > > > > > > reluctanthero104@gmail.com
> > > > > > > > > > >
> > > > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Hey there,
> > > > > > > > > > > > >
> > > > > > > > > > > > > we are going to introduce a minor change to bump
> the
> > > > version
> > > > > > of
> > > > > > > > > > several
> > > > > > > > > > > > > RPCs which are currently not supporting flexible
> > > > versions. It
> > > > > > > is
> > > > > > > > > > > > necessary
> > > > > > > > > > > > > because they need to be able to construct request
> > > header
> > > > with
> > > > > > > > > initial
> > > > > > > > > > > > > principal name and client id as optional fields for
> > > > > > > redirection.
> > > > > > > > > The
> > > > > > > > > > > are
> > > > > > > > > > > > > only two of them:
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1. AlterConfig
> > > > > > > > > > > > > 2. AlterClientQuotas
> > > > > > > > > > > > >
> > > > > > > > > > > > > Let me know if you have any questions.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Boyang
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Fri, Jul 31, 2020 at 11:42 AM Boyang Chen <
> > > > > > > > > > > reluctanthero104@gmail.com
> > > > > > > > > > > > >
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > >> Hey David,
> > > > > > > > > > > > >>
> > > > > > > > > > > > >> After discussing with Colin offline, I would like
> to
> > > > correct
> > > > > > > one
> > > > > > > > > > case
> > > > > > > > > > > in
> > > > > > > > > > > > >> the described workflow, where the CLUSTER_ACTION
> > > > > > authorization
> > > > > > > > > would
> > > > > > > > > > > > not be
> > > > > > > > > > > > >> based on the initial principal field check,
> because
> > it
> > > > is
> > > > > > not
> > > > > > > a
> > > > > > > > > > > secured
> > > > > > > > > > > > >> condition which anyone could forge. The revised
> > > workflow
> > > > > > shall
> > > > > > > > be:
> > > > > > > > > > > > >>
> > > > > > > > > > > > >> Step 1. Filter out resources that are authorized
> > > > > > > > > > > > >>          1.1 Use traditional principals to verify
> > > > first. If
> > > > > > > > > > > authorized,
> > > > > > > > > > > > >> continue
> > > > > > > > > > > > >>          1.2 If not authorized, check whether the
> > > > request is
> > > > > > > > from
> > > > > > > > > > the
> > > > > > > > > > > > >> control plane. Note that this is a best-effort to
> > > verify
> > > > > > > whether
> > > > > > > > > the
> > > > > > > > > > > > >> request is internal.
> > > > > > > > > > > > >>          1.3 If the request is not from the
> control
> > > > plane,
> > > > > > > > return
> > > > > > > > > > > > >> authorization failure
> > > > > > > > > > > > >>          1.4 If the request is from the control
> > plane,
> > > > use
> > > > > > > > > > > > CLUSTER_ACTION
> > > > > > > > > > > > >> to verify and determine the result
> > > > > > > > > > > > >>
> > > > > > > > > > > > >> Step 2. Check the request context to see if this
> is
> > a
> > > > > > > forwarding
> > > > > > > > > > > > request,
> > > > > > > > > > > > >> by checking whether it is from control plane and
> > uses
> > > > extra
> > > > > > > > header
> > > > > > > > > > > > fields
> > > > > > > > > > > > >>         2.1 if the resource is authorized, and if
> > this
> > > > is
> > > > > > the
> > > > > > > > > active
> > > > > > > > > > > > >> controller, process it
> > > > > > > > > > > > >>         2.2 if the resource is authorized but this
> > is
> > > > not
> > > > > > the
> > > > > > > > > active
> > > > > > > > > > > > >> controller, return NOT_CONTROLLER to the sender
> > > > (forwarding
> > > > > > > > > broker)
> > > > > > > > > > > for
> > > > > > > > > > > > >> retry
> > > > > > > > > > > > >>         2.3 if the resource is not authorized,
> > return
> > > > > > > > > > > > >> CLUSTER_AUTHORIZATION_FAILURE to propagate back to
> > the
> > > > > > > original
> > > > > > > > > > client
> > > > > > > > > > > > >> through forwarding broker
> > > > > > > > > > > > >> Step 3. If the request is not a forwarding request
> > > > > > > > > > > > >>         3.1 If the resource is authorized, and
> this
> > is
> > > > the
> > > > > > > > active
> > > > > > > > > > > > >> controller, process it
> > > > > > > > > > > > >>         3.2 If the resource is authorized, but
> this
> > is
> > > > not
> > > > > > > > active
> > > > > > > > > > > > >> controller, put the resource into the preparation
> > for
> > > a
> > > > new
> > > > > > > > > > > AlterConfig
> > > > > > > > > > > > >> request for forwarding
> > > > > > > > > > > > >>         3.3 If the resource is not authorized,
> reply
> > > the
> > > > > > > > original
> > > > > > > > > > > client
> > > > > > > > > > > > >> AUTHORIZATION_FAILURE when the forwarding request
> is
> > > > > > returned
> > > > > > > > > > > > >>
> > > > > > > > > > > > >> On Thu, Jul 30, 2020 at 3:47 PM Boyang Chen <
> > > > > > > > > > > reluctanthero104@gmail.com
> > > > > > > > > > > > >
> > > > > > > > > > > > >> wrote:
> > > > > > > > > > > > >>
> > > > > > > > > > > > >>>
> > > > > > > > > > > > >>>
> > > > > > > > > > > > >>> On Thu, Jul 30, 2020 at 7:18 AM David Jacot <
> > > > > > > > djacot@confluent.io
> > > > > > > > > >
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > >>>
> > > > > > > > > > > > >>>> Hi Boyang,
> > > > > > > > > > > > >>>>
> > > > > > > > > > > > >>>> Thanks for your answers.
> > > > > > > > > > > > >>>>
> > > > > > > > > > > > >>>> > The point for using the listener name is more
> > of a
> > > > > > > security
> > > > > > > > > > > purpose,
> > > > > > > > > > > > >>>> to
> > > > > > > > > > > > >>>> > detect any forged request to our best effort.
> > > > > > > > > > > > >>>> > For throttling I think we could just check the
> > > > request
> > > > > > > > header
> > > > > > > > > > for
> > > > > > > > > > > > >>>> > *InitialClientId* existence, to distinguish
> > > whether
> > > > to
> > > > > > > apply
> > > > > > > > > > > > >>>> > throttling strategy as forwarded request or
> > direct
> > > > > > > request.
> > > > > > > > > > > > >>>>
> > > > > > > > > > > > >>>> Reading "security" and "best effort" in the same
> > > > sentence
> > > > > > > > makes
> > > > > > > > > > me a
> > > > > > > > > > > > >>>> little nervous :).
> > > > > > > > > > > > >>>>
> > > > > > > > > > > > >>>> The identification issue is also valid for quota
> > as
> > > we
> > > > > > don't
> > > > > > > > > want
> > > > > > > > > > > one
> > > > > > > > > > > > >>>> to be
> > > > > > > > > > > > >>>> able to bypass the quota by forging a request as
> > > well,
> > > > > > isn't
> > > > > > > > it?
> > > > > > > > > > > > >>>> Otherwise,
> > > > > > > > > > > > >>>> anyone could just set the InitialPrincipal to
> > bypass
> > > > it. I
> > > > > > > > think
> > > > > > > > > > > that
> > > > > > > > > > > > we
> > > > > > > > > > > > >>>> should
> > > > > > > > > > > > >>>> only use InitialPrincipal and/or InitialClientId
> > > when
> > > > we
> > > > > > > know
> > > > > > > > > that
> > > > > > > > > > > > they
> > > > > > > > > > > > >>>> come
> > > > > > > > > > > > >>>> from another broker. Based on what I read in the
> > > KIP,
> > > > it
> > > > > > > looks
> > > > > > > > > > like
> > > > > > > > > > > we
> > > > > > > > > > > > >>>> could
> > > > > > > > > > > > >>>> only use them when the principal has
> > CLUSTER_ACTION
> > > > > > > privilege.
> > > > > > > > > Do
> > > > > > > > > > I
> > > > > > > > > > > > >>>> understand it correctly?
> > > > > > > > > > > > >>>>
> > > > > > > > > > > > >>>> There is no 100% safe way to distinguish between
> > raw
> > > > > > request
> > > > > > > > and
> > > > > > > > > > > > >>> forward request if you take malicious request
> into
> > > > > > > > consideration,
> > > > > > > > > > > which
> > > > > > > > > > > > >>> could happen
> > > > > > > > > > > > >>> anyway if the attacker prepares those requests to
> > > pass
> > > > the
> > > > > > > > checks
> > > > > > > > > > > for a
> > > > > > > > > > > > >>> pre-KIP-500 cluster.
> > > > > > > > > > > > >>> We could at most know whether it is sent to the
> > > control
> > > > > > > plane,
> > > > > > > > or
> > > > > > > > > > the
> > > > > > > > > > > > >>> data plane, and whether it has extra header
> fields
> > > > such as
> > > > > > > > > initial
> > > > > > > > > > > > >>> principal and client id defined. For a
> > non-malicious
> > > > > > request
> > > > > > > > > > > > >>> going into the control plane, it must be sent
> from
> > a
> > > > valid
> > > > > > > > > broker,
> > > > > > > > > > > > which
> > > > > > > > > > > > >>> is a prerequisite to check its
> > > > > > > > > > > > >>> CLUSTER_ACTION principal. Take AlterConfig for an
> > > > example,
> > > > > > > the
> > > > > > > > > > > intended
> > > > > > > > > > > > >>> workflow for a *KIP-590 broker* would be:
> > > > > > > > > > > > >>>
> > > > > > > > > > > > >>> Step 1. Check the request context to see if this
> > is a
> > > > > > > > forwarding
> > > > > > > > > > > > >>> request, by checking whether it is from control
> > plane
> > > > and
> > > > > > > uses
> > > > > > > > > > extra
> > > > > > > > > > > > header
> > > > > > > > > > > > >>> fields
> > > > > > > > > > > > >>>         1.1 if it is a forwarding request, use
> > > > > > CLUSTER_ACTION
> > > > > > > > to
> > > > > > > > > > > verify
> > > > > > > > > > > > >>> the given resource
> > > > > > > > > > > > >>>         1.2 if the resource is authorized, and if
> > > this
> > > > is
> > > > > > the
> > > > > > > > > > active
> > > > > > > > > > > > >>> controller, process it
> > > > > > > > > > > > >>>         1.3 if the resource is authorized but
> this
> > is
> > > > not
> > > > > > the
> > > > > > > > > > active
> > > > > > > > > > > > >>> controller, return NOT_CONTROLLER to the sender
> > > > (forwarding
> > > > > > > > > broker)
> > > > > > > > > > > for
> > > > > > > > > > > > >>> retry
> > > > > > > > > > > > >>>         1.4 if the resource is not authorized,
> > return
> > > > > > > > > > > > >>> CLUSTER_AUTHORIZATION_FAILURE to propagate back
> to
> > > the
> > > > > > > original
> > > > > > > > > > > client
> > > > > > > > > > > > >>> through forwarding broker
> > > > > > > > > > > > >>> Step 2. If the request is not a forwarding
> request
> > > > > > > > > > > > >>>         2.1 Verify with normal principal as ALTER
> > on
> > > > the
> > > > > > > given
> > > > > > > > > > > > resources
> > > > > > > > > > > > >>>         2.2 If the resource is authorized, and
> this
> > > is
> > > > the
> > > > > > > > active
> > > > > > > > > > > > >>> controller, process it
> > > > > > > > > > > > >>>         2.3 If the resource is authorized, but
> this
> > > is
> > > > not
> > > > > > > > active
> > > > > > > > > > > > >>> controller, put the resource into the preparation
> > for
> > > > a new
> > > > > > > > > > > AlterConfig
> > > > > > > > > > > > >>> request for forwarding
> > > > > > > > > > > > >>>         2.4 If the resource is not authorized,
> > reply
> > > > the
> > > > > > > > original
> > > > > > > > > > > > client
> > > > > > > > > > > > >>> AUTHORIZATION_FAILURE when the forwarding request
> > is
> > > > > > returned
> > > > > > > > > > > > >>>
> > > > > > > > > > > > >>> When the control plane and data plane are using
> the
> > > > same
> > > > > > > > > listener,
> > > > > > > > > > we
> > > > > > > > > > > > >>> couldn't distinguish whether a request is
> forwarded
> > > or
> > > > not
> > > > > > > for
> > > > > > > > > > sure,
> > > > > > > > > > > > so in
> > > > > > > > > > > > >>> terms of the forward request checking, we have to
> > > > require
> > > > > > the
> > > > > > > > > extra
> > > > > > > > > > > > header
> > > > > > > > > > > > >>> fields to present. A stronger checking mechanism
> > > could
> > > > only
> > > > > > > be
> > > > > > > > > > > enforced
> > > > > > > > > > > > >>> when we upgrade to post-KIP-500 with a separate
> > > > controller
> > > > > > > > > network.
> > > > > > > > > > > > >>>
> > > > > > > > > > > > >>>
> > > > > > > > > > > > >>>> I have made another pass on the whole KIP, I
> have
> > > few
> > > > > > nits:
> > > > > > > > > > > > >>>>
> > > > > > > > > > > > >>>> - The sentence "Take AlterConfig as an example
> to
> > > > > > understand
> > > > > > > > the
> > > > > > > > > > > > changes
> > > > > > > > > > > > >>>> we are making." does not make much sense anymore
> > in
> > > > the
> > > > > > > > > beginning
> > > > > > > > > > of
> > > > > > > > > > > > the
> > > > > > > > > > > > >>>> "Proposed Changes" chapter.
> > > > > > > > > > > > >>>>
> > > > > > > > > > > > >>>> Sure, deleted.
> > > > > > > > > > > > >>>
> > > > > > > > > > > > >>>
> > > > > > > > > > > > >>>> - When you say "Existing RPCs which are sending
> > > > directly
> > > > > > to
> > > > > > > > the
> > > > > > > > > > > > >>>> controller
> > > > > > > > > > > > >>>> will
> > > > > > > > > > > > >>>> rely on forwarding as well.". I suggest to
> > > explicitly
> > > > > > > mention
> > > > > > > > > how
> > > > > > > > > > > "old
> > > > > > > > > > > > >>>> admin clients"
> > > > > > > > > > > > >>>> will work here to complement the sentence.
> > Something
> > > > like:
> > > > > > > > They
> > > > > > > > > > will
> > > > > > > > > > > > >>>> get a
> > > > > > > > > > > > >>>> random
> > > > > > > > > > > > >>>> broker id as the controller id in the metadata
> > > > response
> > > > > > and
> > > > > > > > > stick
> > > > > > > > > > to
> > > > > > > > > > > > it
> > > > > > > > > > > > >>>> as
> > > > > > > > > > > > >>>> you explained.
> > > > > > > > > > > > >>>>
> > > > > > > > > > > > >>>> Sg, changed.
> > > > > > > > > > > > >>>
> > > > > > > > > > > > >>>
> > > > > > > > > > > > >>>> - "The purpose of adding principal name is for
> the
> > > > audit
> > > > > > > > > logging,
> > > > > > > > > > > and
> > > > > > > > > > > > >>>> the
> > > > > > > > > > > > >>>> client id is
> > > > > > > > > > > > >>>> being used to throttling according to KIP-599
> > > > > > requirement."
> > > > > > > > > > > Actually,
> > > > > > > > > > > > >>>> KIP-599 needs
> > > > > > > > > > > > >>>> both the principal and the clientId.
> > > > > > > > > > > > >>>>
> > > > > > > > > > > > >>>> Makes sense.
> > > > > > > > > > > > >>>
> > > > > > > > > > > > >>>
> > > > > > > > > > > > >>>> - In the "Routing Request Security" chapter. It
> is
> > > > written
> > > > > > > > that
> > > > > > > > > > the
> > > > > > > > > > > > >>>> forwarding broker
> > > > > > > > > > > > >>>> will verify the request with its own authorizer
> > and
> > > > will
> > > > > > > just
> > > > > > > > > > > forward
> > > > > > > > > > > > >>>> it if
> > > > > > > > > > > > >>>> the request
> > > > > > > > > > > > >>>> looks good. When a request contains for instance
> > > > multiple
> > > > > > > > > topics,
> > > > > > > > > > I
> > > > > > > > > > > > >>>> suppose
> > > > > > > > > > > > >>>> that
> > > > > > > > > > > > >>>> we will forward only the authorized ones and not
> > the
> > > > whole
> > > > > > > > > > original
> > > > > > > > > > > > >>>> request
> > > > > > > > > > > > >>>> as is.
> > > > > > > > > > > > >>>> We may want to reword the sentence to make this
> > > clear.
> > > > > > > > > > > > >>>>
> > > > > > > > > > > > >>>> That makes sense, I will put this more detailed
> > > > version of
> > > > > > > > > routing
> > > > > > > > > > > > into
> > > > > > > > > > > > >>> the design.
> > > > > > > > > > > > >>>
> > > > > > > > > > > > >>>
> > > > > > > > > > > > >>>> - For the record, should we put the previous
> > > proposal
> > > > in
> > > > > > the
> > > > > > > > > > > rejected
> > > > > > > > > > > > >>>> alternatives as
> > > > > > > > > > > > >>>> well?
> > > > > > > > > > > > >>>>
> > > > > > > > > > > > >>>> We do have big changes in this KIP, our current
> > > > strategy
> > > > > > is
> > > > > > > to
> > > > > > > > > > rely
> > > > > > > > > > > on
> > > > > > > > > > > > >>> wiki revisions if people
> > > > > > > > > > > > >>> are interested to figure out our previous design
> > > > strategy.
> > > > > > > > > Putting
> > > > > > > > > > > the
> > > > > > > > > > > > >>> full design in current KIP proposal
> > > > > > > > > > > > >>> would distract too much for readers as we
> discussed
> > > > during
> > > > > > > the
> > > > > > > > > last
> > > > > > > > > > > > time
> > > > > > > > > > > > >>> of big refactoring.
> > > > > > > > > > > > >>>
> > > > > > > > > > > > >>> Best,
> > > > > > > > > > > > >>>> David
> > > > > > > > > > > > >>>>
> > > > > > > > > > > > >>>> On Thu, Jul 30, 2020 at 3:51 AM Boyang Chen <
> > > > > > > > > > > > reluctanthero104@gmail.com
> > > > > > > > > > > > >>>> >
> > > > > > > > > > > > >>>> wrote:
> > > > > > > > > > > > >>>>
> > > > > > > > > > > > >>>> > Thanks David for the feedback!
> > > > > > > > > > > > >>>> >
> > > > > > > > > > > > >>>> > On Wed, Jul 29, 2020 at 7:53 AM David Jacot <
> > > > > > > > > > djacot@confluent.io>
> > > > > > > > > > > > >>>> wrote:
> > > > > > > > > > > > >>>> >
> > > > > > > > > > > > >>>> > > Hi, Colin, Boyang,
> > > > > > > > > > > > >>>> > >
> > > > > > > > > > > > >>>> > > Colin, thanks for the clarification.
> Somehow,
> > I
> > > > > > thought
> > > > > > > > that
> > > > > > > > > > > even
> > > > > > > > > > > > >>>> if the
> > > > > > > > > > > > >>>> > > controller is ran independently, it
> > > > > > > > > > > > >>>> > > would still run the listeners of the broker
> > and
> > > > thus
> > > > > > > would
> > > > > > > > > be
> > > > > > > > > > > > >>>> accessible
> > > > > > > > > > > > >>>> > by
> > > > > > > > > > > > >>>> > > redirecting on the loopback
> > > > > > > > > > > > >>>> > > interface. My mistake.
> > > > > > > > > > > > >>>> > >
> > > > > > > > > > > > >>>> > > Boyang, I have few questions/comments
> > regarding
> > > > the
> > > > > > > > updated
> > > > > > > > > > KIP:
> > > > > > > > > > > > >>>> > >
> > > > > > > > > > > > >>>> > > 1. I think that it would be great if we
> could
> > > > clarify
> > > > > > > how
> > > > > > > > > old
> > > > > > > > > > > > admin
> > > > > > > > > > > > >>>> > clients
> > > > > > > > > > > > >>>> > > which are directly talking to the
> > > > > > > > > > > > >>>> > > controller will work with this KIP. I read
> > > > between the
> > > > > > > > lines
> > > > > > > > > > > that,
> > > > > > > > > > > > >>>> as we
> > > > > > > > > > > > >>>> > > propose to provide a random
> > > > > > > > > > > > >>>> > > broker Id as the controller Id in the
> metadata
> > > > > > response,
> > > > > > > > > they
> > > > > > > > > > > will
> > > > > > > > > > > > >>>> use a
> > > > > > > > > > > > >>>> > > single node as a proxy. Is that
> > > > > > > > > > > > >>>> > > correct? This deserves to be called out more
> > > > > > explicitly
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > > > >>>> design
> > > > > > > > > > > > >>>> > > section instead of being hidden
> > > > > > > > > > > > >>>> > > in the protocol bump of the metadata RPC.
> > > > > > > > > > > > >>>> > >
> > > > > > > > > > > > >>>> > > Makes sense, I stress this point in the
> > > > compatibility
> > > > > > > > > section.
> > > > > > > > > > > > >>>> >
> > > > > > > > > > > > >>>> >
> > > > > > > > > > > > >>>> > > 1.1 If I understand correctly, could we
> assume
> > > > that
> > > > > > old
> > > > > > > > > admin
> > > > > > > > > > > > >>>> clients
> > > > > > > > > > > > >>>> > will
> > > > > > > > > > > > >>>> > > stick to the same "fake controller"
> > > > > > > > > > > > >>>> > > until they refresh their metadata?
> Refreshing
> > > the
> > > > > > > metadata
> > > > > > > > > > > usually
> > > > > > > > > > > > >>>> > happens
> > > > > > > > > > > > >>>> > > when NOT_CONTROLLER
> > > > > > > > > > > > >>>> > > is received but this won't happen anymore so
> > > they
> > > > > > should
> > > > > > > > > > change
> > > > > > > > > > > > >>>> > > infrequently.
> > > > > > > > > > > > >>>> > >
> > > > > > > > > > > > >>>> > > That is correct, old admin clients would not
> > try
> > > > to
> > > > > > > > refresh
> > > > > > > > > > > their
> > > > > > > > > > > > >>>> > metadata
> > > > > > > > > > > > >>>> > due to NOT_CONTROLLER,
> > > > > > > > > > > > >>>> > which is impossible to happen with the new
> > broker
> > > > > > cluster.
> > > > > > > > > > > > >>>> >
> > > > > > > > > > > > >>>> >
> > > > > > > > > > > > >>>> > > 2. For the new admin client, I suppose that
> we
> > > > plan on
> > > > > > > > using
> > > > > > > > > > > > >>>> > > LeastLoadedNodeProvider for the
> > > > > > > > > > > > >>>> > > requests that are using
> > ControllerNodeProvider.
> > > We
> > > > > > could
> > > > > > > > > > perhaps
> > > > > > > > > > > > >>>> mention
> > > > > > > > > > > > >>>> > > it.
> > > > > > > > > > > > >>>> > >
> > > > > > > > > > > > >>>> > > Sure, added.
> > > > > > > > > > > > >>>> >
> > > > > > > > > > > > >>>> >
> > > > > > > > > > > > >>>> > > 3. Pre KIP-500, will we have a way to
> > > distinguish
> > > > if a
> > > > > > > > > request
> > > > > > > > > > > > that
> > > > > > > > > > > > >>>> is
> > > > > > > > > > > > >>>> > > received by the controller is
> > > > > > > > > > > > >>>> > > coming directly from a client or from a
> > broker?
> > > > You
> > > > > > > > mention
> > > > > > > > > > that
> > > > > > > > > > > > the
> > > > > > > > > > > > >>>> > > listener can be used to do
> > > > > > > > > > > > >>>> > > this but as you pointed out, it is not
> > > mandatory.
> > > > Do
> > > > > > we
> > > > > > > > have
> > > > > > > > > > > > another
> > > > > > > > > > > > >>>> > > reliable method? I am asking
> > > > > > > > > > > > >>>> > > in the context of KIP-599 with the current
> > > > controller,
> > > > > > > we
> > > > > > > > > may
> > > > > > > > > > > need
> > > > > > > > > > > > >>>> to
> > > > > > > > > > > > >>>> > > throttle differently if the
> > > > > > > > > > > > >>>> > > request comes from a client or from a
> broker.
> > > > > > > > > > > > >>>> > >
> > > > > > > > > > > > >>>> > > The point for using the listener name is
> more
> > > of a
> > > > > > > > security
> > > > > > > > > > > > >>>> purpose, to
> > > > > > > > > > > > >>>> > detect any forged request to our best effort.
> > > > > > > > > > > > >>>> > For throttling I think we could just check the
> > > > request
> > > > > > > > header
> > > > > > > > > > for
> > > > > > > > > > > > >>>> > *InitialClientId* existence, to distinguish
> > > whether
> > > > to
> > > > > > > apply
> > > > > > > > > > > > >>>> > throttling strategy as forwarded request or
> > direct
> > > > > > > request.
> > > > > > > > > > > > >>>> >
> > > > > > > > > > > > >>>> >
> > > > > > > > > > > > >>>> > > 4. Could we add `InitialClientId` as well?
> > This
> > > > will
> > > > > > be
> > > > > > > > > > required
> > > > > > > > > > > > >>>> for the
> > > > > > > > > > > > >>>> > > quota as we can apply them
> > > > > > > > > > > > >>>> > > by principal and/or clientId.
> > > > > > > > > > > > >>>> > >
> > > > > > > > > > > > >>>> > > Sounds good, added.
> > > > > > > > > > > > >>>> >
> > > > > > > > > > > > >>>> >
> > > > > > > > > > > > >>>> > > 5. A small remark regarding the structure of
> > the
> > > > KIP.
> > > > > > It
> > > > > > > > is
> > > > > > > > > a
> > > > > > > > > > > bit
> > > > > > > > > > > > >>>> weird
> > > > > > > > > > > > >>>> > > that requests that do not go
> > > > > > > > > > > > >>>> > > to the controller are mentioned in the
> > Proposed
> > > > Design
> > > > > > > > > section
> > > > > > > > > > > and
> > > > > > > > > > > > >>>> the
> > > > > > > > > > > > >>>> > > requests that go to the
> > > > > > > > > > > > >>>> > > controller are mentioned in the Public
> > > Interfaces.
> > > > > > When
> > > > > > > > one
> > > > > > > > > > read
> > > > > > > > > > > > the
> > > > > > > > > > > > >>>> > > Proposed Design, it does not
> > > > > > > > > > > > >>>> > > get a full picture of the whole new routing
> > > > proposal
> > > > > > for
> > > > > > > > old
> > > > > > > > > > and
> > > > > > > > > > > > new
> > > > > > > > > > > > >>>> > > clients. It would be great if we
> > > > > > > > > > > > >>>> > > could have a full overview in that section.
> > > > > > > > > > > > >>>> > >
> > > > > > > > > > > > >>>> > > Good point, I will move the pieces around.
> > > > > > > > > > > > >>>> >
> > > > > > > > > > > > >>>> >
> > > > > > > > > > > > >>>> > > Overall the change makes sense to me. I will
> > > work
> > > > on
> > > > > > > > > drafting
> > > > > > > > > > an
> > > > > > > > > > > > >>>> addendum
> > > > > > > > > > > > >>>> > > to KIP-599 to
> > > > > > > > > > > > >>>> > > alter the design to cope with these changes.
> > At
> > > a
> > > > > > first
> > > > > > > > > > glance,
> > > > > > > > > > > > that
> > > > > > > > > > > > >>>> > seems
> > > > > > > > > > > > >>>> > > doable if 1.1, 3
> > > > > > > > > > > > >>>> > > and 4 are OK.
> > > > > > > > > > > > >>>> > >
> > > > > > > > > > > > >>>> > > Thank you for the help!
> > > > > > > > > > > > >>>> >
> > > > > > > > > > > > >>>> >
> > > > > > > > > > > > >>>> > > Thanks,
> > > > > > > > > > > > >>>> > > David
> > > > > > > > > > > > >>>> > >
> > > > > > > > > > > > >>>> > > On Wed, Jul 29, 2020 at 5:29 AM Boyang Chen
> <
> > > > > > > > > > > > >>>> reluctanthero104@gmail.com>
> > > > > > > > > > > > >>>> > > wrote:
> > > > > > > > > > > > >>>> > >
> > > > > > > > > > > > >>>> > > > Thanks for the feedback Colin!
> > > > > > > > > > > > >>>> > > >
> > > > > > > > > > > > >>>> > > > On Tue, Jul 28, 2020 at 2:11 PM Colin
> > McCabe <
> > > > > > > > > > > > cmccabe@apache.org>
> > > > > > > > > > > > >>>> > wrote:
> > > > > > > > > > > > >>>> > > >
> > > > > > > > > > > > >>>> > > > > Hi Boyang,
> > > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > > >>>> > > > > Thanks for updating this.  A few
> comments
> > > > below:
> > > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > > >>>> > > > > In the "Routing Request Security"
> section,
> > > > there
> > > > > > is
> > > > > > > a
> > > > > > > > > > > > reference
> > > > > > > > > > > > >>>> to
> > > > > > > > > > > > >>>> > > "older
> > > > > > > > > > > > >>>> > > > > requests that need redirection."  But
> > after
> > > > these
> > > > > > > new
> > > > > > > > > > > > >>>> revisions, both
> > > > > > > > > > > > >>>> > > new
> > > > > > > > > > > > >>>> > > > > and old requests need redirection.  So
> we
> > > > should
> > > > > > > > > rephrase
> > > > > > > > > > > > this.
> > > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > > >>>> > > > > > In addition, to avoid exposing this
> > > > forwarding
> > > > > > > power
> > > > > > > > > to
> > > > > > > > > > > the
> > > > > > > > > > > > >>>> admin
> > > > > > > > > > > > >>>> > > > > clients,
> > > > > > > > > > > > >>>> > > > > > the routing request shall be forwarded
> > > > towards
> > > > > > the
> > > > > > > > > > > > controller
> > > > > > > > > > > > >>>> > broker
> > > > > > > > > > > > >>>> > > > > internal
> > > > > > > > > > > > >>>> > > > > > endpoint which should be only visible
> to
> > > > other
> > > > > > > > brokers
> > > > > > > > > > > > inside
> > > > > > > > > > > > >>>> the
> > > > > > > > > > > > >>>> > > > > cluster
> > > > > > > > > > > > >>>> > > > > > in the KIP-500 controller. Any admin
> > > > > > configuration
> > > > > > > > > > request
> > > > > > > > > > > > >>>> with
> > > > > > > > > > > > >>>> > > broker
> > > > > > > > > > > > >>>> > > > > > principal should not be going through
> > the
> > > > public
> > > > > > > > > > endpoint
> > > > > > > > > > > > and
> > > > > > > > > > > > >>>> will
> > > > > > > > > > > > >>>> > be
> > > > > > > > > > > > >>>> > > > > > rejected for security purpose.
> > > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > > >>>> > > > > We should also describe how this will
> work
> > > in
> > > > the
> > > > > > > > > > > pre-KIP-500
> > > > > > > > > > > > >>>> case.
> > > > > > > > > > > > >>>> > In
> > > > > > > > > > > > >>>> > > > > that case, CLUSTER_ACTION gets the extra
> > > > > > permissions
> > > > > > > > > > > described
> > > > > > > > > > > > >>>> here
> > > > > > > > > > > > >>>> > > only
> > > > > > > > > > > > >>>> > > > > when the message comes in on the
> > > inter-broker
> > > > > > > > listener.
> > > > > > > > > > We
> > > > > > > > > > > > >>>> should
> > > > > > > > > > > > >>>> > > state
> > > > > > > > > > > > >>>> > > > > that here.
> > > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > > >>>> > > > > (I can see that you have this
> information
> > > > later on
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > > > >>>> "Security
> > > > > > > > > > > > >>>> > > > Access
> > > > > > > > > > > > >>>> > > > > Changes" section, but it would be good
> to
> > > > have it
> > > > > > > here
> > > > > > > > > as
> > > > > > > > > > > > well,
> > > > > > > > > > > > >>>> to
> > > > > > > > > > > > >>>> > > avoid
> > > > > > > > > > > > >>>> > > > > confusion.)
> > > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > > >>>> > > > > > To be more strict of protecting
> > controller
> > > > > > > > > information,
> > > > > > > > > > > the
> > > > > > > > > > > > >>>> > > > > "ControllerId"
> > > > > > > > > > > > >>>> > > > > > field in new MetadataResponse shall be
> > set
> > > > to -1
> > > > > > > > when
> > > > > > > > > > the
> > > > > > > > > > > > >>>> original
> > > > > > > > > > > > >>>> > > > > request
> > > > > > > > > > > > >>>> > > > > > comes from a non-broker client and it
> is
> > > > already
> > > > > > > on
> > > > > > > > > v10.
> > > > > > > > > > > We
> > > > > > > > > > > > >>>> shall
> > > > > > > > > > > > >>>> > use
> > > > > > > > > > > > >>>> > > > the
> > > > > > > > > > > > >>>> > > > > > request listener name to distinguish
> > > > whether a
> > > > > > > given
> > > > > > > > > > > request
> > > > > > > > > > > > >>>> is
> > > > > > > > > > > > >>>> > > > > inter-broker,
> > > > > > > > > > > > >>>> > > > > > or from the client.
> > > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > > >>>> > > > > I'm not sure why we would need to
> > > distinguish
> > > > > > > between
> > > > > > > > > > broker
> > > > > > > > > > > > >>>> clients
> > > > > > > > > > > > >>>> > > and
> > > > > > > > > > > > >>>> > > > > non-broker clients.  Brokers don't
> > generally
> > > > send
> > > > > > > > > > > > >>>> MetadataRequests to
> > > > > > > > > > > > >>>> > > > other
> > > > > > > > > > > > >>>> > > > > brokers, do they?  Brokers learn about
> > > > metadata
> > > > > > from
> > > > > > > > > > > > >>>> > > > UpdateMetadataRequest
> > > > > > > > > > > > >>>> > > > > and LeaderAndIsrRequest, not by sending
> > > > > > > > MetadataRequests
> > > > > > > > > > to
> > > > > > > > > > > > >>>> other
> > > > > > > > > > > > >>>> > > > brokers.
> > > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > > >>>> > > > > We do have one use case where the
> > > > MetadataRequest
> > > > > > > gets
> > > > > > > > > > sent
> > > > > > > > > > > > >>>> between
> > > > > > > > > > > > >>>> > the
> > > > > > > > > > > > >>>> > > > brokers, which is the
> InterBrokerSendThread.
> > > > > > Currently
> > > > > > > > we
> > > > > > > > > > > don't
> > > > > > > > > > > > >>>> rely on
> > > > > > > > > > > > >>>> > > it
> > > > > > > > > > > > >>>> > > > to get the controller id, so I guess your
> > > > suggestion
> > > > > > > > > should
> > > > > > > > > > be
> > > > > > > > > > > > >>>> good to
> > > > > > > > > > > > >>>> > > > enforce. We could use some meta comment on
> > the
> > > > > > > > > NetworkClient
> > > > > > > > > > > > that
> > > > > > > > > > > > >>>> it
> > > > > > > > > > > > >>>> > > should
> > > > > > > > > > > > >>>> > > > not be used to get the controller
> location.
> > > > > > > > > > > > >>>> > > >
> > > > > > > > > > > > >>>> > > > Probably what we want here is: v0-v9:
> > return a
> > > > > > random
> > > > > > > > > broker
> > > > > > > > > > > in
> > > > > > > > > > > > >>>> the
> > > > > > > > > > > > >>>> > > cluster
> > > > > > > > > > > > >>>> > > > > as the controller ID.  v10: no
> > controllerID
> > > > > > present
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > > > >>>> > > > > MetadataResponse.  We should also
> > deprecate
> > > > the
> > > > > > > > > > adminClient
> > > > > > > > > > > > >>>> method
> > > > > > > > > > > > >>>> > > which
> > > > > > > > > > > > >>>> > > > > gets the controllerId.
> > > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > > >>>> > > > > > BROKER_AUTHORIZATION_FAILURE(92,
> > > > "Authorization
> > > > > > > > failed
> > > > > > > > > > for
> > > > > > > > > > > > the
> > > > > > > > > > > > >>>> > > > > > request during forwarding, this
> > indicates
> > > an
> > > > > > > > internal
> > > > > > > > > > > error
> > > > > > > > > > > > >>>> on the
> > > > > > > > > > > > >>>> > > > broker
> > > > > > > > > > > > >>>> > > > > > cluster security setup.",
> > > > > > > > > > > > >>>> > BrokerAuthorizationFailureException::new);
> > > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > > >>>> > > > > Grammar nitpick: It would be good to
> have
> > a
> > > > period
> > > > > > > > > between
> > > > > > > > > > > > >>>> > "forwarding"
> > > > > > > > > > > > >>>> > > > > and "this" to avoid a run-on sentence :)
> > > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > > >>>> > > > > best,
> > > > > > > > > > > > >>>> > > > > Colin
> > > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > > >>>> > > > > On Mon, Jul 27, 2020, at 21:47, Boyang
> > Chen
> > > > wrote:
> > > > > > > > > > > > >>>> > > > > > Hey there,
> > > > > > > > > > > > >>>> > > > > >
> > > > > > > > > > > > >>>> > > > > > I'm re-opening this thread because
> after
> > > > some
> > > > > > > > initial
> > > > > > > > > > > > >>>> > implementations
> > > > > > > > > > > > >>>> > > > > > started, we spotted some gaps in the
> > > > approved
> > > > > > KIP
> > > > > > > as
> > > > > > > > > > well
> > > > > > > > > > > as
> > > > > > > > > > > > >>>> some
> > > > > > > > > > > > >>>> > > > > > inconsistencies with KIP-631
> controller.
> > > > There
> > > > > > > are a
> > > > > > > > > > > couple
> > > > > > > > > > > > of
> > > > > > > > > > > > >>>> > > > addendums
> > > > > > > > > > > > >>>> > > > > to
> > > > > > > > > > > > >>>> > > > > > the existing KIP, specifically:
> > > > > > > > > > > > >>>> > > > > >
> > > > > > > > > > > > >>>> > > > > > 1. As the controller is foreseen to be
> > > only
> > > > > > > > accessible
> > > > > > > > > > to
> > > > > > > > > > > > the
> > > > > > > > > > > > >>>> > > brokers,
> > > > > > > > > > > > >>>> > > > > the
> > > > > > > > > > > > >>>> > > > > > new admin client would not have direct
> > > > access to
> > > > > > > the
> > > > > > > > > > > > >>>> controller. It
> > > > > > > > > > > > >>>> > > is
> > > > > > > > > > > > >>>> > > > > > guaranteed on the MetadataResponse
> level
> > > > which
> > > > > > no
> > > > > > > > > longer
> > > > > > > > > > > > >>>> provides
> > > > > > > > > > > > >>>> > > > > > `ControllerId` to client side
> requests.
> > > > > > > > > > > > >>>> > > > > >
> > > > > > > > > > > > >>>> > > > > > 2. The broker would forward any direct
> > ZK
> > > > path
> > > > > > > > > mutation
> > > > > > > > > > > > >>>> requests,
> > > > > > > > > > > > >>>> > > > > including
> > > > > > > > > > > > >>>> > > > > > topic creation/deletion, reassignment,
> > etc
> > > > since
> > > > > > > we
> > > > > > > > > > > > deprecate
> > > > > > > > > > > > >>>> the
> > > > > > > > > > > > >>>> > > > direct
> > > > > > > > > > > > >>>> > > > > > controller access on the client side.
> No
> > > > more
> > > > > > > > protocol
> > > > > > > > > > > > >>>> version bump
> > > > > > > > > > > > >>>> > > is
> > > > > > > > > > > > >>>> > > > > > necessary for the configuration
> > requests.
> > > > > > > > > > > > >>>> > > > > >
> > > > > > > > > > > > >>>> > > > > > 3. To make sure forwarding requests
> pass
> > > the
> > > > > > > > > > > authorization,
> > > > > > > > > > > > >>>> broker
> > > > > > > > > > > > >>>> > > > > > principal CLUSTER_ACTION would be
> > allowed
> > > > to be
> > > > > > > used
> > > > > > > > > as
> > > > > > > > > > an
> > > > > > > > > > > > >>>> > > alternative
> > > > > > > > > > > > >>>> > > > > > authentication method for a variety of
> > > > principal
> > > > > > > > > > > operations,
> > > > > > > > > > > > >>>> > > including
> > > > > > > > > > > > >>>> > > > > > ALTER, ALTER_CONFIG, DELETE, etc. It
> is
> > > > because
> > > > > > > the
> > > > > > > > > > > > forwarding
> > > > > > > > > > > > >>>> > > request
> > > > > > > > > > > > >>>> > > > > > needs to use the proxy broker's own
> > > > principal,
> > > > > > > which
> > > > > > > > > is
> > > > > > > > > > > > >>>> currently
> > > > > > > > > > > > >>>> > not
> > > > > > > > > > > > >>>> > > > > > supported to be used for many
> > > configuration
> > > > > > change
> > > > > > > > > > > > >>>> authentication
> > > > > > > > > > > > >>>> > > > listed
> > > > > > > > > > > > >>>> > > > > > above. The full list could be found in
> > the
> > > > KIP.
> > > > > > > > > > > > >>>> > > > > >
> > > > > > > > > > > > >>>> > > > > > 4. Add a new
> > BROKER_AUTHORIZATION_FAILURE
> > > > error
> > > > > > > code
> > > > > > > > > to
> > > > > > > > > > > > >>>> indicate
> > > > > > > > > > > > >>>> > any
> > > > > > > > > > > > >>>> > > > > > internal security configuration
> failure,
> > > > when
> > > > > > the
> > > > > > > > > > > forwarded
> > > > > > > > > > > > >>>> request
> > > > > > > > > > > > >>>> > > > > failed
> > > > > > > > > > > > >>>> > > > > > authentication on the controller side.
> > > > > > > > > > > > >>>> > > > > >
> > > > > > > > > > > > >>>> > > > > > Let me know what you think. With such
> a
> > > > major
> > > > > > > > > refinement
> > > > > > > > > > > of
> > > > > > > > > > > > >>>> the
> > > > > > > > > > > > >>>> > KIP,
> > > > > > > > > > > > >>>> > > > I'm
> > > > > > > > > > > > >>>> > > > > > open for re-vote after discussions
> > > converge.
> > > > > > > > > > > > >>>> > > > > >
> > > > > > > > > > > > >>>> > > > > > Boyang
> > > > > > > > > > > > >>>> > > > > >
> > > > > > > > > > > > >>>> > > > > > On Wed, Jul 1, 2020 at 2:17 PM Boyang
> > > Chen <
> > > > > > > > > > > > >>>> > > reluctanthero104@gmail.com
> > > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > > >>>> > > > > > wrote:
> > > > > > > > > > > > >>>> > > > > >
> > > > > > > > > > > > >>>> > > > > > > Hey folks,
> > > > > > > > > > > > >>>> > > > > > >
> > > > > > > > > > > > >>>> > > > > > > I have also synced on the KIP-578
> > which
> > > > was
> > > > > > > doing
> > > > > > > > > the
> > > > > > > > > > > > >>>> partition
> > > > > > > > > > > > >>>> > > > limit,
> > > > > > > > > > > > >>>> > > > > to
> > > > > > > > > > > > >>>> > > > > > > make sure the partition limit error
> > code
> > > > would
> > > > > > > be
> > > > > > > > > > > properly
> > > > > > > > > > > > >>>> > > propagated
> > > > > > > > > > > > >>>> > > > > once
> > > > > > > > > > > > >>>> > > > > > > it is done on top of KIP-590. Let me
> > > know
> > > > if
> > > > > > you
> > > > > > > > > have
> > > > > > > > > > > > >>>> further
> > > > > > > > > > > > >>>> > > > > questions or
> > > > > > > > > > > > >>>> > > > > > > concerns.
> > > > > > > > > > > > >>>> > > > > > >
> > > > > > > > > > > > >>>> > > > > > > Boyang
> > > > > > > > > > > > >>>> > > > > > >
> > > > > > > > > > > > >>>> > > > > > > On Tue, Jun 23, 2020 at 5:08 PM
> Boyang
> > > > Chen <
> > > > > > > > > > > > >>>> > > > > reluctanthero104@gmail.com>
> > > > > > > > > > > > >>>> > > > > > > wrote:
> > > > > > > > > > > > >>>> > > > > > >
> > > > > > > > > > > > >>>> > > > > > >> Thanks for the clarification, Colin
> > and
> > > > > > Ismael.
> > > > > > > > > > > > Personally
> > > > > > > > > > > > >>>> I
> > > > > > > > > > > > >>>> > also
> > > > > > > > > > > > >>>> > > > feel
> > > > > > > > > > > > >>>> > > > > > >> Option A is better to prioritize
> > fixing
> > > > the
> > > > > > > gap.
> > > > > > > > > Just
> > > > > > > > > > > to
> > > > > > > > > > > > be
> > > > > > > > > > > > >>>> > clear,
> > > > > > > > > > > > >>>> > > > the
> > > > > > > > > > > > >>>> > > > > > >> proposed solution would be:
> > > > > > > > > > > > >>>> > > > > > >>
> > > > > > > > > > > > >>>> > > > > > >> 1. Bump the Metadata RPC version to
> > > > return
> > > > > > > > > > > > >>>> POLICY_VIOLATION. In
> > > > > > > > > > > > >>>> > > the
> > > > > > > > > > > > >>>> > > > > > >> application level, we should swap
> the
> > > > error
> > > > > > > > message
> > > > > > > > > > > with
> > > > > > > > > > > > >>>> the
> > > > > > > > > > > > >>>> > > actual
> > > > > > > > > > > > >>>> > > > > failure
> > > > > > > > > > > > >>>> > > > > > >> reason such as "violation of topic
> > > > creation
> > > > > > > > policy
> > > > > > > > > > when
> > > > > > > > > > > > >>>> > attempting
> > > > > > > > > > > > >>>> > > > to
> > > > > > > > > > > > >>>> > > > > auto
> > > > > > > > > > > > >>>> > > > > > >> create internal topic through
> > > > > > MetadataRequest."
> > > > > > > > > > > > >>>> > > > > > >>
> > > > > > > > > > > > >>>> > > > > > >> 2. For older Metadata RPC, return
> > > > > > > > > > AUTHORIZATION_FAILED
> > > > > > > > > > > to
> > > > > > > > > > > > >>>> fail
> > > > > > > > > > > > >>>> > > fast.
> > > > > > > > > > > > >>>> > > > > > >>
> > > > > > > > > > > > >>>> > > > > > >> Will address our other discussed
> > points
> > > > as
> > > > > > well
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > >>>> KIP, let
> > > > > > > > > > > > >>>> > me
> > > > > > > > > > > > >>>> > > > > know
> > > > > > > > > > > > >>>> > > > > > >> if you have further questions.
> > > > > > > > > > > > >>>> > > > > > >>
> > > > > > > > > > > > >>>> > > > > > >> Thanks,
> > > > > > > > > > > > >>>> > > > > > >> Boyang
> > > > > > > > > > > > >>>> > > > > > >>
> > > > > > > > > > > > >>>> > > > > > >> On Tue, Jun 23, 2020 at 10:41 AM
> > Ismael
> > > > Juma
> > > > > > <
> > > > > > > > > > > > >>>> ismael@juma.me.uk
> > > > > > > > > > > > >>>> > >
> > > > > > > > > > > > >>>> > > > > wrote:
> > > > > > > > > > > > >>>> > > > > > >>
> > > > > > > > > > > > >>>> > > > > > >>> Option A is basically what I was
> > > > thinking.
> > > > > > But
> > > > > > > > > with
> > > > > > > > > > a
> > > > > > > > > > > > >>>> slight
> > > > > > > > > > > > >>>> > > > > adjustment:
> > > > > > > > > > > > >>>> > > > > > >>>
> > > > > > > > > > > > >>>> > > > > > >>> New versions of MetadataResponse
> > > return
> > > > > > > > > > > > POLICY_VIOLATION,
> > > > > > > > > > > > >>>> old
> > > > > > > > > > > > >>>> > > > > versions
> > > > > > > > > > > > >>>> > > > > > >>> return AUTHORIZATION_FAILED. The
> > > latter
> > > > > > works
> > > > > > > > > > > correctly
> > > > > > > > > > > > >>>> with
> > > > > > > > > > > > >>>> > old
> > > > > > > > > > > > >>>> > > > Java
> > > > > > > > > > > > >>>> > > > > > >>> clients (i.e. the client fails
> fast
> > > and
> > > > > > > > propagates
> > > > > > > > > > the
> > > > > > > > > > > > >>>> error),
> > > > > > > > > > > > >>>> > > I've
> > > > > > > > > > > > >>>> > > > > > >>> tested
> > > > > > > > > > > > >>>> > > > > > >>> it. Adjust new clients to treat
> > > > > > > POLICY_VIOLATION
> > > > > > > > > > like
> > > > > > > > > > > > >>>> > > > > > >>> AUTHORIZATION_FAILED,
> > > > > > > > > > > > >>>> > > > > > >>> but propagate the custom error
> > > message.
> > > > > > > > > > > > >>>> > > > > > >>>
> > > > > > > > > > > > >>>> > > > > > >>> Ismael
> > > > > > > > > > > > >>>> > > > > > >>>
> > > > > > > > > > > > >>>> > > > > > >>> On Mon, Jun 22, 2020 at 11:00 PM
> > Colin
> > > > > > McCabe
> > > > > > > <
> > > > > > > > > > > > >>>> > > cmccabe@apache.org>
> > > > > > > > > > > > >>>> > > > > > >>> wrote:
> > > > > > > > > > > > >>>> > > > > > >>>
> > > > > > > > > > > > >>>> > > > > > >>> > > > > On Fri, Jun 19, 2020 at
> 3:18
> > > PM
> > > > > > Ismael
> > > > > > > > > Juma
> > > > > > > > > > <
> > > > > > > > > > > > >>>> > > > > ismael@juma.me.uk>
> > > > > > > > > > > > >>>> > > > > > >>> > wrote:
> > > > > > > > > > > > >>>> > > > > > >>> > > > >
> > > > > > > > > > > > >>>> > > > > > >>> > > > > > Hi Colin,
> > > > > > > > > > > > >>>> > > > > > >>> > > > > >
> > > > > > > > > > > > >>>> > > > > > >>> > > > > > The KIP states in the
> > > > > > Compatibility
> > > > > > > > > > section
> > > > > > > > > > > > (not
> > > > > > > > > > > > >>>> > Future
> > > > > > > > > > > > >>>> > > > > work):
> > > > > > > > > > > > >>>> > > > > > >>> > > > > >
> > > > > > > > > > > > >>>> > > > > > >>> > > > > > "To support the proxy of
> > > > requests,
> > > > > > > we
> > > > > > > > > need
> > > > > > > > > > > to
> > > > > > > > > > > > >>>> build a
> > > > > > > > > > > > >>>> > > > > channel
> > > > > > > > > > > > >>>> > > > > > >>> for
> > > > > > > > > > > > >>>> > > > > > >>> > > > > > brokers to talk directly
> > to
> > > > the
> > > > > > > > > > controller.
> > > > > > > > > > > > >>>> This part
> > > > > > > > > > > > >>>> > > of
> > > > > > > > > > > > >>>> > > > > the
> > > > > > > > > > > > >>>> > > > > > >>> design
> > > > > > > > > > > > >>>> > > > > > >>> > > > > > is internal change only
> > and
> > > > won’t
> > > > > > > > block
> > > > > > > > > > the
> > > > > > > > > > > > KIP
> > > > > > > > > > > > >>>> > > > progress."
> > > > > > > > > > > > >>>> > > > > > >>> > > > > >
> > > > > > > > > > > > >>>> > > > > > >>> > > > > > I am clarifying that
> this
> > is
> > > > not
> > > > > > > > > internal
> > > > > > > > > > > only
> > > > > > > > > > > > >>>> due to
> > > > > > > > > > > > >>>> > > the
> > > > > > > > > > > > >>>> > > > > > >>> config.
> > > > > > > > > > > > >>>> > > > > > >>> > If we
> > > > > > > > > > > > >>>> > > > > > >>> > > > > > say that this KIP
> depends
> > on
> > > > > > another
> > > > > > > > KIP
> > > > > > > > > > > > before
> > > > > > > > > > > > >>>> we
> > > > > > > > > > > > >>>> > can
> > > > > > > > > > > > >>>> > > > > merge
> > > > > > > > > > > > >>>> > > > > > >>> > > > > > it, that's fine although
> > it
> > > > feels
> > > > > > a
> > > > > > > > bit
> > > > > > > > > > > > >>>> unnecessary.
> > > > > > > > > > > > >>>> > > > > > >>> > > > > >
> > > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > > >>>> > > > > > >>> > Hi Ismael,
> > > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > > >>>> > > > > > >>> > I didn't realize there was
> still a
> > > > > > reference
> > > > > > > > to
> > > > > > > > > > the
> > > > > > > > > > > > >>>> separate
> > > > > > > > > > > > >>>> > > > > controller
> > > > > > > > > > > > >>>> > > > > > >>> > channel in the "Compatibility,
> > > > > > Deprecation,
> > > > > > > > and
> > > > > > > > > > > > >>>> Migration
> > > > > > > > > > > > >>>> > Plan"
> > > > > > > > > > > > >>>> > > > > > >>> section.  I
> > > > > > > > > > > > >>>> > > > > > >>> > agree that it doesn't really
> > belong
> > > > there.
> > > > > > > > > Given
> > > > > > > > > > > that
> > > > > > > > > > > > >>>> this
> > > > > > > > > > > > >>>> > is
> > > > > > > > > > > > >>>> > > > > creating
> > > > > > > > > > > > >>>> > > > > > >>> > confusion, I would suggest that
> we
> > > > just
> > > > > > drop
> > > > > > > > > this
> > > > > > > > > > > from
> > > > > > > > > > > > >>>> the
> > > > > > > > > > > > >>>> > KIP
> > > > > > > > > > > > >>>> > > > > > >>> entirely.
> > > > > > > > > > > > >>>> > > > > > >>> > It really is orthogonal to what
> > this
> > > > KIP
> > > > > > is
> > > > > > > > > > about--
> > > > > > > > > > > we
> > > > > > > > > > > > >>>> don't
> > > > > > > > > > > > >>>> > > > need a
> > > > > > > > > > > > >>>> > > > > > >>> > separate channel to implement
> > > > redirection.
> > > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > > >>>> > > > > > >>> > Boyang wrote:
> > > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > > >>>> > > > > > >>> > >
> > > > > > > > > > > > >>>> > > > > > >>> > > We are only opening the doors
> > for
> > > > > > specific
> > > > > > > > > > > internal
> > > > > > > > > > > > >>>> topics
> > > > > > > > > > > > >>>> > > > > (offsets,
> > > > > > > > > > > > >>>> > > > > > >>> txn
> > > > > > > > > > > > >>>> > > > > > >>> > > log), which I assume the
> client
> > > > should
> > > > > > > have
> > > > > > > > no
> > > > > > > > > > > > >>>> possibility
> > > > > > > > > > > > >>>> > to
> > > > > > > > > > > > >>>> > > > > mutate
> > > > > > > > > > > > >>>> > > > > > >>> the
> > > > > > > > > > > > >>>> > > > > > >>> > > topic policy?
> > > > > > > > > > > > >>>> > > > > > >>> > >
> > > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > > >>>> > > > > > >>> > Hi Boyang,
> > > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > > >>>> > > > > > >>> > I think you and Ismael are
> talking
> > > > about
> > > > > > > > > different
> > > > > > > > > > > > >>>> scenarios.
> > > > > > > > > > > > >>>> > > > You
> > > > > > > > > > > > >>>> > > > > are
> > > > > > > > > > > > >>>> > > > > > >>> > describing the scenario where
> the
> > > > broker
> > > > > > is
> > > > > > > > > > > > >>>> auto-creating the
> > > > > > > > > > > > >>>> > > > > > >>> transaction
> > > > > > > > > > > > >>>> > > > > > >>> > log topic or consumer offset
> > topic.
> > > > This
> > > > > > > > > scenario
> > > > > > > > > > > > >>>> indeed
> > > > > > > > > > > > >>>> > > should
> > > > > > > > > > > > >>>> > > > > not
> > > > > > > > > > > > >>>> > > > > > >>> happen
> > > > > > > > > > > > >>>> > > > > > >>> > in a properly-configured
> cluster.
> > > > > > However,
> > > > > > > > > Ismael
> > > > > > > > > > > is
> > > > > > > > > > > > >>>> > > describing
> > > > > > > > > > > > >>>> > > > a
> > > > > > > > > > > > >>>> > > > > > >>> scenario
> > > > > > > > > > > > >>>> > > > > > >>> > where the client is
> auto-creating
> > > some
> > > > > > > > arbitrary
> > > > > > > > > > > > >>>> non-internal
> > > > > > > > > > > > >>>> > > > topic
> > > > > > > > > > > > >>>> > > > > > >>> just by
> > > > > > > > > > > > >>>> > > > > > >>> > sending a metadata request.
> > > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > > >>>> > > > > > >>> > As far as I can see, there are
> two
> > > > > > solutions
> > > > > > > > > here:
> > > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > > >>>> > > > > > >>> > A. Close the hole in
> > > > CreateTopicsPolicy
> > > > > > > > > > immediately.
> > > > > > > > > > > > >>>> In new
> > > > > > > > > > > > >>>> > > > > versions,
> > > > > > > > > > > > >>>> > > > > > >>> > allow MetadataResponse to return
> > > > > > > > > > > AUTHORIZATION_FAILED
> > > > > > > > > > > > >>>> if we
> > > > > > > > > > > > >>>> > > tried
> > > > > > > > > > > > >>>> > > > > to
> > > > > > > > > > > > >>>> > > > > > >>> > auto-create a topic and failed.
> > > Find
> > > > some
> > > > > > > > other
> > > > > > > > > > > error
> > > > > > > > > > > > >>>> code
> > > > > > > > > > > > >>>> > to
> > > > > > > > > > > > >>>> > > > > return
> > > > > > > > > > > > >>>> > > > > > >>> for
> > > > > > > > > > > > >>>> > > > > > >>> > existing versions.
> > > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > > >>>> > > > > > >>> > B. Keep the hole in
> > > > CreateTopicsPolicy and
> > > > > > > add
> > > > > > > > > > some
> > > > > > > > > > > > >>>> > > configuration
> > > > > > > > > > > > >>>> > > > > to
> > > > > > > > > > > > >>>> > > > > > >>> allow
> > > > > > > > > > > > >>>> > > > > > >>> > admins to gradually migrate to
> > > > closing it.
> > > > > > > In
> > > > > > > > > > > > >>>> practice, this
> > > > > > > > > > > > >>>> > > > > probably
> > > > > > > > > > > > >>>> > > > > > >>> > means a configuration toggle
> that
> > > > enables
> > > > > > > > direct
> > > > > > > > > > ZK
> > > > > > > > > > > > >>>> access,
> > > > > > > > > > > > >>>> > > that
> > > > > > > > > > > > >>>> > > > > > >>> starts off
> > > > > > > > > > > > >>>> > > > > > >>> > as enabled.  Then we can
> > eventually
> > > > > > default
> > > > > > > it
> > > > > > > > > to
> > > > > > > > > > > > false
> > > > > > > > > > > > >>>> and
> > > > > > > > > > > > >>>> > > then
> > > > > > > > > > > > >>>> > > > > > >>> remove it
> > > > > > > > > > > > >>>> > > > > > >>> > entirely over time.
> > > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > > >>>> > > > > > >>> > best,
> > > > > > > > > > > > >>>> > > > > > >>> > Colin
> > > > > > > > > > > > >>>> > > > > > >>> >
> > > > > > > > > > > > >>>> > > > > > >>>
> > > > > > > > > > > > >>>> > > > > > >>
> > > > > > > > > > > > >>>> > > > > >
> > > > > > > > > > > > >>>> > > > >
> > > > > > > > > > > > >>>> > > >
> > > > > > > > > > > > >>>> > >
> > > > > > > > > > > > >>>> >
> > > > > > > > > > > > >>>>
> > > > > > > > > > > > >>>
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>