Hi Lucas,
Thank you for the thoughtful review and for pointing out these discrepancies.

LB01:
You’re absolutely right — the intent of this KIP is only to move the existing 
application.server validation earlier (from KafkaStreams construction to 
StreamsConfig construction), without changing the current set of accepted 
values or failure cases. In the previous version of the KIP, I unintentionally 
included descriptions that could be read as tightening or otherwise changing 
the effective validation behavior.

To better reflect the intended intent, I updated KIP-1245 to remove those 
descriptions and to clarify that the validator must be behaviorally equivalent 
to the current runtime validation, with the only change being earlier failure 
reporting. I also intentionally did not mention HostInfo in the KIP to avoid 
tying the proposal to implementation details; I’ll cover those specifics in the 
PR.
Thanks again — I’d appreciate it if you could take another look when you get a 
chance.

Best regards,
Sanghyeok An


-----Original Message-----
From: "Lucas Brutschy via dev"<[email protected]>
To: <[email protected]>;
Cc: "Lucas Brutschy"<[email protected]>;
Sent: 2026-01-30 (금) 23:55:48 (GMT+09:00)
Subject: Re: [DISCUSS] KIP-1245 Enforce 'application.server' <server>:<port> 
format at config level

Hi,

Thanks for the KIP!

LB1: I mostly have questions about this KIP's valid values and the
ones allowed by Utils.getPort / Utils.getHost. The KIP states that
there are no new failure modes because validation was merely moved;
however, differences in the validation seem apparent.
LB1a) The KIP states it will validate "Port Range: It ensures the port
is a valid integer within the allowable range (0 to 65535).". However,
the current validation only parses an integer (Utils.getPort) and does
not validate the port range
LB1b) Similarly, we currently allow protocol://host:port, but the KIP
doesn't seem to allow it.
LB1c) The KIP says "The host part is treated as an opaque string." No
validation is performed on the host string itself.". But Utils.getHost
does perform validation, for example, it rejects "!!!:8080".

Cheers,
Lucas



On Fri, Jan 16, 2026 at 6:27 PM Nikita Shupletsov <[email protected]> wrote:
>
> 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
> >

Reply via email to