> On Sept. 8, 2013, 10:47 p.m., Jun Rao wrote:
> > I am wondering if there is any performance degradation due to the changes 
> > in implicit conversion. Could you run kafka.perf.ProducerPerformance and 
> > see if there is any noticeable performance change with and w/o the patch?

both should compile to similar byte code.  Any extra work is done during 
compile time where the compiler must figure out which method should be invoked 
rather then being explicitly told by the source code.

Over seven runs of the producer perf test using 1,000,000 messages, for before 
and after patch, I got 77180.93 and 77350.39 nMsg.sec.  Performance between 
runs was kind of noisy so the delta (which actually improved) is well within 
the margin of error.


> On Sept. 8, 2013, 10:47 p.m., Jun Rao wrote:
> > core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala,
> >  lines 95-97
> > <https://reviews.apache.org/r/14013/diff/3/?file=349538#file349538line95>
> >
> >     Don't you need to specify the java return type explicitly here?

This is not required as the the return type is defined in the trait which the 
class is extending


> On Sept. 8, 2013, 10:47 p.m., Jun Rao wrote:
> > core/src/main/scala/kafka/controller/PartitionStateMachine.scala, lines 
> > 364-365
> > <https://reviews.apache.org/r/14013/diff/3/?file=349531#file349531line364>
> >
> >     Could you move the import of the JavaConversion to within this function 
> > to limit the scope of the implicit conversion?

sure.


> On Sept. 8, 2013, 10:47 p.m., Jun Rao wrote:
> > core/src/main/scala/kafka/utils/Annotations_2.9+.scala, lines 1-38
> > <https://reviews.apache.org/r/14013/diff/3/?file=349557#file349557line1>
> >
> >     How does this affect IDEs like Intellij? Are you able to build the 
> > project in IDE with both version of threadsafe annotation?

I don't normally build with my IDE but I don't see any errors when I compile 
the project with Intellij (Build -> Make project, which invokes an external 
scala compiler).  I also didn't see any issues using the SBT plugin (which 
makes sense as all it is doing is opening a shell and running sbt).


- Christopher


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


On Sept. 8, 2013, 9:47 p.m., Christopher Freeman wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/14013/
> -----------------------------------------------------------
> 
> (Updated Sept. 8, 2013, 9:47 p.m.)
> 
> 
> Review request for kafka and Neha Narkhede.
> 
> 
> Bugs: KAFKA-1046
>     https://issues.apache.org/jira/browse/KAFKA-1046
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> This RB contains necessary changes to the kafka source code to support Scala 
> 2.10.x while still maintaining support for Scala 2.8.x
> 
> The existing cause of source incompatibility between 2.8.x and 2.10.x was 
> primarily due to the use of implicit conversions in the 
> scala.collection.JavaConversions object.  Many of them where deprecated (and 
> replaced with differently named methods) and the deprecated methods where 
> removed in 2.10.  I avoided the source incompatibility by importing the 
> implicits with a wild card.
> 
> Scala annotations moved to a different package in 2.9 and the reference to 
> the old location was removed in 2.10.  I couldn't think of a source 
> compatible way to support both in Annotations.scala so since this source file 
> is very small, I created two copies and configured sbt to use the one 
> appropriate for when building for a particular Scala version.
> 
> I also typed many of the catch block case statements with :Throwable.  This 
> change isn't required but starting with Scala 2.9, the compiler will emit 
> warnings if the type is emitted.  Otherwise "case e =>" and case "e: 
> Throwable =>" are equivalent.
> 
> 
>   
> 
> 
> Diffs
> -----
> 
>   core/build.sbt c54cf44 
>   core/src/main/scala/kafka/Kafka.scala dafb1ee 
>   core/src/main/scala/kafka/admin/AddPartitionsCommand.scala 5757c32 
>   core/src/main/scala/kafka/admin/AdminUtils.scala c399bc7 
>   core/src/main/scala/kafka/admin/CreateTopicCommand.scala 21c1186 
>   core/src/main/scala/kafka/admin/DeleteTopicCommand.scala 3da4518 
>   core/src/main/scala/kafka/admin/ListTopicCommand.scala c760cc0 
>   core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala 
> d5de5f3 
>   core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala aa61fa1 
>   core/src/main/scala/kafka/client/ClientUtils.scala cc526ec 
>   core/src/main/scala/kafka/cluster/Broker.scala b03dea2 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala 719beb5 
>   core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala fa6b213 
>   core/src/main/scala/kafka/consumer/SimpleConsumer.scala 4395fe3 
>   core/src/main/scala/kafka/consumer/TopicCount.scala c8e8406 
>   core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala e7a692a 
>   core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala df83baa 
>   core/src/main/scala/kafka/controller/ControllerChannelManager.scala ed1ce0b 
>   core/src/main/scala/kafka/controller/KafkaController.scala ab18b7a 
>   core/src/main/scala/kafka/controller/PartitionStateMachine.scala a084830 
>   core/src/main/scala/kafka/controller/ReplicaStateMachine.scala c964857 
>   core/src/main/scala/kafka/javaapi/FetchRequest.scala b475240 
>   core/src/main/scala/kafka/javaapi/Implicits.scala ee0a71d 
>   core/src/main/scala/kafka/javaapi/OffsetRequest.scala 1c77ff8 
>   core/src/main/scala/kafka/javaapi/TopicMetadata.scala 97b6dcd 
>   core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala 5f80df7 
>   core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala 
> 14c4c8a 
>   core/src/main/scala/kafka/javaapi/message/ByteBufferMessageSet.scala 
> 0a95248 
>   core/src/main/scala/kafka/javaapi/producer/Producer.scala 7265328 
>   core/src/main/scala/kafka/log/LogManager.scala 4771d11 
>   core/src/main/scala/kafka/network/BoundedByteBufferReceive.scala cab1864 
>   core/src/main/scala/kafka/producer/SyncProducer.scala 306f200 
>   core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala 2e36d3b 
>   core/src/main/scala/kafka/producer/async/ProducerSendThread.scala 2b41a49 
>   core/src/main/scala/kafka/server/AbstractFetcherThread.scala d5addb3 
>   core/src/main/scala/kafka/server/KafkaApis.scala cd02aab 
>   core/src/main/scala/kafka/server/KafkaServerStartable.scala 5be65e9 
>   core/src/main/scala/kafka/server/ReplicaManager.scala f551243 
>   core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala f1f0625 
>   core/src/main/scala/kafka/tools/ImportZkOffsets.scala 55709b5 
>   core/src/main/scala/kafka/tools/JmxTool.scala 7e424e7 
>   core/src/main/scala/kafka/tools/MirrorMaker.scala 6fb545a 
>   core/src/main/scala/kafka/tools/SimpleConsumerShell.scala 3cfa384 
>   core/src/main/scala/kafka/utils/Annotations.scala 28269eb 
>   core/src/main/scala/kafka/utils/Annotations_2.8.scala PRE-CREATION 
>   core/src/main/scala/kafka/utils/Annotations_2.9+.scala PRE-CREATION 
>   core/src/main/scala/kafka/utils/Json.scala f80b2cc 
>   core/src/main/scala/kafka/utils/Mx4jLoader.scala 64d84cc 
>   core/src/main/scala/kafka/utils/Pool.scala 9a86eab 
>   core/src/main/scala/kafka/utils/Utils.scala e83eb5f 
>   core/src/main/scala/kafka/utils/ZkUtils.scala ca1ce12 
>   core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala 06be990 
>   core/src/test/scala/unit/kafka/admin/AdminTest.scala dc0013f 
>   
> core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala 
> fcfc583 
>   
> core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala
>  9f243f0 
>   
> core/src/test/scala/unit/kafka/javaapi/message/BaseMessageSetTestCases.scala 
> abee11b 
>   core/src/test/scala/unit/kafka/log/LogTest.scala 4ed88e8 
>   core/src/test/scala/unit/kafka/metrics/KafkaTimerTest.scala fe5bc09 
>   core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala 1781bc0 
>   core/src/test/scala/unit/kafka/producer/ProducerTest.scala 29331db 
>   core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala b3e89c3 
>   core/src/test/scala/unit/kafka/utils/TestUtils.scala 830608f 
>   perf/src/main/scala/kafka/perf/ConsumerPerformance.scala 3158a22 
>   project/Build.scala b3858f3 
> 
> Diff: https://reviews.apache.org/r/14013/diff/
> 
> 
> Testing
> -------
> 
> successfully performed cross build across all scala versions
> unit tests pass across all scala versions
> 
> 
> Thanks,
> 
> Christopher Freeman
> 
>

Reply via email to