> On July 7, 2015, 2:18 a.m., Jun Rao wrote:
> > Thanks for the patch. A few more comments below.
> > 
> > 1. The patch doesn't apply. Could you rebase?
> > 2. Also, we need the logic to read all existing client configs. Is that in 
> > a separate jira?

1. Will do.
2. Hey Jun - I didn't understand what you meant by "read all existing client 
configs". Can you elaborate? In general, I'm submitting all followup client 
config changes in subsequent patches to avoid making the patches too large. 
This patch will basically refactor the code and make it possible to receive 
client change notifications.


> On July 7, 2015, 2:18 a.m., Jun Rao wrote:
> > core/src/main/scala/kafka/admin/ConfigCommand.scala, line 49
> > <https://reviews.apache.org/r/34554/diff/2/?file=997805#file997805line49>
> >
> >     What is the 1 at the end?

typo.


> On July 7, 2015, 2:18 a.m., Jun Rao wrote:
> > core/src/main/scala/kafka/server/TopicConfigManager.scala, line 157
> > <https://reviews.apache.org/r/34554/diff/2/?file=997814#file997814line157>
> >
> >     It's probably useful to include the orginal json string. Also, could we 
> > make the message string in all IllegalArgumentException consistent? For 
> > example, they should all reference config change notification.

Improved the error messages a bit


> On July 7, 2015, 2:18 a.m., Jun Rao wrote:
> > core/src/main/scala/kafka/admin/ConfigCommand.scala, lines 60-106
> > <https://reviews.apache.org/r/34554/diff/2/?file=997805#file997805line60>
> >
> >     It seems that all those methods can be private.

parseConfigsToBeAdded and parseConfigsToBeDeleted are used in 
ConfigCommandTest. I've made them private to the admin package.


> On July 7, 2015, 2:18 a.m., Jun Rao wrote:
> > core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala, lines 
> > 88-96
> > <https://reviews.apache.org/r/34554/diff/2/?file=997819#file997819line88>
> >
> >     Are we really mocking zkclient calls here?

Nope.. no zkclient calls being mocked here.


- Aditya


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


On July 2, 2015, 1:39 a.m., Aditya Auradkar wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/34554/
> -----------------------------------------------------------
> 
> (Updated July 2, 2015, 1:39 a.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
> 
>

Reply via email to