You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2021/09/02 21:36:08 UTC

[GitHub] [kafka] ccding opened a new pull request #11293: MINOR: defineInternal for KIP-405 configs

ccding opened a new pull request #11293:
URL: https://github.com/apache/kafka/pull/11293


   We haven't finished implementing KIP-405, therefore we should make
   KIP-405 configs as defineInternal.
   
   We may also want to port this change to 3.0 to avoid leaking these
   configs to the doc.
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] junrao merged pull request #11293: MINOR: defineInternal for KIP-405 configs

Posted by GitBox <gi...@apache.org>.
junrao merged pull request #11293:
URL: https://github.com/apache/kafka/pull/11293


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] junrao commented on a change in pull request #11293: MINOR: defineInternal for KIP-405 configs

Posted by GitBox <gi...@apache.org>.
junrao commented on a change in pull request #11293:
URL: https://github.com/apache/kafka/pull/11293#discussion_r712319893



##########
File path: core/src/main/scala/kafka/server/ZkAdminManager.scala
##########
@@ -118,7 +118,7 @@ class ZkAdminManager(val config: KafkaConfig,
     metadataAndConfigs.get(topicName).foreach { result =>
       val logConfig = LogConfig.fromProps(LogConfig.extractLogConfigMap(config), configs)
       val createEntry = configHelper.createTopicConfigEntry(logConfig, configs, includeSynonyms = false, includeDocumentation = false)(_, _)
-      val topicConfigs = logConfig.values.asScala.map { case (k, v) =>
+      val topicConfigs = logConfig.nonInternalValues.asScala.map { case (k, v) =>

Review comment:
       Do we need to fix any test related to this change?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ccding edited a comment on pull request #11293: MINOR: defineInternal for KIP-405 configs

Posted by GitBox <gi...@apache.org>.
ccding edited a comment on pull request #11293:
URL: https://github.com/apache/kafka/pull/11293#issuecomment-920247203


   ~~The comment of `defineInternal` says~~
   ```
   ~~     * Define a new internal configuration. Internal configuration won't show up in the docs and aren't~~
   ~~     * intended for general use.~~
   ```
   ~~I read it as the config should be available and visible to users, but not documented. Therefore, I changed the DescribeTopic call to return internal configs.~~
   
   Per offline discussion with @junrao , we changed CreateTopic to not return internal configs.
   
   PTAL @junrao 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ccding commented on pull request #11293: MINOR: defineInternal for KIP-405 configs

Posted by GitBox <gi...@apache.org>.
ccding commented on pull request #11293:
URL: https://github.com/apache/kafka/pull/11293#issuecomment-920248580


   Will cherry-pick this PR to 3.0 after it is merged to trunk


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ccding commented on pull request #11293: MINOR: defineInternal for KIP-405 configs

Posted by GitBox <gi...@apache.org>.
ccding commented on pull request #11293:
URL: https://github.com/apache/kafka/pull/11293#issuecomment-921051336


   fixed testDescribeAndAlterConfigs


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ccding commented on pull request #11293: MINOR: defineInternal for KIP-405 configs

Posted by GitBox <gi...@apache.org>.
ccding commented on pull request #11293:
URL: https://github.com/apache/kafka/pull/11293#issuecomment-914613398


   It failed here https://github.com/apache/kafka/blob/9d107c174bfb23e5ce0daca9a59796f018f627c6/core[…]t/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala
   ```
   Expected :HashSet(message.timestamp.difference.max.ms, follower.replication.throttled.replicas, file.delete.delay.ms, compression.type, unclean.leader.election.enable, flush.messages, delete.retention.ms, max.message.bytes, segment.index.bytes, segment.jitter.ms, min.cleanable.dirty.ratio, retention.bytes, message.downconversion.enable, max.compaction.lag.ms, min.compaction.lag.ms, flush.ms, cleanup.policy, message.timestamp.type, retention.ms, min.insync.replicas, message.format.version, leader.replication.throttled.replicas, preallocate, index.interval.bytes, segment.bytes, segment.ms)
   Actual   :HashSet(message.timestamp.difference.max.ms, follower.replication.throttled.replicas, file.delete.delay.ms, compression.type, unclean.leader.election.enable, flush.messages, delete.retention.ms, local.retention.ms, max.message.bytes, segment.index.bytes, segment.jitter.ms, min.cleanable.dirty.ratio, retention.bytes, message.downconversion.enable, max.compaction.lag.ms, min.compaction.lag.ms, remote.storage.enable, flush.ms, cleanup.policy, message.timestamp.type, retention.ms, min.insync.replicas, message.format.version, leader.replication.throttled.replicas, local.retention.bytes, preallocate, index.interval.bytes, segment.bytes, segment.ms)
   ```
   the diff between Expected and Actual is the three defineInternal configs.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] junrao commented on pull request #11293: MINOR: defineInternal for KIP-405 configs

Posted by GitBox <gi...@apache.org>.
junrao commented on pull request #11293:
URL: https://github.com/apache/kafka/pull/11293#issuecomment-923147112


   Also cherry-picked the PR to 3.0 branch.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ccding edited a comment on pull request #11293: MINOR: defineInternal for KIP-405 configs

Posted by GitBox <gi...@apache.org>.
ccding edited a comment on pull request #11293:
URL: https://github.com/apache/kafka/pull/11293#issuecomment-920247203


   The comment of `defineInternal` says
   ```
        * Define a new internal configuration. Internal configuration won't show up in the docs and aren't
        * intended for general use.
   ```
   I read it as the config should be available and visible to users, but not documented. Therefore, I changed the DescribeTopic call to return internal configs.
   
   PTAL @junrao 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ccding edited a comment on pull request #11293: MINOR: defineInternal for KIP-405 configs

Posted by GitBox <gi...@apache.org>.
ccding edited a comment on pull request #11293:
URL: https://github.com/apache/kafka/pull/11293#issuecomment-920247203


   ~~The comment of `defineInternal` says~~
   ```
        * Define a new internal configuration. Internal configuration won't show up in the docs and aren't
        * intended for general use.
   ```
   ~~I read it as the config should be available and visible to users, but not documented. Therefore, I changed the DescribeTopic call to return internal configs.~~
   
   Per offline discussion with @junrao , we changed CreateTopic to not return internal configs.
   
   PTAL @junrao 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ccding edited a comment on pull request #11293: MINOR: defineInternal for KIP-405 configs

Posted by GitBox <gi...@apache.org>.
ccding edited a comment on pull request #11293:
URL: https://github.com/apache/kafka/pull/11293#issuecomment-914613398


   It failed here https://github.com/apache/kafka/blob/9d107c174bfb23e5ce0daca9a59796f018f627c6/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala#L432
   ```
   Expected :HashSet(message.timestamp.difference.max.ms, follower.replication.throttled.replicas, file.delete.delay.ms, compression.type, unclean.leader.election.enable, flush.messages, delete.retention.ms, max.message.bytes, segment.index.bytes, segment.jitter.ms, min.cleanable.dirty.ratio, retention.bytes, message.downconversion.enable, max.compaction.lag.ms, min.compaction.lag.ms, flush.ms, cleanup.policy, message.timestamp.type, retention.ms, min.insync.replicas, message.format.version, leader.replication.throttled.replicas, preallocate, index.interval.bytes, segment.bytes, segment.ms)
   Actual   :HashSet(message.timestamp.difference.max.ms, follower.replication.throttled.replicas, file.delete.delay.ms, compression.type, unclean.leader.election.enable, flush.messages, delete.retention.ms, local.retention.ms, max.message.bytes, segment.index.bytes, segment.jitter.ms, min.cleanable.dirty.ratio, retention.bytes, message.downconversion.enable, max.compaction.lag.ms, min.compaction.lag.ms, remote.storage.enable, flush.ms, cleanup.policy, message.timestamp.type, retention.ms, min.insync.replicas, message.format.version, leader.replication.throttled.replicas, local.retention.bytes, preallocate, index.interval.bytes, segment.bytes, segment.ms)
   ```
   the diff between Expected and Actual is the three defineInternal configs.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ccding commented on pull request #11293: MINOR: defineInternal for KIP-405 configs

Posted by GitBox <gi...@apache.org>.
ccding commented on pull request #11293:
URL: https://github.com/apache/kafka/pull/11293#issuecomment-920247203


   The commend of `defineInternal` says
   ```
        * Define a new internal configuration. Internal configuration won't show up in the docs and aren't
        * intended for general use.
   ```
   I read it as the config should be available and visible to users, but not documented. Therefore, I changed the DescribeTopic call to return internal configs.
   
   PTAL @junrao 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ccding commented on pull request #11293: MINOR: defineInternal for KIP-405 configs

Posted by GitBox <gi...@apache.org>.
ccding commented on pull request #11293:
URL: https://github.com/apache/kafka/pull/11293#issuecomment-923097186


   Failed tests passed on my local run
   ```
   Build / JDK 8 and Scala 2.12 / [1] Type=Raft, Name=testTopicPartition, Security=PLAINTEXT – kafka.admin.LeaderElectionCommandTest
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ccding commented on pull request #11293: MINOR: defineInternal for KIP-405 configs

Posted by GitBox <gi...@apache.org>.
ccding commented on pull request #11293:
URL: https://github.com/apache/kafka/pull/11293#issuecomment-921046676


   Actually this cause a new failure I have to fix
   ```
   Build / JDK 17 and Scala 2.13 / testDescribeAndAlterConfigs() – kafka.api.PlaintextAdminIntegrationTest
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ccding commented on pull request #11293: MINOR: defineInternal for KIP-405 configs

Posted by GitBox <gi...@apache.org>.
ccding commented on pull request #11293:
URL: https://github.com/apache/kafka/pull/11293#issuecomment-912200659


   This PR fails `testCreateTopicsResponseMetadataAndConfig()`. DescribeTopic doesn't return the internal configs, while the response of CreateTopic includes the internal configs.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ccding commented on a change in pull request #11293: MINOR: defineInternal for KIP-405 configs

Posted by GitBox <gi...@apache.org>.
ccding commented on a change in pull request #11293:
URL: https://github.com/apache/kafka/pull/11293#discussion_r712339044



##########
File path: core/src/main/scala/kafka/server/ZkAdminManager.scala
##########
@@ -118,7 +118,7 @@ class ZkAdminManager(val config: KafkaConfig,
     metadataAndConfigs.get(topicName).foreach { result =>
       val logConfig = LogConfig.fromProps(LogConfig.extractLogConfigMap(config), configs)
       val createEntry = configHelper.createTopicConfigEntry(logConfig, configs, includeSynonyms = false, includeDocumentation = false)(_, _)
-      val topicConfigs = logConfig.values.asScala.map { case (k, v) =>
+      val topicConfigs = logConfig.nonInternalValues.asScala.map { case (k, v) =>

Review comment:
       No tests need to be fixed.
   
   The test that covers this code is `testCreateTopicsResponseMetadataAndConfig`. The test makes sure the returned values of CreateTopic and DescribeTopic are the same. Previously, the behavior of CreateTopic and DescribeTopic are different for internal configs, but the test passed because we don't have internal configs in LogConfig. Now, we have internal configs and this test failed, then I changed this line to let the test pass: CreateTopic and DescribeTopic both don't return internal configs.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org