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/06/05 21:18:55 UTC

[GitHub] [kafka] ijuma opened a new pull request #10828: MINOR: Only log overridden topic configs during topic creation

ijuma opened a new pull request #10828:
URL: https://github.com/apache/kafka/pull/10828


   I think this behavior changed unintentionally as part of the kraft work.
   Also make sure to redact sensitive and unknown config values.
   
   Unit test included.
   
   ### 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.

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



[GitHub] [kafka] ijuma merged pull request #10828: MINOR: Only log overridden topic configs during topic creation

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


   


-- 
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.

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



[GitHub] [kafka] ijuma commented on pull request #10828: MINOR: Only log overridden topic configs during topic creation

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


   > Also I found you opened this PR in Kafka repo, not your personal repo. It should be a miss. FYI.
   
   We typically use our forks as a convention, but there is no rule against using the main repo if you are a committer.


-- 
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.

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



[GitHub] [kafka] rondagostino commented on pull request #10828: MINOR: Only log overridden topic configs during topic creation

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


   @ijuma The relevant KRaft PRs are https://github.com/apache/kafka/pull/10039 and https://github.com/apache/kafka/pull/10005 (more the latter than the former, though).  I believe this behavior did not change with respect to what is retrieved via `val config = fetchLogConfig(topicPartition.topic)`; I think originally the code would invoke the config provider, which was defined like this:
   
   ```
       val configProvider = new TopicConfigFetcher {
         override def fetch(): Properties = {
           val adminZkClient = new AdminZkClient(replicaManager.zkClient)
           adminZkClient.fetchEntityConfig(ConfigType.Topic, topicPartition.topic)
         }
       }
   ```
   
   Now it just asks the config repository for the topic config.  And then, regardless of what it got, whether via `val props = topicConfigProvider.fetch()` or `val props = configRepository.topicConfig(topic)`, it would invoke this to get the LogConfig:
   
   ```
         LogConfig.fromProps(logManager.currentDefaultConfig.originals, props)
   ```
   
   And then when it logged it it was always invoking `config.originals.asScala.mkString(", ")`.


-- 
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.

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



[GitHub] [kafka] ijuma commented on a change in pull request #10828: MINOR: Only log overridden topic configs during topic creation

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



##########
File path: core/src/test/scala/unit/kafka/log/LogConfigTest.scala
##########
@@ -162,6 +162,21 @@ class LogConfigTest {
     assertNull(nullServerDefault)
   }
 
+  @Test
+  def testOverriddenConfigsAsLoggableString(): Unit = {
+    val kafkaProps = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "")
+    kafkaProps.put("unknown.broker.password.config", "aaaaa")
+    kafkaProps.put(KafkaConfig.SslKeyPasswordProp, "somekeypassword")

Review comment:
       Yes, we can. The goal of this test is not to verify that, but no harm in covering it too.




-- 
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.

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



[GitHub] [kafka] rondagostino commented on pull request #10828: MINOR: Only log overridden topic configs during topic creation

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


   @ijuma I checked out 772f2cfc828d9da56559968a8eed06f2c8c43062 from before those PRs I mentioned above (Wed Feb 3) and I see the same log line in a system test.  I thin checked out 2.7, and again the same log line.  Here they are:
   
   `trunk`
   ```
   [2021-06-07 13:35:33,709] INFO Created log for partition topic-0 in /mnt/kafka/kafka-data-logs-1/topic-0 with properties {compression.type -> producer, message.downconversion.enable -> true, min.insync.replicas -> 1, segment.jitter.ms -> 0, cleanup.policy -> [delete], flush.ms -> 9223372036854775807, segment.bytes -> 1073741824, retention.ms -> 604800000, flush.messages -> 9223372036854775807, message.format.version -> 3.0-IV0, file.delete.delay.ms -> 60000, max.compaction.lag.ms -> 9223372036854775807, max.message.bytes -> 1048588, min.compaction.lag.ms -> 0, message.timestamp.type -> CreateTime, preallocate -> false, min.cleanable.dirty.ratio -> 0.5, index.interval.bytes -> 4096, unclean.leader.election.enable -> false, retention.bytes -> -1, delete.retention.ms -> 86400000, segment.ms -> 604800000, message.timestamp.difference.max.ms -> 9223372036854775807, segment.index.bytes -> 10485760}. (kafka.log.LogManager)
   ```
   `772f2cfc828d9da56559968a8eed06f2c8c43062`
   ```
   [2021-06-07 15:14:27,351] INFO Created log for partition topic-0 in /mnt/kafka/kafka-data-logs-1/topic-0 with properties {compression.type -> producer, message.downconversion.enable -> true, min.insync.replicas -> 1, segment.jitter.ms -> 0, cleanup.policy -> [delete], flush.ms -> 9223372036854775807, segment.bytes -> 1073741824, retention.ms -> 604800000, flush.messages -> 9223372036854775807, message.format.version -> 2.8-IV1, file.delete.delay.ms -> 60000, max.compaction.lag.ms -> 9223372036854775807, max.message.bytes -> 1048588, min.compaction.lag.ms -> 0, message.timestamp.type -> CreateTime, preallocate -> false, min.cleanable.dirty.ratio -> 0.5, index.interval.bytes -> 4096, unclean.leader.election.enable -> false, retention.bytes -> -1, delete.retention.ms -> 86400000, segment.ms -> 604800000, message.timestamp.difference.max.ms -> 9223372036854775807, segment.index.bytes -> 10485760}. (kafka.log.LogManager)
   ```
   `2.7`
   ```
   [2021-06-07 15:27:23,161] INFO Created log for partition topic-0 in /mnt/kafka/kafka-data-logs-1/topic-0 with properties {compression.type -> producer, message.downconversion.enable -> true, min.insync.replicas -> 1, segment.jitter.ms -> 0, cleanup.policy -> [delete], flush.ms -> 9223372036854775807, segment.bytes -> 1073741824, retention.ms -> 604800000, flush.messages -> 9223372036854775807, message.format.version -> 2.7-IV2, file.delete.delay.ms -> 60000, max.compaction.lag.ms -> 9223372036854775807, max.message.bytes -> 1048588, min.compaction.lag.ms -> 0, message.timestamp.type -> CreateTime, preallocate -> false, min.cleanable.dirty.ratio -> 0.5, index.interval.bytes -> 4096, unclean.leader.election.enable -> false, retention.bytes -> -1, delete.retention.ms -> 86400000, segment.ms -> 604800000, message.timestamp.difference.max.ms -> 9223372036854775807, segment.index.bytes -> 10485760}. (kafka.log.LogManager)
   ```


-- 
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.

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



[GitHub] [kafka] ijuma commented on pull request #10828: MINOR: Only log overridden topic configs during topic creation

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


   Thanks @rondagostino 


-- 
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.

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



[GitHub] [kafka] ijuma edited a comment on pull request #10828: MINOR: Only log overridden topic configs during topic creation

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


   @rondagostino but wouldn't `originals` previously only include the overrides? I don't recall seeing the full configs after partition creation in the past.


-- 
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.

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



[GitHub] [kafka] ijuma commented on pull request #10828: MINOR: Only log overridden topic configs during topic creation

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


   @rondagostino Can you double check if what I stated regarding the kraft work is true?


-- 
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.

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



[GitHub] [kafka] rondagostino edited a comment on pull request #10828: MINOR: Only log overridden topic configs during topic creation

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


   @ijuma I checked out 772f2cfc828d9da56559968a8eed06f2c8c43062 from before those PRs I mentioned above (Wed Feb 3) and I see the same log line in a system test.  I then checked out 2.7, and again the same log line.  Here they are:
   
   `trunk`
   ```
   [2021-06-07 13:35:33,709] INFO Created log for partition topic-0 in /mnt/kafka/kafka-data-logs-1/topic-0 with properties {compression.type -> producer, message.downconversion.enable -> true, min.insync.replicas -> 1, segment.jitter.ms -> 0, cleanup.policy -> [delete], flush.ms -> 9223372036854775807, segment.bytes -> 1073741824, retention.ms -> 604800000, flush.messages -> 9223372036854775807, message.format.version -> 3.0-IV0, file.delete.delay.ms -> 60000, max.compaction.lag.ms -> 9223372036854775807, max.message.bytes -> 1048588, min.compaction.lag.ms -> 0, message.timestamp.type -> CreateTime, preallocate -> false, min.cleanable.dirty.ratio -> 0.5, index.interval.bytes -> 4096, unclean.leader.election.enable -> false, retention.bytes -> -1, delete.retention.ms -> 86400000, segment.ms -> 604800000, message.timestamp.difference.max.ms -> 9223372036854775807, segment.index.bytes -> 10485760}. (kafka.log.LogManager)
   ```
   `772f2cfc828d9da56559968a8eed06f2c8c43062`
   ```
   [2021-06-07 15:14:27,351] INFO Created log for partition topic-0 in /mnt/kafka/kafka-data-logs-1/topic-0 with properties {compression.type -> producer, message.downconversion.enable -> true, min.insync.replicas -> 1, segment.jitter.ms -> 0, cleanup.policy -> [delete], flush.ms -> 9223372036854775807, segment.bytes -> 1073741824, retention.ms -> 604800000, flush.messages -> 9223372036854775807, message.format.version -> 2.8-IV1, file.delete.delay.ms -> 60000, max.compaction.lag.ms -> 9223372036854775807, max.message.bytes -> 1048588, min.compaction.lag.ms -> 0, message.timestamp.type -> CreateTime, preallocate -> false, min.cleanable.dirty.ratio -> 0.5, index.interval.bytes -> 4096, unclean.leader.election.enable -> false, retention.bytes -> -1, delete.retention.ms -> 86400000, segment.ms -> 604800000, message.timestamp.difference.max.ms -> 9223372036854775807, segment.index.bytes -> 10485760}. (kafka.log.LogManager)
   ```
   `2.7`
   ```
   [2021-06-07 15:27:23,161] INFO Created log for partition topic-0 in /mnt/kafka/kafka-data-logs-1/topic-0 with properties {compression.type -> producer, message.downconversion.enable -> true, min.insync.replicas -> 1, segment.jitter.ms -> 0, cleanup.policy -> [delete], flush.ms -> 9223372036854775807, segment.bytes -> 1073741824, retention.ms -> 604800000, flush.messages -> 9223372036854775807, message.format.version -> 2.7-IV2, file.delete.delay.ms -> 60000, max.compaction.lag.ms -> 9223372036854775807, max.message.bytes -> 1048588, min.compaction.lag.ms -> 0, message.timestamp.type -> CreateTime, preallocate -> false, min.cleanable.dirty.ratio -> 0.5, index.interval.bytes -> 4096, unclean.leader.election.enable -> false, retention.bytes -> -1, delete.retention.ms -> 86400000, segment.ms -> 604800000, message.timestamp.difference.max.ms -> 9223372036854775807, segment.index.bytes -> 10485760}. (kafka.log.LogManager)
   ```


-- 
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.

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



[GitHub] [kafka] ijuma commented on pull request #10828: MINOR: Only log overridden topic configs during topic creation

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


   Unrelated failures:
   
   > Build / JDK 11 and Scala 2.13 / kafka.server.RaftClusterTest.testCreateClusterAndCreateListDeleteTopic()
   > Build / JDK 15 and Scala 2.13 / org.apache.kafka.connect.integration.RebalanceSourceConnectorsIntegrationTest.testDeleteConnector
   > Build / JDK 15 and Scala 2.13 / kafka.server.epoch.EpochDrivenReplicationProtocolAcceptanceTest.shouldSurviveFastLeaderChange()


-- 
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.

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



[GitHub] [kafka] ijuma commented on pull request #10828: MINOR: Only log overridden topic configs during topic creation

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


   I updated the PR description.


-- 
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.

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



[GitHub] [kafka] ijuma commented on pull request #10828: MINOR: Only log overridden topic configs during topic creation

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


   @rondagostino but wouldn't `originals` previously only include the overrides? I don't recall saying the full configs after partition creation in the past.


-- 
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.

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



[GitHub] [kafka] showuon commented on a change in pull request #10828: MINOR: Only log overridden topic configs during topic creation

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



##########
File path: core/src/test/scala/unit/kafka/log/LogConfigTest.scala
##########
@@ -162,6 +162,21 @@ class LogConfigTest {
     assertNull(nullServerDefault)
   }
 
+  @Test
+  def testOverriddenConfigsAsLoggableString(): Unit = {
+    val kafkaProps = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "")
+    kafkaProps.put("unknown.broker.password.config", "aaaaa")
+    kafkaProps.put(KafkaConfig.SslKeyPasswordProp, "somekeypassword")

Review comment:
       Could we add a line with the same setting as overridden one, but not the same value, so that we can verify the overridden value will be outputted? Ex:
   add this line
   `kafkaProps.put(LogConfig.MinInSyncReplicasProp, "1")`
   then, we already have
   ` topicOverrides.setProperty(LogConfig.MinInSyncReplicasProp, "2")`
   
   So, we can assert the log is output `min.insync.replicas=2`




-- 
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.

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



[GitHub] [kafka] ijuma commented on pull request #10828: MINOR: Only log overridden topic configs during topic creation

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


   Merged to trunk and cherry-picked to 2.8.


-- 
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.

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