> On Nov. 7, 2014, 5 a.m., Gwen Shapira wrote:
> > core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala, 
> > line 144
> > <https://reviews.apache.org/r/27634/diff/3/?file=751668#file751668line144>
> >
> >     String.valueOf(false) should evaluate to "false", right? Why do we need 
> > "nottrue"? 
> >     
> >     If String.valueOf(false) no longer works as expected, it looks like a 
> > bug waiting to happen...
> 
> Dmytro Kostiuchenko wrote:
>     Before my changes boolean values were obtained as StringOps.toBoolean(). 
> Method throws IllegalArgumentException if called on anything other than 
> "true" or "false".
>     Now parsing is handled by ConfigDef. It performs call to 
> Boolean.parseBoolean(String) which returns false for anything but "true" 
> throwing no exception.
>     I updated test to indicate that fact.
> 
> Gwen Shapira wrote:
>     Got it.
>     
>     This changes the behavior of "unclean leader election" property into 
> something less safe (typo in "true" can lead to false value for example). 
>     
>     Perhaps change the type to "STRING" (similar to how we handle 
> CleanupPolicy) and add a validation to enforce the old behavior?

Makes sense. Will implement the former behaviour.


> On Nov. 7, 2014, 5 a.m., Gwen Shapira wrote:
> > core/src/main/scala/kafka/utils/Utils.scala, lines 514-522
> > <https://reviews.apache.org/r/27634/diff/3/?file=751666#file751666line514>
> >
> >     I think I'm not seeing why we need this. 
> >     Shouldn't Scala's JavaConversion class handle this exact case?
> 
> Dmytro Kostiuchenko wrote:
>     Frankly, I'm new to Scala. But I didn't found a way to convert 
> java.util.Properties to Map respecting defaults.
>     java.util.Properties have additional method getProperty() which checks 
> for value in nested Properties object (see Properties(Properties defaults) 
> constructor) if key is absent in this one. But is treated as Map, defaults 
> can't be accessed. Map.get(K, V) is implemented in Properties' parent — 
> HashTable, and thus, get() knows nothing about default properties.
>     
>     Example:
>     val defaults = new Properties()
>     defaults.put("foo", "bar")
>     val props = new Properties(defaults)
>     props.get("foo") // null
>     props.getProperty("foo") // "bar"
> 
> Gwen Shapira wrote:
>     Oh, yuck. Took me a bit to realize what's going on here. Properties 
> inherits from Map, but can actually contains another map for defaults that 
> isn't use when casting...
>     I'd document why this is necessary - since its not 100% trivial (at least 
> not to me)
>     
>     Perhaps instead of a util method, adding another parse method to 
> ConfigDef that will take Properties as a parameter will make things a bit 
> cleaner. This is just my preference, so I'd wait for one of the committers to 
> chime in here.

In my opinion the whole Properties-Map situation is already confusing and 
adding an overloaded method will make things even worse. I prefer to avoid 
overloaded methods if there is a slightest ambiguity.

As for the documentation, I would add a comment at the calling point and change 
wording in javadoc (your description is much shorter and more explicit than 
mine). It would also make sense to add a code example to javadoc.


- Dmytro


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


On Nov. 7, 2014, 1:28 p.m., Dmytro Kostiuchenko wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/27634/
> -----------------------------------------------------------
> 
> (Updated Nov. 7, 2014, 1:28 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1667
>     https://issues.apache.org/jira/browse/KAFKA-1667
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> KAFKA-1667 Fixed bugs in LogConfig. Added test and documentation
> 
> 
> KAFKA-1667 Updated tests to reflect new boolean property parsing logic
> 
> 
> KAFKA-1677 renamed methods to match naming convention
> 
> 
> KAFKA-1667 Added unit test to cover invalid configuration case
> 
> 
> Diffs
> -----
> 
>   clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java 
> c4cea2cc072f4db4ce014b63d226431d3766bef1 
>   core/src/main/scala/kafka/admin/TopicCommand.scala 
> 0b2735e7fc42ef9894bef1997b1f06a8ebee5439 
>   core/src/main/scala/kafka/log/LogConfig.scala 
> e48922a97727dd0b98f3ae630ebb0af3bef2373d 
>   core/src/main/scala/kafka/utils/Utils.scala 
> 23aefb4715b177feae1d2f83e8b910653ea10c5f 
>   core/src/test/scala/kafka/log/LogConfigTest.scala PRE-CREATION 
>   core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala 
> f44568cb25edf25db857415119018fd4c9922f61 
> 
> Diff: https://reviews.apache.org/r/27634/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Dmytro Kostiuchenko
> 
>

Reply via email to