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 <cmcc...@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 <show...@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 <cmcc...@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 <cmcc...@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 <cmcc...@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 < >> cmcc...@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 < >> >> > cmcc...@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