You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Jun Rao <ju...@gmail.com> on 2015/06/27 16:48:25 UTC

Re: Review Request 34554: Patch for KAFKA-2205

-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/34554/#review89623
-----------------------------------------------------------


Thanks for the patch. A few comments below.

We plan to add the AlterConfig/DescribeConfig requests. Will that be handled in a separate jira? Also, are the new requests format listed in the KIP?


core/src/main/scala/kafka/admin/AdminUtils.scala (line 315)
<https://reviews.apache.org/r/34554/#comment142269>

    Currently, the existing json format in ZK uses _ instead of camel case. See https://cwiki.apache.org/confluence/display/KAFKA/Kafka+data+structures+in+Zookeeper



core/src/main/scala/kafka/admin/AdminUtils.scala (line 353)
<https://reviews.apache.org/r/34554/#comment142270>

    We probably want to include entity type in the exception message as well.



core/src/main/scala/kafka/admin/ConfigCommand.scala (lines 42 - 45)
<https://reviews.apache.org/r/34554/#comment142273>

    Perhaps these can be moved into checkArgs()?



core/src/main/scala/kafka/admin/ConfigCommand.scala (lines 122 - 125)
<https://reviews.apache.org/r/34554/#comment142271>

    I think the current naming convention is to use -, instead of camel, e.g., broker-list.



core/src/main/scala/kafka/admin/ConfigCommand.scala (line 145)
<https://reviews.apache.org/r/34554/#comment142272>

    With describe, a few other options like addConfig, deleteConfig are also invalid.



core/src/main/scala/kafka/admin/TopicCommand.scala (line 23)
<https://reviews.apache.org/r/34554/#comment142274>

    Given the new ConfigCommand, we should disallow alter config here, right?



core/src/main/scala/kafka/server/ConfigHandler.scala (line 23)
<https://reviews.apache.org/r/34554/#comment142275>

    Could we use groupBy{ case(key, value) => ...} instead of using tuples to make it clear what we reference?



core/src/main/scala/kafka/server/ConfigHandler.scala (lines 43 - 48)
<https://reviews.apache.org/r/34554/#comment142278>

    We need to store the client configs in a map and provide an api to access them, right?



core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala (line 98)
<https://reviews.apache.org/r/34554/#comment142276>

    Instead of "Topic", perhaps referencing an entityType constant?



core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala (lines 99 - 100)
<https://reviews.apache.org/r/34554/#comment142277>

    Should we ignore it or should we error it like the rest?


- Jun Rao


On May 21, 2015, 6:32 p.m., Aditya Auradkar wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/34554/
> -----------------------------------------------------------
> 
> (Updated May 21, 2015, 6:32 p.m.)
> 
> 
> Review request for kafka and Joel Koshy.
> 
> 
> Bugs: KAFKA-2205
>     https://issues.apache.org/jira/browse/KAFKA-2205
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> KAFKA-2205. Summary of changes:
> 
> 1. Generalized TopicConfigManager to DynamicConfigManager. It is now able to handle multiple types of entities.
> 2. Changed format of the notification znode as described in KIP-21
> 3. Replaced TopicConfigManager with DynamicConfigManager.
> 4. Added new testcases. Existing testcases all pass
> 5. Added ConfigCommand to handle all config changes. Eventually this will make calls to the broker once the new API's are built for now it speaks to ZK directly
> 
> 
> Diffs
> -----
> 
>   core/src/main/scala/kafka/admin/AdminUtils.scala f06edf41c732a7b794e496d0048b0ce6f897e72b 
>   core/src/main/scala/kafka/admin/ConfigCommand.scala PRE-CREATION 
>   core/src/main/scala/kafka/admin/TopicCommand.scala 8e6f18633b25bf1beee3f813b28ef7aa7d779d7b 
>   core/src/main/scala/kafka/cluster/Partition.scala 730a232482fdf77be5704cdf5941cfab3828db88 
>   core/src/main/scala/kafka/controller/KafkaController.scala 69bba243a9a511cc5292b43da0cc48e421a428b0 
>   core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala 3b15ab4eef22c6f50a7483e99a6af40fb55aca9f 
>   core/src/main/scala/kafka/controller/TopicDeletionManager.scala 64ecb499f24bc801d48f86e1612d927cc08e006d 
>   core/src/main/scala/kafka/server/ConfigHandler.scala PRE-CREATION 
>   core/src/main/scala/kafka/server/KafkaServer.scala ea6d165d8e5c3146d2c65e8ad1a513308334bf6f 
>   core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b31b432a226ba79546dd22ef1d2acbb439c2e9a3 
>   core/src/main/scala/kafka/server/TopicConfigManager.scala b675a7e45ea4f4179f8b15fe221fd988aff13aa0 
>   core/src/main/scala/kafka/utils/ZkUtils.scala 2618dd39b925b979ad6e4c0abd5c6eaafb3db5d5 
>   core/src/test/scala/unit/kafka/admin/AdminTest.scala efb2f8e79b3faef78722774b951fea828cd50374 
>   core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala PRE-CREATION 
>   core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala c7136f20972614ac47aa57ab13e3c94ef775a4b7 
>   core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala 7877f6ca1845c2edbf96d4a9783a07a552db8f07 
> 
> Diff: https://reviews.apache.org/r/34554/diff/
> 
> 
> Testing
> -------
> 
> 1. Added new testcases for new code.
> 2. Verified that both topic and client configs can be changed dynamically by starting a local cluster
> 
> 
> Thanks,
> 
> Aditya Auradkar
> 
>


Re: Review Request 34554: Patch for KAFKA-2205

Posted by Aditya Auradkar <aa...@linkedin.com>.

> On June 27, 2015, 2:48 p.m., Jun Rao wrote:
> > Thanks for the patch. A few comments below.
> > 
> > We plan to add the AlterConfig/DescribeConfig requests. Will that be handled in a separate jira? Also, are the new requests format listed in the KIP?

Yeah, I'm planning to tackle the AlterConfig/DescribeConfig stuff in a separate jira. Those requests are listed on KIP-4


> On June 27, 2015, 2:48 p.m., Jun Rao wrote:
> > core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala, lines 99-100
> > <https://reviews.apache.org/r/34554/diff/1/?file=966847#file966847line99>
> >
> >     Should we ignore it or should we error it like the rest?

Actually, I think we should ignore such notifications. The old-style notifications parsed by TopicConfigManager aren't JSON. They simply have a string representing the topic name. The new code can expect to receive these notifications (leftovers prior to the upgrade) and shouldn't fail to process the other more recent notifications IMO.


> On June 27, 2015, 2:48 p.m., Jun Rao wrote:
> > core/src/main/scala/kafka/server/ConfigHandler.scala, line 23
> > <https://reviews.apache.org/r/34554/diff/1/?file=966839#file966839line23>
> >
> >     Could we use groupBy{ case(key, value) => ...} instead of using tuples to make it clear what we reference?

Using tuples now. I'm still using _ for the final map function because it seems quite clear what is being referenced


> On June 27, 2015, 2:48 p.m., Jun Rao wrote:
> > core/src/main/scala/kafka/server/ConfigHandler.scala, lines 43-48
> > <https://reviews.apache.org/r/34554/diff/1/?file=966839#file966839line43>
> >
> >     We need to store the client configs in a map and provide an api to access them, right?

I've added a map to store these configs. This is a no-op right now. I don't think an API is necessary because we can use them within the same class i.e. quota changes can be done within processConfigChanges.


- Aditya


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/34554/#review89623
-----------------------------------------------------------


On May 21, 2015, 6:32 p.m., Aditya Auradkar wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/34554/
> -----------------------------------------------------------
> 
> (Updated May 21, 2015, 6:32 p.m.)
> 
> 
> Review request for kafka and Joel Koshy.
> 
> 
> Bugs: KAFKA-2205
>     https://issues.apache.org/jira/browse/KAFKA-2205
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> KAFKA-2205. Summary of changes:
> 
> 1. Generalized TopicConfigManager to DynamicConfigManager. It is now able to handle multiple types of entities.
> 2. Changed format of the notification znode as described in KIP-21
> 3. Replaced TopicConfigManager with DynamicConfigManager.
> 4. Added new testcases. Existing testcases all pass
> 5. Added ConfigCommand to handle all config changes. Eventually this will make calls to the broker once the new API's are built for now it speaks to ZK directly
> 
> 
> Diffs
> -----
> 
>   core/src/main/scala/kafka/admin/AdminUtils.scala f06edf41c732a7b794e496d0048b0ce6f897e72b 
>   core/src/main/scala/kafka/admin/ConfigCommand.scala PRE-CREATION 
>   core/src/main/scala/kafka/admin/TopicCommand.scala 8e6f18633b25bf1beee3f813b28ef7aa7d779d7b 
>   core/src/main/scala/kafka/cluster/Partition.scala 730a232482fdf77be5704cdf5941cfab3828db88 
>   core/src/main/scala/kafka/controller/KafkaController.scala 69bba243a9a511cc5292b43da0cc48e421a428b0 
>   core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala 3b15ab4eef22c6f50a7483e99a6af40fb55aca9f 
>   core/src/main/scala/kafka/controller/TopicDeletionManager.scala 64ecb499f24bc801d48f86e1612d927cc08e006d 
>   core/src/main/scala/kafka/server/ConfigHandler.scala PRE-CREATION 
>   core/src/main/scala/kafka/server/KafkaServer.scala ea6d165d8e5c3146d2c65e8ad1a513308334bf6f 
>   core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b31b432a226ba79546dd22ef1d2acbb439c2e9a3 
>   core/src/main/scala/kafka/server/TopicConfigManager.scala b675a7e45ea4f4179f8b15fe221fd988aff13aa0 
>   core/src/main/scala/kafka/utils/ZkUtils.scala 2618dd39b925b979ad6e4c0abd5c6eaafb3db5d5 
>   core/src/test/scala/unit/kafka/admin/AdminTest.scala efb2f8e79b3faef78722774b951fea828cd50374 
>   core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala PRE-CREATION 
>   core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala c7136f20972614ac47aa57ab13e3c94ef775a4b7 
>   core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala 7877f6ca1845c2edbf96d4a9783a07a552db8f07 
> 
> Diff: https://reviews.apache.org/r/34554/diff/
> 
> 
> Testing
> -------
> 
> 1. Added new testcases for new code.
> 2. Verified that both topic and client configs can be changed dynamically by starting a local cluster
> 
> 
> Thanks,
> 
> Aditya Auradkar
> 
>