HI Sanghyeok, It makes sense to me to discuss the implementation details in the PR.
We can definitely look at bootstrap.servers separately. Overall, the KIP looks good to me, thanks a lot! On Tue, Jan 13, 2026 at 7:13 AM 안상혁 <[email protected]> wrote: > > Hi Nikita and Matthias! > Thank you both for your valuable feedback and insights and apologies for the > delayed response. > > To Nikita: > Thank you for suggesting ClientUtils. As you noted, reviewing it was very > helpful, and it appears to contain reusable logic. > I think it would be worth considering in the PR. If you are open to it, I > would appreciate discussing the implementation details (for example, > ClientUtils vs HostInfo) further in the PR! > What do you think? > > Regarding bootstrap.servers, I also find the idea interesting. > However, I agree with Matthias that including it in this KIP would expand the > scope too much. > Would it make sense to open a separate Jira ticket for this, and, depending > on interest, consider a follow-up KIP to gather broader feedback from the PMC > and committers? > > Also, I have updated the wiki accordingly following Matthias's opinion. > When you have time, could you take another look? > > To Matthias: > Thank you for taking the time to review this KIP. > I fully agree with your points regarding scope and the fail-fast approach. > > As suggested, I updated the KIP to reduce the emphasis on implementation > details. > Instead, I clarified the Proposed Changes by adding a Validation Scope > section. > This specifies that we will validate the endpoint format and port range, but > will not perform DNS lookups or hostname validation, to keep the scope > appropriately limited. > > I have updated the wiki accordingly. > When you have time, please take another look! > > Best regards, > Sanghyeok An. > > > -----Original Message----- > From: "Matthias J. Sax"<[email protected]> > To: <[email protected]>; > Cc: > Sent: 2026-01-13 (화) 10:05:29 (GMT+09:00) > Subject: Re: [DISCUSS] KIP-1245 Enforce 'application.server' <server>:<port> > format at config level > > Thanks for the KIP! Overall LGTM. > > I agree that we don't need a two-phase approach. > > Interesting question if we would want to verify `bootstrap.server` > earlier too. In general, I am open to this idea but wondering if it > would expand the scope of this KIP too much? Also, if we would want to > do this, would we want to do this for consumer/producer/admin client > too? The problem for theses clients is, that they only accept > `Properties` as a parameter but `ConsumerConfig` (even if technically > public API) is not a parameter the consumer constructor accepts and thus > fail-faster is not really possible w/o make larger changes. So for > consistency, it might be better to also keep Kafka Streams as-is for > now, and if we really want to do this, do it with it's own KIP covering > all clients? > > Re-using `parseAndValidateAddresses` is also an interesting idea, but > sounds more like an implementation detail? Don't think we would need to > make this part of the KIP discussion. -- Some functionality does not > really apply to `application.server` though, in particular the DNS > lookup part. In the end, `application.server` is a config that Kafka > Streams only distributes to all client to allow users to implement a IQ > routing laying, and the user code would use `application.server` to > actually open network connections, but not Kafka Streams. So maybe there > is no reason to go overboard? > > > -Matthias > > > > On 12/12/25 2:50 PM, Nikita Shupletsov wrote: > > Hi Sanghyeok, > > > > Thanks a lot for updating the KIP. > > LGTM, but I would like to ask someone more experienced to take a look > > at the approach. > > Also: the client has this this validation for bootstrap.servers: > > `org.apache.kafka.clients.ClientUtils#parseAndValidateAddresses`, > > which similar to what the KIP is proposing for application.server, but > > with a couple extra features on top. Have you seen it? also, as we > > want to fail fast if application.server is invalid, should we fail > > fast for bootstrap.servers? > > Sorry for asking one question at a time, I am learning on the go. > > > > On Tue, Dec 9, 2025 at 5:08 AM 안상혁 <[email protected]> wrote: > >> > >> Hi Nikita, > >> > >> Thanks again for pointing me to KIP-1161. > >> > >> I have updated the KIP as we discussed: > >> - Removed the two phase rollout with a warning first and an error later > >> - Added a ConfigDef.Validator for application.server in StreamsConfig > >> - Clarified that the validator reuses the existing HostInfo endpoint > >> parsing logic and only moves the existing failure earlier to configuration > >> time > >> > >> Since applications with an invalid application.server value already fail > >> today when HostInfo parses the endpoint, > >> this change should not affect any correctly working applications, but it > >> makes the error surface earlier and more explicit. > >> > >> When you have time, I would appreciate it if you could take another look > >> and let me know whether the updated text matches your expectations. > >> > >> Best regards, > >> Sanghyeok An >
