DUE TO SPAM, SIGN-UP IS DISABLED. Goto Selfserve wiki signup and request an account.
This page is meant as a template for writing a KIP. To create a KIP choose Tools->Copy on this page and modify with your content and replace the heading with the next KIP number and a description of your issue. Replace anything in italics with your own description.
Status
Current state: Under discussion
Discussion thread:
JIRA: here
Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).
Motivation
Currently, StreamsConfig accepts any string for application.server without validation at configuration time.
The check is deferred until the KafkaStreams client is constructed, which delays error detection in terms of "fast-fail".
At the same time, a misconfigured application.server already leads to a failure today – the application does not start successfully once HostInfo tries to parse the endpoint.
In other words, the current behaviour is already "fail on invalid endpoint", but the failure happens later and in a less obvious place.
This KIP proposes to move this validation into StreamsConfig itself by introducing a dedicated ConfigDef.Validator for application.server .
This aligns with the precedent set by KIP-1161 for bootstrap.servers: instead of having an implicit, late failure, we surface an explicit configuration error as early as possible.
This KIP does not change what constitutes a valid value for application.server , and therefore does not introduce new failure modes. It only moves detection of already-invalid values earlier, from KafkaStreams construction time to StreamsConfig construction time, to improve fast-fail and debuggability.
Public Interfaces
- class
org.apache.kafka.streams.StreamsConfig- Update
StreamsConfigto register this validatorApplicationServerConfigValidatorforapplication.server, so invalid values fail fast with ConfigException during config construction.
- Update
Proposed Changes
This KIP proposes to validate application.server at configuration time using a dedicated ConfigDef.Validator, while preserving the current runtime acceptance and rejection behavior.
The change only shifts detection of invalid values earlier.
1. Attach a validator to application.server in StreamsConfig
We add a new ConfigDef.Validator implementation, ApplicationServerConfigValidator class under org.apache.kafka.streams.internals.
This validator will ensure that the configured string is considered valid under the same rules used today at runtime, and will throw ConfigException only for values that would already fail at runtime.
- If
application.serveris an empty string, the validator accepts it as "not configured". For any non-empty value, the validator requires a value that would be accepted today at runtime and throws
ConfigExceptionotherwise.
2. Validation Scope
The validator must preserve the current acceptance and rejection behavior for application.server. It must be behaviorally equivalent to the validation that occurs today at runtime, and must not tighten or relax the effective validity criteria.
The set of values considered valid remains unchanged.
Values that are accepted today will continue to be accepted.
Values that fail today will continue to fail, but the failure will surface earlier as a
ConfigExceptionduringStreamsConfigconstruction.The validation is syntactic only. It does not perform DNS resolution, network reachability checks, or any other environmental validation.
This section is intentionally scoped to ensure the change only shifts the failure point earlier, without introducing new failure modes.
3. Fail fast at configuration time
With the validator attached, constructing a new StreamsConfig will throw ConfigException immediately when application.server is malformed.
This does not introduce a new failure mode: applications with invalid application.server values already fail today once HostInfo attempts to parse the endpoint.
The change only moves the failure point earlier, making it more explicit and easier to debug.
Compatibility, Deprecation, and Migration Plan
Backward Compatibility
This change preserves backward compatibility by keeping the validity rules for application.server unchanged.
Any configuration that is accepted today will continue to be accepted.
Any configuration that fails today will continue to fail, but the failure will surface earlier during StreamsConfig construction as a ConfigException.
No new classes of failure are introduced; the behavior is made more explicit and easier to reason about. Since invalid application.server values already prevent applications from running correctly, this change is a correctness and debuggability improvement rather than a semantic behavior change.
Migration
No explicit migration is required for correctly configured applications.
For applications that currently provide an application.server value that fails validation at runtime today,
Fix the configuration to a value that is accepted by the current runtime validation for application.server.
- Leave
application.serverempty if interactive queries are not used.
These configurations already cause the application to fail today once HostInfo parses application.server ; the change simply moves that failure earlier and makes it more explicit.
Test Plan
- Extend
StreamsConfigTest- Valid Formats: Verify that standard patterns are accepted, including:
Include representative values that are accepted at runtime today, including scheme-prefixed forms if applicable.
Examples: localhost:8080, 127.0.0.1:8080, [::1]:8080, etc.
Empty/unset
Verify that empty strings are accepted (treated as not configured).
- Invalid Formats: Verify that
ConfigExceptionis thrown for:Include representative values rejected by the current parsing rules to ensure behavioral equivalence.
Examples: missing port, non-numeric port, and inputs rejected by current host parsing such as !!!:8080.
- Valid Formats: Verify that standard patterns are accepted, including:
Rejected Alternatives
Two-phase rollout with a warning period
An earlier revision of this KIP proposed a two-phase rollout:
- Phase 1: log a WARN message for invalid application.server values but still accept them, and
- Phase 2: enforce strict validation and throw ConfigException.
After revisiting this approach and looking at KIP-1161 (which changed an implicit failure for an empty bootstrap.servers value to
an explicit configuration error without an intermediate warning phase), we decided that the two-phase strategy adds complexity without clear benefits.
Misconfigured application.server values already cause the application to fail today once HostInfo attempts to use them.
Introducing a warning-only phase would still leave users with a misconfigured application that does not behave correctly, while delaying the point at which the error becomes explicit.
Therefore, we chose the simpler design of enforcing validation immediately at configuration time.