Hi Boyang,

Thanks for the KIP. Overall looks great.

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

A few advantages to this approach come to mind:

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

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

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

What do you think?

Regards,
Ray


On Wed, Apr 8, 2020 at 4:36 PM Boyang Chen <reluctanthero...@gmail.com>
wrote:

> Thanks for the info Agam! Will add to the KIP.
>
> On Wed, Apr 8, 2020 at 4:26 PM Agam Brahma <abra...@confluent.io> wrote:
>
> > Hi Boyang,
> >
> > The KIP already talks about incorporating changes for FindCoordinator
> > request routing, wanted to point out one additional case where internal
> > topics are created "as a side effect":
> >
> > As part of handling metadata requests, if we are looking for metadata for
> > an internal topic and auto-topic-creation is enabled [1], the broker
> > currently goes ahead and creates the internal topic in the same way [2]
> as
> > it would for the FindCoordinator request.
> >
> > -Agam
> >
> > [1]
> >
> >
> https://github.com/apache/kafka/blob/cd1e46c8bb46f1e5303c51f476c74e33b522fce8/core/src/main/scala/kafka/server/KafkaApis.scala#L1096
> > [2]
> >
> >
> https://github.com/apache/kafka/blob/cd1e46c8bb46f1e5303c51f476c74e33b522fce8/core/src/main/scala/kafka/server/KafkaApis.scala#L1041
> >
> >
> >
> > On Mon, Apr 6, 2020 at 8:25 PM Boyang Chen <reluctanthero...@gmail.com>
> > wrote:
> >
> > > Thanks for the various inputs everyone!
> > >
> > > I think Sonke and Colin's suggestions make sense. The tagged field also
> > > avoids the unnecessary protocol changes for affected requests. Will add
> > it
> > > to the header. As for the verification, I'm not sure whether it's
> > necessary
> > > to require a higher permission level, as it is just an ignorable field?
> > >
> > > Guozhang's suggestions about metrics also sound great, I will think
> > through
> > > the use cases and make some changes to the KIP.
> > >
> > > Best,
> > > Boyang
> > >
> > > On Mon, Apr 6, 2020 at 4:28 PM Guozhang Wang <wangg...@gmail.com>
> wrote:
> > >
> > > > Thanks for the KIP Boyang, this looks good to me. Some minor
> comments:
> > > >
> > > > 1) I think in order to implement the forwarding mechanism the brokers
> > > needs
> > > > some purgatory to keep the forwarded requests; if that's true, should
> > we
> > > > add some broker-side metrics for those purgatories for debugging
> > > purposes?
> > > >
> > > > 2) Should we also consider adding some extra metric counting old
> > > versioned
> > > > admin client request rates (this goes beyond
> > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-511%3A+Collect+and+Expose+Client%27s+Name+and+Version+in+the+Brokers
> > > > since
> > > > old versioned client would not report its Kafka version anyways); one
> > use
> > > > case I can think of besides debugging purposes, is that if we ever
> > > decides
> > > > to break compatibility in future versions way after the bridge
> > releases,
> > > to
> > > > reject any v1 requests and hence can totally remove this forwarding
> > logic
> > > > on brokers, we can leverage on this metric to find a safe time to
> > > upgrade.
> > > >
> > > >
> > > > Guozhang
> > > >
> > > >
> > > >
> > > > On Mon, Apr 6, 2020 at 3:50 PM Colin McCabe <cmcc...@apache.org>
> > wrote:
> > > >
> > > > > Hi Sönke,
> > > > >
> > > > > Yeah, that was my thought too.  The request has already been
> > validated
> > > on
> > > > > the forwarding broker, so we don't need to validate it again.
> > However,
> > > > you
> > > > > make a good point that it's unfortunate that the audit log would
> lose
> > > the
> > > > > principal information if we didn't forward it as well.
> > > > >
> > > > > Perhaps we could add a tagged field to the request header for all
> > > > > messages.  This field would contain the principal name.  Of course,
> > > this
> > > > > field should only be allowed if the request arrives with the
> highest
> > > > > permission levels (Probably ClusterAction on Cluster, since that's
> > what
> > > > all
> > > > > the brokers have.)
> > > > >
> > > > > regards,
> > > > > Colin
> > > > >
> > > > >
> > > > > On Mon, Apr 6, 2020, at 14:37, Sönke Liebau wrote:
> > > > > > Hi Boyang,
> > > > > >
> > > > > > thanks for the KIP. Sounds good overall.
> > > > > >
> > > > > > @Tom: I thought about your remark a little and think that in
> > > principle
> > > > we
> > > > > > can get away without forwarding the principal at all. Brokers
> > > currently
> > > > > > authenticate and authorize requests before performing writes to
> > > > > Zookeeper -
> > > > > > as long as we don't change that it shouldn't matter, whether the
> > > write
> > > > > goes
> > > > > > to ZK or the controller, as long as that request is properly
> > > > > authenticated.
> > > > > > So the broker would simply authorize and authenticate the
> original
> > > > > request
> > > > > > and then forward it to the controller using its own credentials.
> > And
> > > > the
> > > > > > controller could simply trust that this is a bona-fide request,
> > > because
> > > > > it
> > > > > > came from a trusted peer.
> > > > > >
> > > > > > I can see two issues here, one is a bit academic I think..
> > > > > >
> > > > > > 1. The controller would be unable to write a proper audit log,
> > > because
> > > > it
> > > > > > cannot know who sent the original request.
> > > > > >
> > > > > > 2. In theory, clusters could use Plaintext Listeners for inter
> > broker
> > > > > > traffic because that is on a separate, secure network or similar
> > > > reasons.
> > > > > > In that case, the forwarded request would be unauthenticated -
> then
> > > > > again,
> > > > > > so are all other requests between brokers, so nothing lost
> really.
> > > > > >
> > > > > > Overall though, I think that sending the principal along with the
> > > > request
> > > > > > shouldn't be a large issue though, it is just two Strings and a
> > > > boolean.
> > > > > > And the controller could bypass the PrincipalBuilder and just
> pass
> > > the
> > > > > > Principal that was built and sent by the remote broker straight
> to
> > > the
> > > > > > Authorizer. Since PrincipalBuilders are the same on all brokers
> it
> > > > > > shouldn't matter who does the processing I think.
> > > > > >
> > > > > > Best regards,
> > > > > > Sönke
> > > > > >
> > > > > >
> > > > > > On Mon, 6 Apr 2020 at 22:30, Boyang Chen <
> > reluctanthero...@gmail.com
> > > >
> > > > > wrote:
> > > > > >
> > > > > > > Thanks Tom for the question! I'm not super familiar with the
> > > > Principal
> > > > > > > stuff, could you elaborate more on the two points you proposed
> > > here?
> > > > > > >
> > > > > > > I looked up Admin client and just take `createDelegationToken`
> > API
> > > > for
> > > > > an
> > > > > > > example, the request data encodes the principal information
> > > already,
> > > > so
> > > > > > > broker should also leverage that information to proxy the
> request
> > > > IMHO.
> > > > > > >
> > > > > > > Boyang
> > > > > > >
> > > > > > > On Mon, Apr 6, 2020 at 9:21 AM Tom Bentley <
> tbent...@redhat.com>
> > > > > wrote:
> > > > > > >
> > > > > > > > Hi Boyang,
> > > > > > > >
> > > > > > > > Thanks for the KIP!
> > > > > > > >
> > > > > > > > When a broker proxies a request to the controller how does
> the
> > > > > > > > authenticated principal get propagated? I think a couple of
> > > things
> > > > > might
> > > > > > > > complicate this:
> > > > > > > >
> > > > > > > > 1. A PrincipalBuilder might be in use,
> > > > > > > > 2. A Principal does not have to be serializable.
> > > > > > > >
> > > > > > > >
> > > > > > > > Kind regards,
> > > > > > > >
> > > > > > > > Tom
> > > > > > > >
> > > > > > > > On Sat, Apr 4, 2020 at 12:52 AM Boyang Chen <
> > > > > reluctanthero...@gmail.com>
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Hey all,
> > > > > > > > >
> > > > > > > > > I would like to start off the discussion for KIP-590, a
> > > follow-up
> > > > > > > > > initiative after KIP-500:
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-590%3A+Redirect+Zookeeper+Mutation+Protocols+to+The+Controller
> > > > > > > > >
> > > > > > > > > This KIP proposes to migrate existing Zookeeper mutation
> > paths,
> > > > > > > including
> > > > > > > > > configuration, security and quota changes, to
> controller-only
> > > by
> > > > > always
> > > > > > > > > routing these alterations to the controller.
> > > > > > > > >
> > > > > > > > > Let me know your thoughts!
> > > > > > > > >
> > > > > > > > > Best,
> > > > > > > > > Boyang
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > > >
> > > > > > --
> > > > > > Sönke Liebau
> > > > > > Partner
> > > > > > Tel. +49 179 7940878
> > > > > > OpenCore GmbH & Co. KG - Thomas-Mann-Straße 8 - 22880 Wedel -
> > Germany
> > > > > >
> > > > >
> > > >
> > > >
> > > > --
> > > > -- Guozhang
> > > >
> > >
> >
> >
> > --
> > -Agam
> >
>

Reply via email to