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 2020/07/01 08:30:22 UTC

[GitHub] [kafka] showuon opened a new pull request #8966: KAFKA-10220: add null check for configurationKey

showuon opened a new pull request #8966:
URL: https://github.com/apache/kafka/pull/8966


   Add null check for configurationKey to avoid NPE, and add test for it.
   
   ### 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] showuon commented on a change in pull request #8966: KAFKA-10220: add null check for configurationKey

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



##########
File path: core/src/main/scala/kafka/server/AdminManager.scala
##########
@@ -354,10 +354,13 @@ class AdminManager(val config: KafkaConfig,
       }
       def createResponseConfig(configs: Map[String, Any],
                                createConfigEntry: (String, Any) => DescribeConfigsResponseData.DescribeConfigsResourceResult): DescribeConfigsResponseData.DescribeConfigsResult = {
-        val filteredConfigPairs = configs.filter { case (configName, _) =>
-          /* Always returns true if configNames is None */
-          resource.configurationKeys.asScala.forall(_.contains(configName))
-        }.toBuffer
+        val filteredConfigPairs = if (resource.configurationKeys == null)
+          configs.toBuffer
+        else
+          configs.filter { case (configName, _) =>
+            /* Always returns true if configurationKeys is null */

Review comment:
       Oh, nice catch!




----------------------------------------------------------------
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] omkreddy commented on pull request #8966: KAFKA-10220: add null check for configurationKey

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


   @tombentley Thanks for the explanation. I will go ahead and merge the PR, 


----------------------------------------------------------------
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 #8966: KAFKA-10220: add null check for configurationKey

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



##########
File path: core/src/main/scala/kafka/server/AdminManager.scala
##########
@@ -355,8 +355,8 @@ class AdminManager(val config: KafkaConfig,
       def createResponseConfig(configs: Map[String, Any],
                                createConfigEntry: (String, Any) => DescribeConfigsResponseData.DescribeConfigsResourceResult): DescribeConfigsResponseData.DescribeConfigsResult = {
         val filteredConfigPairs = configs.filter { case (configName, _) =>
-          /* Always returns true if configNames is None */
-          resource.configurationKeys.asScala.forall(_.contains(configName))
+          /* Always returns true if configurationKeys is null */
+          if (resource.configurationKeys != null) resource.configurationKeys.asScala.forall(_.contains(configName)) else true

Review comment:
       good suggestion. Thanks.




----------------------------------------------------------------
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 pull request #8966: KAFKA-10220: add null check for configurationKey

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


   @omkreddy , sorry, I saw the PR is closed, instead of merging. Could you please check again. Thanks.


----------------------------------------------------------------
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] tombentley commented on pull request #8966: KAFKA-10220: add null check for configurationKey

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


   @omkreddy that is... an excellent question. 
   
   * In `HEAD` the only place it's set to non-null is in `RequestResponseTest`
   * In 9a4f00f78b (i.e. just before the change to use the generated message classes the situation is the same: Only non-null call sites are two tests in `RequestResponseTest`
   * ... and I chased it all the way back to 972b7545363ae, when the RPC was added, and it seems that this has always been the situation. One or two callers with non-null keys in `RequestResponseTest`. Thus supported (but without any test coverage) at the protocol level, but never exposed in the Java API. 


----------------------------------------------------------------
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] huxihx commented on pull request #8966: KAFKA-10220: add null check for configurationKey

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


   retest this please.


----------------------------------------------------------------
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] omkreddy edited a comment on pull request #8966: KAFKA-10220: add null check for configurationKey

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


   @tombentley @showuon Can you help me to understand where are we calling `DescribeConfigsRequestData.setConfigurationKeys()` in AdminClient?  looks like `ConfigurationKeys` are always null.


----------------------------------------------------------------
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 #8966: KAFKA-10220: add null check for configurationKey

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



##########
File path: core/src/main/scala/kafka/server/AdminManager.scala
##########
@@ -354,10 +354,13 @@ class AdminManager(val config: KafkaConfig,
       }
       def createResponseConfig(configs: Map[String, Any],
                                createConfigEntry: (String, Any) => DescribeConfigsResponseData.DescribeConfigsResourceResult): DescribeConfigsResponseData.DescribeConfigsResult = {
-        val filteredConfigPairs = configs.filter { case (configName, _) =>
-          /* Always returns true if configNames is None */
-          resource.configurationKeys.asScala.forall(_.contains(configName))
-        }.toBuffer
+        val filteredConfigPairs = if (resource.configurationKeys == null)
+          configs.toBuffer
+        else
+          configs.filter { case (configName, _) =>
+            /* Always returns true if configurationKeys is null */

Review comment:
       Oh, nice catch! Fixed in https://github.com/apache/kafka/pull/8966/commits/582d02bd67bbe112c4407b0c5b665dee97eceb2a. Thanks.




----------------------------------------------------------------
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 pull request #8966: KAFKA-10220: add null check for configurationKey

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


   hi @huxihx @omkreddy, do you have other comments for this PR? And I cannot check the jenkins testing results now since they are out-dated. Should we re-run tests again?
   Thanks.


----------------------------------------------------------------
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 pull request #8966: KAFKA-10220: add null check for configurationKey

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


   Cool! Thanks, @omkreddy !


----------------------------------------------------------------
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] omkreddy commented on pull request #8966: KAFKA-10220: add null check for configurationKey

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


   ok to test


----------------------------------------------------------------
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 pull request #8966: KAFKA-10220: add null check for configurationKey

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


   jdk 11 -
       org.apache.kafka.streams.integration.EosBetaUpgradeIntegrationTest.shouldUpgradeFromEosAlphaToEosBeta[false]
       org.apache.kafka.streams.integration.EosBetaUpgradeIntegrationTest.shouldUpgradeFromEosAlphaToEosBeta[true]
       org.apache.kafka.streams.integration.EosBetaUpgradeIntegrationTest.shouldUpgradeFromEosAlphaToEosBeta[false]
       org.apache.kafka.streams.integration.EosBetaUpgradeIntegrationTest.shouldUpgradeFromEosAlphaToEosBeta[true]
   
   jdk 14 - 
   org.apache.kafka.streams.integration.EosBetaUpgradeIntegrationTest.shouldUpgradeFromEosAlphaToEosBeta[true]
   
   jdk 8 -
       org.apache.kafka.streams.integration.EosBetaUpgradeIntegrationTest.shouldUpgradeFromEosAlphaToEosBeta[false]
       org.apache.kafka.streams.integration.EosBetaUpgradeIntegrationTest.shouldUpgradeFromEosAlphaToEosBeta[false]
       org.apache.kafka.streams.integration.EosBetaUpgradeIntegrationTest.shouldUpgradeFromEosAlphaToEosBeta[true]
       org.apache.kafka.streams.integration.EosBetaUpgradeIntegrationTest.shouldUpgradeFromEosAlphaToEosBeta[true]
   
   
   All tracked in KAFKA-10017. Thanks.


----------------------------------------------------------------
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] omkreddy edited a comment on pull request #8966: KAFKA-10220: add null check for configurationKey

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


   @showuon  I normally use `kafka-merge-pr.py` script for merging PRs. This will do a direct push from local. In this case, github marks it as closed. You can check the commit on 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.

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



[GitHub] [kafka] omkreddy commented on pull request #8966: KAFKA-10220: add null check for configurationKey

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


   @tombentley @showuon Can you help me to understand where are we calling `DescribeConfigsRequestData.setConfigurationKeys()` in AdminClient?  looks like we are always setting to null.


----------------------------------------------------------------
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] omkreddy commented on pull request #8966: KAFKA-10220: add null check for configurationKey

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


   @showuon  I normally use `kafka-merge-pr.py` script for merging PRs. This will do a direct push from local. In this case, github marks it as closed. You can check the commit of 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.

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



[GitHub] [kafka] tombentley commented on a change in pull request #8966: KAFKA-10220: add null check for configurationKey

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



##########
File path: core/src/main/scala/kafka/server/AdminManager.scala
##########
@@ -354,10 +354,13 @@ class AdminManager(val config: KafkaConfig,
       }
       def createResponseConfig(configs: Map[String, Any],
                                createConfigEntry: (String, Any) => DescribeConfigsResponseData.DescribeConfigsResourceResult): DescribeConfigsResponseData.DescribeConfigsResult = {
-        val filteredConfigPairs = configs.filter { case (configName, _) =>
-          /* Always returns true if configNames is None */
-          resource.configurationKeys.asScala.forall(_.contains(configName))
-        }.toBuffer
+        val filteredConfigPairs = if (resource.configurationKeys == null)
+          configs.toBuffer
+        else
+          configs.filter { case (configName, _) =>
+            /* Always returns true if configurationKeys is null */

Review comment:
       This comment is no longer valid




----------------------------------------------------------------
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 pull request #8966: KAFKA-10220: add null check for configurationKey

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


   @omkreddy , I can take a look tomorrow ( my time). Thanks.


----------------------------------------------------------------
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] omkreddy closed pull request #8966: KAFKA-10220: add null check for configurationKey

Posted by GitBox <gi...@apache.org>.
omkreddy closed pull request #8966:
URL: https://github.com/apache/kafka/pull/8966


   


----------------------------------------------------------------
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 pull request #8966: KAFKA-10220: add null check for configurationKey

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


   @huxihx @tombentley , could you help review this PR? Thanks.


----------------------------------------------------------------
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 #8966: KAFKA-10220: add null check for configurationKey

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



##########
File path: core/src/main/scala/kafka/server/AdminManager.scala
##########
@@ -354,10 +354,12 @@ class AdminManager(val config: KafkaConfig,
       }
       def createResponseConfig(configs: Map[String, Any],
                                createConfigEntry: (String, Any) => DescribeConfigsResponseData.DescribeConfigsResourceResult): DescribeConfigsResponseData.DescribeConfigsResult = {
-        val filteredConfigPairs = configs.filter { case (configName, _) =>
-          /* Always returns true if configNames is None */
-          resource.configurationKeys.asScala.forall(_.contains(configName))
-        }.toBuffer
+        val filteredConfigPairs = if (resource.configurationKeys == null)

Review comment:
       Thanks @huxihx , I agree your suggestion makes the code less and simplified. But for better readability, I'd prefer the @tombentley 's suggestion. Thank you.




----------------------------------------------------------------
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] tombentley commented on a change in pull request #8966: KAFKA-10220: add null check for configurationKey

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



##########
File path: core/src/test/scala/unit/kafka/server/AdminManagerTest.scala
##########
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package kafka.server
+
+import kafka.zk.KafkaZkClient
+import org.apache.kafka.common.metrics.Metrics
+import org.easymock.EasyMock
+import kafka.utils.TestUtils
+import org.apache.kafka.common.config.ConfigResource
+import org.apache.kafka.common.message.DescribeConfigsRequestData
+import org.apache.kafka.common.message.DescribeConfigsResponseData
+import org.apache.kafka.common.protocol.Errors
+
+import org.junit.{After, Test}
+import org.junit.Assert.assertEquals
+
+class AdminManagerTest {
+
+  private val zkClient: KafkaZkClient = EasyMock.createNiceMock(classOf[KafkaZkClient])
+  private val metrics = new Metrics()
+  private val brokerId = 1
+  private val topic = "topic-1"
+  private val metadataCache: MetadataCache = EasyMock.createNiceMock(classOf[MetadataCache])
+
+  @After
+  def tearDown(): Unit = {
+    metrics.close()
+  }
+
+  def createAdminManager(): AdminManager = {
+    val props = TestUtils.createBrokerConfig(brokerId, "zk")
+    new AdminManager(KafkaConfig.fromProps(props), metrics, metadataCache, zkClient)
+  }
+
+  @Test
+  def testDescribeConfigs(): Unit = {

Review comment:
       ```suggestion
     def testDescribeConfigsWithNullConfigurationKeys(): Unit = {
   ```

##########
File path: core/src/main/scala/kafka/server/AdminManager.scala
##########
@@ -355,8 +355,8 @@ class AdminManager(val config: KafkaConfig,
       def createResponseConfig(configs: Map[String, Any],
                                createConfigEntry: (String, Any) => DescribeConfigsResponseData.DescribeConfigsResourceResult): DescribeConfigsResponseData.DescribeConfigsResult = {
         val filteredConfigPairs = configs.filter { case (configName, _) =>
-          /* Always returns true if configNames is None */
-          resource.configurationKeys.asScala.forall(_.contains(configName))
+          /* Always returns true if configurationKeys is null */
+          if (resource.configurationKeys != null) resource.configurationKeys.asScala.forall(_.contains(configName)) else true

Review comment:
       Since null means "all the keys" perhaps it would be better to move the `if` to the topic level to avoid an unneccessary traversal in the null case:
   
   ```scala
           val filteredConfigPairs = if (resource.configurationKeys == null)
             configs.toBuffer
           else 
             configs.filter { case (configName, _) =>
               /* Always returns true if configNames is None */
               resource.configurationKeys.asScala.forall(_.contains(configName))
             }.toBuffer
   ```




----------------------------------------------------------------
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 pull request #8966: KAFKA-10220: add null check for configurationKey

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


   hi @tombentley , thanks for the comments. I've updated in this commit: https://github.com/apache/kafka/pull/8966/commits/ea249ab061fa2db7ba40963ff032c85a3919ecdb. Thanks.


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