You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Colin McCabe <cm...@apache.org> on 2023/04/19 21:17:53 UTC

KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum

Hi all,

I wrote a short KIP about allowing AdminClient to talk directly with the KRaft controller quorum. Check it out here:

https://cwiki.apache.org/confluence/x/Owo0Dw

best,
Colin

Re: KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum

Posted by ziming deng <de...@gmail.com>.
Hello Colin,
There is a mistake that we use `—bootstrap-server` instead of `—bootstrap-server(s)`, so should we also change the new argument `—bootstrap-controller` (no s).

--
Ziming

> On Apr 20, 2023, at 05:17, Colin McCabe <cm...@apache.org> wrote:
> 
> Hi all,
> 
> I wrote a short KIP about allowing AdminClient to talk directly with the KRaft controller quorum. Check it out here:
> 
> https://cwiki.apache.org/confluence/x/Owo0Dw
> 
> best,
> Colin


Re: KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum

Posted by Colin McCabe <cm...@apache.org>.
On Wed, Apr 19, 2023, at 14:37, Ron Dagostino wrote:
> Thanks for the KIP, Colin.
>
> There seems to be some inconsistency between sometimes referring to
> "TargetKRaftControllerQuorum" and other times referring to
> "DirectToKRaftControllerQuorum".  Aside from that, it looks good to
> me.  The symmetry of bootstrap servers and bootstrap controllers feels
> right.
>

Hi Ron,

Good point. I will replace all of them with DirectToKRaftControllerQuorum.

> What happens in the combined case?  Will it depend on the listener
> that gets the request?  So if you put the port for a controller
> listener in bootstrap servers and that listener gets the request then
> it fails, and vice-versa if you put the port for a non-controller
> listener in bootstrap controllers and that listener gets the request
> it also fails?

Correct. Controller listeners are always distinct from broker listeners.

best,
Colin

>
> Ron
>
> On Wed, Apr 19, 2023 at 5:18 PM Colin McCabe <cm...@apache.org> wrote:
>>
>> Hi all,
>>
>> I wrote a short KIP about allowing AdminClient to talk directly with the KRaft controller quorum. Check it out here:
>>
>> https://cwiki.apache.org/confluence/x/Owo0Dw
>>
>> best,
>> Colin

Re: KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum

Posted by Ron Dagostino <rn...@gmail.com>.
Thanks for the KIP, Colin.

There seems to be some inconsistency between sometimes referring to
"TargetKRaftControllerQuorum" and other times referring to
"DirectToKRaftControllerQuorum".  Aside from that, it looks good to
me.  The symmetry of bootstrap servers and bootstrap controllers feels
right.

What happens in the combined case?  Will it depend on the listener
that gets the request?  So if you put the port for a controller
listener in bootstrap servers and that listener gets the request then
it fails, and vice-versa if you put the port for a non-controller
listener in bootstrap controllers and that listener gets the request
it also fails?

Ron

On Wed, Apr 19, 2023 at 5:18 PM Colin McCabe <cm...@apache.org> wrote:
>
> Hi all,
>
> I wrote a short KIP about allowing AdminClient to talk directly with the KRaft controller quorum. Check it out here:
>
> https://cwiki.apache.org/confluence/x/Owo0Dw
>
> best,
> Colin

Re: KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum

Posted by Colin McCabe <co...@cmccabe.xyz>.
On Tue, Jul 4, 2023, at 04:46, Tom Bentley wrote:
> Hi Colin,
>
> Thanks for the KIP.
>
> 1. It mentions kafka-configs.sh as one of the affected tools, but doesn't
> mention that ControllerApis doesn't currently support DESCRIBE_CONFIGS. I
> think this is worth noting as it is, in effect, a change to the wire
> protocol accepted by the controller, even if it's an existing RPC.

Hi Tom,

Good point. I added a section mentioning that we will support DESCRIBE_CONFIGS on the controller.

> 2. The diff you show for the MetadataRequest.son doesn't show a change to
> the top-level "listeners" key, presumably this should add "controller"?
> Similarly, per the above point, I guess we'd also be updating the JSON for
> DescribeConfigs.

I have reworked this section to reflect the fact that I am now using DESCRIBE_CLUSTER to give AdminClient information about the controller nodes.

> 3. Do you have any timeline for calling a vote for this KIP?
>

Yes, good question. I will call a vote soon since I know the KIP freeze is coming up.

best,
Colin

> Many thanks,
>
> Tom
>
> On Thu, 27 Apr 2023 at 18:51, Colin McCabe <cm...@apache.org> wrote:
>
>> On Wed, Apr 26, 2023, at 22:08, Luke Chen wrote:
>> > Hi Colin,
>> >
>> > Some comments:
>> > 1. I agree we should set "top-level" errors for metadata response
>> >
>> > 2. In the "brokers" field of metadata response from controller, it'll
>> > respond with "Controller endpoint information as given in
>> > controller.quorum.voters", instead of the "alive" controllers(voters).
>> That
>> > will break the existing admin client because in admin client, we'll rely
>> on
>> > the metadata response to build the "current alive brokers" list, and
>> choose
>> > one from them to connect (either least load or other criteria). That
>> means,
>> > if now, we return the value in `controller.quorum.voters`, but one of
>> them
>> > is down. We might choose it to connect and get connection errors. Should
>> we
>> > return the "alive" controllers(voters) to client?
>>
>> Hi Luke,
>>
>> Good question. When talking to the controllers directly, the AdminClient
>> needs to always send its RPCs to the active controller. There is one
>> exception: configuring ephemeral log4j settings with
>> incrementalAlterConfigs must be done by sending them to the specified
>> controller node.
>>
>> I will add this to a section called "AdminClient Implementation Notes" so
>> that it's captured in the KIP.
>>
>> >
>> > 3. In the KIP, we list the command-line tools will get a new
>> > --bootstrap-controllers argument, but without explaining why these tools
>> > need to talk to controller directly. Could we add some explanation about
>> > them? I tried but cannot know why some tools are listed here:
>> >     - kafka-acls.sh -> Allow clients to update ACLs via controller before
>> > brokers up
>> >
>> >     - kafka-cluster.sh -> Reasonable to get/update cluster info via
>> > controller
>> >
>> >     - kafka-configs.sh -> Allow clients to dynamically update
>> > configs/describe configs from controller. But in this script, client can
>> > still set quota for users/clients/topics... is client also able to update
>> > via controllers? Or we only allow partial actions in the script to talk
>> to
>> > controllers?
>> >
>> >     - kafka-delegation-tokens.sh -> Reasonable to update
>> delegation-tokens
>> > via controllers
>> >
>> >     - kafka-features.sh -> Reasonable
>> >     - kafka-metadata-quorum.sh -> Reasonable
>> >     - kafka-metadata-shell.sh -> Reasonable
>> >
>> >     - kafka-reassign-partitions.sh -> Why should we allow clients to move
>> > metadata log partitions in controller nodes? What's the use-case?
>> >
>>
>> Yes, the common thread here is that all of these shell commands perform
>> operations can be done without the broker. So it's reasonable to allow them
>> to be done without going through the broker. I don't know if we need a
>> separate note for each since the rationale is really the same for all (is
>> it reasonable? if so allow it.)
>>
>> kafka-reassign-partitions.sh cannot be used to move the __cluster_metadata
>> topic. However, it can be used to move partitions that reside on the
>> brokers, even when using --bootstrap-controllers to talk directly to the
>> quorum.
>>
>> Colin
>>
>> >
>> > Thank you.
>> > Luke
>> >
>> > On Thu, Apr 27, 2023 at 8:04 AM Colin McCabe <cm...@apache.org> wrote:
>> >
>> >> On Tue, Apr 25, 2023, at 04:59, Divij Vaidya wrote:
>> >> > Thank you for the KIP Colin.
>> >> >
>> >> > In general, I like the idea of having the ability to interact directly
>> >> with
>> >> > the controllers. I agree with your observation that it helps in
>> >> situations
>> >> > where you would want to get data directly from the controller instead
>> of
>> >> > going via a broker. I have some general comments but the main concern
>> I
>> >> > have is with the piggy-backing of error code with response of
>> >> > __cluster_metadata topic.
>> >> >
>> >> > 1. With this change, how are we guarding against the possibility of
>> >> > misbehaving client traffic from disrupting the controller (that you
>> >> > mentioned as a motivation of earlier behaviour)? One solution could
>> be to
>> >> > have default values set for request throttling on the controller.
>> >>
>> >> Hi Divij,
>> >>
>> >> Thanks for the comments.
>> >>
>> >> Our guards against client misbehavior remain the same:
>> >> 1. our recommendation to put the clients on a separate network
>> >> 2. the fact that producers and consumers can't interact directly with
>> the
>> >> controller
>> >> 3. the authorizer.
>> >>
>> >> Re: #3, I will spell out in the KIP that clients must have DESCRIBE on
>> the
>> >> CLUSTER resource to send a METADATA request to the controller.
>> >>
>> >> > 2. This KIP also increases the network attack surface area. Prior to
>> this
>> >> > KIP, it was possible to have firewall rules setup for controllers such
>> >> that
>> >> > only the brokers can talk to it. But now, the controller would require
>> >> > access from other endpoints other than brokers as well. Can we add a
>> >> > suggestion to the upgrade documentation and inform users
>> >>
>> >> There is no requirement for access from other endpoints. It is still
>> >> possible to set up firewall rules such that only the brokers can talk to
>> >> the controller. In fact I would even say this is desirable. Since this
>> >> faculty is intended for infrequent manual administrative operations,
>> >> needing to log into the broker to use it seems perfectly fine.
>> >>
>> >> > 3. In section KRaft Controller MetadataResponse, row 3, "There is no
>> >> > top-level error code in MetadataResponse, so we use the
>> >> __cluster_metadata
>> >> > topic to send back our error.". This will definitely confuse the
>> users.
>> >> Can
>> >> > we introduce a top level error field instead?
>> >>
>> >> Let me check how we're handling this in other places. I recall some
>> other
>> >> cases where we used the dummy topic approach, but I can't find them just
>> >> now.
>> >>
>> >> > 4. As part of the KIP, could we please add some documentation for
>> users
>> >> > with the suggestion of when to get information directly from the
>> >> controller
>> >> > and when not to (and associated tradeoffs)?
>> >>
>> >> I think most users will not face this tradeoff because they simply won't
>> >> have network access to the controller servers.
>> >>
>> >> For those who do want more information, we'll have command-line
>> >> documentation for --boostrap-controllers and the bootstrap.controllers
>> >> configuration key.
>> >>
>> >> > 5. Why do we need the `FromKRaftController` field in the response?
>> What
>> >> do
>> >> > we expect the users to do with this information?
>> >>
>> >> The field is present so that we can throw an exception in the client if
>> we
>> >> have received a response that is not from the controller.
>> >>
>> >> > 6. Can we drop `KRaft` from the fields `FromKRaftController` and
>> >> > `DirectToKRaftControllerQuorum`? My suggestion will be to rename it as
>> >> > `DirectToController`.
>> >>
>> >> I like the idea, but wouldn't that create confusion in the ZK cluster
>> case?
>> >>
>> >> > 7.  "kafka-metadata-shell.sh  is at an "evolving" level of interface
>> >> > stability" -> I thought that with KRaft being production ready, the
>> >> > evolving mode for kraft-related tools has also moved to production.
>> Do we
>> >> > have a timeline when this would move to production?
>> >>
>> >> That's a good question, but I think we should tackle it separately from
>> >> this KIP. The metadata shell is pretty different from other parts of
>> kafka
>> >> since it interacts so closely with internals.
>> >>
>> >> best,
>> >> Colin
>> >>
>> >>
>> >> >
>> >> > --
>> >> > Divij Vaidya
>> >> >
>> >> >
>> >> >
>> >> > On Tue, Apr 25, 2023 at 1:38 AM Colin McCabe <cm...@apache.org>
>> wrote:
>> >> >
>> >> >> On Fri, Apr 21, 2023, at 14:17, Jason Gustafson wrote:
>> >> >> > Hey Colin,
>> >> >> >
>> >> >> > The KIP makes sense overall. Nice to clarify the contract between
>> >> clients
>> >> >> > and the controllers. The use of `DirectToKRaftControllerQuorum`
>> will
>> >> help
>> >> >> > prevent misconfiguration. In fact, I wonder if we can return a
>> fatal
>> >> >> error
>> >> >> > instead of NOT_CONTROLLER so that the client would immediately
>> fail.
>> >> For
>> >> >> > example, could we use INVALID_REQUEST or something like that?
>> Either
>> >> that
>> >> >> > or we need to make sure clients treat NOT_CONTROLLER as a fatal
>> error.
>> >> >> > Without that, it would probably get picked up with default retry
>> logic
>> >> >> and
>> >> >> > the user might not see the problem.
>> >> >>
>> >> >> Hi Jason,
>> >> >>
>> >> >> Yes, this is a good point. It should return INVALID_REQUEST since
>> that
>> >> is
>> >> >> not retryable. I'll change it.
>> >> >>
>> >> >> >
>> >> >> > I also wonder if we can relax the requirements on the Metadata
>> >> request so
>> >> >> > that we can use it to list topics and partition state (e.g.
>> URPs).  It
>> >> >> > would be useful to query the controllers as the metadata source of
>> >> truth
>> >> >> > when we suspect that the broker metadata may have diverged.
>> >> >> >
>> >> >>
>> >> >> Fair enough. I will document that we can return topics.
>> >> >>
>> >> >> I also added a "future work" section about maybe using the
>> controllers
>> >> as
>> >> >> bootstrap servers for the broker cluster. To be clear, that is NOT in
>> >> scope
>> >> >> here, but it's interesting to think about potentially doing in the
>> >> future.
>> >> >> The major problem is what to do when the broker endpoints we're
>> >> returning
>> >> >> have different security settings from the controller endpoint the
>> client
>> >> >> initially talked to.
>> >> >>
>> >> >> best,
>> >> >> Colin
>> >> >>
>> >> >>
>> >> >> >
>> >> >> > Thanks,
>> >> >> > Jason
>> >> >> >
>> >> >> > On Thu, Apr 20, 2023 at 5:53 PM Colin McCabe <cm...@apache.org>
>> >> wrote:
>> >> >> >
>> >> >> >> On Wed, Apr 19, 2023, at 20:56, Philip Nee wrote:
>> >> >> >> > Hey Colin,
>> >> >> >> >
>> >> >> >> > I still need to finish reading and understanding the KIP, but I
>> >> have a
>> >> >> >> > couple of comments despite being ignorant of most of the KRaft
>> >> stuff.
>> >> >> >> > (Sorry!)
>> >> >> >> >
>> >> >> >> > Firstly, does it make sense to create an extension of the
>> current
>> >> >> >> > AdminClient only to handle these specific KRaft use cases? It
>> seems
>> >> >> >> > cumbersome to have two sets of bootstrap configurations to make
>> the
>> >> >> >> > AdminClient generic enough to handle these specific cases,
>> instead,
>> >> >> maybe
>> >> >> >> > it is more obvious (to me) to just extend the AdminClient. What
>> I'm
>> >> >> >> > thinking is KraftAdminClient which continuously uses
>> >> >> *bootstrap.servers*,
>> >> >> >> > but make this class only serves the Kraft controllers APIs.
>> >> >> >>
>> >> >> >> Hi Philip,
>> >> >> >>
>> >> >> >> Thanks for taking a look.
>> >> >> >>
>> >> >> >> We would not want to create a new Admin client class in order to
>> >> >> >> communicate directly with the controllers. The RPCs accepted by
>> the
>> >> >> >> controllers have the same format as the those accepted by the
>> >> brokers.
>> >> >> >> There is no difference in what is sent over the wire or the data
>> >> >> structures
>> >> >> >> that are used in the client.
>> >> >> >>
>> >> >> >> I know you mentioned you haven't had time to read all the KRaft
>> stuff
>> >> >> (and
>> >> >> >> there is a lot, I understand). But this is one area that would
>> >> probably
>> >> >> be
>> >> >> >> clarified if you were able to read at least KIP-500 and KIP-590.
>> RPCs
>> >> >> sent
>> >> >> >> to the broker are forwarded to the controller (mostly) without
>> >> >> modification.
>> >> >> >>
>> >> >> >> >
>> >> >> >> > Secondly, if we want to continue with the design, I'm not yet
>> sure
>> >> >> why we
>> >> >> >> > can't continue using the *bootstrap.servers*? I assume when the
>> >> client
>> >> >> >> gets
>> >> >> >> > the metadata, it should know who it is talking to. I'm just
>> >> >> reconsidering
>> >> >> >> > your alternative again.
>> >> >> >> >
>> >> >> >> > A bad idea: Why don't we continue using *bootstrap.servers* but
>> >> have a
>> >> >> >> > separated config like *kraft.controller* = true/false. I feel
>> like
>> >> >> most
>> >> >> >> > users might not know what is a controller and causes some
>> mistakes
>> >> >> down
>> >> >> >> the
>> >> >> >> > road.
>> >> >> >> >
>> >> >> >>
>> >> >> >> Well, you called it a bad idea, and I can't help but agree! :)
>> >> >> >>
>> >> >> >> I think "the user might not know what a controller is" is a good
>> sign
>> >> >> that
>> >> >> >> they shouldn't be interacting with the controller. Like many
>> >> AdminClient
>> >> >> >> APIs, this one is intended for use by administrators only. Most
>> users
>> >> >> >> indeed should not need to know what a controller is or interact
>> with
>> >> it
>> >> >> >> directly. If they do interact it should be very clear that they
>> are
>> >> >> doing
>> >> >> >> so. The --controller-bootstrap flag makes it very clear, as does
>> the
>> >> >> >> separate configuration.
>> >> >> >>
>> >> >> >> Let me give an example of the kind of problems that arise if you
>> >> want to
>> >> >> >> reuse bootstrap.servers for this purpose.
>> >> >> >>
>> >> >> >> If the user grasb a 3.4 Kafka AdminClient and set
>> bootstrap.servers
>> >> to a
>> >> >> >> set of controller servers, and set direct.to.controller to true,
>> the
>> >> >> >> unknown (in 3.4) configuration will be ignored, and a normal
>> metadata
>> >> >> >> request will be sent without the direct to controller flag. In
>> that
>> >> >> case it
>> >> >> >> will give back an error. Confusing, right?
>> >> >> >>
>> >> >> >> Using controller.servers clarifies this situation because the 3.4
>> >> client
>> >> >> >> will not support that config, and will complain about the lack of
>> >> >> >> bootstrap.servers.
>> >> >> >>
>> >> >> >> Actually, the situation could get even more confusing than what I
>> >> >> >> described since some older preproduction versions of the KRaft
>> >> >> controller
>> >> >> >> did implement the METADATA RPC. So if you send them a METADATA
>> >> request
>> >> >> >> without any special information, it's not clear what you'll get.
>> >> >> Indeed,
>> >> >> >> it would be dependent on the client version and the controller
>> >> version.
>> >> >> >>
>> >> >> >> The bottom line is that reusing the bootstrap.servers
>> configuration
>> >> here
>> >> >> >> is not a good idea.
>> >> >> >>
>> >> >> >> best,
>> >> >> >> Colin
>> >> >> >>
>> >> >> >> > Thanks,
>> >> >> >> > P
>> >> >> >> >
>> >> >> >> > On Wed, Apr 19, 2023 at 2:18 PM Colin McCabe <
>> cmccabe@apache.org>
>> >> >> wrote:
>> >> >> >> >
>> >> >> >> >> Hi all,
>> >> >> >> >>
>> >> >> >> >> I wrote a short KIP about allowing AdminClient to talk directly
>> >> with
>> >> >> the
>> >> >> >> >> KRaft controller quorum. Check it out here:
>> >> >> >> >>
>> >> >> >> >> https://cwiki.apache.org/confluence/x/Owo0Dw
>> >> >> >> >>
>> >> >> >> >> best,
>> >> >> >> >> Colin
>> >> >> >> >>
>> >> >> >>
>> >> >>
>> >>
>>
>>

Re: KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum

Posted by Tom Bentley <tb...@redhat.com>.
Hi Colin,

Thanks for the KIP.

1. It mentions kafka-configs.sh as one of the affected tools, but doesn't
mention that ControllerApis doesn't currently support DESCRIBE_CONFIGS. I
think this is worth noting as it is, in effect, a change to the wire
protocol accepted by the controller, even if it's an existing RPC.
2. The diff you show for the MetadataRequest.son doesn't show a change to
the top-level "listeners" key, presumably this should add "controller"?
Similarly, per the above point, I guess we'd also be updating the JSON for
DescribeConfigs.
3. Do you have any timeline for calling a vote for this KIP?

Many thanks,

Tom

On Thu, 27 Apr 2023 at 18:51, Colin McCabe <cm...@apache.org> wrote:

> On Wed, Apr 26, 2023, at 22:08, Luke Chen wrote:
> > Hi Colin,
> >
> > Some comments:
> > 1. I agree we should set "top-level" errors for metadata response
> >
> > 2. In the "brokers" field of metadata response from controller, it'll
> > respond with "Controller endpoint information as given in
> > controller.quorum.voters", instead of the "alive" controllers(voters).
> That
> > will break the existing admin client because in admin client, we'll rely
> on
> > the metadata response to build the "current alive brokers" list, and
> choose
> > one from them to connect (either least load or other criteria). That
> means,
> > if now, we return the value in `controller.quorum.voters`, but one of
> them
> > is down. We might choose it to connect and get connection errors. Should
> we
> > return the "alive" controllers(voters) to client?
>
> Hi Luke,
>
> Good question. When talking to the controllers directly, the AdminClient
> needs to always send its RPCs to the active controller. There is one
> exception: configuring ephemeral log4j settings with
> incrementalAlterConfigs must be done by sending them to the specified
> controller node.
>
> I will add this to a section called "AdminClient Implementation Notes" so
> that it's captured in the KIP.
>
> >
> > 3. In the KIP, we list the command-line tools will get a new
> > --bootstrap-controllers argument, but without explaining why these tools
> > need to talk to controller directly. Could we add some explanation about
> > them? I tried but cannot know why some tools are listed here:
> >     - kafka-acls.sh -> Allow clients to update ACLs via controller before
> > brokers up
> >
> >     - kafka-cluster.sh -> Reasonable to get/update cluster info via
> > controller
> >
> >     - kafka-configs.sh -> Allow clients to dynamically update
> > configs/describe configs from controller. But in this script, client can
> > still set quota for users/clients/topics... is client also able to update
> > via controllers? Or we only allow partial actions in the script to talk
> to
> > controllers?
> >
> >     - kafka-delegation-tokens.sh -> Reasonable to update
> delegation-tokens
> > via controllers
> >
> >     - kafka-features.sh -> Reasonable
> >     - kafka-metadata-quorum.sh -> Reasonable
> >     - kafka-metadata-shell.sh -> Reasonable
> >
> >     - kafka-reassign-partitions.sh -> Why should we allow clients to move
> > metadata log partitions in controller nodes? What's the use-case?
> >
>
> Yes, the common thread here is that all of these shell commands perform
> operations can be done without the broker. So it's reasonable to allow them
> to be done without going through the broker. I don't know if we need a
> separate note for each since the rationale is really the same for all (is
> it reasonable? if so allow it.)
>
> kafka-reassign-partitions.sh cannot be used to move the __cluster_metadata
> topic. However, it can be used to move partitions that reside on the
> brokers, even when using --bootstrap-controllers to talk directly to the
> quorum.
>
> Colin
>
> >
> > Thank you.
> > Luke
> >
> > On Thu, Apr 27, 2023 at 8:04 AM Colin McCabe <cm...@apache.org> wrote:
> >
> >> On Tue, Apr 25, 2023, at 04:59, Divij Vaidya wrote:
> >> > Thank you for the KIP Colin.
> >> >
> >> > In general, I like the idea of having the ability to interact directly
> >> with
> >> > the controllers. I agree with your observation that it helps in
> >> situations
> >> > where you would want to get data directly from the controller instead
> of
> >> > going via a broker. I have some general comments but the main concern
> I
> >> > have is with the piggy-backing of error code with response of
> >> > __cluster_metadata topic.
> >> >
> >> > 1. With this change, how are we guarding against the possibility of
> >> > misbehaving client traffic from disrupting the controller (that you
> >> > mentioned as a motivation of earlier behaviour)? One solution could
> be to
> >> > have default values set for request throttling on the controller.
> >>
> >> Hi Divij,
> >>
> >> Thanks for the comments.
> >>
> >> Our guards against client misbehavior remain the same:
> >> 1. our recommendation to put the clients on a separate network
> >> 2. the fact that producers and consumers can't interact directly with
> the
> >> controller
> >> 3. the authorizer.
> >>
> >> Re: #3, I will spell out in the KIP that clients must have DESCRIBE on
> the
> >> CLUSTER resource to send a METADATA request to the controller.
> >>
> >> > 2. This KIP also increases the network attack surface area. Prior to
> this
> >> > KIP, it was possible to have firewall rules setup for controllers such
> >> that
> >> > only the brokers can talk to it. But now, the controller would require
> >> > access from other endpoints other than brokers as well. Can we add a
> >> > suggestion to the upgrade documentation and inform users
> >>
> >> There is no requirement for access from other endpoints. It is still
> >> possible to set up firewall rules such that only the brokers can talk to
> >> the controller. In fact I would even say this is desirable. Since this
> >> faculty is intended for infrequent manual administrative operations,
> >> needing to log into the broker to use it seems perfectly fine.
> >>
> >> > 3. In section KRaft Controller MetadataResponse, row 3, "There is no
> >> > top-level error code in MetadataResponse, so we use the
> >> __cluster_metadata
> >> > topic to send back our error.". This will definitely confuse the
> users.
> >> Can
> >> > we introduce a top level error field instead?
> >>
> >> Let me check how we're handling this in other places. I recall some
> other
> >> cases where we used the dummy topic approach, but I can't find them just
> >> now.
> >>
> >> > 4. As part of the KIP, could we please add some documentation for
> users
> >> > with the suggestion of when to get information directly from the
> >> controller
> >> > and when not to (and associated tradeoffs)?
> >>
> >> I think most users will not face this tradeoff because they simply won't
> >> have network access to the controller servers.
> >>
> >> For those who do want more information, we'll have command-line
> >> documentation for --boostrap-controllers and the bootstrap.controllers
> >> configuration key.
> >>
> >> > 5. Why do we need the `FromKRaftController` field in the response?
> What
> >> do
> >> > we expect the users to do with this information?
> >>
> >> The field is present so that we can throw an exception in the client if
> we
> >> have received a response that is not from the controller.
> >>
> >> > 6. Can we drop `KRaft` from the fields `FromKRaftController` and
> >> > `DirectToKRaftControllerQuorum`? My suggestion will be to rename it as
> >> > `DirectToController`.
> >>
> >> I like the idea, but wouldn't that create confusion in the ZK cluster
> case?
> >>
> >> > 7.  "kafka-metadata-shell.sh  is at an "evolving" level of interface
> >> > stability" -> I thought that with KRaft being production ready, the
> >> > evolving mode for kraft-related tools has also moved to production.
> Do we
> >> > have a timeline when this would move to production?
> >>
> >> That's a good question, but I think we should tackle it separately from
> >> this KIP. The metadata shell is pretty different from other parts of
> kafka
> >> since it interacts so closely with internals.
> >>
> >> best,
> >> Colin
> >>
> >>
> >> >
> >> > --
> >> > Divij Vaidya
> >> >
> >> >
> >> >
> >> > On Tue, Apr 25, 2023 at 1:38 AM Colin McCabe <cm...@apache.org>
> wrote:
> >> >
> >> >> On Fri, Apr 21, 2023, at 14:17, Jason Gustafson wrote:
> >> >> > Hey Colin,
> >> >> >
> >> >> > The KIP makes sense overall. Nice to clarify the contract between
> >> clients
> >> >> > and the controllers. The use of `DirectToKRaftControllerQuorum`
> will
> >> help
> >> >> > prevent misconfiguration. In fact, I wonder if we can return a
> fatal
> >> >> error
> >> >> > instead of NOT_CONTROLLER so that the client would immediately
> fail.
> >> For
> >> >> > example, could we use INVALID_REQUEST or something like that?
> Either
> >> that
> >> >> > or we need to make sure clients treat NOT_CONTROLLER as a fatal
> error.
> >> >> > Without that, it would probably get picked up with default retry
> logic
> >> >> and
> >> >> > the user might not see the problem.
> >> >>
> >> >> Hi Jason,
> >> >>
> >> >> Yes, this is a good point. It should return INVALID_REQUEST since
> that
> >> is
> >> >> not retryable. I'll change it.
> >> >>
> >> >> >
> >> >> > I also wonder if we can relax the requirements on the Metadata
> >> request so
> >> >> > that we can use it to list topics and partition state (e.g.
> URPs).  It
> >> >> > would be useful to query the controllers as the metadata source of
> >> truth
> >> >> > when we suspect that the broker metadata may have diverged.
> >> >> >
> >> >>
> >> >> Fair enough. I will document that we can return topics.
> >> >>
> >> >> I also added a "future work" section about maybe using the
> controllers
> >> as
> >> >> bootstrap servers for the broker cluster. To be clear, that is NOT in
> >> scope
> >> >> here, but it's interesting to think about potentially doing in the
> >> future.
> >> >> The major problem is what to do when the broker endpoints we're
> >> returning
> >> >> have different security settings from the controller endpoint the
> client
> >> >> initially talked to.
> >> >>
> >> >> best,
> >> >> Colin
> >> >>
> >> >>
> >> >> >
> >> >> > Thanks,
> >> >> > Jason
> >> >> >
> >> >> > On Thu, Apr 20, 2023 at 5:53 PM Colin McCabe <cm...@apache.org>
> >> wrote:
> >> >> >
> >> >> >> On Wed, Apr 19, 2023, at 20:56, Philip Nee wrote:
> >> >> >> > Hey Colin,
> >> >> >> >
> >> >> >> > I still need to finish reading and understanding the KIP, but I
> >> have a
> >> >> >> > couple of comments despite being ignorant of most of the KRaft
> >> stuff.
> >> >> >> > (Sorry!)
> >> >> >> >
> >> >> >> > Firstly, does it make sense to create an extension of the
> current
> >> >> >> > AdminClient only to handle these specific KRaft use cases? It
> seems
> >> >> >> > cumbersome to have two sets of bootstrap configurations to make
> the
> >> >> >> > AdminClient generic enough to handle these specific cases,
> instead,
> >> >> maybe
> >> >> >> > it is more obvious (to me) to just extend the AdminClient. What
> I'm
> >> >> >> > thinking is KraftAdminClient which continuously uses
> >> >> *bootstrap.servers*,
> >> >> >> > but make this class only serves the Kraft controllers APIs.
> >> >> >>
> >> >> >> Hi Philip,
> >> >> >>
> >> >> >> Thanks for taking a look.
> >> >> >>
> >> >> >> We would not want to create a new Admin client class in order to
> >> >> >> communicate directly with the controllers. The RPCs accepted by
> the
> >> >> >> controllers have the same format as the those accepted by the
> >> brokers.
> >> >> >> There is no difference in what is sent over the wire or the data
> >> >> structures
> >> >> >> that are used in the client.
> >> >> >>
> >> >> >> I know you mentioned you haven't had time to read all the KRaft
> stuff
> >> >> (and
> >> >> >> there is a lot, I understand). But this is one area that would
> >> probably
> >> >> be
> >> >> >> clarified if you were able to read at least KIP-500 and KIP-590.
> RPCs
> >> >> sent
> >> >> >> to the broker are forwarded to the controller (mostly) without
> >> >> modification.
> >> >> >>
> >> >> >> >
> >> >> >> > Secondly, if we want to continue with the design, I'm not yet
> sure
> >> >> why we
> >> >> >> > can't continue using the *bootstrap.servers*? I assume when the
> >> client
> >> >> >> gets
> >> >> >> > the metadata, it should know who it is talking to. I'm just
> >> >> reconsidering
> >> >> >> > your alternative again.
> >> >> >> >
> >> >> >> > A bad idea: Why don't we continue using *bootstrap.servers* but
> >> have a
> >> >> >> > separated config like *kraft.controller* = true/false. I feel
> like
> >> >> most
> >> >> >> > users might not know what is a controller and causes some
> mistakes
> >> >> down
> >> >> >> the
> >> >> >> > road.
> >> >> >> >
> >> >> >>
> >> >> >> Well, you called it a bad idea, and I can't help but agree! :)
> >> >> >>
> >> >> >> I think "the user might not know what a controller is" is a good
> sign
> >> >> that
> >> >> >> they shouldn't be interacting with the controller. Like many
> >> AdminClient
> >> >> >> APIs, this one is intended for use by administrators only. Most
> users
> >> >> >> indeed should not need to know what a controller is or interact
> with
> >> it
> >> >> >> directly. If they do interact it should be very clear that they
> are
> >> >> doing
> >> >> >> so. The --controller-bootstrap flag makes it very clear, as does
> the
> >> >> >> separate configuration.
> >> >> >>
> >> >> >> Let me give an example of the kind of problems that arise if you
> >> want to
> >> >> >> reuse bootstrap.servers for this purpose.
> >> >> >>
> >> >> >> If the user grasb a 3.4 Kafka AdminClient and set
> bootstrap.servers
> >> to a
> >> >> >> set of controller servers, and set direct.to.controller to true,
> the
> >> >> >> unknown (in 3.4) configuration will be ignored, and a normal
> metadata
> >> >> >> request will be sent without the direct to controller flag. In
> that
> >> >> case it
> >> >> >> will give back an error. Confusing, right?
> >> >> >>
> >> >> >> Using controller.servers clarifies this situation because the 3.4
> >> client
> >> >> >> will not support that config, and will complain about the lack of
> >> >> >> bootstrap.servers.
> >> >> >>
> >> >> >> Actually, the situation could get even more confusing than what I
> >> >> >> described since some older preproduction versions of the KRaft
> >> >> controller
> >> >> >> did implement the METADATA RPC. So if you send them a METADATA
> >> request
> >> >> >> without any special information, it's not clear what you'll get.
> >> >> Indeed,
> >> >> >> it would be dependent on the client version and the controller
> >> version.
> >> >> >>
> >> >> >> The bottom line is that reusing the bootstrap.servers
> configuration
> >> here
> >> >> >> is not a good idea.
> >> >> >>
> >> >> >> best,
> >> >> >> Colin
> >> >> >>
> >> >> >> > Thanks,
> >> >> >> > P
> >> >> >> >
> >> >> >> > On Wed, Apr 19, 2023 at 2:18 PM Colin McCabe <
> cmccabe@apache.org>
> >> >> wrote:
> >> >> >> >
> >> >> >> >> Hi all,
> >> >> >> >>
> >> >> >> >> I wrote a short KIP about allowing AdminClient to talk directly
> >> with
> >> >> the
> >> >> >> >> KRaft controller quorum. Check it out here:
> >> >> >> >>
> >> >> >> >> https://cwiki.apache.org/confluence/x/Owo0Dw
> >> >> >> >>
> >> >> >> >> best,
> >> >> >> >> Colin
> >> >> >> >>
> >> >> >>
> >> >>
> >>
>
>

Re: KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum

Posted by Colin McCabe <cm...@apache.org>.
On Fri, Jul 21, 2023, at 08:14, David Arthur wrote:
> Hey Colin, thanks for the KIP! Some questions
>

Hi David,

Thanks for reviewing.

> 1) "This registration will include information about the endpoints which
> they possess"  Will this include all endpoints, or only those configured in
> "advertised.listeners"
>

The registration information includes all controller endpoints. Keep in mind, it is not valid for any controller listeners to appear in advertised.listeners.

>
> 2) "Periodically, each controller will check that the controller
> registration for its ID is as expected."  Does this need to be a periodic
> check? Since the controller registration state will be in the log, can't
> the follower just react to unexpected incarnation IDs (after it's caught
> up)?
>

I think it will hang off of a metadata publisher. You are right that it can mostly be event-driven. See a metadata update that overwrites your registration => act to re-register.

The "periodic" stuff comes in in cases where we fail to register and have to try again. I'll try to clarify the wording.

>
> 3) ControllerRegistrationRequest has a typo in the listeners section (it
> mentions "broker")
>

Fixed

>
> 4) Since we can't rely on the ApiVersions data, should we remove the field
> we added to ApiVersionsResponse in KIP-866?
>

Yes, this is a good point. I'll mark it as deprecated.

>
> 5)I filed https://issues.apache.org/jira/browse/KAFKA-15230 for the issues
> mentioned under "Controller Changes" in case you want to link it
>

Added

>
> 6) I don't see it explicitly mentioned, but I think it's the case that the
> active controller must accept and persist any controller registration it
> receives. This is unlike the behavior of broker registrations where we can
> reject brokers we don't want. For controllers, I don't think we have that
> option unless we go for some tighter Raft integration. Since the followers
> must be participating in Raft to learn about the leader (and therefore,
> will have replayed the full log), we can't really say "no" at that point.
>

Agreed. I added some wording to this effect in the ControllerRegistrationRequest  section. Also specified that we can return NOT_CONTROLLER from this API, when not active.

cheers,
Colin
 
>
> Cheers,
> David
>
>
> On Thu, Jul 20, 2023 at 7:23 PM Colin McCabe <cm...@apache.org> wrote:
>
>> On Tue, Jul 18, 2023, at 09:30, Mickael Maison wrote:
>> > H Colin,
>> >
>> > Thanks for the KIP.
>> >
>> > Just a few points:
>> > 1. As Tom mentioned it would be good to clarify the APIs we expect
>> > available on controllers. I assume we want to add DESCRIBE_CONFIGS as
>> > part of this KIP.
>>
>> Hi Mickael,
>>
>> Yes, this is a good point. I added a table describing the APIs that will
>> now be added.
>>
>> > 2. Currently we have no way of retrieving the list of configs that
>> > apply to controllers. It would be good to have an object, so we can
>> > add that to the docs but also use that in kafka-configs.
>>
>> I think this is out of scope.
>>
>> > 3. Should we have a new entity-type in kafka-configs for setting
>> > controller configs?
>>
>> The BROKER entity type already applies to controllers. It probably needs a
>> new name (NODE would be better) but that's out of scope for this KIP, I
>> think.
>>
>> best,
>> Colin
>>
>>
>> >
>> > Thanks,
>> > Mickael
>> >
>> > On Tue, Jul 4, 2023 at 2:20 PM Luke Chen <sh...@gmail.com> wrote:
>> >>
>> >> Hi Colin,
>> >>
>> >> Thanks for the answers to my previous questions.
>> >>
>> >> > Yes, the common thread here is that all of these shell commands
>> perform
>> >> operations can be done without the broker. So it's reasonable to allow
>> them
>> >> to be done without going through the broker. I don't know if we need a
>> >> separate note for each since the rationale is really the same for all
>> (is
>> >> it reasonable? if so allow it.)
>> >>
>> >> Yes, it makes sense. Could we make a note about the main rationale for
>> >> selecting these command-line tools in the KIP to make it clear?
>> >> Ex: The following command-line tools will get a new
>> --bootstrap-controllers
>> >> argument (because these shell commands perform operations can be done
>> >> without the broker):
>> >>
>> >> > kafka-reassign-partitions.sh cannot be used to move the
>> >> __cluster_metadata topic. However, it can be used to move partitions
>> that
>> >> reside on the brokers, even when using --bootstrap-controllers to talk
>> >> directly to the quorum.
>> >>
>> >> Fair enough.
>> >>
>> >>
>> >> 4. Does all the command-line tools with `--bootstrap-controllers`
>> support
>> >> all the options in the tool?
>> >> For example, kafka-configs.sh, In addition to the `--alter` option you
>> >> mentioned in the example, do we also support `--describe` or `--delete`
>> >> option?
>> >> If so, do we also support setting "quota" for users/clients/topics...
>> via
>> >> `--bootstrap-controllers`? (not intuitive, but maybe we just directly
>> >> commit the change into the metadata from controller?)
>> >>
>> >> 5. Do we have any plan for this feature to be completed? v3.6.0?
>> >>
>> >>
>> >> Thank you.
>> >> Luke
>> >>
>> >>
>> >> On Fri, Apr 28, 2023 at 1:42 AM Colin McCabe <cm...@apache.org>
>> wrote:
>> >>
>> >> > On Wed, Apr 26, 2023, at 22:08, Luke Chen wrote:
>> >> > > Hi Colin,
>> >> > >
>> >> > > Some comments:
>> >> > > 1. I agree we should set "top-level" errors for metadata response
>> >> > >
>> >> > > 2. In the "brokers" field of metadata response from controller,
>> it'll
>> >> > > respond with "Controller endpoint information as given in
>> >> > > controller.quorum.voters", instead of the "alive"
>> controllers(voters).
>> >> > That
>> >> > > will break the existing admin client because in admin client, we'll
>> rely
>> >> > on
>> >> > > the metadata response to build the "current alive brokers" list, and
>> >> > choose
>> >> > > one from them to connect (either least load or other criteria). That
>> >> > means,
>> >> > > if now, we return the value in `controller.quorum.voters`, but one
>> of
>> >> > them
>> >> > > is down. We might choose it to connect and get connection errors.
>> Should
>> >> > we
>> >> > > return the "alive" controllers(voters) to client?
>> >> >
>> >> > Hi Luke,
>> >> >
>> >> > Good question. When talking to the controllers directly, the
>> AdminClient
>> >> > needs to always send its RPCs to the active controller. There is one
>> >> > exception: configuring ephemeral log4j settings with
>> >> > incrementalAlterConfigs must be done by sending them to the specified
>> >> > controller node.
>> >> >
>> >> > I will add this to a section called "AdminClient Implementation
>> Notes" so
>> >> > that it's captured in the KIP.
>> >> >
>> >> > >
>> >> > > 3. In the KIP, we list the command-line tools will get a new
>> >> > > --bootstrap-controllers argument, but without explaining why these
>> tools
>> >> > > need to talk to controller directly. Could we add some explanation
>> about
>> >> > > them? I tried but cannot know why some tools are listed here:
>> >> > >     - kafka-acls.sh -> Allow clients to update ACLs via controller
>> before
>> >> > > brokers up
>> >> > >
>> >> > >     - kafka-cluster.sh -> Reasonable to get/update cluster info via
>> >> > > controller
>> >> > >
>> >> > >     - kafka-configs.sh -> Allow clients to dynamically update
>> >> > > configs/describe configs from controller. But in this script,
>> client can
>> >> > > still set quota for users/clients/topics... is client also able to
>> update
>> >> > > via controllers? Or we only allow partial actions in the script to
>> talk
>> >> > to
>> >> > > controllers?
>> >> > >
>> >> > >     - kafka-delegation-tokens.sh -> Reasonable to update
>> >> > delegation-tokens
>> >> > > via controllers
>> >> > >
>> >> > >     - kafka-features.sh -> Reasonable
>> >> > >     - kafka-metadata-quorum.sh -> Reasonable
>> >> > >     - kafka-metadata-shell.sh -> Reasonable
>> >> > >
>> >> > >     - kafka-reassign-partitions.sh -> Why should we allow clients
>> to move
>> >> > > metadata log partitions in controller nodes? What's the use-case?
>> >> > >
>> >> >
>> >> > Yes, the common thread here is that all of these shell commands
>> perform
>> >> > operations can be done without the broker. So it's reasonable to
>> allow them
>> >> > to be done without going through the broker. I don't know if we need a
>> >> > separate note for each since the rationale is really the same for all
>> (is
>> >> > it reasonable? if so allow it.)
>> >> >
>> >> > kafka-reassign-partitions.sh cannot be used to move the
>> __cluster_metadata
>> >> > topic. However, it can be used to move partitions that reside on the
>> >> > brokers, even when using --bootstrap-controllers to talk directly to
>> the
>> >> > quorum.
>> >> >
>> >> > Colin
>> >> >
>> >> > >
>> >> > > Thank you.
>> >> > > Luke
>> >> > >
>> >> > > On Thu, Apr 27, 2023 at 8:04 AM Colin McCabe <cm...@apache.org>
>> wrote:
>> >> > >
>> >> > >> On Tue, Apr 25, 2023, at 04:59, Divij Vaidya wrote:
>> >> > >> > Thank you for the KIP Colin.
>> >> > >> >
>> >> > >> > In general, I like the idea of having the ability to interact
>> directly
>> >> > >> with
>> >> > >> > the controllers. I agree with your observation that it helps in
>> >> > >> situations
>> >> > >> > where you would want to get data directly from the controller
>> instead
>> >> > of
>> >> > >> > going via a broker. I have some general comments but the main
>> concern
>> >> > I
>> >> > >> > have is with the piggy-backing of error code with response of
>> >> > >> > __cluster_metadata topic.
>> >> > >> >
>> >> > >> > 1. With this change, how are we guarding against the possibility
>> of
>> >> > >> > misbehaving client traffic from disrupting the controller (that
>> you
>> >> > >> > mentioned as a motivation of earlier behaviour)? One solution
>> could
>> >> > be to
>> >> > >> > have default values set for request throttling on the controller.
>> >> > >>
>> >> > >> Hi Divij,
>> >> > >>
>> >> > >> Thanks for the comments.
>> >> > >>
>> >> > >> Our guards against client misbehavior remain the same:
>> >> > >> 1. our recommendation to put the clients on a separate network
>> >> > >> 2. the fact that producers and consumers can't interact directly
>> with
>> >> > the
>> >> > >> controller
>> >> > >> 3. the authorizer.
>> >> > >>
>> >> > >> Re: #3, I will spell out in the KIP that clients must have
>> DESCRIBE on
>> >> > the
>> >> > >> CLUSTER resource to send a METADATA request to the controller.
>> >> > >>
>> >> > >> > 2. This KIP also increases the network attack surface area.
>> Prior to
>> >> > this
>> >> > >> > KIP, it was possible to have firewall rules setup for
>> controllers such
>> >> > >> that
>> >> > >> > only the brokers can talk to it. But now, the controller would
>> require
>> >> > >> > access from other endpoints other than brokers as well. Can we
>> add a
>> >> > >> > suggestion to the upgrade documentation and inform users
>> >> > >>
>> >> > >> There is no requirement for access from other endpoints. It is
>> still
>> >> > >> possible to set up firewall rules such that only the brokers can
>> talk to
>> >> > >> the controller. In fact I would even say this is desirable. Since
>> this
>> >> > >> faculty is intended for infrequent manual administrative
>> operations,
>> >> > >> needing to log into the broker to use it seems perfectly fine.
>> >> > >>
>> >> > >> > 3. In section KRaft Controller MetadataResponse, row 3, "There
>> is no
>> >> > >> > top-level error code in MetadataResponse, so we use the
>> >> > >> __cluster_metadata
>> >> > >> > topic to send back our error.". This will definitely confuse the
>> >> > users.
>> >> > >> Can
>> >> > >> > we introduce a top level error field instead?
>> >> > >>
>> >> > >> Let me check how we're handling this in other places. I recall some
>> >> > other
>> >> > >> cases where we used the dummy topic approach, but I can't find
>> them just
>> >> > >> now.
>> >> > >>
>> >> > >> > 4. As part of the KIP, could we please add some documentation for
>> >> > users
>> >> > >> > with the suggestion of when to get information directly from the
>> >> > >> controller
>> >> > >> > and when not to (and associated tradeoffs)?
>> >> > >>
>> >> > >> I think most users will not face this tradeoff because they simply
>> won't
>> >> > >> have network access to the controller servers.
>> >> > >>
>> >> > >> For those who do want more information, we'll have command-line
>> >> > >> documentation for --boostrap-controllers and the
>> bootstrap.controllers
>> >> > >> configuration key.
>> >> > >>
>> >> > >> > 5. Why do we need the `FromKRaftController` field in the
>> response?
>> >> > What
>> >> > >> do
>> >> > >> > we expect the users to do with this information?
>> >> > >>
>> >> > >> The field is present so that we can throw an exception in the
>> client if
>> >> > we
>> >> > >> have received a response that is not from the controller.
>> >> > >>
>> >> > >> > 6. Can we drop `KRaft` from the fields `FromKRaftController` and
>> >> > >> > `DirectToKRaftControllerQuorum`? My suggestion will be to rename
>> it as
>> >> > >> > `DirectToController`.
>> >> > >>
>> >> > >> I like the idea, but wouldn't that create confusion in the ZK
>> cluster
>> >> > case?
>> >> > >>
>> >> > >> > 7.  "kafka-metadata-shell.sh  is at an "evolving" level of
>> interface
>> >> > >> > stability" -> I thought that with KRaft being production ready,
>> the
>> >> > >> > evolving mode for kraft-related tools has also moved to
>> production.
>> >> > Do we
>> >> > >> > have a timeline when this would move to production?
>> >> > >>
>> >> > >> That's a good question, but I think we should tackle it separately
>> from
>> >> > >> this KIP. The metadata shell is pretty different from other parts
>> of
>> >> > kafka
>> >> > >> since it interacts so closely with internals.
>> >> > >>
>> >> > >> best,
>> >> > >> Colin
>> >> > >>
>> >> > >>
>> >> > >> >
>> >> > >> > --
>> >> > >> > Divij Vaidya
>> >> > >> >
>> >> > >> >
>> >> > >> >
>> >> > >> > On Tue, Apr 25, 2023 at 1:38 AM Colin McCabe <cmccabe@apache.org
>> >
>> >> > wrote:
>> >> > >> >
>> >> > >> >> On Fri, Apr 21, 2023, at 14:17, Jason Gustafson wrote:
>> >> > >> >> > Hey Colin,
>> >> > >> >> >
>> >> > >> >> > The KIP makes sense overall. Nice to clarify the contract
>> between
>> >> > >> clients
>> >> > >> >> > and the controllers. The use of
>> `DirectToKRaftControllerQuorum`
>> >> > will
>> >> > >> help
>> >> > >> >> > prevent misconfiguration. In fact, I wonder if we can return a
>> >> > fatal
>> >> > >> >> error
>> >> > >> >> > instead of NOT_CONTROLLER so that the client would immediately
>> >> > fail.
>> >> > >> For
>> >> > >> >> > example, could we use INVALID_REQUEST or something like that?
>> >> > Either
>> >> > >> that
>> >> > >> >> > or we need to make sure clients treat NOT_CONTROLLER as a
>> fatal
>> >> > error.
>> >> > >> >> > Without that, it would probably get picked up with default
>> retry
>> >> > logic
>> >> > >> >> and
>> >> > >> >> > the user might not see the problem.
>> >> > >> >>
>> >> > >> >> Hi Jason,
>> >> > >> >>
>> >> > >> >> Yes, this is a good point. It should return INVALID_REQUEST
>> since
>> >> > that
>> >> > >> is
>> >> > >> >> not retryable. I'll change it.
>> >> > >> >>
>> >> > >> >> >
>> >> > >> >> > I also wonder if we can relax the requirements on the Metadata
>> >> > >> request so
>> >> > >> >> > that we can use it to list topics and partition state (e.g.
>> >> > URPs).  It
>> >> > >> >> > would be useful to query the controllers as the metadata
>> source of
>> >> > >> truth
>> >> > >> >> > when we suspect that the broker metadata may have diverged.
>> >> > >> >> >
>> >> > >> >>
>> >> > >> >> Fair enough. I will document that we can return topics.
>> >> > >> >>
>> >> > >> >> I also added a "future work" section about maybe using the
>> >> > controllers
>> >> > >> as
>> >> > >> >> bootstrap servers for the broker cluster. To be clear, that is
>> NOT in
>> >> > >> scope
>> >> > >> >> here, but it's interesting to think about potentially doing in
>> the
>> >> > >> future.
>> >> > >> >> The major problem is what to do when the broker endpoints we're
>> >> > >> returning
>> >> > >> >> have different security settings from the controller endpoint
>> the
>> >> > client
>> >> > >> >> initially talked to.
>> >> > >> >>
>> >> > >> >> best,
>> >> > >> >> Colin
>> >> > >> >>
>> >> > >> >>
>> >> > >> >> >
>> >> > >> >> > Thanks,
>> >> > >> >> > Jason
>> >> > >> >> >
>> >> > >> >> > On Thu, Apr 20, 2023 at 5:53 PM Colin McCabe <
>> cmccabe@apache.org>
>> >> > >> wrote:
>> >> > >> >> >
>> >> > >> >> >> On Wed, Apr 19, 2023, at 20:56, Philip Nee wrote:
>> >> > >> >> >> > Hey Colin,
>> >> > >> >> >> >
>> >> > >> >> >> > I still need to finish reading and understanding the KIP,
>> but I
>> >> > >> have a
>> >> > >> >> >> > couple of comments despite being ignorant of most of the
>> KRaft
>> >> > >> stuff.
>> >> > >> >> >> > (Sorry!)
>> >> > >> >> >> >
>> >> > >> >> >> > Firstly, does it make sense to create an extension of the
>> >> > current
>> >> > >> >> >> > AdminClient only to handle these specific KRaft use cases?
>> It
>> >> > seems
>> >> > >> >> >> > cumbersome to have two sets of bootstrap configurations to
>> make
>> >> > the
>> >> > >> >> >> > AdminClient generic enough to handle these specific cases,
>> >> > instead,
>> >> > >> >> maybe
>> >> > >> >> >> > it is more obvious (to me) to just extend the AdminClient.
>> What
>> >> > I'm
>> >> > >> >> >> > thinking is KraftAdminClient which continuously uses
>> >> > >> >> *bootstrap.servers*,
>> >> > >> >> >> > but make this class only serves the Kraft controllers APIs.
>> >> > >> >> >>
>> >> > >> >> >> Hi Philip,
>> >> > >> >> >>
>> >> > >> >> >> Thanks for taking a look.
>> >> > >> >> >>
>> >> > >> >> >> We would not want to create a new Admin client class in
>> order to
>> >> > >> >> >> communicate directly with the controllers. The RPCs accepted
>> by
>> >> > the
>> >> > >> >> >> controllers have the same format as the those accepted by the
>> >> > >> brokers.
>> >> > >> >> >> There is no difference in what is sent over the wire or the
>> data
>> >> > >> >> structures
>> >> > >> >> >> that are used in the client.
>> >> > >> >> >>
>> >> > >> >> >> I know you mentioned you haven't had time to read all the
>> KRaft
>> >> > stuff
>> >> > >> >> (and
>> >> > >> >> >> there is a lot, I understand). But this is one area that
>> would
>> >> > >> probably
>> >> > >> >> be
>> >> > >> >> >> clarified if you were able to read at least KIP-500 and
>> KIP-590.
>> >> > RPCs
>> >> > >> >> sent
>> >> > >> >> >> to the broker are forwarded to the controller (mostly)
>> without
>> >> > >> >> modification.
>> >> > >> >> >>
>> >> > >> >> >> >
>> >> > >> >> >> > Secondly, if we want to continue with the design, I'm not
>> yet
>> >> > sure
>> >> > >> >> why we
>> >> > >> >> >> > can't continue using the *bootstrap.servers*? I assume
>> when the
>> >> > >> client
>> >> > >> >> >> gets
>> >> > >> >> >> > the metadata, it should know who it is talking to. I'm just
>> >> > >> >> reconsidering
>> >> > >> >> >> > your alternative again.
>> >> > >> >> >> >
>> >> > >> >> >> > A bad idea: Why don't we continue using
>> *bootstrap.servers* but
>> >> > >> have a
>> >> > >> >> >> > separated config like *kraft.controller* = true/false. I
>> feel
>> >> > like
>> >> > >> >> most
>> >> > >> >> >> > users might not know what is a controller and causes some
>> >> > mistakes
>> >> > >> >> down
>> >> > >> >> >> the
>> >> > >> >> >> > road.
>> >> > >> >> >> >
>> >> > >> >> >>
>> >> > >> >> >> Well, you called it a bad idea, and I can't help but agree!
>> :)
>> >> > >> >> >>
>> >> > >> >> >> I think "the user might not know what a controller is" is a
>> good
>> >> > sign
>> >> > >> >> that
>> >> > >> >> >> they shouldn't be interacting with the controller. Like many
>> >> > >> AdminClient
>> >> > >> >> >> APIs, this one is intended for use by administrators only.
>> Most
>> >> > users
>> >> > >> >> >> indeed should not need to know what a controller is or
>> interact
>> >> > with
>> >> > >> it
>> >> > >> >> >> directly. If they do interact it should be very clear that
>> they
>> >> > are
>> >> > >> >> doing
>> >> > >> >> >> so. The --controller-bootstrap flag makes it very clear, as
>> does
>> >> > the
>> >> > >> >> >> separate configuration.
>> >> > >> >> >>
>> >> > >> >> >> Let me give an example of the kind of problems that arise if
>> you
>> >> > >> want to
>> >> > >> >> >> reuse bootstrap.servers for this purpose.
>> >> > >> >> >>
>> >> > >> >> >> If the user grasb a 3.4 Kafka AdminClient and set
>> >> > bootstrap.servers
>> >> > >> to a
>> >> > >> >> >> set of controller servers, and set direct.to.controller to
>> true,
>> >> > the
>> >> > >> >> >> unknown (in 3.4) configuration will be ignored, and a normal
>> >> > metadata
>> >> > >> >> >> request will be sent without the direct to controller flag.
>> In
>> >> > that
>> >> > >> >> case it
>> >> > >> >> >> will give back an error. Confusing, right?
>> >> > >> >> >>
>> >> > >> >> >> Using controller.servers clarifies this situation because
>> the 3.4
>> >> > >> client
>> >> > >> >> >> will not support that config, and will complain about the
>> lack of
>> >> > >> >> >> bootstrap.servers.
>> >> > >> >> >>
>> >> > >> >> >> Actually, the situation could get even more confusing than
>> what I
>> >> > >> >> >> described since some older preproduction versions of the
>> KRaft
>> >> > >> >> controller
>> >> > >> >> >> did implement the METADATA RPC. So if you send them a
>> METADATA
>> >> > >> request
>> >> > >> >> >> without any special information, it's not clear what you'll
>> get.
>> >> > >> >> Indeed,
>> >> > >> >> >> it would be dependent on the client version and the
>> controller
>> >> > >> version.
>> >> > >> >> >>
>> >> > >> >> >> The bottom line is that reusing the bootstrap.servers
>> >> > configuration
>> >> > >> here
>> >> > >> >> >> is not a good idea.
>> >> > >> >> >>
>> >> > >> >> >> best,
>> >> > >> >> >> Colin
>> >> > >> >> >>
>> >> > >> >> >> > Thanks,
>> >> > >> >> >> > P
>> >> > >> >> >> >
>> >> > >> >> >> > On Wed, Apr 19, 2023 at 2:18 PM Colin McCabe <
>> >> > cmccabe@apache.org>
>> >> > >> >> wrote:
>> >> > >> >> >> >
>> >> > >> >> >> >> Hi all,
>> >> > >> >> >> >>
>> >> > >> >> >> >> I wrote a short KIP about allowing AdminClient to talk
>> directly
>> >> > >> with
>> >> > >> >> the
>> >> > >> >> >> >> KRaft controller quorum. Check it out here:
>> >> > >> >> >> >>
>> >> > >> >> >> >> https://cwiki.apache.org/confluence/x/Owo0Dw
>> >> > >> >> >> >>
>> >> > >> >> >> >> best,
>> >> > >> >> >> >> Colin
>> >> > >> >> >> >>
>> >> > >> >> >>
>> >> > >> >>
>> >> > >>
>> >> >
>>
>
>
> -- 
> -David

Re: KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum

Posted by "Nelson B." <ba...@gmail.com>.
Hi Colin,

Thanks for the KIP!

I'm sorry for reopening this old discussion thread but I don't know where
else I can ask my question.

I wanted to use the newly updated kafka-configs.sh tool to dynamically
update the ssl keystore in the controller node, but I still couldn't
figure it out.
Is it possible to use kafka-configs.sh to dynamically update
controller configuration?

In my setup, I have a controller node with node id 1 listening on port
9093. Below is the log of kafka-configs.sh tool:

[image: image.png]

As you can see, for some reason node is not recognized as a controller and
I'm not sure why this happens. Could you please help me?

Thanks! Regards

On Wed, Jul 26, 2023 at 3:43 AM Colin McCabe <cm...@apache.org> wrote:

> On Tue, Jul 25, 2023, at 05:30, Luke Chen wrote:
> > Hi Colin,
> >
> > Some more comments:
> > 1. In the KIP, we mentioned "controller heartbeats", but it is not
> > explained anywhere.
> > I think "controller heartbeats" = controller registration", is that
> > correct?
> > If no, please explain more about it in the KIP.
>
> Hi Luke,
>
> Sorry, this was an artifact of earlier revisions. I have changed it to
> "ControllerRegistration" in all the cases where I didn't update it before.
>
> >
> > 2. Following this question:
> > > Which endpoint will the inactive controllers use to send the
> > > ControllerRegistrationRequest?
> > > A: They will use the endpoint in controller.quorum.voters.
> > If the registration request will include controller.quorum.voters, why
> > bother sending this information to active controller again?
> > The active controller should already have all the
> controller.quorum.voters
> > when start up.
> > Any purpose of that design? For validation?
>
> The controllers don't know which endpoint controller.quorum.voters is
> referencing.
>
> >
> > 3. If a controller node is not part of `controller.quorum.voters`, when
> it
> > sends ControllerRegistrationRequest, what will we respond to it?
> >
>
> Good point. I added a new error, UNKNOWN_CONTROLLER_ID, for this case.
>
> > 4. Nice and clear compatibility matrix!
> >
>
> Thanks!
> Colin
>
> > Thank you.
> > Luke
> >
> > On Sat, Jul 22, 2023 at 3:38 AM Colin McCabe <cm...@apache.org> wrote:
> >
> >> On Fri, Jul 21, 2023, at 09:43, José Armando García Sancio wrote:
> >> > Thanks for the KIP Colin. Apologies if some of these points have
> >> > already been made. I have not followed the discussion closely:
> >> >
> >> > 1. Re: Periodically, each controller will check that the controller
> >> > registration for its ID is as expected
> >> >
> >> > Does this need to be periodic? Can't the controller schedule this RPC,
> >> > retry etc, when it finds that the incarnation ID doesn't match its
> >> > own?
> >> >
> >>
> >> Hi José,
> >>
> >> Thanks for the reviews.
> >>
> >> David had the same question. I agree that it should be event-driven
> rather
> >> than periodic (except for retries, etc.)
> >>
> >> >
> >> > 2. Did you consider including the active controller's epoch in the
> >> > ControllerRegistrationRequest?
> >> >
> >> > This would allow the active controller to reject registration from
> >> > controllers that are not part of the active quorum and don't know the
> >> > latest controller epoch. This should mitigate some of the concerns you
> >> > raised in bullet point 1.
> >> >
> >>
> >> Good idea. I will add the active controller epoch to the registration
> >> request.
> >>
> >> >
> >> > 3. Which endpoint will the inactive controllers use to send the
> >> > ControllerRegistrationRequest?
> >> >
> >> > Will it use the first endpoint described in the cluster metadata
> >> > controller registration record? Or would it use the endpoint described
> >> > in the server configuration at controller.quorum.voters?
> >> >
> >>
> >> They will use the endpoint in controller.quorum.voters. In general, the
> >> endpoints from the registration are only used for responding to
> >> DESCRIBE_CLUSTER. Since, after all, we may not even have the
> registration
> >> endpoints when we start up.
> >>
> >> >
> >> > 4. Re: Raft integration in the rejected alternatives
> >> >
> >> > Yes, The KRaft layer needs to solve a similar problem like endpoint
> >> > discovery to support dynamic controller membership change. As you
> >> > point out the requirements are different and the set of information
> >> > that needs to be tracked is different. I think it is okay to use a
> >> > different solution for each of these problems.
> >>
> >> Yeah that was my feeling too. Thanks for taking a look.
> >>
> >> regards,
> >> Colin
> >>
> >> >
> >> > Thanks!
> >> > --
> >> > -José
> >>
>

Re: KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum

Posted by Colin McCabe <cm...@apache.org>.
On Tue, Jul 25, 2023, at 05:30, Luke Chen wrote:
> Hi Colin,
>
> Some more comments:
> 1. In the KIP, we mentioned "controller heartbeats", but it is not
> explained anywhere.
> I think "controller heartbeats" = controller registration", is that
> correct?
> If no, please explain more about it in the KIP.

Hi Luke,

Sorry, this was an artifact of earlier revisions. I have changed it to "ControllerRegistration" in all the cases where I didn't update it before.

>
> 2. Following this question:
> > Which endpoint will the inactive controllers use to send the
> > ControllerRegistrationRequest?
> > A: They will use the endpoint in controller.quorum.voters.
> If the registration request will include controller.quorum.voters, why
> bother sending this information to active controller again?
> The active controller should already have all the controller.quorum.voters
> when start up.
> Any purpose of that design? For validation?

The controllers don't know which endpoint controller.quorum.voters is referencing.

>
> 3. If a controller node is not part of `controller.quorum.voters`, when it
> sends ControllerRegistrationRequest, what will we respond to it?
>

Good point. I added a new error, UNKNOWN_CONTROLLER_ID, for this case.

> 4. Nice and clear compatibility matrix!
>

Thanks!
Colin

> Thank you.
> Luke
>
> On Sat, Jul 22, 2023 at 3:38 AM Colin McCabe <cm...@apache.org> wrote:
>
>> On Fri, Jul 21, 2023, at 09:43, José Armando García Sancio wrote:
>> > Thanks for the KIP Colin. Apologies if some of these points have
>> > already been made. I have not followed the discussion closely:
>> >
>> > 1. Re: Periodically, each controller will check that the controller
>> > registration for its ID is as expected
>> >
>> > Does this need to be periodic? Can't the controller schedule this RPC,
>> > retry etc, when it finds that the incarnation ID doesn't match its
>> > own?
>> >
>>
>> Hi José,
>>
>> Thanks for the reviews.
>>
>> David had the same question. I agree that it should be event-driven rather
>> than periodic (except for retries, etc.)
>>
>> >
>> > 2. Did you consider including the active controller's epoch in the
>> > ControllerRegistrationRequest?
>> >
>> > This would allow the active controller to reject registration from
>> > controllers that are not part of the active quorum and don't know the
>> > latest controller epoch. This should mitigate some of the concerns you
>> > raised in bullet point 1.
>> >
>>
>> Good idea. I will add the active controller epoch to the registration
>> request.
>>
>> >
>> > 3. Which endpoint will the inactive controllers use to send the
>> > ControllerRegistrationRequest?
>> >
>> > Will it use the first endpoint described in the cluster metadata
>> > controller registration record? Or would it use the endpoint described
>> > in the server configuration at controller.quorum.voters?
>> >
>>
>> They will use the endpoint in controller.quorum.voters. In general, the
>> endpoints from the registration are only used for responding to
>> DESCRIBE_CLUSTER. Since, after all, we may not even have the registration
>> endpoints when we start up.
>>
>> >
>> > 4. Re: Raft integration in the rejected alternatives
>> >
>> > Yes, The KRaft layer needs to solve a similar problem like endpoint
>> > discovery to support dynamic controller membership change. As you
>> > point out the requirements are different and the set of information
>> > that needs to be tracked is different. I think it is okay to use a
>> > different solution for each of these problems.
>>
>> Yeah that was my feeling too. Thanks for taking a look.
>>
>> regards,
>> Colin
>>
>> >
>> > Thanks!
>> > --
>> > -José
>>

Re: KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum

Posted by Luke Chen <sh...@gmail.com>.
Hi Colin,

Some more comments:
1. In the KIP, we mentioned "controller heartbeats", but it is not
explained anywhere.
I think "controller heartbeats" = controller registration", is that
correct?
If no, please explain more about it in the KIP.

2. Following this question:
> Which endpoint will the inactive controllers use to send the
ControllerRegistrationRequest?
> A: They will use the endpoint in controller.quorum.voters.
If the registration request will include controller.quorum.voters, why
bother sending this information to active controller again?
The active controller should already have all the controller.quorum.voters
when start up.
Any purpose of that design? For validation?

3. If a controller node is not part of `controller.quorum.voters`, when it
sends ControllerRegistrationRequest, what will we respond to it?

4. Nice and clear compatibility matrix!

Thank you.
Luke

On Sat, Jul 22, 2023 at 3:38 AM Colin McCabe <cm...@apache.org> wrote:

> On Fri, Jul 21, 2023, at 09:43, José Armando García Sancio wrote:
> > Thanks for the KIP Colin. Apologies if some of these points have
> > already been made. I have not followed the discussion closely:
> >
> > 1. Re: Periodically, each controller will check that the controller
> > registration for its ID is as expected
> >
> > Does this need to be periodic? Can't the controller schedule this RPC,
> > retry etc, when it finds that the incarnation ID doesn't match its
> > own?
> >
>
> Hi José,
>
> Thanks for the reviews.
>
> David had the same question. I agree that it should be event-driven rather
> than periodic (except for retries, etc.)
>
> >
> > 2. Did you consider including the active controller's epoch in the
> > ControllerRegistrationRequest?
> >
> > This would allow the active controller to reject registration from
> > controllers that are not part of the active quorum and don't know the
> > latest controller epoch. This should mitigate some of the concerns you
> > raised in bullet point 1.
> >
>
> Good idea. I will add the active controller epoch to the registration
> request.
>
> >
> > 3. Which endpoint will the inactive controllers use to send the
> > ControllerRegistrationRequest?
> >
> > Will it use the first endpoint described in the cluster metadata
> > controller registration record? Or would it use the endpoint described
> > in the server configuration at controller.quorum.voters?
> >
>
> They will use the endpoint in controller.quorum.voters. In general, the
> endpoints from the registration are only used for responding to
> DESCRIBE_CLUSTER. Since, after all, we may not even have the registration
> endpoints when we start up.
>
> >
> > 4. Re: Raft integration in the rejected alternatives
> >
> > Yes, The KRaft layer needs to solve a similar problem like endpoint
> > discovery to support dynamic controller membership change. As you
> > point out the requirements are different and the set of information
> > that needs to be tracked is different. I think it is okay to use a
> > different solution for each of these problems.
>
> Yeah that was my feeling too. Thanks for taking a look.
>
> regards,
> Colin
>
> >
> > Thanks!
> > --
> > -José
>

Re: KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum

Posted by Colin McCabe <cm...@apache.org>.
On Fri, Jul 21, 2023, at 09:43, José Armando García Sancio wrote:
> Thanks for the KIP Colin. Apologies if some of these points have
> already been made. I have not followed the discussion closely:
>
> 1. Re: Periodically, each controller will check that the controller
> registration for its ID is as expected
>
> Does this need to be periodic? Can't the controller schedule this RPC,
> retry etc, when it finds that the incarnation ID doesn't match its
> own?
>

Hi José,

Thanks for the reviews.

David had the same question. I agree that it should be event-driven rather than periodic (except for retries, etc.)

>
> 2. Did you consider including the active controller's epoch in the
> ControllerRegistrationRequest?
>
> This would allow the active controller to reject registration from
> controllers that are not part of the active quorum and don't know the
> latest controller epoch. This should mitigate some of the concerns you
> raised in bullet point 1.
>

Good idea. I will add the active controller epoch to the registration request.

>
> 3. Which endpoint will the inactive controllers use to send the
> ControllerRegistrationRequest?
>
> Will it use the first endpoint described in the cluster metadata
> controller registration record? Or would it use the endpoint described
> in the server configuration at controller.quorum.voters?
>

They will use the endpoint in controller.quorum.voters. In general, the endpoints from the registration are only used for responding to DESCRIBE_CLUSTER. Since, after all, we may not even have the registration endpoints when we start up.

>
> 4. Re: Raft integration in the rejected alternatives
>
> Yes, The KRaft layer needs to solve a similar problem like endpoint
> discovery to support dynamic controller membership change. As you
> point out the requirements are different and the set of information
> that needs to be tracked is different. I think it is okay to use a
> different solution for each of these problems.

Yeah that was my feeling too. Thanks for taking a look.

regards,
Colin

>
> Thanks!
> -- 
> -José

Re: KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum

Posted by José Armando García Sancio <js...@confluent.io.INVALID>.
Thanks for the KIP Colin. Apologies if some of these points have
already been made. I have not followed the discussion closely:

1. Re: Periodically, each controller will check that the controller
registration for its ID is as expected

Does this need to be periodic? Can't the controller schedule this RPC,
retry etc, when it finds that the incarnation ID doesn't match its
own?

2. Did you consider including the active controller's epoch in the
ControllerRegistrationRequest?

This would allow the active controller to reject registration from
controllers that are not part of the active quorum and don't know the
latest controller epoch. This should mitigate some of the concerns you
raised in bullet point 1.

3. Which endpoint will the inactive controllers use to send the
ControllerRegistrationRequest?

Will it use the first endpoint described in the cluster metadata
controller registration record? Or would it use the endpoint described
in the server configuration at controller.quorum.voters?

4. Re: Raft integration in the rejected alternatives

Yes, The KRaft layer needs to solve a similar problem like endpoint
discovery to support dynamic controller membership change. As you
point out the requirements are different and the set of information
that needs to be tracked is different. I think it is okay to use a
different solution for each of these problems.

Thanks!
-- 
-José

Re: KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum

Posted by David Arthur <da...@confluent.io.INVALID>.
Hey Colin, thanks for the KIP! Some questions

1) "This registration will include information about the endpoints which
they possess"  Will this include all endpoints, or only those configured in
"advertised.listeners"

2) "Periodically, each controller will check that the controller
registration for its ID is as expected."  Does this need to be a periodic
check? Since the controller registration state will be in the log, can't
the follower just react to unexpected incarnation IDs (after it's caught
up)?

3) ControllerRegistrationRequest has a typo in the listeners section (it
mentions "broker")

4) Since we can't rely on the ApiVersions data, should we remove the field
we added to ApiVersionsResponse in KIP-866?

5)I filed https://issues.apache.org/jira/browse/KAFKA-15230 for the issues
mentioned under "Controller Changes" in case you want to link it

6) I don't see it explicitly mentioned, but I think it's the case that the
active controller must accept and persist any controller registration it
receives. This is unlike the behavior of broker registrations where we can
reject brokers we don't want. For controllers, I don't think we have that
option unless we go for some tighter Raft integration. Since the followers
must be participating in Raft to learn about the leader (and therefore,
will have replayed the full log), we can't really say "no" at that point.


Cheers,
David


On Thu, Jul 20, 2023 at 7:23 PM Colin McCabe <cm...@apache.org> wrote:

> On Tue, Jul 18, 2023, at 09:30, Mickael Maison wrote:
> > H Colin,
> >
> > Thanks for the KIP.
> >
> > Just a few points:
> > 1. As Tom mentioned it would be good to clarify the APIs we expect
> > available on controllers. I assume we want to add DESCRIBE_CONFIGS as
> > part of this KIP.
>
> Hi Mickael,
>
> Yes, this is a good point. I added a table describing the APIs that will
> now be added.
>
> > 2. Currently we have no way of retrieving the list of configs that
> > apply to controllers. It would be good to have an object, so we can
> > add that to the docs but also use that in kafka-configs.
>
> I think this is out of scope.
>
> > 3. Should we have a new entity-type in kafka-configs for setting
> > controller configs?
>
> The BROKER entity type already applies to controllers. It probably needs a
> new name (NODE would be better) but that's out of scope for this KIP, I
> think.
>
> best,
> Colin
>
>
> >
> > Thanks,
> > Mickael
> >
> > On Tue, Jul 4, 2023 at 2:20 PM Luke Chen <sh...@gmail.com> wrote:
> >>
> >> Hi Colin,
> >>
> >> Thanks for the answers to my previous questions.
> >>
> >> > Yes, the common thread here is that all of these shell commands
> perform
> >> operations can be done without the broker. So it's reasonable to allow
> them
> >> to be done without going through the broker. I don't know if we need a
> >> separate note for each since the rationale is really the same for all
> (is
> >> it reasonable? if so allow it.)
> >>
> >> Yes, it makes sense. Could we make a note about the main rationale for
> >> selecting these command-line tools in the KIP to make it clear?
> >> Ex: The following command-line tools will get a new
> --bootstrap-controllers
> >> argument (because these shell commands perform operations can be done
> >> without the broker):
> >>
> >> > kafka-reassign-partitions.sh cannot be used to move the
> >> __cluster_metadata topic. However, it can be used to move partitions
> that
> >> reside on the brokers, even when using --bootstrap-controllers to talk
> >> directly to the quorum.
> >>
> >> Fair enough.
> >>
> >>
> >> 4. Does all the command-line tools with `--bootstrap-controllers`
> support
> >> all the options in the tool?
> >> For example, kafka-configs.sh, In addition to the `--alter` option you
> >> mentioned in the example, do we also support `--describe` or `--delete`
> >> option?
> >> If so, do we also support setting "quota" for users/clients/topics...
> via
> >> `--bootstrap-controllers`? (not intuitive, but maybe we just directly
> >> commit the change into the metadata from controller?)
> >>
> >> 5. Do we have any plan for this feature to be completed? v3.6.0?
> >>
> >>
> >> Thank you.
> >> Luke
> >>
> >>
> >> On Fri, Apr 28, 2023 at 1:42 AM Colin McCabe <cm...@apache.org>
> wrote:
> >>
> >> > On Wed, Apr 26, 2023, at 22:08, Luke Chen wrote:
> >> > > Hi Colin,
> >> > >
> >> > > Some comments:
> >> > > 1. I agree we should set "top-level" errors for metadata response
> >> > >
> >> > > 2. In the "brokers" field of metadata response from controller,
> it'll
> >> > > respond with "Controller endpoint information as given in
> >> > > controller.quorum.voters", instead of the "alive"
> controllers(voters).
> >> > That
> >> > > will break the existing admin client because in admin client, we'll
> rely
> >> > on
> >> > > the metadata response to build the "current alive brokers" list, and
> >> > choose
> >> > > one from them to connect (either least load or other criteria). That
> >> > means,
> >> > > if now, we return the value in `controller.quorum.voters`, but one
> of
> >> > them
> >> > > is down. We might choose it to connect and get connection errors.
> Should
> >> > we
> >> > > return the "alive" controllers(voters) to client?
> >> >
> >> > Hi Luke,
> >> >
> >> > Good question. When talking to the controllers directly, the
> AdminClient
> >> > needs to always send its RPCs to the active controller. There is one
> >> > exception: configuring ephemeral log4j settings with
> >> > incrementalAlterConfigs must be done by sending them to the specified
> >> > controller node.
> >> >
> >> > I will add this to a section called "AdminClient Implementation
> Notes" so
> >> > that it's captured in the KIP.
> >> >
> >> > >
> >> > > 3. In the KIP, we list the command-line tools will get a new
> >> > > --bootstrap-controllers argument, but without explaining why these
> tools
> >> > > need to talk to controller directly. Could we add some explanation
> about
> >> > > them? I tried but cannot know why some tools are listed here:
> >> > >     - kafka-acls.sh -> Allow clients to update ACLs via controller
> before
> >> > > brokers up
> >> > >
> >> > >     - kafka-cluster.sh -> Reasonable to get/update cluster info via
> >> > > controller
> >> > >
> >> > >     - kafka-configs.sh -> Allow clients to dynamically update
> >> > > configs/describe configs from controller. But in this script,
> client can
> >> > > still set quota for users/clients/topics... is client also able to
> update
> >> > > via controllers? Or we only allow partial actions in the script to
> talk
> >> > to
> >> > > controllers?
> >> > >
> >> > >     - kafka-delegation-tokens.sh -> Reasonable to update
> >> > delegation-tokens
> >> > > via controllers
> >> > >
> >> > >     - kafka-features.sh -> Reasonable
> >> > >     - kafka-metadata-quorum.sh -> Reasonable
> >> > >     - kafka-metadata-shell.sh -> Reasonable
> >> > >
> >> > >     - kafka-reassign-partitions.sh -> Why should we allow clients
> to move
> >> > > metadata log partitions in controller nodes? What's the use-case?
> >> > >
> >> >
> >> > Yes, the common thread here is that all of these shell commands
> perform
> >> > operations can be done without the broker. So it's reasonable to
> allow them
> >> > to be done without going through the broker. I don't know if we need a
> >> > separate note for each since the rationale is really the same for all
> (is
> >> > it reasonable? if so allow it.)
> >> >
> >> > kafka-reassign-partitions.sh cannot be used to move the
> __cluster_metadata
> >> > topic. However, it can be used to move partitions that reside on the
> >> > brokers, even when using --bootstrap-controllers to talk directly to
> the
> >> > quorum.
> >> >
> >> > Colin
> >> >
> >> > >
> >> > > Thank you.
> >> > > Luke
> >> > >
> >> > > On Thu, Apr 27, 2023 at 8:04 AM Colin McCabe <cm...@apache.org>
> wrote:
> >> > >
> >> > >> On Tue, Apr 25, 2023, at 04:59, Divij Vaidya wrote:
> >> > >> > Thank you for the KIP Colin.
> >> > >> >
> >> > >> > In general, I like the idea of having the ability to interact
> directly
> >> > >> with
> >> > >> > the controllers. I agree with your observation that it helps in
> >> > >> situations
> >> > >> > where you would want to get data directly from the controller
> instead
> >> > of
> >> > >> > going via a broker. I have some general comments but the main
> concern
> >> > I
> >> > >> > have is with the piggy-backing of error code with response of
> >> > >> > __cluster_metadata topic.
> >> > >> >
> >> > >> > 1. With this change, how are we guarding against the possibility
> of
> >> > >> > misbehaving client traffic from disrupting the controller (that
> you
> >> > >> > mentioned as a motivation of earlier behaviour)? One solution
> could
> >> > be to
> >> > >> > have default values set for request throttling on the controller.
> >> > >>
> >> > >> Hi Divij,
> >> > >>
> >> > >> Thanks for the comments.
> >> > >>
> >> > >> Our guards against client misbehavior remain the same:
> >> > >> 1. our recommendation to put the clients on a separate network
> >> > >> 2. the fact that producers and consumers can't interact directly
> with
> >> > the
> >> > >> controller
> >> > >> 3. the authorizer.
> >> > >>
> >> > >> Re: #3, I will spell out in the KIP that clients must have
> DESCRIBE on
> >> > the
> >> > >> CLUSTER resource to send a METADATA request to the controller.
> >> > >>
> >> > >> > 2. This KIP also increases the network attack surface area.
> Prior to
> >> > this
> >> > >> > KIP, it was possible to have firewall rules setup for
> controllers such
> >> > >> that
> >> > >> > only the brokers can talk to it. But now, the controller would
> require
> >> > >> > access from other endpoints other than brokers as well. Can we
> add a
> >> > >> > suggestion to the upgrade documentation and inform users
> >> > >>
> >> > >> There is no requirement for access from other endpoints. It is
> still
> >> > >> possible to set up firewall rules such that only the brokers can
> talk to
> >> > >> the controller. In fact I would even say this is desirable. Since
> this
> >> > >> faculty is intended for infrequent manual administrative
> operations,
> >> > >> needing to log into the broker to use it seems perfectly fine.
> >> > >>
> >> > >> > 3. In section KRaft Controller MetadataResponse, row 3, "There
> is no
> >> > >> > top-level error code in MetadataResponse, so we use the
> >> > >> __cluster_metadata
> >> > >> > topic to send back our error.". This will definitely confuse the
> >> > users.
> >> > >> Can
> >> > >> > we introduce a top level error field instead?
> >> > >>
> >> > >> Let me check how we're handling this in other places. I recall some
> >> > other
> >> > >> cases where we used the dummy topic approach, but I can't find
> them just
> >> > >> now.
> >> > >>
> >> > >> > 4. As part of the KIP, could we please add some documentation for
> >> > users
> >> > >> > with the suggestion of when to get information directly from the
> >> > >> controller
> >> > >> > and when not to (and associated tradeoffs)?
> >> > >>
> >> > >> I think most users will not face this tradeoff because they simply
> won't
> >> > >> have network access to the controller servers.
> >> > >>
> >> > >> For those who do want more information, we'll have command-line
> >> > >> documentation for --boostrap-controllers and the
> bootstrap.controllers
> >> > >> configuration key.
> >> > >>
> >> > >> > 5. Why do we need the `FromKRaftController` field in the
> response?
> >> > What
> >> > >> do
> >> > >> > we expect the users to do with this information?
> >> > >>
> >> > >> The field is present so that we can throw an exception in the
> client if
> >> > we
> >> > >> have received a response that is not from the controller.
> >> > >>
> >> > >> > 6. Can we drop `KRaft` from the fields `FromKRaftController` and
> >> > >> > `DirectToKRaftControllerQuorum`? My suggestion will be to rename
> it as
> >> > >> > `DirectToController`.
> >> > >>
> >> > >> I like the idea, but wouldn't that create confusion in the ZK
> cluster
> >> > case?
> >> > >>
> >> > >> > 7.  "kafka-metadata-shell.sh  is at an "evolving" level of
> interface
> >> > >> > stability" -> I thought that with KRaft being production ready,
> the
> >> > >> > evolving mode for kraft-related tools has also moved to
> production.
> >> > Do we
> >> > >> > have a timeline when this would move to production?
> >> > >>
> >> > >> That's a good question, but I think we should tackle it separately
> from
> >> > >> this KIP. The metadata shell is pretty different from other parts
> of
> >> > kafka
> >> > >> since it interacts so closely with internals.
> >> > >>
> >> > >> best,
> >> > >> Colin
> >> > >>
> >> > >>
> >> > >> >
> >> > >> > --
> >> > >> > Divij Vaidya
> >> > >> >
> >> > >> >
> >> > >> >
> >> > >> > On Tue, Apr 25, 2023 at 1:38 AM Colin McCabe <cmccabe@apache.org
> >
> >> > wrote:
> >> > >> >
> >> > >> >> On Fri, Apr 21, 2023, at 14:17, Jason Gustafson wrote:
> >> > >> >> > Hey Colin,
> >> > >> >> >
> >> > >> >> > The KIP makes sense overall. Nice to clarify the contract
> between
> >> > >> clients
> >> > >> >> > and the controllers. The use of
> `DirectToKRaftControllerQuorum`
> >> > will
> >> > >> help
> >> > >> >> > prevent misconfiguration. In fact, I wonder if we can return a
> >> > fatal
> >> > >> >> error
> >> > >> >> > instead of NOT_CONTROLLER so that the client would immediately
> >> > fail.
> >> > >> For
> >> > >> >> > example, could we use INVALID_REQUEST or something like that?
> >> > Either
> >> > >> that
> >> > >> >> > or we need to make sure clients treat NOT_CONTROLLER as a
> fatal
> >> > error.
> >> > >> >> > Without that, it would probably get picked up with default
> retry
> >> > logic
> >> > >> >> and
> >> > >> >> > the user might not see the problem.
> >> > >> >>
> >> > >> >> Hi Jason,
> >> > >> >>
> >> > >> >> Yes, this is a good point. It should return INVALID_REQUEST
> since
> >> > that
> >> > >> is
> >> > >> >> not retryable. I'll change it.
> >> > >> >>
> >> > >> >> >
> >> > >> >> > I also wonder if we can relax the requirements on the Metadata
> >> > >> request so
> >> > >> >> > that we can use it to list topics and partition state (e.g.
> >> > URPs).  It
> >> > >> >> > would be useful to query the controllers as the metadata
> source of
> >> > >> truth
> >> > >> >> > when we suspect that the broker metadata may have diverged.
> >> > >> >> >
> >> > >> >>
> >> > >> >> Fair enough. I will document that we can return topics.
> >> > >> >>
> >> > >> >> I also added a "future work" section about maybe using the
> >> > controllers
> >> > >> as
> >> > >> >> bootstrap servers for the broker cluster. To be clear, that is
> NOT in
> >> > >> scope
> >> > >> >> here, but it's interesting to think about potentially doing in
> the
> >> > >> future.
> >> > >> >> The major problem is what to do when the broker endpoints we're
> >> > >> returning
> >> > >> >> have different security settings from the controller endpoint
> the
> >> > client
> >> > >> >> initially talked to.
> >> > >> >>
> >> > >> >> best,
> >> > >> >> Colin
> >> > >> >>
> >> > >> >>
> >> > >> >> >
> >> > >> >> > Thanks,
> >> > >> >> > Jason
> >> > >> >> >
> >> > >> >> > On Thu, Apr 20, 2023 at 5:53 PM Colin McCabe <
> cmccabe@apache.org>
> >> > >> wrote:
> >> > >> >> >
> >> > >> >> >> On Wed, Apr 19, 2023, at 20:56, Philip Nee wrote:
> >> > >> >> >> > Hey Colin,
> >> > >> >> >> >
> >> > >> >> >> > I still need to finish reading and understanding the KIP,
> but I
> >> > >> have a
> >> > >> >> >> > couple of comments despite being ignorant of most of the
> KRaft
> >> > >> stuff.
> >> > >> >> >> > (Sorry!)
> >> > >> >> >> >
> >> > >> >> >> > Firstly, does it make sense to create an extension of the
> >> > current
> >> > >> >> >> > AdminClient only to handle these specific KRaft use cases?
> It
> >> > seems
> >> > >> >> >> > cumbersome to have two sets of bootstrap configurations to
> make
> >> > the
> >> > >> >> >> > AdminClient generic enough to handle these specific cases,
> >> > instead,
> >> > >> >> maybe
> >> > >> >> >> > it is more obvious (to me) to just extend the AdminClient.
> What
> >> > I'm
> >> > >> >> >> > thinking is KraftAdminClient which continuously uses
> >> > >> >> *bootstrap.servers*,
> >> > >> >> >> > but make this class only serves the Kraft controllers APIs.
> >> > >> >> >>
> >> > >> >> >> Hi Philip,
> >> > >> >> >>
> >> > >> >> >> Thanks for taking a look.
> >> > >> >> >>
> >> > >> >> >> We would not want to create a new Admin client class in
> order to
> >> > >> >> >> communicate directly with the controllers. The RPCs accepted
> by
> >> > the
> >> > >> >> >> controllers have the same format as the those accepted by the
> >> > >> brokers.
> >> > >> >> >> There is no difference in what is sent over the wire or the
> data
> >> > >> >> structures
> >> > >> >> >> that are used in the client.
> >> > >> >> >>
> >> > >> >> >> I know you mentioned you haven't had time to read all the
> KRaft
> >> > stuff
> >> > >> >> (and
> >> > >> >> >> there is a lot, I understand). But this is one area that
> would
> >> > >> probably
> >> > >> >> be
> >> > >> >> >> clarified if you were able to read at least KIP-500 and
> KIP-590.
> >> > RPCs
> >> > >> >> sent
> >> > >> >> >> to the broker are forwarded to the controller (mostly)
> without
> >> > >> >> modification.
> >> > >> >> >>
> >> > >> >> >> >
> >> > >> >> >> > Secondly, if we want to continue with the design, I'm not
> yet
> >> > sure
> >> > >> >> why we
> >> > >> >> >> > can't continue using the *bootstrap.servers*? I assume
> when the
> >> > >> client
> >> > >> >> >> gets
> >> > >> >> >> > the metadata, it should know who it is talking to. I'm just
> >> > >> >> reconsidering
> >> > >> >> >> > your alternative again.
> >> > >> >> >> >
> >> > >> >> >> > A bad idea: Why don't we continue using
> *bootstrap.servers* but
> >> > >> have a
> >> > >> >> >> > separated config like *kraft.controller* = true/false. I
> feel
> >> > like
> >> > >> >> most
> >> > >> >> >> > users might not know what is a controller and causes some
> >> > mistakes
> >> > >> >> down
> >> > >> >> >> the
> >> > >> >> >> > road.
> >> > >> >> >> >
> >> > >> >> >>
> >> > >> >> >> Well, you called it a bad idea, and I can't help but agree!
> :)
> >> > >> >> >>
> >> > >> >> >> I think "the user might not know what a controller is" is a
> good
> >> > sign
> >> > >> >> that
> >> > >> >> >> they shouldn't be interacting with the controller. Like many
> >> > >> AdminClient
> >> > >> >> >> APIs, this one is intended for use by administrators only.
> Most
> >> > users
> >> > >> >> >> indeed should not need to know what a controller is or
> interact
> >> > with
> >> > >> it
> >> > >> >> >> directly. If they do interact it should be very clear that
> they
> >> > are
> >> > >> >> doing
> >> > >> >> >> so. The --controller-bootstrap flag makes it very clear, as
> does
> >> > the
> >> > >> >> >> separate configuration.
> >> > >> >> >>
> >> > >> >> >> Let me give an example of the kind of problems that arise if
> you
> >> > >> want to
> >> > >> >> >> reuse bootstrap.servers for this purpose.
> >> > >> >> >>
> >> > >> >> >> If the user grasb a 3.4 Kafka AdminClient and set
> >> > bootstrap.servers
> >> > >> to a
> >> > >> >> >> set of controller servers, and set direct.to.controller to
> true,
> >> > the
> >> > >> >> >> unknown (in 3.4) configuration will be ignored, and a normal
> >> > metadata
> >> > >> >> >> request will be sent without the direct to controller flag.
> In
> >> > that
> >> > >> >> case it
> >> > >> >> >> will give back an error. Confusing, right?
> >> > >> >> >>
> >> > >> >> >> Using controller.servers clarifies this situation because
> the 3.4
> >> > >> client
> >> > >> >> >> will not support that config, and will complain about the
> lack of
> >> > >> >> >> bootstrap.servers.
> >> > >> >> >>
> >> > >> >> >> Actually, the situation could get even more confusing than
> what I
> >> > >> >> >> described since some older preproduction versions of the
> KRaft
> >> > >> >> controller
> >> > >> >> >> did implement the METADATA RPC. So if you send them a
> METADATA
> >> > >> request
> >> > >> >> >> without any special information, it's not clear what you'll
> get.
> >> > >> >> Indeed,
> >> > >> >> >> it would be dependent on the client version and the
> controller
> >> > >> version.
> >> > >> >> >>
> >> > >> >> >> The bottom line is that reusing the bootstrap.servers
> >> > configuration
> >> > >> here
> >> > >> >> >> is not a good idea.
> >> > >> >> >>
> >> > >> >> >> best,
> >> > >> >> >> Colin
> >> > >> >> >>
> >> > >> >> >> > Thanks,
> >> > >> >> >> > P
> >> > >> >> >> >
> >> > >> >> >> > On Wed, Apr 19, 2023 at 2:18 PM Colin McCabe <
> >> > cmccabe@apache.org>
> >> > >> >> wrote:
> >> > >> >> >> >
> >> > >> >> >> >> Hi all,
> >> > >> >> >> >>
> >> > >> >> >> >> I wrote a short KIP about allowing AdminClient to talk
> directly
> >> > >> with
> >> > >> >> the
> >> > >> >> >> >> KRaft controller quorum. Check it out here:
> >> > >> >> >> >>
> >> > >> >> >> >> https://cwiki.apache.org/confluence/x/Owo0Dw
> >> > >> >> >> >>
> >> > >> >> >> >> best,
> >> > >> >> >> >> Colin
> >> > >> >> >> >>
> >> > >> >> >>
> >> > >> >>
> >> > >>
> >> >
>


-- 
-David

Re: KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum

Posted by Colin McCabe <cm...@apache.org>.
On Tue, Jul 18, 2023, at 09:30, Mickael Maison wrote:
> H Colin,
>
> Thanks for the KIP.
>
> Just a few points:
> 1. As Tom mentioned it would be good to clarify the APIs we expect
> available on controllers. I assume we want to add DESCRIBE_CONFIGS as
> part of this KIP.

Hi Mickael,

Yes, this is a good point. I added a table describing the APIs that will now be added.

> 2. Currently we have no way of retrieving the list of configs that
> apply to controllers. It would be good to have an object, so we can
> add that to the docs but also use that in kafka-configs.

I think this is out of scope.

> 3. Should we have a new entity-type in kafka-configs for setting
> controller configs?

The BROKER entity type already applies to controllers. It probably needs a new name (NODE would be better) but that's out of scope for this KIP, I think.

best,
Colin


>
> Thanks,
> Mickael
>
> On Tue, Jul 4, 2023 at 2:20 PM Luke Chen <sh...@gmail.com> wrote:
>>
>> Hi Colin,
>>
>> Thanks for the answers to my previous questions.
>>
>> > Yes, the common thread here is that all of these shell commands perform
>> operations can be done without the broker. So it's reasonable to allow them
>> to be done without going through the broker. I don't know if we need a
>> separate note for each since the rationale is really the same for all (is
>> it reasonable? if so allow it.)
>>
>> Yes, it makes sense. Could we make a note about the main rationale for
>> selecting these command-line tools in the KIP to make it clear?
>> Ex: The following command-line tools will get a new --bootstrap-controllers
>> argument (because these shell commands perform operations can be done
>> without the broker):
>>
>> > kafka-reassign-partitions.sh cannot be used to move the
>> __cluster_metadata topic. However, it can be used to move partitions that
>> reside on the brokers, even when using --bootstrap-controllers to talk
>> directly to the quorum.
>>
>> Fair enough.
>>
>>
>> 4. Does all the command-line tools with `--bootstrap-controllers` support
>> all the options in the tool?
>> For example, kafka-configs.sh, In addition to the `--alter` option you
>> mentioned in the example, do we also support `--describe` or `--delete`
>> option?
>> If so, do we also support setting "quota" for users/clients/topics... via
>> `--bootstrap-controllers`? (not intuitive, but maybe we just directly
>> commit the change into the metadata from controller?)
>>
>> 5. Do we have any plan for this feature to be completed? v3.6.0?
>>
>>
>> Thank you.
>> Luke
>>
>>
>> On Fri, Apr 28, 2023 at 1:42 AM Colin McCabe <cm...@apache.org> wrote:
>>
>> > On Wed, Apr 26, 2023, at 22:08, Luke Chen wrote:
>> > > Hi Colin,
>> > >
>> > > Some comments:
>> > > 1. I agree we should set "top-level" errors for metadata response
>> > >
>> > > 2. In the "brokers" field of metadata response from controller, it'll
>> > > respond with "Controller endpoint information as given in
>> > > controller.quorum.voters", instead of the "alive" controllers(voters).
>> > That
>> > > will break the existing admin client because in admin client, we'll rely
>> > on
>> > > the metadata response to build the "current alive brokers" list, and
>> > choose
>> > > one from them to connect (either least load or other criteria). That
>> > means,
>> > > if now, we return the value in `controller.quorum.voters`, but one of
>> > them
>> > > is down. We might choose it to connect and get connection errors. Should
>> > we
>> > > return the "alive" controllers(voters) to client?
>> >
>> > Hi Luke,
>> >
>> > Good question. When talking to the controllers directly, the AdminClient
>> > needs to always send its RPCs to the active controller. There is one
>> > exception: configuring ephemeral log4j settings with
>> > incrementalAlterConfigs must be done by sending them to the specified
>> > controller node.
>> >
>> > I will add this to a section called "AdminClient Implementation Notes" so
>> > that it's captured in the KIP.
>> >
>> > >
>> > > 3. In the KIP, we list the command-line tools will get a new
>> > > --bootstrap-controllers argument, but without explaining why these tools
>> > > need to talk to controller directly. Could we add some explanation about
>> > > them? I tried but cannot know why some tools are listed here:
>> > >     - kafka-acls.sh -> Allow clients to update ACLs via controller before
>> > > brokers up
>> > >
>> > >     - kafka-cluster.sh -> Reasonable to get/update cluster info via
>> > > controller
>> > >
>> > >     - kafka-configs.sh -> Allow clients to dynamically update
>> > > configs/describe configs from controller. But in this script, client can
>> > > still set quota for users/clients/topics... is client also able to update
>> > > via controllers? Or we only allow partial actions in the script to talk
>> > to
>> > > controllers?
>> > >
>> > >     - kafka-delegation-tokens.sh -> Reasonable to update
>> > delegation-tokens
>> > > via controllers
>> > >
>> > >     - kafka-features.sh -> Reasonable
>> > >     - kafka-metadata-quorum.sh -> Reasonable
>> > >     - kafka-metadata-shell.sh -> Reasonable
>> > >
>> > >     - kafka-reassign-partitions.sh -> Why should we allow clients to move
>> > > metadata log partitions in controller nodes? What's the use-case?
>> > >
>> >
>> > Yes, the common thread here is that all of these shell commands perform
>> > operations can be done without the broker. So it's reasonable to allow them
>> > to be done without going through the broker. I don't know if we need a
>> > separate note for each since the rationale is really the same for all (is
>> > it reasonable? if so allow it.)
>> >
>> > kafka-reassign-partitions.sh cannot be used to move the __cluster_metadata
>> > topic. However, it can be used to move partitions that reside on the
>> > brokers, even when using --bootstrap-controllers to talk directly to the
>> > quorum.
>> >
>> > Colin
>> >
>> > >
>> > > Thank you.
>> > > Luke
>> > >
>> > > On Thu, Apr 27, 2023 at 8:04 AM Colin McCabe <cm...@apache.org> wrote:
>> > >
>> > >> On Tue, Apr 25, 2023, at 04:59, Divij Vaidya wrote:
>> > >> > Thank you for the KIP Colin.
>> > >> >
>> > >> > In general, I like the idea of having the ability to interact directly
>> > >> with
>> > >> > the controllers. I agree with your observation that it helps in
>> > >> situations
>> > >> > where you would want to get data directly from the controller instead
>> > of
>> > >> > going via a broker. I have some general comments but the main concern
>> > I
>> > >> > have is with the piggy-backing of error code with response of
>> > >> > __cluster_metadata topic.
>> > >> >
>> > >> > 1. With this change, how are we guarding against the possibility of
>> > >> > misbehaving client traffic from disrupting the controller (that you
>> > >> > mentioned as a motivation of earlier behaviour)? One solution could
>> > be to
>> > >> > have default values set for request throttling on the controller.
>> > >>
>> > >> Hi Divij,
>> > >>
>> > >> Thanks for the comments.
>> > >>
>> > >> Our guards against client misbehavior remain the same:
>> > >> 1. our recommendation to put the clients on a separate network
>> > >> 2. the fact that producers and consumers can't interact directly with
>> > the
>> > >> controller
>> > >> 3. the authorizer.
>> > >>
>> > >> Re: #3, I will spell out in the KIP that clients must have DESCRIBE on
>> > the
>> > >> CLUSTER resource to send a METADATA request to the controller.
>> > >>
>> > >> > 2. This KIP also increases the network attack surface area. Prior to
>> > this
>> > >> > KIP, it was possible to have firewall rules setup for controllers such
>> > >> that
>> > >> > only the brokers can talk to it. But now, the controller would require
>> > >> > access from other endpoints other than brokers as well. Can we add a
>> > >> > suggestion to the upgrade documentation and inform users
>> > >>
>> > >> There is no requirement for access from other endpoints. It is still
>> > >> possible to set up firewall rules such that only the brokers can talk to
>> > >> the controller. In fact I would even say this is desirable. Since this
>> > >> faculty is intended for infrequent manual administrative operations,
>> > >> needing to log into the broker to use it seems perfectly fine.
>> > >>
>> > >> > 3. In section KRaft Controller MetadataResponse, row 3, "There is no
>> > >> > top-level error code in MetadataResponse, so we use the
>> > >> __cluster_metadata
>> > >> > topic to send back our error.". This will definitely confuse the
>> > users.
>> > >> Can
>> > >> > we introduce a top level error field instead?
>> > >>
>> > >> Let me check how we're handling this in other places. I recall some
>> > other
>> > >> cases where we used the dummy topic approach, but I can't find them just
>> > >> now.
>> > >>
>> > >> > 4. As part of the KIP, could we please add some documentation for
>> > users
>> > >> > with the suggestion of when to get information directly from the
>> > >> controller
>> > >> > and when not to (and associated tradeoffs)?
>> > >>
>> > >> I think most users will not face this tradeoff because they simply won't
>> > >> have network access to the controller servers.
>> > >>
>> > >> For those who do want more information, we'll have command-line
>> > >> documentation for --boostrap-controllers and the bootstrap.controllers
>> > >> configuration key.
>> > >>
>> > >> > 5. Why do we need the `FromKRaftController` field in the response?
>> > What
>> > >> do
>> > >> > we expect the users to do with this information?
>> > >>
>> > >> The field is present so that we can throw an exception in the client if
>> > we
>> > >> have received a response that is not from the controller.
>> > >>
>> > >> > 6. Can we drop `KRaft` from the fields `FromKRaftController` and
>> > >> > `DirectToKRaftControllerQuorum`? My suggestion will be to rename it as
>> > >> > `DirectToController`.
>> > >>
>> > >> I like the idea, but wouldn't that create confusion in the ZK cluster
>> > case?
>> > >>
>> > >> > 7.  "kafka-metadata-shell.sh  is at an "evolving" level of interface
>> > >> > stability" -> I thought that with KRaft being production ready, the
>> > >> > evolving mode for kraft-related tools has also moved to production.
>> > Do we
>> > >> > have a timeline when this would move to production?
>> > >>
>> > >> That's a good question, but I think we should tackle it separately from
>> > >> this KIP. The metadata shell is pretty different from other parts of
>> > kafka
>> > >> since it interacts so closely with internals.
>> > >>
>> > >> best,
>> > >> Colin
>> > >>
>> > >>
>> > >> >
>> > >> > --
>> > >> > Divij Vaidya
>> > >> >
>> > >> >
>> > >> >
>> > >> > On Tue, Apr 25, 2023 at 1:38 AM Colin McCabe <cm...@apache.org>
>> > wrote:
>> > >> >
>> > >> >> On Fri, Apr 21, 2023, at 14:17, Jason Gustafson wrote:
>> > >> >> > Hey Colin,
>> > >> >> >
>> > >> >> > The KIP makes sense overall. Nice to clarify the contract between
>> > >> clients
>> > >> >> > and the controllers. The use of `DirectToKRaftControllerQuorum`
>> > will
>> > >> help
>> > >> >> > prevent misconfiguration. In fact, I wonder if we can return a
>> > fatal
>> > >> >> error
>> > >> >> > instead of NOT_CONTROLLER so that the client would immediately
>> > fail.
>> > >> For
>> > >> >> > example, could we use INVALID_REQUEST or something like that?
>> > Either
>> > >> that
>> > >> >> > or we need to make sure clients treat NOT_CONTROLLER as a fatal
>> > error.
>> > >> >> > Without that, it would probably get picked up with default retry
>> > logic
>> > >> >> and
>> > >> >> > the user might not see the problem.
>> > >> >>
>> > >> >> Hi Jason,
>> > >> >>
>> > >> >> Yes, this is a good point. It should return INVALID_REQUEST since
>> > that
>> > >> is
>> > >> >> not retryable. I'll change it.
>> > >> >>
>> > >> >> >
>> > >> >> > I also wonder if we can relax the requirements on the Metadata
>> > >> request so
>> > >> >> > that we can use it to list topics and partition state (e.g.
>> > URPs).  It
>> > >> >> > would be useful to query the controllers as the metadata source of
>> > >> truth
>> > >> >> > when we suspect that the broker metadata may have diverged.
>> > >> >> >
>> > >> >>
>> > >> >> Fair enough. I will document that we can return topics.
>> > >> >>
>> > >> >> I also added a "future work" section about maybe using the
>> > controllers
>> > >> as
>> > >> >> bootstrap servers for the broker cluster. To be clear, that is NOT in
>> > >> scope
>> > >> >> here, but it's interesting to think about potentially doing in the
>> > >> future.
>> > >> >> The major problem is what to do when the broker endpoints we're
>> > >> returning
>> > >> >> have different security settings from the controller endpoint the
>> > client
>> > >> >> initially talked to.
>> > >> >>
>> > >> >> best,
>> > >> >> Colin
>> > >> >>
>> > >> >>
>> > >> >> >
>> > >> >> > Thanks,
>> > >> >> > Jason
>> > >> >> >
>> > >> >> > On Thu, Apr 20, 2023 at 5:53 PM Colin McCabe <cm...@apache.org>
>> > >> wrote:
>> > >> >> >
>> > >> >> >> On Wed, Apr 19, 2023, at 20:56, Philip Nee wrote:
>> > >> >> >> > Hey Colin,
>> > >> >> >> >
>> > >> >> >> > I still need to finish reading and understanding the KIP, but I
>> > >> have a
>> > >> >> >> > couple of comments despite being ignorant of most of the KRaft
>> > >> stuff.
>> > >> >> >> > (Sorry!)
>> > >> >> >> >
>> > >> >> >> > Firstly, does it make sense to create an extension of the
>> > current
>> > >> >> >> > AdminClient only to handle these specific KRaft use cases? It
>> > seems
>> > >> >> >> > cumbersome to have two sets of bootstrap configurations to make
>> > the
>> > >> >> >> > AdminClient generic enough to handle these specific cases,
>> > instead,
>> > >> >> maybe
>> > >> >> >> > it is more obvious (to me) to just extend the AdminClient. What
>> > I'm
>> > >> >> >> > thinking is KraftAdminClient which continuously uses
>> > >> >> *bootstrap.servers*,
>> > >> >> >> > but make this class only serves the Kraft controllers APIs.
>> > >> >> >>
>> > >> >> >> Hi Philip,
>> > >> >> >>
>> > >> >> >> Thanks for taking a look.
>> > >> >> >>
>> > >> >> >> We would not want to create a new Admin client class in order to
>> > >> >> >> communicate directly with the controllers. The RPCs accepted by
>> > the
>> > >> >> >> controllers have the same format as the those accepted by the
>> > >> brokers.
>> > >> >> >> There is no difference in what is sent over the wire or the data
>> > >> >> structures
>> > >> >> >> that are used in the client.
>> > >> >> >>
>> > >> >> >> I know you mentioned you haven't had time to read all the KRaft
>> > stuff
>> > >> >> (and
>> > >> >> >> there is a lot, I understand). But this is one area that would
>> > >> probably
>> > >> >> be
>> > >> >> >> clarified if you were able to read at least KIP-500 and KIP-590.
>> > RPCs
>> > >> >> sent
>> > >> >> >> to the broker are forwarded to the controller (mostly) without
>> > >> >> modification.
>> > >> >> >>
>> > >> >> >> >
>> > >> >> >> > Secondly, if we want to continue with the design, I'm not yet
>> > sure
>> > >> >> why we
>> > >> >> >> > can't continue using the *bootstrap.servers*? I assume when the
>> > >> client
>> > >> >> >> gets
>> > >> >> >> > the metadata, it should know who it is talking to. I'm just
>> > >> >> reconsidering
>> > >> >> >> > your alternative again.
>> > >> >> >> >
>> > >> >> >> > A bad idea: Why don't we continue using *bootstrap.servers* but
>> > >> have a
>> > >> >> >> > separated config like *kraft.controller* = true/false. I feel
>> > like
>> > >> >> most
>> > >> >> >> > users might not know what is a controller and causes some
>> > mistakes
>> > >> >> down
>> > >> >> >> the
>> > >> >> >> > road.
>> > >> >> >> >
>> > >> >> >>
>> > >> >> >> Well, you called it a bad idea, and I can't help but agree! :)
>> > >> >> >>
>> > >> >> >> I think "the user might not know what a controller is" is a good
>> > sign
>> > >> >> that
>> > >> >> >> they shouldn't be interacting with the controller. Like many
>> > >> AdminClient
>> > >> >> >> APIs, this one is intended for use by administrators only. Most
>> > users
>> > >> >> >> indeed should not need to know what a controller is or interact
>> > with
>> > >> it
>> > >> >> >> directly. If they do interact it should be very clear that they
>> > are
>> > >> >> doing
>> > >> >> >> so. The --controller-bootstrap flag makes it very clear, as does
>> > the
>> > >> >> >> separate configuration.
>> > >> >> >>
>> > >> >> >> Let me give an example of the kind of problems that arise if you
>> > >> want to
>> > >> >> >> reuse bootstrap.servers for this purpose.
>> > >> >> >>
>> > >> >> >> If the user grasb a 3.4 Kafka AdminClient and set
>> > bootstrap.servers
>> > >> to a
>> > >> >> >> set of controller servers, and set direct.to.controller to true,
>> > the
>> > >> >> >> unknown (in 3.4) configuration will be ignored, and a normal
>> > metadata
>> > >> >> >> request will be sent without the direct to controller flag. In
>> > that
>> > >> >> case it
>> > >> >> >> will give back an error. Confusing, right?
>> > >> >> >>
>> > >> >> >> Using controller.servers clarifies this situation because the 3.4
>> > >> client
>> > >> >> >> will not support that config, and will complain about the lack of
>> > >> >> >> bootstrap.servers.
>> > >> >> >>
>> > >> >> >> Actually, the situation could get even more confusing than what I
>> > >> >> >> described since some older preproduction versions of the KRaft
>> > >> >> controller
>> > >> >> >> did implement the METADATA RPC. So if you send them a METADATA
>> > >> request
>> > >> >> >> without any special information, it's not clear what you'll get.
>> > >> >> Indeed,
>> > >> >> >> it would be dependent on the client version and the controller
>> > >> version.
>> > >> >> >>
>> > >> >> >> The bottom line is that reusing the bootstrap.servers
>> > configuration
>> > >> here
>> > >> >> >> is not a good idea.
>> > >> >> >>
>> > >> >> >> best,
>> > >> >> >> Colin
>> > >> >> >>
>> > >> >> >> > Thanks,
>> > >> >> >> > P
>> > >> >> >> >
>> > >> >> >> > On Wed, Apr 19, 2023 at 2:18 PM Colin McCabe <
>> > cmccabe@apache.org>
>> > >> >> wrote:
>> > >> >> >> >
>> > >> >> >> >> Hi all,
>> > >> >> >> >>
>> > >> >> >> >> I wrote a short KIP about allowing AdminClient to talk directly
>> > >> with
>> > >> >> the
>> > >> >> >> >> KRaft controller quorum. Check it out here:
>> > >> >> >> >>
>> > >> >> >> >> https://cwiki.apache.org/confluence/x/Owo0Dw
>> > >> >> >> >>
>> > >> >> >> >> best,
>> > >> >> >> >> Colin
>> > >> >> >> >>
>> > >> >> >>
>> > >> >>
>> > >>
>> >

Re: KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum

Posted by Mickael Maison <mi...@gmail.com>.
H Colin,

Thanks for the KIP.

Just a few points:
1. As Tom mentioned it would be good to clarify the APIs we expect
available on controllers. I assume we want to add DESCRIBE_CONFIGS as
part of this KIP.
2. Currently we have no way of retrieving the list of configs that
apply to controllers. It would be good to have an object, so we can
add that to the docs but also use that in kafka-configs.
3. Should we have a new entity-type in kafka-configs for setting
controller configs?

Thanks,
Mickael

On Tue, Jul 4, 2023 at 2:20 PM Luke Chen <sh...@gmail.com> wrote:
>
> Hi Colin,
>
> Thanks for the answers to my previous questions.
>
> > Yes, the common thread here is that all of these shell commands perform
> operations can be done without the broker. So it's reasonable to allow them
> to be done without going through the broker. I don't know if we need a
> separate note for each since the rationale is really the same for all (is
> it reasonable? if so allow it.)
>
> Yes, it makes sense. Could we make a note about the main rationale for
> selecting these command-line tools in the KIP to make it clear?
> Ex: The following command-line tools will get a new --bootstrap-controllers
> argument (because these shell commands perform operations can be done
> without the broker):
>
> > kafka-reassign-partitions.sh cannot be used to move the
> __cluster_metadata topic. However, it can be used to move partitions that
> reside on the brokers, even when using --bootstrap-controllers to talk
> directly to the quorum.
>
> Fair enough.
>
>
> 4. Does all the command-line tools with `--bootstrap-controllers` support
> all the options in the tool?
> For example, kafka-configs.sh, In addition to the `--alter` option you
> mentioned in the example, do we also support `--describe` or `--delete`
> option?
> If so, do we also support setting "quota" for users/clients/topics... via
> `--bootstrap-controllers`? (not intuitive, but maybe we just directly
> commit the change into the metadata from controller?)
>
> 5. Do we have any plan for this feature to be completed? v3.6.0?
>
>
> Thank you.
> Luke
>
>
> On Fri, Apr 28, 2023 at 1:42 AM Colin McCabe <cm...@apache.org> wrote:
>
> > On Wed, Apr 26, 2023, at 22:08, Luke Chen wrote:
> > > Hi Colin,
> > >
> > > Some comments:
> > > 1. I agree we should set "top-level" errors for metadata response
> > >
> > > 2. In the "brokers" field of metadata response from controller, it'll
> > > respond with "Controller endpoint information as given in
> > > controller.quorum.voters", instead of the "alive" controllers(voters).
> > That
> > > will break the existing admin client because in admin client, we'll rely
> > on
> > > the metadata response to build the "current alive brokers" list, and
> > choose
> > > one from them to connect (either least load or other criteria). That
> > means,
> > > if now, we return the value in `controller.quorum.voters`, but one of
> > them
> > > is down. We might choose it to connect and get connection errors. Should
> > we
> > > return the "alive" controllers(voters) to client?
> >
> > Hi Luke,
> >
> > Good question. When talking to the controllers directly, the AdminClient
> > needs to always send its RPCs to the active controller. There is one
> > exception: configuring ephemeral log4j settings with
> > incrementalAlterConfigs must be done by sending them to the specified
> > controller node.
> >
> > I will add this to a section called "AdminClient Implementation Notes" so
> > that it's captured in the KIP.
> >
> > >
> > > 3. In the KIP, we list the command-line tools will get a new
> > > --bootstrap-controllers argument, but without explaining why these tools
> > > need to talk to controller directly. Could we add some explanation about
> > > them? I tried but cannot know why some tools are listed here:
> > >     - kafka-acls.sh -> Allow clients to update ACLs via controller before
> > > brokers up
> > >
> > >     - kafka-cluster.sh -> Reasonable to get/update cluster info via
> > > controller
> > >
> > >     - kafka-configs.sh -> Allow clients to dynamically update
> > > configs/describe configs from controller. But in this script, client can
> > > still set quota for users/clients/topics... is client also able to update
> > > via controllers? Or we only allow partial actions in the script to talk
> > to
> > > controllers?
> > >
> > >     - kafka-delegation-tokens.sh -> Reasonable to update
> > delegation-tokens
> > > via controllers
> > >
> > >     - kafka-features.sh -> Reasonable
> > >     - kafka-metadata-quorum.sh -> Reasonable
> > >     - kafka-metadata-shell.sh -> Reasonable
> > >
> > >     - kafka-reassign-partitions.sh -> Why should we allow clients to move
> > > metadata log partitions in controller nodes? What's the use-case?
> > >
> >
> > Yes, the common thread here is that all of these shell commands perform
> > operations can be done without the broker. So it's reasonable to allow them
> > to be done without going through the broker. I don't know if we need a
> > separate note for each since the rationale is really the same for all (is
> > it reasonable? if so allow it.)
> >
> > kafka-reassign-partitions.sh cannot be used to move the __cluster_metadata
> > topic. However, it can be used to move partitions that reside on the
> > brokers, even when using --bootstrap-controllers to talk directly to the
> > quorum.
> >
> > Colin
> >
> > >
> > > Thank you.
> > > Luke
> > >
> > > On Thu, Apr 27, 2023 at 8:04 AM Colin McCabe <cm...@apache.org> wrote:
> > >
> > >> On Tue, Apr 25, 2023, at 04:59, Divij Vaidya wrote:
> > >> > Thank you for the KIP Colin.
> > >> >
> > >> > In general, I like the idea of having the ability to interact directly
> > >> with
> > >> > the controllers. I agree with your observation that it helps in
> > >> situations
> > >> > where you would want to get data directly from the controller instead
> > of
> > >> > going via a broker. I have some general comments but the main concern
> > I
> > >> > have is with the piggy-backing of error code with response of
> > >> > __cluster_metadata topic.
> > >> >
> > >> > 1. With this change, how are we guarding against the possibility of
> > >> > misbehaving client traffic from disrupting the controller (that you
> > >> > mentioned as a motivation of earlier behaviour)? One solution could
> > be to
> > >> > have default values set for request throttling on the controller.
> > >>
> > >> Hi Divij,
> > >>
> > >> Thanks for the comments.
> > >>
> > >> Our guards against client misbehavior remain the same:
> > >> 1. our recommendation to put the clients on a separate network
> > >> 2. the fact that producers and consumers can't interact directly with
> > the
> > >> controller
> > >> 3. the authorizer.
> > >>
> > >> Re: #3, I will spell out in the KIP that clients must have DESCRIBE on
> > the
> > >> CLUSTER resource to send a METADATA request to the controller.
> > >>
> > >> > 2. This KIP also increases the network attack surface area. Prior to
> > this
> > >> > KIP, it was possible to have firewall rules setup for controllers such
> > >> that
> > >> > only the brokers can talk to it. But now, the controller would require
> > >> > access from other endpoints other than brokers as well. Can we add a
> > >> > suggestion to the upgrade documentation and inform users
> > >>
> > >> There is no requirement for access from other endpoints. It is still
> > >> possible to set up firewall rules such that only the brokers can talk to
> > >> the controller. In fact I would even say this is desirable. Since this
> > >> faculty is intended for infrequent manual administrative operations,
> > >> needing to log into the broker to use it seems perfectly fine.
> > >>
> > >> > 3. In section KRaft Controller MetadataResponse, row 3, "There is no
> > >> > top-level error code in MetadataResponse, so we use the
> > >> __cluster_metadata
> > >> > topic to send back our error.". This will definitely confuse the
> > users.
> > >> Can
> > >> > we introduce a top level error field instead?
> > >>
> > >> Let me check how we're handling this in other places. I recall some
> > other
> > >> cases where we used the dummy topic approach, but I can't find them just
> > >> now.
> > >>
> > >> > 4. As part of the KIP, could we please add some documentation for
> > users
> > >> > with the suggestion of when to get information directly from the
> > >> controller
> > >> > and when not to (and associated tradeoffs)?
> > >>
> > >> I think most users will not face this tradeoff because they simply won't
> > >> have network access to the controller servers.
> > >>
> > >> For those who do want more information, we'll have command-line
> > >> documentation for --boostrap-controllers and the bootstrap.controllers
> > >> configuration key.
> > >>
> > >> > 5. Why do we need the `FromKRaftController` field in the response?
> > What
> > >> do
> > >> > we expect the users to do with this information?
> > >>
> > >> The field is present so that we can throw an exception in the client if
> > we
> > >> have received a response that is not from the controller.
> > >>
> > >> > 6. Can we drop `KRaft` from the fields `FromKRaftController` and
> > >> > `DirectToKRaftControllerQuorum`? My suggestion will be to rename it as
> > >> > `DirectToController`.
> > >>
> > >> I like the idea, but wouldn't that create confusion in the ZK cluster
> > case?
> > >>
> > >> > 7.  "kafka-metadata-shell.sh  is at an "evolving" level of interface
> > >> > stability" -> I thought that with KRaft being production ready, the
> > >> > evolving mode for kraft-related tools has also moved to production.
> > Do we
> > >> > have a timeline when this would move to production?
> > >>
> > >> That's a good question, but I think we should tackle it separately from
> > >> this KIP. The metadata shell is pretty different from other parts of
> > kafka
> > >> since it interacts so closely with internals.
> > >>
> > >> best,
> > >> Colin
> > >>
> > >>
> > >> >
> > >> > --
> > >> > Divij Vaidya
> > >> >
> > >> >
> > >> >
> > >> > On Tue, Apr 25, 2023 at 1:38 AM Colin McCabe <cm...@apache.org>
> > wrote:
> > >> >
> > >> >> On Fri, Apr 21, 2023, at 14:17, Jason Gustafson wrote:
> > >> >> > Hey Colin,
> > >> >> >
> > >> >> > The KIP makes sense overall. Nice to clarify the contract between
> > >> clients
> > >> >> > and the controllers. The use of `DirectToKRaftControllerQuorum`
> > will
> > >> help
> > >> >> > prevent misconfiguration. In fact, I wonder if we can return a
> > fatal
> > >> >> error
> > >> >> > instead of NOT_CONTROLLER so that the client would immediately
> > fail.
> > >> For
> > >> >> > example, could we use INVALID_REQUEST or something like that?
> > Either
> > >> that
> > >> >> > or we need to make sure clients treat NOT_CONTROLLER as a fatal
> > error.
> > >> >> > Without that, it would probably get picked up with default retry
> > logic
> > >> >> and
> > >> >> > the user might not see the problem.
> > >> >>
> > >> >> Hi Jason,
> > >> >>
> > >> >> Yes, this is a good point. It should return INVALID_REQUEST since
> > that
> > >> is
> > >> >> not retryable. I'll change it.
> > >> >>
> > >> >> >
> > >> >> > I also wonder if we can relax the requirements on the Metadata
> > >> request so
> > >> >> > that we can use it to list topics and partition state (e.g.
> > URPs).  It
> > >> >> > would be useful to query the controllers as the metadata source of
> > >> truth
> > >> >> > when we suspect that the broker metadata may have diverged.
> > >> >> >
> > >> >>
> > >> >> Fair enough. I will document that we can return topics.
> > >> >>
> > >> >> I also added a "future work" section about maybe using the
> > controllers
> > >> as
> > >> >> bootstrap servers for the broker cluster. To be clear, that is NOT in
> > >> scope
> > >> >> here, but it's interesting to think about potentially doing in the
> > >> future.
> > >> >> The major problem is what to do when the broker endpoints we're
> > >> returning
> > >> >> have different security settings from the controller endpoint the
> > client
> > >> >> initially talked to.
> > >> >>
> > >> >> best,
> > >> >> Colin
> > >> >>
> > >> >>
> > >> >> >
> > >> >> > Thanks,
> > >> >> > Jason
> > >> >> >
> > >> >> > On Thu, Apr 20, 2023 at 5:53 PM Colin McCabe <cm...@apache.org>
> > >> wrote:
> > >> >> >
> > >> >> >> On Wed, Apr 19, 2023, at 20:56, Philip Nee wrote:
> > >> >> >> > Hey Colin,
> > >> >> >> >
> > >> >> >> > I still need to finish reading and understanding the KIP, but I
> > >> have a
> > >> >> >> > couple of comments despite being ignorant of most of the KRaft
> > >> stuff.
> > >> >> >> > (Sorry!)
> > >> >> >> >
> > >> >> >> > Firstly, does it make sense to create an extension of the
> > current
> > >> >> >> > AdminClient only to handle these specific KRaft use cases? It
> > seems
> > >> >> >> > cumbersome to have two sets of bootstrap configurations to make
> > the
> > >> >> >> > AdminClient generic enough to handle these specific cases,
> > instead,
> > >> >> maybe
> > >> >> >> > it is more obvious (to me) to just extend the AdminClient. What
> > I'm
> > >> >> >> > thinking is KraftAdminClient which continuously uses
> > >> >> *bootstrap.servers*,
> > >> >> >> > but make this class only serves the Kraft controllers APIs.
> > >> >> >>
> > >> >> >> Hi Philip,
> > >> >> >>
> > >> >> >> Thanks for taking a look.
> > >> >> >>
> > >> >> >> We would not want to create a new Admin client class in order to
> > >> >> >> communicate directly with the controllers. The RPCs accepted by
> > the
> > >> >> >> controllers have the same format as the those accepted by the
> > >> brokers.
> > >> >> >> There is no difference in what is sent over the wire or the data
> > >> >> structures
> > >> >> >> that are used in the client.
> > >> >> >>
> > >> >> >> I know you mentioned you haven't had time to read all the KRaft
> > stuff
> > >> >> (and
> > >> >> >> there is a lot, I understand). But this is one area that would
> > >> probably
> > >> >> be
> > >> >> >> clarified if you were able to read at least KIP-500 and KIP-590.
> > RPCs
> > >> >> sent
> > >> >> >> to the broker are forwarded to the controller (mostly) without
> > >> >> modification.
> > >> >> >>
> > >> >> >> >
> > >> >> >> > Secondly, if we want to continue with the design, I'm not yet
> > sure
> > >> >> why we
> > >> >> >> > can't continue using the *bootstrap.servers*? I assume when the
> > >> client
> > >> >> >> gets
> > >> >> >> > the metadata, it should know who it is talking to. I'm just
> > >> >> reconsidering
> > >> >> >> > your alternative again.
> > >> >> >> >
> > >> >> >> > A bad idea: Why don't we continue using *bootstrap.servers* but
> > >> have a
> > >> >> >> > separated config like *kraft.controller* = true/false. I feel
> > like
> > >> >> most
> > >> >> >> > users might not know what is a controller and causes some
> > mistakes
> > >> >> down
> > >> >> >> the
> > >> >> >> > road.
> > >> >> >> >
> > >> >> >>
> > >> >> >> Well, you called it a bad idea, and I can't help but agree! :)
> > >> >> >>
> > >> >> >> I think "the user might not know what a controller is" is a good
> > sign
> > >> >> that
> > >> >> >> they shouldn't be interacting with the controller. Like many
> > >> AdminClient
> > >> >> >> APIs, this one is intended for use by administrators only. Most
> > users
> > >> >> >> indeed should not need to know what a controller is or interact
> > with
> > >> it
> > >> >> >> directly. If they do interact it should be very clear that they
> > are
> > >> >> doing
> > >> >> >> so. The --controller-bootstrap flag makes it very clear, as does
> > the
> > >> >> >> separate configuration.
> > >> >> >>
> > >> >> >> Let me give an example of the kind of problems that arise if you
> > >> want to
> > >> >> >> reuse bootstrap.servers for this purpose.
> > >> >> >>
> > >> >> >> If the user grasb a 3.4 Kafka AdminClient and set
> > bootstrap.servers
> > >> to a
> > >> >> >> set of controller servers, and set direct.to.controller to true,
> > the
> > >> >> >> unknown (in 3.4) configuration will be ignored, and a normal
> > metadata
> > >> >> >> request will be sent without the direct to controller flag. In
> > that
> > >> >> case it
> > >> >> >> will give back an error. Confusing, right?
> > >> >> >>
> > >> >> >> Using controller.servers clarifies this situation because the 3.4
> > >> client
> > >> >> >> will not support that config, and will complain about the lack of
> > >> >> >> bootstrap.servers.
> > >> >> >>
> > >> >> >> Actually, the situation could get even more confusing than what I
> > >> >> >> described since some older preproduction versions of the KRaft
> > >> >> controller
> > >> >> >> did implement the METADATA RPC. So if you send them a METADATA
> > >> request
> > >> >> >> without any special information, it's not clear what you'll get.
> > >> >> Indeed,
> > >> >> >> it would be dependent on the client version and the controller
> > >> version.
> > >> >> >>
> > >> >> >> The bottom line is that reusing the bootstrap.servers
> > configuration
> > >> here
> > >> >> >> is not a good idea.
> > >> >> >>
> > >> >> >> best,
> > >> >> >> Colin
> > >> >> >>
> > >> >> >> > Thanks,
> > >> >> >> > P
> > >> >> >> >
> > >> >> >> > On Wed, Apr 19, 2023 at 2:18 PM Colin McCabe <
> > cmccabe@apache.org>
> > >> >> wrote:
> > >> >> >> >
> > >> >> >> >> Hi all,
> > >> >> >> >>
> > >> >> >> >> I wrote a short KIP about allowing AdminClient to talk directly
> > >> with
> > >> >> the
> > >> >> >> >> KRaft controller quorum. Check it out here:
> > >> >> >> >>
> > >> >> >> >> https://cwiki.apache.org/confluence/x/Owo0Dw
> > >> >> >> >>
> > >> >> >> >> best,
> > >> >> >> >> Colin
> > >> >> >> >>
> > >> >> >>
> > >> >>
> > >>
> >

Re: KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum

Posted by Colin McCabe <cm...@apache.org>.
On Tue, Jul 4, 2023, at 05:19, Luke Chen wrote:
> 4. Does all the command-line tools with `--bootstrap-controllers` support
> all the options in the tool?
> For example, kafka-configs.sh, In addition to the `--alter` option you
> mentioned in the example, do we also support `--describe` or `--delete`
> option?

Hi Luke,

Yes, we do.

> If so, do we also support setting "quota" for users/clients/topics... via
> `--bootstrap-controllers`? (not intuitive, but maybe we just directly
> commit the change into the metadata from controller?)

Yes. This would be supported via the alterClientQuotas API.

>
> 5. Do we have any plan for this feature to be completed? v3.6.0?
>

Yes, I would like to target 3.6. If time gets tight one or two command conversions might slip. The most critical command line tools to support are the ones for getting the quorum status itself.

best,
Colin

>
> Thank you.
> Luke
>
>
> On Fri, Apr 28, 2023 at 1:42 AM Colin McCabe <cm...@apache.org> wrote:
>
>> On Wed, Apr 26, 2023, at 22:08, Luke Chen wrote:
>> > Hi Colin,
>> >
>> > Some comments:
>> > 1. I agree we should set "top-level" errors for metadata response
>> >
>> > 2. In the "brokers" field of metadata response from controller, it'll
>> > respond with "Controller endpoint information as given in
>> > controller.quorum.voters", instead of the "alive" controllers(voters).
>> That
>> > will break the existing admin client because in admin client, we'll rely
>> on
>> > the metadata response to build the "current alive brokers" list, and
>> choose
>> > one from them to connect (either least load or other criteria). That
>> means,
>> > if now, we return the value in `controller.quorum.voters`, but one of
>> them
>> > is down. We might choose it to connect and get connection errors. Should
>> we
>> > return the "alive" controllers(voters) to client?
>>
>> Hi Luke,
>>
>> Good question. When talking to the controllers directly, the AdminClient
>> needs to always send its RPCs to the active controller. There is one
>> exception: configuring ephemeral log4j settings with
>> incrementalAlterConfigs must be done by sending them to the specified
>> controller node.
>>
>> I will add this to a section called "AdminClient Implementation Notes" so
>> that it's captured in the KIP.
>>
>> >
>> > 3. In the KIP, we list the command-line tools will get a new
>> > --bootstrap-controllers argument, but without explaining why these tools
>> > need to talk to controller directly. Could we add some explanation about
>> > them? I tried but cannot know why some tools are listed here:
>> >     - kafka-acls.sh -> Allow clients to update ACLs via controller before
>> > brokers up
>> >
>> >     - kafka-cluster.sh -> Reasonable to get/update cluster info via
>> > controller
>> >
>> >     - kafka-configs.sh -> Allow clients to dynamically update
>> > configs/describe configs from controller. But in this script, client can
>> > still set quota for users/clients/topics... is client also able to update
>> > via controllers? Or we only allow partial actions in the script to talk
>> to
>> > controllers?
>> >
>> >     - kafka-delegation-tokens.sh -> Reasonable to update
>> delegation-tokens
>> > via controllers
>> >
>> >     - kafka-features.sh -> Reasonable
>> >     - kafka-metadata-quorum.sh -> Reasonable
>> >     - kafka-metadata-shell.sh -> Reasonable
>> >
>> >     - kafka-reassign-partitions.sh -> Why should we allow clients to move
>> > metadata log partitions in controller nodes? What's the use-case?
>> >
>>
>> Yes, the common thread here is that all of these shell commands perform
>> operations can be done without the broker. So it's reasonable to allow them
>> to be done without going through the broker. I don't know if we need a
>> separate note for each since the rationale is really the same for all (is
>> it reasonable? if so allow it.)
>>
>> kafka-reassign-partitions.sh cannot be used to move the __cluster_metadata
>> topic. However, it can be used to move partitions that reside on the
>> brokers, even when using --bootstrap-controllers to talk directly to the
>> quorum.
>>
>> Colin
>>
>> >
>> > Thank you.
>> > Luke
>> >
>> > On Thu, Apr 27, 2023 at 8:04 AM Colin McCabe <cm...@apache.org> wrote:
>> >
>> >> On Tue, Apr 25, 2023, at 04:59, Divij Vaidya wrote:
>> >> > Thank you for the KIP Colin.
>> >> >
>> >> > In general, I like the idea of having the ability to interact directly
>> >> with
>> >> > the controllers. I agree with your observation that it helps in
>> >> situations
>> >> > where you would want to get data directly from the controller instead
>> of
>> >> > going via a broker. I have some general comments but the main concern
>> I
>> >> > have is with the piggy-backing of error code with response of
>> >> > __cluster_metadata topic.
>> >> >
>> >> > 1. With this change, how are we guarding against the possibility of
>> >> > misbehaving client traffic from disrupting the controller (that you
>> >> > mentioned as a motivation of earlier behaviour)? One solution could
>> be to
>> >> > have default values set for request throttling on the controller.
>> >>
>> >> Hi Divij,
>> >>
>> >> Thanks for the comments.
>> >>
>> >> Our guards against client misbehavior remain the same:
>> >> 1. our recommendation to put the clients on a separate network
>> >> 2. the fact that producers and consumers can't interact directly with
>> the
>> >> controller
>> >> 3. the authorizer.
>> >>
>> >> Re: #3, I will spell out in the KIP that clients must have DESCRIBE on
>> the
>> >> CLUSTER resource to send a METADATA request to the controller.
>> >>
>> >> > 2. This KIP also increases the network attack surface area. Prior to
>> this
>> >> > KIP, it was possible to have firewall rules setup for controllers such
>> >> that
>> >> > only the brokers can talk to it. But now, the controller would require
>> >> > access from other endpoints other than brokers as well. Can we add a
>> >> > suggestion to the upgrade documentation and inform users
>> >>
>> >> There is no requirement for access from other endpoints. It is still
>> >> possible to set up firewall rules such that only the brokers can talk to
>> >> the controller. In fact I would even say this is desirable. Since this
>> >> faculty is intended for infrequent manual administrative operations,
>> >> needing to log into the broker to use it seems perfectly fine.
>> >>
>> >> > 3. In section KRaft Controller MetadataResponse, row 3, "There is no
>> >> > top-level error code in MetadataResponse, so we use the
>> >> __cluster_metadata
>> >> > topic to send back our error.". This will definitely confuse the
>> users.
>> >> Can
>> >> > we introduce a top level error field instead?
>> >>
>> >> Let me check how we're handling this in other places. I recall some
>> other
>> >> cases where we used the dummy topic approach, but I can't find them just
>> >> now.
>> >>
>> >> > 4. As part of the KIP, could we please add some documentation for
>> users
>> >> > with the suggestion of when to get information directly from the
>> >> controller
>> >> > and when not to (and associated tradeoffs)?
>> >>
>> >> I think most users will not face this tradeoff because they simply won't
>> >> have network access to the controller servers.
>> >>
>> >> For those who do want more information, we'll have command-line
>> >> documentation for --boostrap-controllers and the bootstrap.controllers
>> >> configuration key.
>> >>
>> >> > 5. Why do we need the `FromKRaftController` field in the response?
>> What
>> >> do
>> >> > we expect the users to do with this information?
>> >>
>> >> The field is present so that we can throw an exception in the client if
>> we
>> >> have received a response that is not from the controller.
>> >>
>> >> > 6. Can we drop `KRaft` from the fields `FromKRaftController` and
>> >> > `DirectToKRaftControllerQuorum`? My suggestion will be to rename it as
>> >> > `DirectToController`.
>> >>
>> >> I like the idea, but wouldn't that create confusion in the ZK cluster
>> case?
>> >>
>> >> > 7.  "kafka-metadata-shell.sh  is at an "evolving" level of interface
>> >> > stability" -> I thought that with KRaft being production ready, the
>> >> > evolving mode for kraft-related tools has also moved to production.
>> Do we
>> >> > have a timeline when this would move to production?
>> >>
>> >> That's a good question, but I think we should tackle it separately from
>> >> this KIP. The metadata shell is pretty different from other parts of
>> kafka
>> >> since it interacts so closely with internals.
>> >>
>> >> best,
>> >> Colin
>> >>
>> >>
>> >> >
>> >> > --
>> >> > Divij Vaidya
>> >> >
>> >> >
>> >> >
>> >> > On Tue, Apr 25, 2023 at 1:38 AM Colin McCabe <cm...@apache.org>
>> wrote:
>> >> >
>> >> >> On Fri, Apr 21, 2023, at 14:17, Jason Gustafson wrote:
>> >> >> > Hey Colin,
>> >> >> >
>> >> >> > The KIP makes sense overall. Nice to clarify the contract between
>> >> clients
>> >> >> > and the controllers. The use of `DirectToKRaftControllerQuorum`
>> will
>> >> help
>> >> >> > prevent misconfiguration. In fact, I wonder if we can return a
>> fatal
>> >> >> error
>> >> >> > instead of NOT_CONTROLLER so that the client would immediately
>> fail.
>> >> For
>> >> >> > example, could we use INVALID_REQUEST or something like that?
>> Either
>> >> that
>> >> >> > or we need to make sure clients treat NOT_CONTROLLER as a fatal
>> error.
>> >> >> > Without that, it would probably get picked up with default retry
>> logic
>> >> >> and
>> >> >> > the user might not see the problem.
>> >> >>
>> >> >> Hi Jason,
>> >> >>
>> >> >> Yes, this is a good point. It should return INVALID_REQUEST since
>> that
>> >> is
>> >> >> not retryable. I'll change it.
>> >> >>
>> >> >> >
>> >> >> > I also wonder if we can relax the requirements on the Metadata
>> >> request so
>> >> >> > that we can use it to list topics and partition state (e.g.
>> URPs).  It
>> >> >> > would be useful to query the controllers as the metadata source of
>> >> truth
>> >> >> > when we suspect that the broker metadata may have diverged.
>> >> >> >
>> >> >>
>> >> >> Fair enough. I will document that we can return topics.
>> >> >>
>> >> >> I also added a "future work" section about maybe using the
>> controllers
>> >> as
>> >> >> bootstrap servers for the broker cluster. To be clear, that is NOT in
>> >> scope
>> >> >> here, but it's interesting to think about potentially doing in the
>> >> future.
>> >> >> The major problem is what to do when the broker endpoints we're
>> >> returning
>> >> >> have different security settings from the controller endpoint the
>> client
>> >> >> initially talked to.
>> >> >>
>> >> >> best,
>> >> >> Colin
>> >> >>
>> >> >>
>> >> >> >
>> >> >> > Thanks,
>> >> >> > Jason
>> >> >> >
>> >> >> > On Thu, Apr 20, 2023 at 5:53 PM Colin McCabe <cm...@apache.org>
>> >> wrote:
>> >> >> >
>> >> >> >> On Wed, Apr 19, 2023, at 20:56, Philip Nee wrote:
>> >> >> >> > Hey Colin,
>> >> >> >> >
>> >> >> >> > I still need to finish reading and understanding the KIP, but I
>> >> have a
>> >> >> >> > couple of comments despite being ignorant of most of the KRaft
>> >> stuff.
>> >> >> >> > (Sorry!)
>> >> >> >> >
>> >> >> >> > Firstly, does it make sense to create an extension of the
>> current
>> >> >> >> > AdminClient only to handle these specific KRaft use cases? It
>> seems
>> >> >> >> > cumbersome to have two sets of bootstrap configurations to make
>> the
>> >> >> >> > AdminClient generic enough to handle these specific cases,
>> instead,
>> >> >> maybe
>> >> >> >> > it is more obvious (to me) to just extend the AdminClient. What
>> I'm
>> >> >> >> > thinking is KraftAdminClient which continuously uses
>> >> >> *bootstrap.servers*,
>> >> >> >> > but make this class only serves the Kraft controllers APIs.
>> >> >> >>
>> >> >> >> Hi Philip,
>> >> >> >>
>> >> >> >> Thanks for taking a look.
>> >> >> >>
>> >> >> >> We would not want to create a new Admin client class in order to
>> >> >> >> communicate directly with the controllers. The RPCs accepted by
>> the
>> >> >> >> controllers have the same format as the those accepted by the
>> >> brokers.
>> >> >> >> There is no difference in what is sent over the wire or the data
>> >> >> structures
>> >> >> >> that are used in the client.
>> >> >> >>
>> >> >> >> I know you mentioned you haven't had time to read all the KRaft
>> stuff
>> >> >> (and
>> >> >> >> there is a lot, I understand). But this is one area that would
>> >> probably
>> >> >> be
>> >> >> >> clarified if you were able to read at least KIP-500 and KIP-590.
>> RPCs
>> >> >> sent
>> >> >> >> to the broker are forwarded to the controller (mostly) without
>> >> >> modification.
>> >> >> >>
>> >> >> >> >
>> >> >> >> > Secondly, if we want to continue with the design, I'm not yet
>> sure
>> >> >> why we
>> >> >> >> > can't continue using the *bootstrap.servers*? I assume when the
>> >> client
>> >> >> >> gets
>> >> >> >> > the metadata, it should know who it is talking to. I'm just
>> >> >> reconsidering
>> >> >> >> > your alternative again.
>> >> >> >> >
>> >> >> >> > A bad idea: Why don't we continue using *bootstrap.servers* but
>> >> have a
>> >> >> >> > separated config like *kraft.controller* = true/false. I feel
>> like
>> >> >> most
>> >> >> >> > users might not know what is a controller and causes some
>> mistakes
>> >> >> down
>> >> >> >> the
>> >> >> >> > road.
>> >> >> >> >
>> >> >> >>
>> >> >> >> Well, you called it a bad idea, and I can't help but agree! :)
>> >> >> >>
>> >> >> >> I think "the user might not know what a controller is" is a good
>> sign
>> >> >> that
>> >> >> >> they shouldn't be interacting with the controller. Like many
>> >> AdminClient
>> >> >> >> APIs, this one is intended for use by administrators only. Most
>> users
>> >> >> >> indeed should not need to know what a controller is or interact
>> with
>> >> it
>> >> >> >> directly. If they do interact it should be very clear that they
>> are
>> >> >> doing
>> >> >> >> so. The --controller-bootstrap flag makes it very clear, as does
>> the
>> >> >> >> separate configuration.
>> >> >> >>
>> >> >> >> Let me give an example of the kind of problems that arise if you
>> >> want to
>> >> >> >> reuse bootstrap.servers for this purpose.
>> >> >> >>
>> >> >> >> If the user grasb a 3.4 Kafka AdminClient and set
>> bootstrap.servers
>> >> to a
>> >> >> >> set of controller servers, and set direct.to.controller to true,
>> the
>> >> >> >> unknown (in 3.4) configuration will be ignored, and a normal
>> metadata
>> >> >> >> request will be sent without the direct to controller flag. In
>> that
>> >> >> case it
>> >> >> >> will give back an error. Confusing, right?
>> >> >> >>
>> >> >> >> Using controller.servers clarifies this situation because the 3.4
>> >> client
>> >> >> >> will not support that config, and will complain about the lack of
>> >> >> >> bootstrap.servers.
>> >> >> >>
>> >> >> >> Actually, the situation could get even more confusing than what I
>> >> >> >> described since some older preproduction versions of the KRaft
>> >> >> controller
>> >> >> >> did implement the METADATA RPC. So if you send them a METADATA
>> >> request
>> >> >> >> without any special information, it's not clear what you'll get.
>> >> >> Indeed,
>> >> >> >> it would be dependent on the client version and the controller
>> >> version.
>> >> >> >>
>> >> >> >> The bottom line is that reusing the bootstrap.servers
>> configuration
>> >> here
>> >> >> >> is not a good idea.
>> >> >> >>
>> >> >> >> best,
>> >> >> >> Colin
>> >> >> >>
>> >> >> >> > Thanks,
>> >> >> >> > P
>> >> >> >> >
>> >> >> >> > On Wed, Apr 19, 2023 at 2:18 PM Colin McCabe <
>> cmccabe@apache.org>
>> >> >> wrote:
>> >> >> >> >
>> >> >> >> >> Hi all,
>> >> >> >> >>
>> >> >> >> >> I wrote a short KIP about allowing AdminClient to talk directly
>> >> with
>> >> >> the
>> >> >> >> >> KRaft controller quorum. Check it out here:
>> >> >> >> >>
>> >> >> >> >> https://cwiki.apache.org/confluence/x/Owo0Dw
>> >> >> >> >>
>> >> >> >> >> best,
>> >> >> >> >> Colin
>> >> >> >> >>
>> >> >> >>
>> >> >>
>> >>
>>

Re: KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum

Posted by Luke Chen <sh...@gmail.com>.
Hi Colin,

Thanks for the answers to my previous questions.

> Yes, the common thread here is that all of these shell commands perform
operations can be done without the broker. So it's reasonable to allow them
to be done without going through the broker. I don't know if we need a
separate note for each since the rationale is really the same for all (is
it reasonable? if so allow it.)

Yes, it makes sense. Could we make a note about the main rationale for
selecting these command-line tools in the KIP to make it clear?
Ex: The following command-line tools will get a new --bootstrap-controllers
argument (because these shell commands perform operations can be done
without the broker):

> kafka-reassign-partitions.sh cannot be used to move the
__cluster_metadata topic. However, it can be used to move partitions that
reside on the brokers, even when using --bootstrap-controllers to talk
directly to the quorum.

Fair enough.


4. Does all the command-line tools with `--bootstrap-controllers` support
all the options in the tool?
For example, kafka-configs.sh, In addition to the `--alter` option you
mentioned in the example, do we also support `--describe` or `--delete`
option?
If so, do we also support setting "quota" for users/clients/topics... via
`--bootstrap-controllers`? (not intuitive, but maybe we just directly
commit the change into the metadata from controller?)

5. Do we have any plan for this feature to be completed? v3.6.0?


Thank you.
Luke


On Fri, Apr 28, 2023 at 1:42 AM Colin McCabe <cm...@apache.org> wrote:

> On Wed, Apr 26, 2023, at 22:08, Luke Chen wrote:
> > Hi Colin,
> >
> > Some comments:
> > 1. I agree we should set "top-level" errors for metadata response
> >
> > 2. In the "brokers" field of metadata response from controller, it'll
> > respond with "Controller endpoint information as given in
> > controller.quorum.voters", instead of the "alive" controllers(voters).
> That
> > will break the existing admin client because in admin client, we'll rely
> on
> > the metadata response to build the "current alive brokers" list, and
> choose
> > one from them to connect (either least load or other criteria). That
> means,
> > if now, we return the value in `controller.quorum.voters`, but one of
> them
> > is down. We might choose it to connect and get connection errors. Should
> we
> > return the "alive" controllers(voters) to client?
>
> Hi Luke,
>
> Good question. When talking to the controllers directly, the AdminClient
> needs to always send its RPCs to the active controller. There is one
> exception: configuring ephemeral log4j settings with
> incrementalAlterConfigs must be done by sending them to the specified
> controller node.
>
> I will add this to a section called "AdminClient Implementation Notes" so
> that it's captured in the KIP.
>
> >
> > 3. In the KIP, we list the command-line tools will get a new
> > --bootstrap-controllers argument, but without explaining why these tools
> > need to talk to controller directly. Could we add some explanation about
> > them? I tried but cannot know why some tools are listed here:
> >     - kafka-acls.sh -> Allow clients to update ACLs via controller before
> > brokers up
> >
> >     - kafka-cluster.sh -> Reasonable to get/update cluster info via
> > controller
> >
> >     - kafka-configs.sh -> Allow clients to dynamically update
> > configs/describe configs from controller. But in this script, client can
> > still set quota for users/clients/topics... is client also able to update
> > via controllers? Or we only allow partial actions in the script to talk
> to
> > controllers?
> >
> >     - kafka-delegation-tokens.sh -> Reasonable to update
> delegation-tokens
> > via controllers
> >
> >     - kafka-features.sh -> Reasonable
> >     - kafka-metadata-quorum.sh -> Reasonable
> >     - kafka-metadata-shell.sh -> Reasonable
> >
> >     - kafka-reassign-partitions.sh -> Why should we allow clients to move
> > metadata log partitions in controller nodes? What's the use-case?
> >
>
> Yes, the common thread here is that all of these shell commands perform
> operations can be done without the broker. So it's reasonable to allow them
> to be done without going through the broker. I don't know if we need a
> separate note for each since the rationale is really the same for all (is
> it reasonable? if so allow it.)
>
> kafka-reassign-partitions.sh cannot be used to move the __cluster_metadata
> topic. However, it can be used to move partitions that reside on the
> brokers, even when using --bootstrap-controllers to talk directly to the
> quorum.
>
> Colin
>
> >
> > Thank you.
> > Luke
> >
> > On Thu, Apr 27, 2023 at 8:04 AM Colin McCabe <cm...@apache.org> wrote:
> >
> >> On Tue, Apr 25, 2023, at 04:59, Divij Vaidya wrote:
> >> > Thank you for the KIP Colin.
> >> >
> >> > In general, I like the idea of having the ability to interact directly
> >> with
> >> > the controllers. I agree with your observation that it helps in
> >> situations
> >> > where you would want to get data directly from the controller instead
> of
> >> > going via a broker. I have some general comments but the main concern
> I
> >> > have is with the piggy-backing of error code with response of
> >> > __cluster_metadata topic.
> >> >
> >> > 1. With this change, how are we guarding against the possibility of
> >> > misbehaving client traffic from disrupting the controller (that you
> >> > mentioned as a motivation of earlier behaviour)? One solution could
> be to
> >> > have default values set for request throttling on the controller.
> >>
> >> Hi Divij,
> >>
> >> Thanks for the comments.
> >>
> >> Our guards against client misbehavior remain the same:
> >> 1. our recommendation to put the clients on a separate network
> >> 2. the fact that producers and consumers can't interact directly with
> the
> >> controller
> >> 3. the authorizer.
> >>
> >> Re: #3, I will spell out in the KIP that clients must have DESCRIBE on
> the
> >> CLUSTER resource to send a METADATA request to the controller.
> >>
> >> > 2. This KIP also increases the network attack surface area. Prior to
> this
> >> > KIP, it was possible to have firewall rules setup for controllers such
> >> that
> >> > only the brokers can talk to it. But now, the controller would require
> >> > access from other endpoints other than brokers as well. Can we add a
> >> > suggestion to the upgrade documentation and inform users
> >>
> >> There is no requirement for access from other endpoints. It is still
> >> possible to set up firewall rules such that only the brokers can talk to
> >> the controller. In fact I would even say this is desirable. Since this
> >> faculty is intended for infrequent manual administrative operations,
> >> needing to log into the broker to use it seems perfectly fine.
> >>
> >> > 3. In section KRaft Controller MetadataResponse, row 3, "There is no
> >> > top-level error code in MetadataResponse, so we use the
> >> __cluster_metadata
> >> > topic to send back our error.". This will definitely confuse the
> users.
> >> Can
> >> > we introduce a top level error field instead?
> >>
> >> Let me check how we're handling this in other places. I recall some
> other
> >> cases where we used the dummy topic approach, but I can't find them just
> >> now.
> >>
> >> > 4. As part of the KIP, could we please add some documentation for
> users
> >> > with the suggestion of when to get information directly from the
> >> controller
> >> > and when not to (and associated tradeoffs)?
> >>
> >> I think most users will not face this tradeoff because they simply won't
> >> have network access to the controller servers.
> >>
> >> For those who do want more information, we'll have command-line
> >> documentation for --boostrap-controllers and the bootstrap.controllers
> >> configuration key.
> >>
> >> > 5. Why do we need the `FromKRaftController` field in the response?
> What
> >> do
> >> > we expect the users to do with this information?
> >>
> >> The field is present so that we can throw an exception in the client if
> we
> >> have received a response that is not from the controller.
> >>
> >> > 6. Can we drop `KRaft` from the fields `FromKRaftController` and
> >> > `DirectToKRaftControllerQuorum`? My suggestion will be to rename it as
> >> > `DirectToController`.
> >>
> >> I like the idea, but wouldn't that create confusion in the ZK cluster
> case?
> >>
> >> > 7.  "kafka-metadata-shell.sh  is at an "evolving" level of interface
> >> > stability" -> I thought that with KRaft being production ready, the
> >> > evolving mode for kraft-related tools has also moved to production.
> Do we
> >> > have a timeline when this would move to production?
> >>
> >> That's a good question, but I think we should tackle it separately from
> >> this KIP. The metadata shell is pretty different from other parts of
> kafka
> >> since it interacts so closely with internals.
> >>
> >> best,
> >> Colin
> >>
> >>
> >> >
> >> > --
> >> > Divij Vaidya
> >> >
> >> >
> >> >
> >> > On Tue, Apr 25, 2023 at 1:38 AM Colin McCabe <cm...@apache.org>
> wrote:
> >> >
> >> >> On Fri, Apr 21, 2023, at 14:17, Jason Gustafson wrote:
> >> >> > Hey Colin,
> >> >> >
> >> >> > The KIP makes sense overall. Nice to clarify the contract between
> >> clients
> >> >> > and the controllers. The use of `DirectToKRaftControllerQuorum`
> will
> >> help
> >> >> > prevent misconfiguration. In fact, I wonder if we can return a
> fatal
> >> >> error
> >> >> > instead of NOT_CONTROLLER so that the client would immediately
> fail.
> >> For
> >> >> > example, could we use INVALID_REQUEST or something like that?
> Either
> >> that
> >> >> > or we need to make sure clients treat NOT_CONTROLLER as a fatal
> error.
> >> >> > Without that, it would probably get picked up with default retry
> logic
> >> >> and
> >> >> > the user might not see the problem.
> >> >>
> >> >> Hi Jason,
> >> >>
> >> >> Yes, this is a good point. It should return INVALID_REQUEST since
> that
> >> is
> >> >> not retryable. I'll change it.
> >> >>
> >> >> >
> >> >> > I also wonder if we can relax the requirements on the Metadata
> >> request so
> >> >> > that we can use it to list topics and partition state (e.g.
> URPs).  It
> >> >> > would be useful to query the controllers as the metadata source of
> >> truth
> >> >> > when we suspect that the broker metadata may have diverged.
> >> >> >
> >> >>
> >> >> Fair enough. I will document that we can return topics.
> >> >>
> >> >> I also added a "future work" section about maybe using the
> controllers
> >> as
> >> >> bootstrap servers for the broker cluster. To be clear, that is NOT in
> >> scope
> >> >> here, but it's interesting to think about potentially doing in the
> >> future.
> >> >> The major problem is what to do when the broker endpoints we're
> >> returning
> >> >> have different security settings from the controller endpoint the
> client
> >> >> initially talked to.
> >> >>
> >> >> best,
> >> >> Colin
> >> >>
> >> >>
> >> >> >
> >> >> > Thanks,
> >> >> > Jason
> >> >> >
> >> >> > On Thu, Apr 20, 2023 at 5:53 PM Colin McCabe <cm...@apache.org>
> >> wrote:
> >> >> >
> >> >> >> On Wed, Apr 19, 2023, at 20:56, Philip Nee wrote:
> >> >> >> > Hey Colin,
> >> >> >> >
> >> >> >> > I still need to finish reading and understanding the KIP, but I
> >> have a
> >> >> >> > couple of comments despite being ignorant of most of the KRaft
> >> stuff.
> >> >> >> > (Sorry!)
> >> >> >> >
> >> >> >> > Firstly, does it make sense to create an extension of the
> current
> >> >> >> > AdminClient only to handle these specific KRaft use cases? It
> seems
> >> >> >> > cumbersome to have two sets of bootstrap configurations to make
> the
> >> >> >> > AdminClient generic enough to handle these specific cases,
> instead,
> >> >> maybe
> >> >> >> > it is more obvious (to me) to just extend the AdminClient. What
> I'm
> >> >> >> > thinking is KraftAdminClient which continuously uses
> >> >> *bootstrap.servers*,
> >> >> >> > but make this class only serves the Kraft controllers APIs.
> >> >> >>
> >> >> >> Hi Philip,
> >> >> >>
> >> >> >> Thanks for taking a look.
> >> >> >>
> >> >> >> We would not want to create a new Admin client class in order to
> >> >> >> communicate directly with the controllers. The RPCs accepted by
> the
> >> >> >> controllers have the same format as the those accepted by the
> >> brokers.
> >> >> >> There is no difference in what is sent over the wire or the data
> >> >> structures
> >> >> >> that are used in the client.
> >> >> >>
> >> >> >> I know you mentioned you haven't had time to read all the KRaft
> stuff
> >> >> (and
> >> >> >> there is a lot, I understand). But this is one area that would
> >> probably
> >> >> be
> >> >> >> clarified if you were able to read at least KIP-500 and KIP-590.
> RPCs
> >> >> sent
> >> >> >> to the broker are forwarded to the controller (mostly) without
> >> >> modification.
> >> >> >>
> >> >> >> >
> >> >> >> > Secondly, if we want to continue with the design, I'm not yet
> sure
> >> >> why we
> >> >> >> > can't continue using the *bootstrap.servers*? I assume when the
> >> client
> >> >> >> gets
> >> >> >> > the metadata, it should know who it is talking to. I'm just
> >> >> reconsidering
> >> >> >> > your alternative again.
> >> >> >> >
> >> >> >> > A bad idea: Why don't we continue using *bootstrap.servers* but
> >> have a
> >> >> >> > separated config like *kraft.controller* = true/false. I feel
> like
> >> >> most
> >> >> >> > users might not know what is a controller and causes some
> mistakes
> >> >> down
> >> >> >> the
> >> >> >> > road.
> >> >> >> >
> >> >> >>
> >> >> >> Well, you called it a bad idea, and I can't help but agree! :)
> >> >> >>
> >> >> >> I think "the user might not know what a controller is" is a good
> sign
> >> >> that
> >> >> >> they shouldn't be interacting with the controller. Like many
> >> AdminClient
> >> >> >> APIs, this one is intended for use by administrators only. Most
> users
> >> >> >> indeed should not need to know what a controller is or interact
> with
> >> it
> >> >> >> directly. If they do interact it should be very clear that they
> are
> >> >> doing
> >> >> >> so. The --controller-bootstrap flag makes it very clear, as does
> the
> >> >> >> separate configuration.
> >> >> >>
> >> >> >> Let me give an example of the kind of problems that arise if you
> >> want to
> >> >> >> reuse bootstrap.servers for this purpose.
> >> >> >>
> >> >> >> If the user grasb a 3.4 Kafka AdminClient and set
> bootstrap.servers
> >> to a
> >> >> >> set of controller servers, and set direct.to.controller to true,
> the
> >> >> >> unknown (in 3.4) configuration will be ignored, and a normal
> metadata
> >> >> >> request will be sent without the direct to controller flag. In
> that
> >> >> case it
> >> >> >> will give back an error. Confusing, right?
> >> >> >>
> >> >> >> Using controller.servers clarifies this situation because the 3.4
> >> client
> >> >> >> will not support that config, and will complain about the lack of
> >> >> >> bootstrap.servers.
> >> >> >>
> >> >> >> Actually, the situation could get even more confusing than what I
> >> >> >> described since some older preproduction versions of the KRaft
> >> >> controller
> >> >> >> did implement the METADATA RPC. So if you send them a METADATA
> >> request
> >> >> >> without any special information, it's not clear what you'll get.
> >> >> Indeed,
> >> >> >> it would be dependent on the client version and the controller
> >> version.
> >> >> >>
> >> >> >> The bottom line is that reusing the bootstrap.servers
> configuration
> >> here
> >> >> >> is not a good idea.
> >> >> >>
> >> >> >> best,
> >> >> >> Colin
> >> >> >>
> >> >> >> > Thanks,
> >> >> >> > P
> >> >> >> >
> >> >> >> > On Wed, Apr 19, 2023 at 2:18 PM Colin McCabe <
> cmccabe@apache.org>
> >> >> wrote:
> >> >> >> >
> >> >> >> >> Hi all,
> >> >> >> >>
> >> >> >> >> I wrote a short KIP about allowing AdminClient to talk directly
> >> with
> >> >> the
> >> >> >> >> KRaft controller quorum. Check it out here:
> >> >> >> >>
> >> >> >> >> https://cwiki.apache.org/confluence/x/Owo0Dw
> >> >> >> >>
> >> >> >> >> best,
> >> >> >> >> Colin
> >> >> >> >>
> >> >> >>
> >> >>
> >>
>

Re: KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum

Posted by Colin McCabe <cm...@apache.org>.
On Wed, Apr 26, 2023, at 22:08, Luke Chen wrote:
> Hi Colin,
>
> Some comments:
> 1. I agree we should set "top-level" errors for metadata response
>
> 2. In the "brokers" field of metadata response from controller, it'll
> respond with "Controller endpoint information as given in
> controller.quorum.voters", instead of the "alive" controllers(voters). That
> will break the existing admin client because in admin client, we'll rely on
> the metadata response to build the "current alive brokers" list, and choose
> one from them to connect (either least load or other criteria). That means,
> if now, we return the value in `controller.quorum.voters`, but one of them
> is down. We might choose it to connect and get connection errors. Should we
> return the "alive" controllers(voters) to client?

Hi Luke,

Good question. When talking to the controllers directly, the AdminClient needs to always send its RPCs to the active controller. There is one exception: configuring ephemeral log4j settings with incrementalAlterConfigs must be done by sending them to the specified controller node.

I will add this to a section called "AdminClient Implementation Notes" so that it's captured in the KIP.

>
> 3. In the KIP, we list the command-line tools will get a new
> --bootstrap-controllers argument, but without explaining why these tools
> need to talk to controller directly. Could we add some explanation about
> them? I tried but cannot know why some tools are listed here:
>     - kafka-acls.sh -> Allow clients to update ACLs via controller before
> brokers up
>
>     - kafka-cluster.sh -> Reasonable to get/update cluster info via
> controller
>
>     - kafka-configs.sh -> Allow clients to dynamically update
> configs/describe configs from controller. But in this script, client can
> still set quota for users/clients/topics... is client also able to update
> via controllers? Or we only allow partial actions in the script to talk to
> controllers?
>
>     - kafka-delegation-tokens.sh -> Reasonable to update delegation-tokens
> via controllers
>
>     - kafka-features.sh -> Reasonable
>     - kafka-metadata-quorum.sh -> Reasonable
>     - kafka-metadata-shell.sh -> Reasonable
>
>     - kafka-reassign-partitions.sh -> Why should we allow clients to move
> metadata log partitions in controller nodes? What's the use-case?
>

Yes, the common thread here is that all of these shell commands perform operations can be done without the broker. So it's reasonable to allow them to be done without going through the broker. I don't know if we need a separate note for each since the rationale is really the same for all (is it reasonable? if so allow it.)

kafka-reassign-partitions.sh cannot be used to move the __cluster_metadata topic. However, it can be used to move partitions that reside on the brokers, even when using --bootstrap-controllers to talk directly to the quorum.

Colin

>
> Thank you.
> Luke
>
> On Thu, Apr 27, 2023 at 8:04 AM Colin McCabe <cm...@apache.org> wrote:
>
>> On Tue, Apr 25, 2023, at 04:59, Divij Vaidya wrote:
>> > Thank you for the KIP Colin.
>> >
>> > In general, I like the idea of having the ability to interact directly
>> with
>> > the controllers. I agree with your observation that it helps in
>> situations
>> > where you would want to get data directly from the controller instead of
>> > going via a broker. I have some general comments but the main concern I
>> > have is with the piggy-backing of error code with response of
>> > __cluster_metadata topic.
>> >
>> > 1. With this change, how are we guarding against the possibility of
>> > misbehaving client traffic from disrupting the controller (that you
>> > mentioned as a motivation of earlier behaviour)? One solution could be to
>> > have default values set for request throttling on the controller.
>>
>> Hi Divij,
>>
>> Thanks for the comments.
>>
>> Our guards against client misbehavior remain the same:
>> 1. our recommendation to put the clients on a separate network
>> 2. the fact that producers and consumers can't interact directly with the
>> controller
>> 3. the authorizer.
>>
>> Re: #3, I will spell out in the KIP that clients must have DESCRIBE on the
>> CLUSTER resource to send a METADATA request to the controller.
>>
>> > 2. This KIP also increases the network attack surface area. Prior to this
>> > KIP, it was possible to have firewall rules setup for controllers such
>> that
>> > only the brokers can talk to it. But now, the controller would require
>> > access from other endpoints other than brokers as well. Can we add a
>> > suggestion to the upgrade documentation and inform users
>>
>> There is no requirement for access from other endpoints. It is still
>> possible to set up firewall rules such that only the brokers can talk to
>> the controller. In fact I would even say this is desirable. Since this
>> faculty is intended for infrequent manual administrative operations,
>> needing to log into the broker to use it seems perfectly fine.
>>
>> > 3. In section KRaft Controller MetadataResponse, row 3, "There is no
>> > top-level error code in MetadataResponse, so we use the
>> __cluster_metadata
>> > topic to send back our error.". This will definitely confuse the users.
>> Can
>> > we introduce a top level error field instead?
>>
>> Let me check how we're handling this in other places. I recall some other
>> cases where we used the dummy topic approach, but I can't find them just
>> now.
>>
>> > 4. As part of the KIP, could we please add some documentation for users
>> > with the suggestion of when to get information directly from the
>> controller
>> > and when not to (and associated tradeoffs)?
>>
>> I think most users will not face this tradeoff because they simply won't
>> have network access to the controller servers.
>>
>> For those who do want more information, we'll have command-line
>> documentation for --boostrap-controllers and the bootstrap.controllers
>> configuration key.
>>
>> > 5. Why do we need the `FromKRaftController` field in the response? What
>> do
>> > we expect the users to do with this information?
>>
>> The field is present so that we can throw an exception in the client if we
>> have received a response that is not from the controller.
>>
>> > 6. Can we drop `KRaft` from the fields `FromKRaftController` and
>> > `DirectToKRaftControllerQuorum`? My suggestion will be to rename it as
>> > `DirectToController`.
>>
>> I like the idea, but wouldn't that create confusion in the ZK cluster case?
>>
>> > 7.  "kafka-metadata-shell.sh  is at an "evolving" level of interface
>> > stability" -> I thought that with KRaft being production ready, the
>> > evolving mode for kraft-related tools has also moved to production. Do we
>> > have a timeline when this would move to production?
>>
>> That's a good question, but I think we should tackle it separately from
>> this KIP. The metadata shell is pretty different from other parts of kafka
>> since it interacts so closely with internals.
>>
>> best,
>> Colin
>>
>>
>> >
>> > --
>> > Divij Vaidya
>> >
>> >
>> >
>> > On Tue, Apr 25, 2023 at 1:38 AM Colin McCabe <cm...@apache.org> wrote:
>> >
>> >> On Fri, Apr 21, 2023, at 14:17, Jason Gustafson wrote:
>> >> > Hey Colin,
>> >> >
>> >> > The KIP makes sense overall. Nice to clarify the contract between
>> clients
>> >> > and the controllers. The use of `DirectToKRaftControllerQuorum` will
>> help
>> >> > prevent misconfiguration. In fact, I wonder if we can return a fatal
>> >> error
>> >> > instead of NOT_CONTROLLER so that the client would immediately fail.
>> For
>> >> > example, could we use INVALID_REQUEST or something like that? Either
>> that
>> >> > or we need to make sure clients treat NOT_CONTROLLER as a fatal error.
>> >> > Without that, it would probably get picked up with default retry logic
>> >> and
>> >> > the user might not see the problem.
>> >>
>> >> Hi Jason,
>> >>
>> >> Yes, this is a good point. It should return INVALID_REQUEST since that
>> is
>> >> not retryable. I'll change it.
>> >>
>> >> >
>> >> > I also wonder if we can relax the requirements on the Metadata
>> request so
>> >> > that we can use it to list topics and partition state (e.g. URPs).  It
>> >> > would be useful to query the controllers as the metadata source of
>> truth
>> >> > when we suspect that the broker metadata may have diverged.
>> >> >
>> >>
>> >> Fair enough. I will document that we can return topics.
>> >>
>> >> I also added a "future work" section about maybe using the controllers
>> as
>> >> bootstrap servers for the broker cluster. To be clear, that is NOT in
>> scope
>> >> here, but it's interesting to think about potentially doing in the
>> future.
>> >> The major problem is what to do when the broker endpoints we're
>> returning
>> >> have different security settings from the controller endpoint the client
>> >> initially talked to.
>> >>
>> >> best,
>> >> Colin
>> >>
>> >>
>> >> >
>> >> > Thanks,
>> >> > Jason
>> >> >
>> >> > On Thu, Apr 20, 2023 at 5:53 PM Colin McCabe <cm...@apache.org>
>> wrote:
>> >> >
>> >> >> On Wed, Apr 19, 2023, at 20:56, Philip Nee wrote:
>> >> >> > Hey Colin,
>> >> >> >
>> >> >> > I still need to finish reading and understanding the KIP, but I
>> have a
>> >> >> > couple of comments despite being ignorant of most of the KRaft
>> stuff.
>> >> >> > (Sorry!)
>> >> >> >
>> >> >> > Firstly, does it make sense to create an extension of the current
>> >> >> > AdminClient only to handle these specific KRaft use cases? It seems
>> >> >> > cumbersome to have two sets of bootstrap configurations to make the
>> >> >> > AdminClient generic enough to handle these specific cases, instead,
>> >> maybe
>> >> >> > it is more obvious (to me) to just extend the AdminClient. What I'm
>> >> >> > thinking is KraftAdminClient which continuously uses
>> >> *bootstrap.servers*,
>> >> >> > but make this class only serves the Kraft controllers APIs.
>> >> >>
>> >> >> Hi Philip,
>> >> >>
>> >> >> Thanks for taking a look.
>> >> >>
>> >> >> We would not want to create a new Admin client class in order to
>> >> >> communicate directly with the controllers. The RPCs accepted by the
>> >> >> controllers have the same format as the those accepted by the
>> brokers.
>> >> >> There is no difference in what is sent over the wire or the data
>> >> structures
>> >> >> that are used in the client.
>> >> >>
>> >> >> I know you mentioned you haven't had time to read all the KRaft stuff
>> >> (and
>> >> >> there is a lot, I understand). But this is one area that would
>> probably
>> >> be
>> >> >> clarified if you were able to read at least KIP-500 and KIP-590. RPCs
>> >> sent
>> >> >> to the broker are forwarded to the controller (mostly) without
>> >> modification.
>> >> >>
>> >> >> >
>> >> >> > Secondly, if we want to continue with the design, I'm not yet sure
>> >> why we
>> >> >> > can't continue using the *bootstrap.servers*? I assume when the
>> client
>> >> >> gets
>> >> >> > the metadata, it should know who it is talking to. I'm just
>> >> reconsidering
>> >> >> > your alternative again.
>> >> >> >
>> >> >> > A bad idea: Why don't we continue using *bootstrap.servers* but
>> have a
>> >> >> > separated config like *kraft.controller* = true/false. I feel like
>> >> most
>> >> >> > users might not know what is a controller and causes some mistakes
>> >> down
>> >> >> the
>> >> >> > road.
>> >> >> >
>> >> >>
>> >> >> Well, you called it a bad idea, and I can't help but agree! :)
>> >> >>
>> >> >> I think "the user might not know what a controller is" is a good sign
>> >> that
>> >> >> they shouldn't be interacting with the controller. Like many
>> AdminClient
>> >> >> APIs, this one is intended for use by administrators only. Most users
>> >> >> indeed should not need to know what a controller is or interact with
>> it
>> >> >> directly. If they do interact it should be very clear that they are
>> >> doing
>> >> >> so. The --controller-bootstrap flag makes it very clear, as does the
>> >> >> separate configuration.
>> >> >>
>> >> >> Let me give an example of the kind of problems that arise if you
>> want to
>> >> >> reuse bootstrap.servers for this purpose.
>> >> >>
>> >> >> If the user grasb a 3.4 Kafka AdminClient and set bootstrap.servers
>> to a
>> >> >> set of controller servers, and set direct.to.controller to true, the
>> >> >> unknown (in 3.4) configuration will be ignored, and a normal metadata
>> >> >> request will be sent without the direct to controller flag. In that
>> >> case it
>> >> >> will give back an error. Confusing, right?
>> >> >>
>> >> >> Using controller.servers clarifies this situation because the 3.4
>> client
>> >> >> will not support that config, and will complain about the lack of
>> >> >> bootstrap.servers.
>> >> >>
>> >> >> Actually, the situation could get even more confusing than what I
>> >> >> described since some older preproduction versions of the KRaft
>> >> controller
>> >> >> did implement the METADATA RPC. So if you send them a METADATA
>> request
>> >> >> without any special information, it's not clear what you'll get.
>> >> Indeed,
>> >> >> it would be dependent on the client version and the controller
>> version.
>> >> >>
>> >> >> The bottom line is that reusing the bootstrap.servers configuration
>> here
>> >> >> is not a good idea.
>> >> >>
>> >> >> best,
>> >> >> Colin
>> >> >>
>> >> >> > Thanks,
>> >> >> > P
>> >> >> >
>> >> >> > On Wed, Apr 19, 2023 at 2:18 PM Colin McCabe <cm...@apache.org>
>> >> wrote:
>> >> >> >
>> >> >> >> Hi all,
>> >> >> >>
>> >> >> >> I wrote a short KIP about allowing AdminClient to talk directly
>> with
>> >> the
>> >> >> >> KRaft controller quorum. Check it out here:
>> >> >> >>
>> >> >> >> https://cwiki.apache.org/confluence/x/Owo0Dw
>> >> >> >>
>> >> >> >> best,
>> >> >> >> Colin
>> >> >> >>
>> >> >>
>> >>
>>

Re: KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum

Posted by Luke Chen <sh...@gmail.com>.
Hi Colin,

Some comments:
1. I agree we should set "top-level" errors for metadata response

2. In the "brokers" field of metadata response from controller, it'll
respond with "Controller endpoint information as given in
controller.quorum.voters", instead of the "alive" controllers(voters). That
will break the existing admin client because in admin client, we'll rely on
the metadata response to build the "current alive brokers" list, and choose
one from them to connect (either least load or other criteria). That means,
if now, we return the value in `controller.quorum.voters`, but one of them
is down. We might choose it to connect and get connection errors. Should we
return the "alive" controllers(voters) to client?

3. In the KIP, we list the command-line tools will get a new
--bootstrap-controllers argument, but without explaining why these tools
need to talk to controller directly. Could we add some explanation about
them? I tried but cannot know why some tools are listed here:
    - kafka-acls.sh -> Allow clients to update ACLs via controller before
brokers up

    - kafka-cluster.sh -> Reasonable to get/update cluster info via
controller

    - kafka-configs.sh -> Allow clients to dynamically update
configs/describe configs from controller. But in this script, client can
still set quota for users/clients/topics... is client also able to update
via controllers? Or we only allow partial actions in the script to talk to
controllers?

    - kafka-delegation-tokens.sh -> Reasonable to update delegation-tokens
via controllers

    - kafka-features.sh -> Reasonable
    - kafka-metadata-quorum.sh -> Reasonable
    - kafka-metadata-shell.sh -> Reasonable

    - kafka-reassign-partitions.sh -> Why should we allow clients to move
metadata log partitions in controller nodes? What's the use-case?


Thank you.
Luke

On Thu, Apr 27, 2023 at 8:04 AM Colin McCabe <cm...@apache.org> wrote:

> On Tue, Apr 25, 2023, at 04:59, Divij Vaidya wrote:
> > Thank you for the KIP Colin.
> >
> > In general, I like the idea of having the ability to interact directly
> with
> > the controllers. I agree with your observation that it helps in
> situations
> > where you would want to get data directly from the controller instead of
> > going via a broker. I have some general comments but the main concern I
> > have is with the piggy-backing of error code with response of
> > __cluster_metadata topic.
> >
> > 1. With this change, how are we guarding against the possibility of
> > misbehaving client traffic from disrupting the controller (that you
> > mentioned as a motivation of earlier behaviour)? One solution could be to
> > have default values set for request throttling on the controller.
>
> Hi Divij,
>
> Thanks for the comments.
>
> Our guards against client misbehavior remain the same:
> 1. our recommendation to put the clients on a separate network
> 2. the fact that producers and consumers can't interact directly with the
> controller
> 3. the authorizer.
>
> Re: #3, I will spell out in the KIP that clients must have DESCRIBE on the
> CLUSTER resource to send a METADATA request to the controller.
>
> > 2. This KIP also increases the network attack surface area. Prior to this
> > KIP, it was possible to have firewall rules setup for controllers such
> that
> > only the brokers can talk to it. But now, the controller would require
> > access from other endpoints other than brokers as well. Can we add a
> > suggestion to the upgrade documentation and inform users
>
> There is no requirement for access from other endpoints. It is still
> possible to set up firewall rules such that only the brokers can talk to
> the controller. In fact I would even say this is desirable. Since this
> faculty is intended for infrequent manual administrative operations,
> needing to log into the broker to use it seems perfectly fine.
>
> > 3. In section KRaft Controller MetadataResponse, row 3, "There is no
> > top-level error code in MetadataResponse, so we use the
> __cluster_metadata
> > topic to send back our error.". This will definitely confuse the users.
> Can
> > we introduce a top level error field instead?
>
> Let me check how we're handling this in other places. I recall some other
> cases where we used the dummy topic approach, but I can't find them just
> now.
>
> > 4. As part of the KIP, could we please add some documentation for users
> > with the suggestion of when to get information directly from the
> controller
> > and when not to (and associated tradeoffs)?
>
> I think most users will not face this tradeoff because they simply won't
> have network access to the controller servers.
>
> For those who do want more information, we'll have command-line
> documentation for --boostrap-controllers and the bootstrap.controllers
> configuration key.
>
> > 5. Why do we need the `FromKRaftController` field in the response? What
> do
> > we expect the users to do with this information?
>
> The field is present so that we can throw an exception in the client if we
> have received a response that is not from the controller.
>
> > 6. Can we drop `KRaft` from the fields `FromKRaftController` and
> > `DirectToKRaftControllerQuorum`? My suggestion will be to rename it as
> > `DirectToController`.
>
> I like the idea, but wouldn't that create confusion in the ZK cluster case?
>
> > 7.  "kafka-metadata-shell.sh  is at an "evolving" level of interface
> > stability" -> I thought that with KRaft being production ready, the
> > evolving mode for kraft-related tools has also moved to production. Do we
> > have a timeline when this would move to production?
>
> That's a good question, but I think we should tackle it separately from
> this KIP. The metadata shell is pretty different from other parts of kafka
> since it interacts so closely with internals.
>
> best,
> Colin
>
>
> >
> > --
> > Divij Vaidya
> >
> >
> >
> > On Tue, Apr 25, 2023 at 1:38 AM Colin McCabe <cm...@apache.org> wrote:
> >
> >> On Fri, Apr 21, 2023, at 14:17, Jason Gustafson wrote:
> >> > Hey Colin,
> >> >
> >> > The KIP makes sense overall. Nice to clarify the contract between
> clients
> >> > and the controllers. The use of `DirectToKRaftControllerQuorum` will
> help
> >> > prevent misconfiguration. In fact, I wonder if we can return a fatal
> >> error
> >> > instead of NOT_CONTROLLER so that the client would immediately fail.
> For
> >> > example, could we use INVALID_REQUEST or something like that? Either
> that
> >> > or we need to make sure clients treat NOT_CONTROLLER as a fatal error.
> >> > Without that, it would probably get picked up with default retry logic
> >> and
> >> > the user might not see the problem.
> >>
> >> Hi Jason,
> >>
> >> Yes, this is a good point. It should return INVALID_REQUEST since that
> is
> >> not retryable. I'll change it.
> >>
> >> >
> >> > I also wonder if we can relax the requirements on the Metadata
> request so
> >> > that we can use it to list topics and partition state (e.g. URPs).  It
> >> > would be useful to query the controllers as the metadata source of
> truth
> >> > when we suspect that the broker metadata may have diverged.
> >> >
> >>
> >> Fair enough. I will document that we can return topics.
> >>
> >> I also added a "future work" section about maybe using the controllers
> as
> >> bootstrap servers for the broker cluster. To be clear, that is NOT in
> scope
> >> here, but it's interesting to think about potentially doing in the
> future.
> >> The major problem is what to do when the broker endpoints we're
> returning
> >> have different security settings from the controller endpoint the client
> >> initially talked to.
> >>
> >> best,
> >> Colin
> >>
> >>
> >> >
> >> > Thanks,
> >> > Jason
> >> >
> >> > On Thu, Apr 20, 2023 at 5:53 PM Colin McCabe <cm...@apache.org>
> wrote:
> >> >
> >> >> On Wed, Apr 19, 2023, at 20:56, Philip Nee wrote:
> >> >> > Hey Colin,
> >> >> >
> >> >> > I still need to finish reading and understanding the KIP, but I
> have a
> >> >> > couple of comments despite being ignorant of most of the KRaft
> stuff.
> >> >> > (Sorry!)
> >> >> >
> >> >> > Firstly, does it make sense to create an extension of the current
> >> >> > AdminClient only to handle these specific KRaft use cases? It seems
> >> >> > cumbersome to have two sets of bootstrap configurations to make the
> >> >> > AdminClient generic enough to handle these specific cases, instead,
> >> maybe
> >> >> > it is more obvious (to me) to just extend the AdminClient. What I'm
> >> >> > thinking is KraftAdminClient which continuously uses
> >> *bootstrap.servers*,
> >> >> > but make this class only serves the Kraft controllers APIs.
> >> >>
> >> >> Hi Philip,
> >> >>
> >> >> Thanks for taking a look.
> >> >>
> >> >> We would not want to create a new Admin client class in order to
> >> >> communicate directly with the controllers. The RPCs accepted by the
> >> >> controllers have the same format as the those accepted by the
> brokers.
> >> >> There is no difference in what is sent over the wire or the data
> >> structures
> >> >> that are used in the client.
> >> >>
> >> >> I know you mentioned you haven't had time to read all the KRaft stuff
> >> (and
> >> >> there is a lot, I understand). But this is one area that would
> probably
> >> be
> >> >> clarified if you were able to read at least KIP-500 and KIP-590. RPCs
> >> sent
> >> >> to the broker are forwarded to the controller (mostly) without
> >> modification.
> >> >>
> >> >> >
> >> >> > Secondly, if we want to continue with the design, I'm not yet sure
> >> why we
> >> >> > can't continue using the *bootstrap.servers*? I assume when the
> client
> >> >> gets
> >> >> > the metadata, it should know who it is talking to. I'm just
> >> reconsidering
> >> >> > your alternative again.
> >> >> >
> >> >> > A bad idea: Why don't we continue using *bootstrap.servers* but
> have a
> >> >> > separated config like *kraft.controller* = true/false. I feel like
> >> most
> >> >> > users might not know what is a controller and causes some mistakes
> >> down
> >> >> the
> >> >> > road.
> >> >> >
> >> >>
> >> >> Well, you called it a bad idea, and I can't help but agree! :)
> >> >>
> >> >> I think "the user might not know what a controller is" is a good sign
> >> that
> >> >> they shouldn't be interacting with the controller. Like many
> AdminClient
> >> >> APIs, this one is intended for use by administrators only. Most users
> >> >> indeed should not need to know what a controller is or interact with
> it
> >> >> directly. If they do interact it should be very clear that they are
> >> doing
> >> >> so. The --controller-bootstrap flag makes it very clear, as does the
> >> >> separate configuration.
> >> >>
> >> >> Let me give an example of the kind of problems that arise if you
> want to
> >> >> reuse bootstrap.servers for this purpose.
> >> >>
> >> >> If the user grasb a 3.4 Kafka AdminClient and set bootstrap.servers
> to a
> >> >> set of controller servers, and set direct.to.controller to true, the
> >> >> unknown (in 3.4) configuration will be ignored, and a normal metadata
> >> >> request will be sent without the direct to controller flag. In that
> >> case it
> >> >> will give back an error. Confusing, right?
> >> >>
> >> >> Using controller.servers clarifies this situation because the 3.4
> client
> >> >> will not support that config, and will complain about the lack of
> >> >> bootstrap.servers.
> >> >>
> >> >> Actually, the situation could get even more confusing than what I
> >> >> described since some older preproduction versions of the KRaft
> >> controller
> >> >> did implement the METADATA RPC. So if you send them a METADATA
> request
> >> >> without any special information, it's not clear what you'll get.
> >> Indeed,
> >> >> it would be dependent on the client version and the controller
> version.
> >> >>
> >> >> The bottom line is that reusing the bootstrap.servers configuration
> here
> >> >> is not a good idea.
> >> >>
> >> >> best,
> >> >> Colin
> >> >>
> >> >> > Thanks,
> >> >> > P
> >> >> >
> >> >> > On Wed, Apr 19, 2023 at 2:18 PM Colin McCabe <cm...@apache.org>
> >> wrote:
> >> >> >
> >> >> >> Hi all,
> >> >> >>
> >> >> >> I wrote a short KIP about allowing AdminClient to talk directly
> with
> >> the
> >> >> >> KRaft controller quorum. Check it out here:
> >> >> >>
> >> >> >> https://cwiki.apache.org/confluence/x/Owo0Dw
> >> >> >>
> >> >> >> best,
> >> >> >> Colin
> >> >> >>
> >> >>
> >>
>

Re: KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum

Posted by Colin McCabe <cm...@apache.org>.
On Tue, Apr 25, 2023, at 04:59, Divij Vaidya wrote:
> Thank you for the KIP Colin.
>
> In general, I like the idea of having the ability to interact directly with
> the controllers. I agree with your observation that it helps in situations
> where you would want to get data directly from the controller instead of
> going via a broker. I have some general comments but the main concern I
> have is with the piggy-backing of error code with response of
> __cluster_metadata topic.
>
> 1. With this change, how are we guarding against the possibility of
> misbehaving client traffic from disrupting the controller (that you
> mentioned as a motivation of earlier behaviour)? One solution could be to
> have default values set for request throttling on the controller.

Hi Divij,

Thanks for the comments.

Our guards against client misbehavior remain the same:
1. our recommendation to put the clients on a separate network
2. the fact that producers and consumers can't interact directly with the controller
3. the authorizer.

Re: #3, I will spell out in the KIP that clients must have DESCRIBE on the CLUSTER resource to send a METADATA request to the controller.

> 2. This KIP also increases the network attack surface area. Prior to this
> KIP, it was possible to have firewall rules setup for controllers such that
> only the brokers can talk to it. But now, the controller would require
> access from other endpoints other than brokers as well. Can we add a
> suggestion to the upgrade documentation and inform users

There is no requirement for access from other endpoints. It is still possible to set up firewall rules such that only the brokers can talk to the controller. In fact I would even say this is desirable. Since this faculty is intended for infrequent manual administrative operations, needing to log into the broker to use it seems perfectly fine.

> 3. In section KRaft Controller MetadataResponse, row 3, "There is no
> top-level error code in MetadataResponse, so we use the __cluster_metadata
> topic to send back our error.". This will definitely confuse the users. Can
> we introduce a top level error field instead?

Let me check how we're handling this in other places. I recall some other cases where we used the dummy topic approach, but I can't find them just now.

> 4. As part of the KIP, could we please add some documentation for users
> with the suggestion of when to get information directly from the controller
> and when not to (and associated tradeoffs)?

I think most users will not face this tradeoff because they simply won't have network access to the controller servers.

For those who do want more information, we'll have command-line documentation for --boostrap-controllers and the bootstrap.controllers configuration key.

> 5. Why do we need the `FromKRaftController` field in the response? What do
> we expect the users to do with this information?

The field is present so that we can throw an exception in the client if we have received a response that is not from the controller.

> 6. Can we drop `KRaft` from the fields `FromKRaftController` and
> `DirectToKRaftControllerQuorum`? My suggestion will be to rename it as
> `DirectToController`.

I like the idea, but wouldn't that create confusion in the ZK cluster case?

> 7.  "kafka-metadata-shell.sh  is at an "evolving" level of interface
> stability" -> I thought that with KRaft being production ready, the
> evolving mode for kraft-related tools has also moved to production. Do we
> have a timeline when this would move to production?

That's a good question, but I think we should tackle it separately from this KIP. The metadata shell is pretty different from other parts of kafka since it interacts so closely with internals.

best,
Colin


>
> --
> Divij Vaidya
>
>
>
> On Tue, Apr 25, 2023 at 1:38 AM Colin McCabe <cm...@apache.org> wrote:
>
>> On Fri, Apr 21, 2023, at 14:17, Jason Gustafson wrote:
>> > Hey Colin,
>> >
>> > The KIP makes sense overall. Nice to clarify the contract between clients
>> > and the controllers. The use of `DirectToKRaftControllerQuorum` will help
>> > prevent misconfiguration. In fact, I wonder if we can return a fatal
>> error
>> > instead of NOT_CONTROLLER so that the client would immediately fail. For
>> > example, could we use INVALID_REQUEST or something like that? Either that
>> > or we need to make sure clients treat NOT_CONTROLLER as a fatal error.
>> > Without that, it would probably get picked up with default retry logic
>> and
>> > the user might not see the problem.
>>
>> Hi Jason,
>>
>> Yes, this is a good point. It should return INVALID_REQUEST since that is
>> not retryable. I'll change it.
>>
>> >
>> > I also wonder if we can relax the requirements on the Metadata request so
>> > that we can use it to list topics and partition state (e.g. URPs).  It
>> > would be useful to query the controllers as the metadata source of truth
>> > when we suspect that the broker metadata may have diverged.
>> >
>>
>> Fair enough. I will document that we can return topics.
>>
>> I also added a "future work" section about maybe using the controllers as
>> bootstrap servers for the broker cluster. To be clear, that is NOT in scope
>> here, but it's interesting to think about potentially doing in the future.
>> The major problem is what to do when the broker endpoints we're returning
>> have different security settings from the controller endpoint the client
>> initially talked to.
>>
>> best,
>> Colin
>>
>>
>> >
>> > Thanks,
>> > Jason
>> >
>> > On Thu, Apr 20, 2023 at 5:53 PM Colin McCabe <cm...@apache.org> wrote:
>> >
>> >> On Wed, Apr 19, 2023, at 20:56, Philip Nee wrote:
>> >> > Hey Colin,
>> >> >
>> >> > I still need to finish reading and understanding the KIP, but I have a
>> >> > couple of comments despite being ignorant of most of the KRaft stuff.
>> >> > (Sorry!)
>> >> >
>> >> > Firstly, does it make sense to create an extension of the current
>> >> > AdminClient only to handle these specific KRaft use cases? It seems
>> >> > cumbersome to have two sets of bootstrap configurations to make the
>> >> > AdminClient generic enough to handle these specific cases, instead,
>> maybe
>> >> > it is more obvious (to me) to just extend the AdminClient. What I'm
>> >> > thinking is KraftAdminClient which continuously uses
>> *bootstrap.servers*,
>> >> > but make this class only serves the Kraft controllers APIs.
>> >>
>> >> Hi Philip,
>> >>
>> >> Thanks for taking a look.
>> >>
>> >> We would not want to create a new Admin client class in order to
>> >> communicate directly with the controllers. The RPCs accepted by the
>> >> controllers have the same format as the those accepted by the brokers.
>> >> There is no difference in what is sent over the wire or the data
>> structures
>> >> that are used in the client.
>> >>
>> >> I know you mentioned you haven't had time to read all the KRaft stuff
>> (and
>> >> there is a lot, I understand). But this is one area that would probably
>> be
>> >> clarified if you were able to read at least KIP-500 and KIP-590. RPCs
>> sent
>> >> to the broker are forwarded to the controller (mostly) without
>> modification.
>> >>
>> >> >
>> >> > Secondly, if we want to continue with the design, I'm not yet sure
>> why we
>> >> > can't continue using the *bootstrap.servers*? I assume when the client
>> >> gets
>> >> > the metadata, it should know who it is talking to. I'm just
>> reconsidering
>> >> > your alternative again.
>> >> >
>> >> > A bad idea: Why don't we continue using *bootstrap.servers* but have a
>> >> > separated config like *kraft.controller* = true/false. I feel like
>> most
>> >> > users might not know what is a controller and causes some mistakes
>> down
>> >> the
>> >> > road.
>> >> >
>> >>
>> >> Well, you called it a bad idea, and I can't help but agree! :)
>> >>
>> >> I think "the user might not know what a controller is" is a good sign
>> that
>> >> they shouldn't be interacting with the controller. Like many AdminClient
>> >> APIs, this one is intended for use by administrators only. Most users
>> >> indeed should not need to know what a controller is or interact with it
>> >> directly. If they do interact it should be very clear that they are
>> doing
>> >> so. The --controller-bootstrap flag makes it very clear, as does the
>> >> separate configuration.
>> >>
>> >> Let me give an example of the kind of problems that arise if you want to
>> >> reuse bootstrap.servers for this purpose.
>> >>
>> >> If the user grasb a 3.4 Kafka AdminClient and set bootstrap.servers to a
>> >> set of controller servers, and set direct.to.controller to true, the
>> >> unknown (in 3.4) configuration will be ignored, and a normal metadata
>> >> request will be sent without the direct to controller flag. In that
>> case it
>> >> will give back an error. Confusing, right?
>> >>
>> >> Using controller.servers clarifies this situation because the 3.4 client
>> >> will not support that config, and will complain about the lack of
>> >> bootstrap.servers.
>> >>
>> >> Actually, the situation could get even more confusing than what I
>> >> described since some older preproduction versions of the KRaft
>> controller
>> >> did implement the METADATA RPC. So if you send them a METADATA request
>> >> without any special information, it's not clear what you'll get.
>> Indeed,
>> >> it would be dependent on the client version and the controller version.
>> >>
>> >> The bottom line is that reusing the bootstrap.servers configuration here
>> >> is not a good idea.
>> >>
>> >> best,
>> >> Colin
>> >>
>> >> > Thanks,
>> >> > P
>> >> >
>> >> > On Wed, Apr 19, 2023 at 2:18 PM Colin McCabe <cm...@apache.org>
>> wrote:
>> >> >
>> >> >> Hi all,
>> >> >>
>> >> >> I wrote a short KIP about allowing AdminClient to talk directly with
>> the
>> >> >> KRaft controller quorum. Check it out here:
>> >> >>
>> >> >> https://cwiki.apache.org/confluence/x/Owo0Dw
>> >> >>
>> >> >> best,
>> >> >> Colin
>> >> >>
>> >>
>>

Re: KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum

Posted by Divij Vaidya <di...@gmail.com>.
Thank you for the KIP Colin.

In general, I like the idea of having the ability to interact directly with
the controllers. I agree with your observation that it helps in situations
where you would want to get data directly from the controller instead of
going via a broker. I have some general comments but the main concern I
have is with the piggy-backing of error code with response of
__cluster_metadata topic.

1. With this change, how are we guarding against the possibility of
misbehaving client traffic from disrupting the controller (that you
mentioned as a motivation of earlier behaviour)? One solution could be to
have default values set for request throttling on the controller.
2. This KIP also increases the network attack surface area. Prior to this
KIP, it was possible to have firewall rules setup for controllers such that
only the brokers can talk to it. But now, the controller would require
access from other endpoints other than brokers as well. Can we add a
suggestion to the upgrade documentation and inform users
3. In section KRaft Controller MetadataResponse, row 3, "There is no
top-level error code in MetadataResponse, so we use the __cluster_metadata
topic to send back our error.". This will definitely confuse the users. Can
we introduce a top level error field instead?
4. As part of the KIP, could we please add some documentation for users
with the suggestion of when to get information directly from the controller
and when not to (and associated tradeoffs)?
5. Why do we need the `FromKRaftController` field in the response? What do
we expect the users to do with this information?
6. Can we drop `KRaft` from the fields `FromKRaftController` and
`DirectToKRaftControllerQuorum`? My suggestion will be to rename it as
`DirectToController`.
7.  "kafka-metadata-shell.sh  is at an "evolving" level of interface
stability" -> I thought that with KRaft being production ready, the
evolving mode for kraft-related tools has also moved to production. Do we
have a timeline when this would move to production?

--
Divij Vaidya



On Tue, Apr 25, 2023 at 1:38 AM Colin McCabe <cm...@apache.org> wrote:

> On Fri, Apr 21, 2023, at 14:17, Jason Gustafson wrote:
> > Hey Colin,
> >
> > The KIP makes sense overall. Nice to clarify the contract between clients
> > and the controllers. The use of `DirectToKRaftControllerQuorum` will help
> > prevent misconfiguration. In fact, I wonder if we can return a fatal
> error
> > instead of NOT_CONTROLLER so that the client would immediately fail. For
> > example, could we use INVALID_REQUEST or something like that? Either that
> > or we need to make sure clients treat NOT_CONTROLLER as a fatal error.
> > Without that, it would probably get picked up with default retry logic
> and
> > the user might not see the problem.
>
> Hi Jason,
>
> Yes, this is a good point. It should return INVALID_REQUEST since that is
> not retryable. I'll change it.
>
> >
> > I also wonder if we can relax the requirements on the Metadata request so
> > that we can use it to list topics and partition state (e.g. URPs).  It
> > would be useful to query the controllers as the metadata source of truth
> > when we suspect that the broker metadata may have diverged.
> >
>
> Fair enough. I will document that we can return topics.
>
> I also added a "future work" section about maybe using the controllers as
> bootstrap servers for the broker cluster. To be clear, that is NOT in scope
> here, but it's interesting to think about potentially doing in the future.
> The major problem is what to do when the broker endpoints we're returning
> have different security settings from the controller endpoint the client
> initially talked to.
>
> best,
> Colin
>
>
> >
> > Thanks,
> > Jason
> >
> > On Thu, Apr 20, 2023 at 5:53 PM Colin McCabe <cm...@apache.org> wrote:
> >
> >> On Wed, Apr 19, 2023, at 20:56, Philip Nee wrote:
> >> > Hey Colin,
> >> >
> >> > I still need to finish reading and understanding the KIP, but I have a
> >> > couple of comments despite being ignorant of most of the KRaft stuff.
> >> > (Sorry!)
> >> >
> >> > Firstly, does it make sense to create an extension of the current
> >> > AdminClient only to handle these specific KRaft use cases? It seems
> >> > cumbersome to have two sets of bootstrap configurations to make the
> >> > AdminClient generic enough to handle these specific cases, instead,
> maybe
> >> > it is more obvious (to me) to just extend the AdminClient. What I'm
> >> > thinking is KraftAdminClient which continuously uses
> *bootstrap.servers*,
> >> > but make this class only serves the Kraft controllers APIs.
> >>
> >> Hi Philip,
> >>
> >> Thanks for taking a look.
> >>
> >> We would not want to create a new Admin client class in order to
> >> communicate directly with the controllers. The RPCs accepted by the
> >> controllers have the same format as the those accepted by the brokers.
> >> There is no difference in what is sent over the wire or the data
> structures
> >> that are used in the client.
> >>
> >> I know you mentioned you haven't had time to read all the KRaft stuff
> (and
> >> there is a lot, I understand). But this is one area that would probably
> be
> >> clarified if you were able to read at least KIP-500 and KIP-590. RPCs
> sent
> >> to the broker are forwarded to the controller (mostly) without
> modification.
> >>
> >> >
> >> > Secondly, if we want to continue with the design, I'm not yet sure
> why we
> >> > can't continue using the *bootstrap.servers*? I assume when the client
> >> gets
> >> > the metadata, it should know who it is talking to. I'm just
> reconsidering
> >> > your alternative again.
> >> >
> >> > A bad idea: Why don't we continue using *bootstrap.servers* but have a
> >> > separated config like *kraft.controller* = true/false. I feel like
> most
> >> > users might not know what is a controller and causes some mistakes
> down
> >> the
> >> > road.
> >> >
> >>
> >> Well, you called it a bad idea, and I can't help but agree! :)
> >>
> >> I think "the user might not know what a controller is" is a good sign
> that
> >> they shouldn't be interacting with the controller. Like many AdminClient
> >> APIs, this one is intended for use by administrators only. Most users
> >> indeed should not need to know what a controller is or interact with it
> >> directly. If they do interact it should be very clear that they are
> doing
> >> so. The --controller-bootstrap flag makes it very clear, as does the
> >> separate configuration.
> >>
> >> Let me give an example of the kind of problems that arise if you want to
> >> reuse bootstrap.servers for this purpose.
> >>
> >> If the user grasb a 3.4 Kafka AdminClient and set bootstrap.servers to a
> >> set of controller servers, and set direct.to.controller to true, the
> >> unknown (in 3.4) configuration will be ignored, and a normal metadata
> >> request will be sent without the direct to controller flag. In that
> case it
> >> will give back an error. Confusing, right?
> >>
> >> Using controller.servers clarifies this situation because the 3.4 client
> >> will not support that config, and will complain about the lack of
> >> bootstrap.servers.
> >>
> >> Actually, the situation could get even more confusing than what I
> >> described since some older preproduction versions of the KRaft
> controller
> >> did implement the METADATA RPC. So if you send them a METADATA request
> >> without any special information, it's not clear what you'll get.
> Indeed,
> >> it would be dependent on the client version and the controller version.
> >>
> >> The bottom line is that reusing the bootstrap.servers configuration here
> >> is not a good idea.
> >>
> >> best,
> >> Colin
> >>
> >> > Thanks,
> >> > P
> >> >
> >> > On Wed, Apr 19, 2023 at 2:18 PM Colin McCabe <cm...@apache.org>
> wrote:
> >> >
> >> >> Hi all,
> >> >>
> >> >> I wrote a short KIP about allowing AdminClient to talk directly with
> the
> >> >> KRaft controller quorum. Check it out here:
> >> >>
> >> >> https://cwiki.apache.org/confluence/x/Owo0Dw
> >> >>
> >> >> best,
> >> >> Colin
> >> >>
> >>
>

Re: KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum

Posted by Colin McCabe <cm...@apache.org>.
On Fri, Apr 21, 2023, at 14:17, Jason Gustafson wrote:
> Hey Colin,
>
> The KIP makes sense overall. Nice to clarify the contract between clients
> and the controllers. The use of `DirectToKRaftControllerQuorum` will help
> prevent misconfiguration. In fact, I wonder if we can return a fatal error
> instead of NOT_CONTROLLER so that the client would immediately fail. For
> example, could we use INVALID_REQUEST or something like that? Either that
> or we need to make sure clients treat NOT_CONTROLLER as a fatal error.
> Without that, it would probably get picked up with default retry logic and
> the user might not see the problem.

Hi Jason,

Yes, this is a good point. It should return INVALID_REQUEST since that is not retryable. I'll change it.

>
> I also wonder if we can relax the requirements on the Metadata request so
> that we can use it to list topics and partition state (e.g. URPs).  It
> would be useful to query the controllers as the metadata source of truth
> when we suspect that the broker metadata may have diverged.
>

Fair enough. I will document that we can return topics.

I also added a "future work" section about maybe using the controllers as bootstrap servers for the broker cluster. To be clear, that is NOT in scope here, but it's interesting to think about potentially doing in the future. The major problem is what to do when the broker endpoints we're returning have different security settings from the controller endpoint the client initially talked to.

best,
Colin


>
> Thanks,
> Jason
>
> On Thu, Apr 20, 2023 at 5:53 PM Colin McCabe <cm...@apache.org> wrote:
>
>> On Wed, Apr 19, 2023, at 20:56, Philip Nee wrote:
>> > Hey Colin,
>> >
>> > I still need to finish reading and understanding the KIP, but I have a
>> > couple of comments despite being ignorant of most of the KRaft stuff.
>> > (Sorry!)
>> >
>> > Firstly, does it make sense to create an extension of the current
>> > AdminClient only to handle these specific KRaft use cases? It seems
>> > cumbersome to have two sets of bootstrap configurations to make the
>> > AdminClient generic enough to handle these specific cases, instead, maybe
>> > it is more obvious (to me) to just extend the AdminClient. What I'm
>> > thinking is KraftAdminClient which continuously uses *bootstrap.servers*,
>> > but make this class only serves the Kraft controllers APIs.
>>
>> Hi Philip,
>>
>> Thanks for taking a look.
>>
>> We would not want to create a new Admin client class in order to
>> communicate directly with the controllers. The RPCs accepted by the
>> controllers have the same format as the those accepted by the brokers.
>> There is no difference in what is sent over the wire or the data structures
>> that are used in the client.
>>
>> I know you mentioned you haven't had time to read all the KRaft stuff (and
>> there is a lot, I understand). But this is one area that would probably be
>> clarified if you were able to read at least KIP-500 and KIP-590. RPCs sent
>> to the broker are forwarded to the controller (mostly) without modification.
>>
>> >
>> > Secondly, if we want to continue with the design, I'm not yet sure why we
>> > can't continue using the *bootstrap.servers*? I assume when the client
>> gets
>> > the metadata, it should know who it is talking to. I'm just reconsidering
>> > your alternative again.
>> >
>> > A bad idea: Why don't we continue using *bootstrap.servers* but have a
>> > separated config like *kraft.controller* = true/false. I feel like most
>> > users might not know what is a controller and causes some mistakes down
>> the
>> > road.
>> >
>>
>> Well, you called it a bad idea, and I can't help but agree! :)
>>
>> I think "the user might not know what a controller is" is a good sign that
>> they shouldn't be interacting with the controller. Like many AdminClient
>> APIs, this one is intended for use by administrators only. Most users
>> indeed should not need to know what a controller is or interact with it
>> directly. If they do interact it should be very clear that they are doing
>> so. The --controller-bootstrap flag makes it very clear, as does the
>> separate configuration.
>>
>> Let me give an example of the kind of problems that arise if you want to
>> reuse bootstrap.servers for this purpose.
>>
>> If the user grasb a 3.4 Kafka AdminClient and set bootstrap.servers to a
>> set of controller servers, and set direct.to.controller to true, the
>> unknown (in 3.4) configuration will be ignored, and a normal metadata
>> request will be sent without the direct to controller flag. In that case it
>> will give back an error. Confusing, right?
>>
>> Using controller.servers clarifies this situation because the 3.4 client
>> will not support that config, and will complain about the lack of
>> bootstrap.servers.
>>
>> Actually, the situation could get even more confusing than what I
>> described since some older preproduction versions of the KRaft controller
>> did implement the METADATA RPC. So if you send them a METADATA request
>> without any special information, it's not clear what you'll get.  Indeed,
>> it would be dependent on the client version and the controller version.
>>
>> The bottom line is that reusing the bootstrap.servers configuration here
>> is not a good idea.
>>
>> best,
>> Colin
>>
>> > Thanks,
>> > P
>> >
>> > On Wed, Apr 19, 2023 at 2:18 PM Colin McCabe <cm...@apache.org> wrote:
>> >
>> >> Hi all,
>> >>
>> >> I wrote a short KIP about allowing AdminClient to talk directly with the
>> >> KRaft controller quorum. Check it out here:
>> >>
>> >> https://cwiki.apache.org/confluence/x/Owo0Dw
>> >>
>> >> best,
>> >> Colin
>> >>
>>

Re: KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum

Posted by Jason Gustafson <ja...@confluent.io.INVALID>.
Hey Colin,

The KIP makes sense overall. Nice to clarify the contract between clients
and the controllers. The use of `DirectToKRaftControllerQuorum` will help
prevent misconfiguration. In fact, I wonder if we can return a fatal error
instead of NOT_CONTROLLER so that the client would immediately fail. For
example, could we use INVALID_REQUEST or something like that? Either that
or we need to make sure clients treat NOT_CONTROLLER as a fatal error.
Without that, it would probably get picked up with default retry logic and
the user might not see the problem.

I also wonder if we can relax the requirements on the Metadata request so
that we can use it to list topics and partition state (e.g. URPs).  It
would be useful to query the controllers as the metadata source of truth
when we suspect that the broker metadata may have diverged.

Thanks,
Jason

On Thu, Apr 20, 2023 at 5:53 PM Colin McCabe <cm...@apache.org> wrote:

> On Wed, Apr 19, 2023, at 20:56, Philip Nee wrote:
> > Hey Colin,
> >
> > I still need to finish reading and understanding the KIP, but I have a
> > couple of comments despite being ignorant of most of the KRaft stuff.
> > (Sorry!)
> >
> > Firstly, does it make sense to create an extension of the current
> > AdminClient only to handle these specific KRaft use cases? It seems
> > cumbersome to have two sets of bootstrap configurations to make the
> > AdminClient generic enough to handle these specific cases, instead, maybe
> > it is more obvious (to me) to just extend the AdminClient. What I'm
> > thinking is KraftAdminClient which continuously uses *bootstrap.servers*,
> > but make this class only serves the Kraft controllers APIs.
>
> Hi Philip,
>
> Thanks for taking a look.
>
> We would not want to create a new Admin client class in order to
> communicate directly with the controllers. The RPCs accepted by the
> controllers have the same format as the those accepted by the brokers.
> There is no difference in what is sent over the wire or the data structures
> that are used in the client.
>
> I know you mentioned you haven't had time to read all the KRaft stuff (and
> there is a lot, I understand). But this is one area that would probably be
> clarified if you were able to read at least KIP-500 and KIP-590. RPCs sent
> to the broker are forwarded to the controller (mostly) without modification.
>
> >
> > Secondly, if we want to continue with the design, I'm not yet sure why we
> > can't continue using the *bootstrap.servers*? I assume when the client
> gets
> > the metadata, it should know who it is talking to. I'm just reconsidering
> > your alternative again.
> >
> > A bad idea: Why don't we continue using *bootstrap.servers* but have a
> > separated config like *kraft.controller* = true/false. I feel like most
> > users might not know what is a controller and causes some mistakes down
> the
> > road.
> >
>
> Well, you called it a bad idea, and I can't help but agree! :)
>
> I think "the user might not know what a controller is" is a good sign that
> they shouldn't be interacting with the controller. Like many AdminClient
> APIs, this one is intended for use by administrators only. Most users
> indeed should not need to know what a controller is or interact with it
> directly. If they do interact it should be very clear that they are doing
> so. The --controller-bootstrap flag makes it very clear, as does the
> separate configuration.
>
> Let me give an example of the kind of problems that arise if you want to
> reuse bootstrap.servers for this purpose.
>
> If the user grasb a 3.4 Kafka AdminClient and set bootstrap.servers to a
> set of controller servers, and set direct.to.controller to true, the
> unknown (in 3.4) configuration will be ignored, and a normal metadata
> request will be sent without the direct to controller flag. In that case it
> will give back an error. Confusing, right?
>
> Using controller.servers clarifies this situation because the 3.4 client
> will not support that config, and will complain about the lack of
> bootstrap.servers.
>
> Actually, the situation could get even more confusing than what I
> described since some older preproduction versions of the KRaft controller
> did implement the METADATA RPC. So if you send them a METADATA request
> without any special information, it's not clear what you'll get.  Indeed,
> it would be dependent on the client version and the controller version.
>
> The bottom line is that reusing the bootstrap.servers configuration here
> is not a good idea.
>
> best,
> Colin
>
> > Thanks,
> > P
> >
> > On Wed, Apr 19, 2023 at 2:18 PM Colin McCabe <cm...@apache.org> wrote:
> >
> >> Hi all,
> >>
> >> I wrote a short KIP about allowing AdminClient to talk directly with the
> >> KRaft controller quorum. Check it out here:
> >>
> >> https://cwiki.apache.org/confluence/x/Owo0Dw
> >>
> >> best,
> >> Colin
> >>
>

Re: KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum

Posted by Colin McCabe <cm...@apache.org>.
On Wed, Apr 19, 2023, at 20:56, Philip Nee wrote:
> Hey Colin,
>
> I still need to finish reading and understanding the KIP, but I have a
> couple of comments despite being ignorant of most of the KRaft stuff.
> (Sorry!)
>
> Firstly, does it make sense to create an extension of the current
> AdminClient only to handle these specific KRaft use cases? It seems
> cumbersome to have two sets of bootstrap configurations to make the
> AdminClient generic enough to handle these specific cases, instead, maybe
> it is more obvious (to me) to just extend the AdminClient. What I'm
> thinking is KraftAdminClient which continuously uses *bootstrap.servers*,
> but make this class only serves the Kraft controllers APIs.

Hi Philip,

Thanks for taking a look.

We would not want to create a new Admin client class in order to communicate directly with the controllers. The RPCs accepted by the controllers have the same format as the those accepted by the brokers. There is no difference in what is sent over the wire or the data structures that are used in the client.

I know you mentioned you haven't had time to read all the KRaft stuff (and there is a lot, I understand). But this is one area that would probably be clarified if you were able to read at least KIP-500 and KIP-590. RPCs sent to the broker are forwarded to the controller (mostly) without modification.

>
> Secondly, if we want to continue with the design, I'm not yet sure why we
> can't continue using the *bootstrap.servers*? I assume when the client gets
> the metadata, it should know who it is talking to. I'm just reconsidering
> your alternative again.
>
> A bad idea: Why don't we continue using *bootstrap.servers* but have a
> separated config like *kraft.controller* = true/false. I feel like most
> users might not know what is a controller and causes some mistakes down the
> road.
>

Well, you called it a bad idea, and I can't help but agree! :)

I think "the user might not know what a controller is" is a good sign that they shouldn't be interacting with the controller. Like many AdminClient APIs, this one is intended for use by administrators only. Most users indeed should not need to know what a controller is or interact with it directly. If they do interact it should be very clear that they are doing so. The --controller-bootstrap flag makes it very clear, as does the separate configuration.

Let me give an example of the kind of problems that arise if you want to reuse bootstrap.servers for this purpose.

If the user grasb a 3.4 Kafka AdminClient and set bootstrap.servers to a set of controller servers, and set direct.to.controller to true, the unknown (in 3.4) configuration will be ignored, and a normal metadata request will be sent without the direct to controller flag. In that case it will give back an error. Confusing, right?

Using controller.servers clarifies this situation because the 3.4 client will not support that config, and will complain about the lack of bootstrap.servers.

Actually, the situation could get even more confusing than what I described since some older preproduction versions of the KRaft controller did implement the METADATA RPC. So if you send them a METADATA request without any special information, it's not clear what you'll get.  Indeed, it would be dependent on the client version and the controller version.

The bottom line is that reusing the bootstrap.servers configuration here is not a good idea.

best,
Colin

> Thanks,
> P
>
> On Wed, Apr 19, 2023 at 2:18 PM Colin McCabe <cm...@apache.org> wrote:
>
>> Hi all,
>>
>> I wrote a short KIP about allowing AdminClient to talk directly with the
>> KRaft controller quorum. Check it out here:
>>
>> https://cwiki.apache.org/confluence/x/Owo0Dw
>>
>> best,
>> Colin
>>

Re: KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum

Posted by Philip Nee <ph...@gmail.com>.
Hey Colin,

I still need to finish reading and understanding the KIP, but I have a
couple of comments despite being ignorant of most of the KRaft stuff.
(Sorry!)

Firstly, does it make sense to create an extension of the current
AdminClient only to handle these specific KRaft use cases? It seems
cumbersome to have two sets of bootstrap configurations to make the
AdminClient generic enough to handle these specific cases, instead, maybe
it is more obvious (to me) to just extend the AdminClient. What I'm
thinking is KraftAdminClient which continuously uses *bootstrap.servers*,
but make this class only serves the Kraft controllers APIs.

Secondly, if we want to continue with the design, I'm not yet sure why we
can't continue using the *bootstrap.servers*? I assume when the client gets
the metadata, it should know who it is talking to. I'm just reconsidering
your alternative again.

A bad idea: Why don't we continue using *bootstrap.servers* but have a
separated config like *kraft.controller* = true/false. I feel like most
users might not know what is a controller and causes some mistakes down the
road.

Thanks,
P

On Wed, Apr 19, 2023 at 2:18 PM Colin McCabe <cm...@apache.org> wrote:

> Hi all,
>
> I wrote a short KIP about allowing AdminClient to talk directly with the
> KRaft controller quorum. Check it out here:
>
> https://cwiki.apache.org/confluence/x/Owo0Dw
>
> best,
> Colin
>