Hi Kafka Community,
I would like to start a discussion on KIP-1245 Enforce 'application.server' 
<server>:<port> format at config level.

Currently, StreamsConfig accepts any string for the application.server 
configuration without validation. 
The actual format check (<host>:<port>) is deferred until the KafkaStreams 
client is constructed. 
It would be beneficial to adopt a 'fail-fast' approach here. Also, From a 
design perspective, StreamsConfig should take responsibility for validating its 
own parameters, ensuring that the configuration object is valid upon creation.
This would help users identify failure points more clearly.

I propose moving this validation logic into StreamsConfig. However, enforcing 
strict validation immediately would be a breaking change, which is not 
desirable for a minor release. 
Therefore, this KIP suggests a phased approach:

Phase 1 (Target: Upcoming Minor Version): Modify StreamsConfig to check the 
format upon initialization. If the format is invalid, we will log a WARN 
message instead of throwing an exception. This serves as a deprecation notice 
for the current lenient behavior without breaking existing applications.

Phase 2 (Target: Kafka 5.0.0): Replace the warning logic with a strict 
ConfigDef.Validator. In this major release, invalid configurations will throw a 
ConfigException, strictly enforcing the format.


You can find the full details, including the implementation plan and 
compatibility notes, in here
KIP WIKI - https://cwiki.apache.org/confluence/x/WIPMFw

Thanks for your attention! 

Reply via email to