Skip to content

Conversation

@m1a2st
Copy link
Collaborator

@m1a2st m1a2st commented Aug 10, 2025

We add the three main changes in this PR

  • Disallowing null values for most LIST-type configurations makes sense,
    since users cannot explicitly set a configuration to null in a
    properties file. Therefore, only configurations with a default value of
    null should be allowed to accept null.
  • Disallowing duplicate values is reasonable, as there are currently no
    known configurations in Kafka that require specifying the same value
    multiple times. Allowing duplicates is both rare in practice and
    potentially confusing to users.
  • Disallowing empty list, even though many configurations currently
    accept them. In practice, setting an empty list for several of these
    configurations can lead to server startup failures or unexpected
    behavior. Therefore, enforcing non-empty lists helps prevent
    misconfiguration and improves system robustness.
    These changes may introduce some backward incompatibility, but this
    trade-off is justified by the significant improvements in safety,
    consistency, and overall user experience.

Additionally, we introduce two minor adjustments:

  • Reclassify some STRING-type configurations as LIST-type, particularly
    those using comma-separated values to represent multiple entries. This
    change reflects the actual semantics used in Kafka.
  • Update the default values for some configurations to better align with
    other configs.
    These changes will not introduce any compatibility issues.

Reviewers: Jun Rao junrao@gmail.com, Chia-Ping Tsai
chia7712@gmail.com

@chia7712
Copy link
Member

The screenshot of the upgrade.html page

@m1a2st Could you please highlight the changes of cleanup.policy?

@m1a2st
Copy link
Collaborator Author

m1a2st commented Aug 31, 2025

CleanShot 2025-08-31 at 15 59 31

@chia7712
Copy link
Member

chia7712 commented Sep 4, 2025

I'm running an e2e for this patch. Will merge it if everything looks good

@chia7712
Copy link
Member

chia7712 commented Sep 5, 2025

network_degrade_test.py and replica_verification_test.py remain unstable. Others pass on my local.

@chia7712 chia7712 merged commit 0a12eaa into apache:trunk Sep 5, 2025
23 checks passed
Copy link
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@m1a2st : Thanks for the updated PR. Just a few minor comments.

assertEquals(1, MockConsumerInterceptor.CONFIG_COUNT.get());
assertEquals(1, MockConsumerInterceptor.CLOSE_COUNT.get());

assertEquals(1, CloseInterceptor.CLOSE_COUNT.get());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is an existing issue. So, the interceptors are called in reverse ordering?

Copy link
Collaborator Author

@m1a2st m1a2st Sep 6, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, this is an existing issue, I can file a Jira issue to trace it.

Updated: If the ordering is changed so that MockConsumerInterceptor comes first and CloseInterceptor second, then when MockConsumerInterceptor throws an exception, CloseInterceptor will not be initialized. As a result, its close() method will not be executed. The following test will be passed

    @EnumSource(GroupProtocol.class)
    public void testInterceptorConstructorConfigurationWithExceptionShouldCloseRemainingInstances(GroupProtocol groupProtocol) {
        final int targetInterceptor = 1;

        try {
            Properties props = new Properties();
            // skip
            props.setProperty(ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG,
                    MockConsumerInterceptor.class.getName() + "," + CloseInterceptor.class.getName());

            MockConsumerInterceptor.setThrowOnConfigExceptionThreshold(targetInterceptor);

            assertThrows(KafkaException.class, () -> newConsumer(
                    props, new StringDeserializer(), new StringDeserializer()));

            assertEquals(1, MockConsumerInterceptor.CONFIG_COUNT.get());
            assertEquals(1, MockConsumerInterceptor.CLOSE_COUNT.get());

            assertEquals(0, CloseInterceptor.INIT_COUNT.get());
            assertEquals(0, CloseInterceptor.CONFIG_COUNT.get());
            assertEquals(0, CloseInterceptor.CLOSE_COUNT.get());
        } finally {
            MockConsumerInterceptor.resetCounters();
            CloseInterceptor.resetCounters();
        }
    }
    
    public static class CloseInterceptor implements ConsumerInterceptor<String, String> {

        public static final AtomicInteger CLOSE_COUNT = new AtomicInteger(0);
        public static final AtomicInteger INIT_COUNT = new AtomicInteger(0);
        public static final AtomicInteger CONFIG_COUNT = new AtomicInteger(0);

        public CloseInterceptor() {
            INIT_COUNT.incrementAndGet();
        }

       // skip

        @Override
        public void close() {
            CLOSE_COUNT.incrementAndGet();
        }

        @Override
        public void configure(Map<String, ?> configs) {
            CONFIG_COUNT.incrementAndGet();
        }

        public static void resetCounters() {
            CLOSE_COUNT.set(0);
        }
    }

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Excuse me, I may have misunderstood your discussion. In this test case, the MockConsumerInterceptor is created after the CloseInterceptor. As a result, CloseInterceptor.CLOSE_COUNT is updated because the getConfiguredInstances method closes the created objects in the exception handler

settings.
</li>
<li>
The <code>cleanup.policy</code> is empty and <code>remote.storage.enable</code> is set to true, the
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The cleanup.policy is empty => If cleanup.policy is empty

Also, could we also add "cleanup.policy supports empty, which means infinite retention."

} else {
// If cleanup.policy is empty and remote storage is disabled, we should not delete any local
// log segments
return deleteLogStartOffsetBreachedSegments();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There's another issue we missed. the RPC DeleteRecordsRequest is not supported if the delete mode is disabled. If this KIP declares that the empty policy is supported, should we also allow DeleteRecordsRequest to pass on topics with the empty policy.

        if (!leaderLog.config.delete)
          throw new PolicyViolationException(s"Records of partition $topicPartition can not be deleted due to the configured policy")

if (!leaderLog.config.delete)

Copy link
Collaborator Author

@m1a2st m1a2st Sep 8, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Agree, we should allow an empty cleanup.policy when receiving a DeleteRecordsRequest. I will update this change in the follow up PR.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please add the changes to #20492

.define(PLUGIN_PATH_CONFIG,
Type.LIST,
null,
ConfigDef.ValidList.anyNonDuplicateValues(false, true),
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

quoted from KIP-898

This will be equivalent to extracting the plugin.path configuration from the worker properties and specifying --plugin-path .

Perhaps we should add manual check for the tool.

String pluginPath = properties.getProperty(WorkerConfig.PLUGIN_PATH_CONFIG);

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@majialoong Could you please file a minor patch for it?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@majialoong Could you please file a minor patch for it?

Sure, I'll fix it later.

chia7712 pushed a commit that referenced this pull request Oct 6, 2025
[In this PR](#20334), we added some
validation checks for the connect config, such as ensuring that
`plugin.path` cannot be empty.

 However, currently, Connect first loads the plugin and then creates the
configuration. Even if `plugin.path` is empty, it still attempts to load
the plugin first, and then throws an exception when creating the
configuration.

The approach should be to first create a configuration to validate that
the config meet the requirements, and then load the plugin only if the
validation passes. This allows for early detection of problems and
avoids unnecessary plugin loading processes.

Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
ConfigException exception5 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndNull.ensureValid("test.config", List.of("")));
assertEquals("Configuration 'test.config' values must not be empty.", exception5.getMessage());

ConfigDef.ValidList allowAnyNonDuplicateValuesAndEmptyList = ConfigDef.ValidList.anyNonDuplicateValues(true, false);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is there a test for anyNonDuplicateValues(false, false);

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I file a patch for it #20844

Copy link
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@m1a2st and @chia7712 : Left another followup comment.

<ul>
<li>Null values are no longer accepted for most LIST-type configurations, except those that explicitly
allow a null default value or where a null value has a well-defined semantic meaning.</li>
<li>Duplicate entries within the same list are no longer permitted.</li>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It seems that configs like HeaderFrom.HEADERS_FIELD still allow duplicates. So, we want to say sth like "Duplicate entries are no longer accepted for most LIST-type configurations, except for those that explicitly stated". We also want to change the description for HeaderFrom.HEADERS_FIELD to explicitly say that duplicates are allowed.

Thinking a bit more about this. Since we typically only introduce breaking changes in major releases, it might be useful to take a more conservative approach by only logging a warning for duplicated entries for now and rejecting them starting in 5.0.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks, I also addressed in #20844

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@m1a2st please update the title of #20844

Also, please open a jira for the breaking change mentioned by @junrao

Copy link
Member

@chia7712 chia7712 Nov 9, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thinking a bit more about this. Since we typically only introduce breaking changes in major releases, it might be useful to take a more conservative approach by only logging a warning for duplicated entries for now and rejecting them starting in 5.0.

perhaps we should print warnings instead of throwing exception in 4.x.

            if (Set.copyOf(values).size() != values.size()) {
                throw new ConfigException("Configuration '" + name + "' values must not be duplicated.");
            }

and then we could change it to exception in 5.x

@junrao @m1a2st WDYT?

Copy link
Collaborator Author

@m1a2st m1a2st Nov 9, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thinking a bit more about this. Since we typically only introduce breaking changes in major releases, it might be useful to take a more conservative approach by only logging a warning for duplicated entries for now and rejecting them starting in 5.0.

For HeaderFrom fields, headers
Both configurations should allow duplicate values. we can see the test case for reference.

perhaps we should print warnings instead of throwing exception in 4.x.

If we want to avoid breaking compatibility due to the stricter duplicate-value validation, printing a warning message is a good option.

We could also update the following configurations, which should include duplicate-value warnings as well, since their validators are customized.

TopicCreationConfig#INCLUDE_REGEX_CONFIG
TopicCreationConfig#EXCLUDE_REGEX_CONFIG
DistributedConfig#INTER_WORKER_VERIFICATION_ALGORITHMS_CONFIG
RestServerConfig#LISTENERS_DEFAULT
Cast#SPEC_CONFIG
ReplaceField#RENAMES
QuorumConfig#QUORUM_VOTERS_CONFIG
QuorumConfig#QUORUM_BOOTSTRAP_SERVERS_CONFIG
QuotaConfig#LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG
QuotaConfig#FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'd like to revisit the lower bound change. We face a core conflict: ignoring the dynamical value creates broker inconsistency, while skipping validation renders the boundary useless. In fact, this appears to be a hard limit we encounter when developing configuration.

Could we address this by allowing configuration breaking changes in a major release, but offering a tool to generate a compatibility report? This would enable users to adjust their configuration for the next major release before updating the binary.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is definitely a possible way forward. If v4.last had a tool to generate a compatibility report, or a way to generate the report during broker start, that would help users migrate to 5.0 without us having to violate the consistency checks for the newly incompatible configs.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If v4.last had a tool to generate a compatibility report,

I assume the tool will be implemented in 4.3 and will be enhanced with more checks over time. Users will be able to ask it whether their configuration file and exising cluster are compatible with its version

that would help users migrate to 5.0 without us having to violate the consistency checks for the newly incompatible configs.

Users will still violate the incompatible configs if they fail to adjust their existing cluster (or configuration file) before upgrading

The tool could also address KAFKA-19851. The compatibility report should remind users that message.format.version is no longer supported in the new version, prompting them to use the Admin Client to remove the configuration before upgrading to 4.x

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

To move 4.2 forward, I filed KAFKA-19879 to discuss compatibility for configuration changes. Also, @m1a2st will fix the breaking changes ( duplicate items ) in the subsequent PR #20844

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

100%. It's never ok to fail to start a broker due to existing config state.

I think we want to be reasonable about that. Currently, if one has a ZK-only based config, one can't start that broker in 4.0. But we provide warning and a migration path in the release notes. For duplicate config entries, it is not to hard to deal with it in a non-breaking way. But for things that are too hard to maintain compatibility, it's probably better to allow a breaking change, but provide enough warning and a migration path.

@AndrewJSchofield
Copy link
Member

There's a blocker defect related to this KIP. https://issues.apache.org/jira/browse/KAFKA-19875

If I create a topic in an earlier release with a duplicated config, and then try to start the broker with AK 4.2, the broker fails to start because the stricter validation is applied. @m1a2st

@chia7712
Copy link
Member

If I create a topic in an earlier release with a duplicated config, and then try to start the broker with AK 4.2, the broker fails to start because the stricter validation is applied.

yes - That is exactly what we are discussing in #20334 (comment).

eduwercamacaro pushed a commit to littlehorse-enterprises/kafka that referenced this pull request Nov 12, 2025
…alues (apache#20334)

We add the three main changes in this PR

- Disallowing null values for most LIST-type configurations makes sense,
since users cannot explicitly set a configuration to null in a
properties file. Therefore, only configurations with a default value of
null should be allowed to accept null.
- Disallowing duplicate values is reasonable, as there are currently no
known configurations in Kafka that require specifying the same value
multiple times. Allowing duplicates is both rare in practice and
potentially confusing to users.
- Disallowing empty list, even though many configurations currently
accept them. In practice, setting an empty list for several of these
configurations can lead to server startup failures or unexpected
behavior. Therefore, enforcing non-empty lists helps prevent
misconfiguration and improves system robustness.
These changes may introduce some backward incompatibility, but this
trade-off is justified by the significant improvements in safety,
consistency, and overall user experience.

Additionally, we introduce two minor adjustments:

- Reclassify some STRING-type configurations as LIST-type, particularly
those using comma-separated values to represent multiple entries. This
change reflects the actual semantics used in Kafka.
- Update the default values for some configurations to better align with
other configs.
These changes will not introduce any compatibility issues.

Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
eduwercamacaro pushed a commit to littlehorse-enterprises/kafka that referenced this pull request Nov 12, 2025
[In this PR](apache#20334), we added some
validation checks for the connect config, such as ensuring that
`plugin.path` cannot be empty.

 However, currently, Connect first loads the plugin and then creates the
configuration. Even if `plugin.path` is empty, it still attempts to load
the plugin first, and then throws an exception when creating the
configuration.

The approach should be to first create a configuration to validate that
the config meet the requirements, and then load the plugin only if the
validation passes. This allows for early detection of problems and
avoids unnecessary plugin loading processes.

Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
chia7712 pushed a commit that referenced this pull request Nov 14, 2025
…sh (#20638)

[KAFKA-19112](#20334) defines the
`plugin.path` config as non-empty. This PR add validation for
`plugin.path` related config in the `connect-plugin-path.sh` tool to
satisfy the same non-empty semantics, addressing the issue [mentioned
here](https://github.com/apache/kafka/pull/20334/files#r2395222944).

Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

6 participants