[jira] [Commented] (KAFKA-170) Support for non-blocking polling on multiple streams

2014-05-27 Thread Guozhang Wang (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-170?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14009835#comment-14009835
 ] 

Guozhang Wang commented on KAFKA-170:
-

Claude, in 0.9 we are going to rewrite our consumer client and it will use 
non-blocking poll APIs, do you want to check its design and see if it satisfies 
your use case?

https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.9+Consumer+Rewrite+Design

> Support for non-blocking polling on multiple streams
> 
>
> Key: KAFKA-170
> URL: https://issues.apache.org/jira/browse/KAFKA-170
> Project: Kafka
>  Issue Type: Sub-task
>  Components: core
>Affects Versions: 0.8.0
>Reporter: Jay Kreps
>Priority: Critical
>  Labels: replication
>
> Currently we provide a blocking iterator in the consumer. This is a good 
> mechanism for consuming data from a single topic, but is limited as a 
> mechanism for polling multiple streams.
> For example if one wants to implement a non-blocking union across multiple 
> streams this is hard to do because calls may block indefinitely. A similar 
> situation arrises if trying to implement a streaming join of between two 
> streams.
> I would propose two changes:
> 1. Implement a next(timeout) interface on KafkaMessageStream. This will 
> easily handle some simple cases with minimal change. This handles certain 
> limited cases nicely and is easy to implement, but doesn't actually cover the 
> two cases above.
> 2. Add an interface to poll streams.
> I don't know the best approach for the later api, but it is important to get 
> it right. One option would be to add a 
> ConsumerConnector.drainTopics("topic1", "topic2", ...) which blocks until 
> there is at least one message and then returns a list of triples (topic, 
> partition, message).



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Resolved] (KAFKA-1444) kafka.javaapi.TopicMetadata and PartitionMetadata doesn't forward the toString method

2014-05-27 Thread Neha Narkhede (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1444?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Neha Narkhede resolved KAFKA-1444.
--

Resolution: Fixed

Thanks for the patch, pushed to trunk

> kafka.javaapi.TopicMetadata and PartitionMetadata doesn't forward the 
> toString method
> -
>
> Key: KAFKA-1444
> URL: https://issues.apache.org/jira/browse/KAFKA-1444
> Project: Kafka
>  Issue Type: Bug
>Reporter: Simon Cooper
>Assignee: Sriharsha Chintalapani
>Priority: Minor
>  Labels: newbie
> Attachments: KAFKA-1444.patch
>
>
> the kafka.javaapi.PartitionMetadata and TopicMetadata classes don't forward 
> the toString method to the underlying 
> kafka.api.PartitionMetadata/TopicMetadata classes along with the other 
> methods. This means toString on these classes doesn't work properly.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Closed] (KAFKA-1444) kafka.javaapi.TopicMetadata and PartitionMetadata doesn't forward the toString method

2014-05-27 Thread Neha Narkhede (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1444?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Neha Narkhede closed KAFKA-1444.



> kafka.javaapi.TopicMetadata and PartitionMetadata doesn't forward the 
> toString method
> -
>
> Key: KAFKA-1444
> URL: https://issues.apache.org/jira/browse/KAFKA-1444
> Project: Kafka
>  Issue Type: Bug
>Reporter: Simon Cooper
>Assignee: Sriharsha Chintalapani
>Priority: Minor
>  Labels: newbie
> Attachments: KAFKA-1444.patch
>
>
> the kafka.javaapi.PartitionMetadata and TopicMetadata classes don't forward 
> the toString method to the underlying 
> kafka.api.PartitionMetadata/TopicMetadata classes along with the other 
> methods. This means toString on these classes doesn't work properly.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


Re: Review Request 21865: Patch for KAFKA-1438

2014-05-27 Thread Neha Narkhede

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


We also want to move the *Performance out of perf into tools and remove the 
perf subproject 


config/tools-log4j.properties


Why do we need this?



perf/src/main/scala/kafka/perf/ConsumerPerformance.scala


Why not let the getClass() remain?



perf/src/main/scala/kafka/perf/ProducerPerformance.scala


same here


- Neha Narkhede


On May 23, 2014, 3:52 p.m., Sriharsha Chintalapani wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/21865/
> ---
> 
> (Updated May 23, 2014, 3:52 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1438
> https://issues.apache.org/jira/browse/KAFKA-1438
> 
> 
> Repository: kafka
> 
> 
> Description
> ---
> 
> KAFKA-1438. Migrate Kafka Client tools.
> 
> 
> Diffs
> -
> 
>   bin/kafka-console-consumer.sh e410dde271eea996416f7a67561e45b78f8e8a69 
>   bin/kafka-consumer-perf-test.sh 4ed3ed9fb12edf00a82c06c4e59270a60df43ecd 
>   bin/kafka-producer-perf-test.sh b4efc29d76ff8f53a310f812b9ed3de6875465b5 
>   bin/kafka-run-class.sh d2fc8c0e413769bb89ada99790b90ad616044f28 
>   bin/kafka-simple-consumer-perf-test.sh 
> 2d3e3d3be21065a4d446b9f8ed2e9d31c0685456 
>   bin/windows/kafka-console-consumer.bat 
> 94b20a4d1d2d043b4c4280d00b807d1d27089ac1 
>   bin/windows/kafka-run-class.bat f4d2904a3320ae756ed4171f6e99566f1f0cf963 
>   config/tools-log4j.properties 52f07c96019b4083fc78f62cfb0a81080327e847 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala 
> 24c92877cc7df20b82b7a438d39120b1caa3924d 
>   core/src/main/scala/kafka/server/OffsetManager.scala 
> 54176283e5f199d9db08e2181e7ddfbb9c801fab 
>   perf/config/log4j.properties 542b7391550b2a8bb091a38d08583bea9b3111b6 
>   perf/src/main/scala/kafka/perf/ConsumerPerformance.scala 
> 4dde4687bada1202eaf54be2b06c2156e7726b23 
>   perf/src/main/scala/kafka/perf/PerfConfig.scala 
> a8fc6b9ec81c49c73c35172ddc5a67c6a96c799a 
>   perf/src/main/scala/kafka/perf/ProducerPerformance.scala 
> 00fa90bb286900fb33a05f55e3c34c2cbb5620a1 
>   perf/src/main/scala/kafka/perf/SimpleConsumerPerformance.scala 
> c52ada0a30ae49c8d576703859b9d49348f62feb 
> 
> Diff: https://reviews.apache.org/r/21865/diff/
> 
> 
> Testing
> ---
> 
> 
> Thanks,
> 
> Sriharsha Chintalapani
> 
>



Re: Review Request 21865: Patch for KAFKA-1438

2014-05-27 Thread Sriharsha Chintalapani


> On May 27, 2014, 5:26 p.m., Neha Narkhede wrote:
> > perf/src/main/scala/kafka/perf/ProducerPerformance.scala, line 42
> > 
> >
> > same here

current perf project has its own log4j.properties under perf/config and it 
writes to perf.log . I thought users expects the logs goto perf.log and 
tools-log4j.properties being included in the classpath by kafka-run-class.sh. I 
added specific logging properties for perf related classes and using that by 
referring to kafka.tools.performance. By default tools-log4j uses 
ConsoleAppender and this config used by other tools to write error messages to 
the console.


- Sriharsha


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


On May 23, 2014, 3:52 p.m., Sriharsha Chintalapani wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/21865/
> ---
> 
> (Updated May 23, 2014, 3:52 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1438
> https://issues.apache.org/jira/browse/KAFKA-1438
> 
> 
> Repository: kafka
> 
> 
> Description
> ---
> 
> KAFKA-1438. Migrate Kafka Client tools.
> 
> 
> Diffs
> -
> 
>   bin/kafka-console-consumer.sh e410dde271eea996416f7a67561e45b78f8e8a69 
>   bin/kafka-consumer-perf-test.sh 4ed3ed9fb12edf00a82c06c4e59270a60df43ecd 
>   bin/kafka-producer-perf-test.sh b4efc29d76ff8f53a310f812b9ed3de6875465b5 
>   bin/kafka-run-class.sh d2fc8c0e413769bb89ada99790b90ad616044f28 
>   bin/kafka-simple-consumer-perf-test.sh 
> 2d3e3d3be21065a4d446b9f8ed2e9d31c0685456 
>   bin/windows/kafka-console-consumer.bat 
> 94b20a4d1d2d043b4c4280d00b807d1d27089ac1 
>   bin/windows/kafka-run-class.bat f4d2904a3320ae756ed4171f6e99566f1f0cf963 
>   config/tools-log4j.properties 52f07c96019b4083fc78f62cfb0a81080327e847 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala 
> 24c92877cc7df20b82b7a438d39120b1caa3924d 
>   core/src/main/scala/kafka/server/OffsetManager.scala 
> 54176283e5f199d9db08e2181e7ddfbb9c801fab 
>   perf/config/log4j.properties 542b7391550b2a8bb091a38d08583bea9b3111b6 
>   perf/src/main/scala/kafka/perf/ConsumerPerformance.scala 
> 4dde4687bada1202eaf54be2b06c2156e7726b23 
>   perf/src/main/scala/kafka/perf/PerfConfig.scala 
> a8fc6b9ec81c49c73c35172ddc5a67c6a96c799a 
>   perf/src/main/scala/kafka/perf/ProducerPerformance.scala 
> 00fa90bb286900fb33a05f55e3c34c2cbb5620a1 
>   perf/src/main/scala/kafka/perf/SimpleConsumerPerformance.scala 
> c52ada0a30ae49c8d576703859b9d49348f62feb 
> 
> Diff: https://reviews.apache.org/r/21865/diff/
> 
> 
> Testing
> ---
> 
> 
> Thanks,
> 
> Sriharsha Chintalapani
> 
>



Re: Review Request 21865: Patch for KAFKA-1438

2014-05-27 Thread Sriharsha Chintalapani


> On May 27, 2014, 5:26 p.m., Neha Narkhede wrote:
> > We also want to move the *Performance out of perf into tools and remove the 
> > perf subproject

The diff contains changes necessary for moving perf classes under tools. 


- Sriharsha


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


On May 23, 2014, 3:52 p.m., Sriharsha Chintalapani wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/21865/
> ---
> 
> (Updated May 23, 2014, 3:52 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1438
> https://issues.apache.org/jira/browse/KAFKA-1438
> 
> 
> Repository: kafka
> 
> 
> Description
> ---
> 
> KAFKA-1438. Migrate Kafka Client tools.
> 
> 
> Diffs
> -
> 
>   bin/kafka-console-consumer.sh e410dde271eea996416f7a67561e45b78f8e8a69 
>   bin/kafka-consumer-perf-test.sh 4ed3ed9fb12edf00a82c06c4e59270a60df43ecd 
>   bin/kafka-producer-perf-test.sh b4efc29d76ff8f53a310f812b9ed3de6875465b5 
>   bin/kafka-run-class.sh d2fc8c0e413769bb89ada99790b90ad616044f28 
>   bin/kafka-simple-consumer-perf-test.sh 
> 2d3e3d3be21065a4d446b9f8ed2e9d31c0685456 
>   bin/windows/kafka-console-consumer.bat 
> 94b20a4d1d2d043b4c4280d00b807d1d27089ac1 
>   bin/windows/kafka-run-class.bat f4d2904a3320ae756ed4171f6e99566f1f0cf963 
>   config/tools-log4j.properties 52f07c96019b4083fc78f62cfb0a81080327e847 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala 
> 24c92877cc7df20b82b7a438d39120b1caa3924d 
>   core/src/main/scala/kafka/server/OffsetManager.scala 
> 54176283e5f199d9db08e2181e7ddfbb9c801fab 
>   perf/config/log4j.properties 542b7391550b2a8bb091a38d08583bea9b3111b6 
>   perf/src/main/scala/kafka/perf/ConsumerPerformance.scala 
> 4dde4687bada1202eaf54be2b06c2156e7726b23 
>   perf/src/main/scala/kafka/perf/PerfConfig.scala 
> a8fc6b9ec81c49c73c35172ddc5a67c6a96c799a 
>   perf/src/main/scala/kafka/perf/ProducerPerformance.scala 
> 00fa90bb286900fb33a05f55e3c34c2cbb5620a1 
>   perf/src/main/scala/kafka/perf/SimpleConsumerPerformance.scala 
> c52ada0a30ae49c8d576703859b9d49348f62feb 
> 
> Diff: https://reviews.apache.org/r/21865/diff/
> 
> 
> Testing
> ---
> 
> 
> Thanks,
> 
> Sriharsha Chintalapani
> 
>



Re: Review Request 21865: Patch for KAFKA-1438

2014-05-27 Thread Neha Narkhede


> On May 27, 2014, 5:26 p.m., Neha Narkhede wrote:
> > We also want to move the *Performance out of perf into tools and remove the 
> > perf subproject
> 
> Sriharsha Chintalapani wrote:
> The diff contains changes necessary for moving perf classes under tools.

I applied the patch. It failed to apply on the .bat file and didn't get rid of 
the perf subproject. Would you mind uploading an updated patch?


> On May 27, 2014, 5:26 p.m., Neha Narkhede wrote:
> > perf/src/main/scala/kafka/perf/ProducerPerformance.scala, line 42
> > 
> >
> > same here
> 
> Sriharsha Chintalapani wrote:
> current perf project has its own log4j.properties under perf/config and 
> it writes to perf.log . I thought users expects the logs goto perf.log and 
> tools-log4j.properties being included in the classpath by kafka-run-class.sh. 
> I added specific logging properties for perf related classes and using that 
> by referring to kafka.tools.performance. By default tools-log4j uses 
> ConsoleAppender and this config used by other tools to write error messages 
> to the console.

I guess it will be convenient to have the same ConsoleAppender behavior for 
perf tools as well. Users can pipe the output to other files as they see fit 
right?


- Neha


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


On May 23, 2014, 3:52 p.m., Sriharsha Chintalapani wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/21865/
> ---
> 
> (Updated May 23, 2014, 3:52 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1438
> https://issues.apache.org/jira/browse/KAFKA-1438
> 
> 
> Repository: kafka
> 
> 
> Description
> ---
> 
> KAFKA-1438. Migrate Kafka Client tools.
> 
> 
> Diffs
> -
> 
>   bin/kafka-console-consumer.sh e410dde271eea996416f7a67561e45b78f8e8a69 
>   bin/kafka-consumer-perf-test.sh 4ed3ed9fb12edf00a82c06c4e59270a60df43ecd 
>   bin/kafka-producer-perf-test.sh b4efc29d76ff8f53a310f812b9ed3de6875465b5 
>   bin/kafka-run-class.sh d2fc8c0e413769bb89ada99790b90ad616044f28 
>   bin/kafka-simple-consumer-perf-test.sh 
> 2d3e3d3be21065a4d446b9f8ed2e9d31c0685456 
>   bin/windows/kafka-console-consumer.bat 
> 94b20a4d1d2d043b4c4280d00b807d1d27089ac1 
>   bin/windows/kafka-run-class.bat f4d2904a3320ae756ed4171f6e99566f1f0cf963 
>   config/tools-log4j.properties 52f07c96019b4083fc78f62cfb0a81080327e847 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala 
> 24c92877cc7df20b82b7a438d39120b1caa3924d 
>   core/src/main/scala/kafka/server/OffsetManager.scala 
> 54176283e5f199d9db08e2181e7ddfbb9c801fab 
>   perf/config/log4j.properties 542b7391550b2a8bb091a38d08583bea9b3111b6 
>   perf/src/main/scala/kafka/perf/ConsumerPerformance.scala 
> 4dde4687bada1202eaf54be2b06c2156e7726b23 
>   perf/src/main/scala/kafka/perf/PerfConfig.scala 
> a8fc6b9ec81c49c73c35172ddc5a67c6a96c799a 
>   perf/src/main/scala/kafka/perf/ProducerPerformance.scala 
> 00fa90bb286900fb33a05f55e3c34c2cbb5620a1 
>   perf/src/main/scala/kafka/perf/SimpleConsumerPerformance.scala 
> c52ada0a30ae49c8d576703859b9d49348f62feb 
> 
> Diff: https://reviews.apache.org/r/21865/diff/
> 
> 
> Testing
> ---
> 
> 
> Thanks,
> 
> Sriharsha Chintalapani
> 
>



Re: Review Request 21883: Patch for KAFKA-1459

2014-05-27 Thread Neha Narkhede

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

Ship it!


Ship It!

- Neha Narkhede


On May 24, 2014, 2:05 a.m., Sriharsha Chintalapani wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/21883/
> ---
> 
> (Updated May 24, 2014, 2:05 a.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1459
> https://issues.apache.org/jira/browse/KAFKA-1459
> 
> 
> Repository: kafka
> 
> 
> Description
> ---
> 
> KAFKA-1459. kafka.tools.ConsumerOffsetChecker throws NoNodeException.
> 
> 
> Diffs
> -
> 
>   core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala 
> 88f824fa5afe7964efc569bc0732fd40a55c5c93 
> 
> Diff: https://reviews.apache.org/r/21883/diff/
> 
> 
> Testing
> ---
> 
> 
> Thanks,
> 
> Sriharsha Chintalapani
> 
>



[jira] [Closed] (KAFKA-1311) Add a flag to turn off delete topic until it is stable

2014-05-27 Thread Joel Koshy (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1311?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Joel Koshy closed KAFKA-1311.
-


> Add a flag to turn off delete topic until it is stable
> --
>
> Key: KAFKA-1311
> URL: https://issues.apache.org/jira/browse/KAFKA-1311
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8.1
>Reporter: Neha Narkhede
>Assignee: Neha Narkhede
>Priority: Blocker
> Fix For: 0.8.1.1
>
> Attachments: KAFKA-1311.patch, KAFKA-1311_2014-03-18_17:14:30.patch
>
>
> Currently delete topic is checked in as beta but is turned on by default. 
> Since it is not well tested right now, I'm proposing turning the feature off 
> until it is. This will introduce a config temporarily that we can get rid of 
> once we are confident that delete topic is stable



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Closed] (KAFKA-1317) KafkaServer 0.8.1 not responding to .shutdown() cleanly, possibly related to TopicDeletionManager or MetricsMeter state

2014-05-27 Thread Joel Koshy (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1317?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Joel Koshy closed KAFKA-1317.
-


> KafkaServer 0.8.1 not responding to .shutdown() cleanly, possibly related to 
> TopicDeletionManager or MetricsMeter state
> ---
>
> Key: KAFKA-1317
> URL: https://issues.apache.org/jira/browse/KAFKA-1317
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8.1
>Reporter: Brent Bradbury
>Assignee: Timothy Chen
>Priority: Blocker
>  Labels: newbie
> Fix For: 0.8.1.1
>
> Attachments: KAFKA-1317.patch, KAFKA-1317.patch, 
> KAFKA-1317_2014-03-23_23:48:28.patch, KAFKA-1317_2014-03-24_11:06:15.patch, 
> KAFKA-1317_2014-03-25_15:20:14.patch, KAFKA-1317_2014-03-26_09:48:03.patch, 
> KAFKA-1317_2014-03-26_11:30:57.patch, KAFKA-1317_2014-03-26_15:09:48.patch, 
> KAFKA-1317_2014-03-26_15:18:52.patch, KAFKA-1317_2014-03-27_15:15:05.patch, 
> KAFKA-1317_2014-03-28_09:34:02.patch, threaddump.txt
>
>
> When I run an in-process instance of KafkaServer, send a message through it, 
> then call shutdown(), some threads never exit and the process hangs until the 
> process is killed manually. The same scenario does not result in a hang on 
> 0.8.0. The hang happens when calling both shutdown() by itself as well as 
> shutdown() and awaitShutdown() together. I have seen similar behavior 
> shutting down a deployed kafka server as well, but haven't had time to 
> diagnose whether or not it is the same symptom.
> I suspect either the metrics-meter-tick-thread-1 & 2 or delete-topics-thread
>  (waiting in 
> kafka.controller.TopicDeletionManager.kafka$controller$TopicDeletionManager$$awaitTopicDeletionNotification(TopicDeletionManager.scala:178)
>  is to blame. Since the TopicDeletionManager is new, it seems more suspicious 
> to me. A complete thread dump is attached; the suspect threads are below.
> "delete-topics-thread" prio=5 tid=0x7fb3e31d2800 nid=0x6b03 waiting on 
> condition [0x00013c3b3000]
>java.lang.Thread.State: WAITING (parking)
>   at sun.misc.Unsafe.park(Native Method)
>   - parking to wait for  <0x00012e6e6920> (a 
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>   at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186)
>   at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2043)
>   at 
> kafka.controller.TopicDeletionManager.kafka$controller$TopicDeletionManager$$awaitTopicDeletionNotification(TopicDeletionManager.scala:178)
>   at 
> kafka.controller.TopicDeletionManager$DeleteTopicsThread$$anonfun$doWork$1.apply$mcV$sp(TopicDeletionManager.scala:334)
>   at 
> kafka.controller.TopicDeletionManager$DeleteTopicsThread$$anonfun$doWork$1.apply(TopicDeletionManager.scala:333)
>   at 
> kafka.controller.TopicDeletionManager$DeleteTopicsThread$$anonfun$doWork$1.apply(TopicDeletionManager.scala:333)
>   at kafka.utils.Utils$.inLock(Utils.scala:538)
>   at 
> kafka.controller.TopicDeletionManager$DeleteTopicsThread.doWork(TopicDeletionManager.scala:333)
>   at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:51)
>Locked ownable synchronizers:
>   - None
> "metrics-meter-tick-thread-2" daemon prio=5 tid=0x7fb3e31c1000 nid=0x5f03 
> runnable [0x00013ab8f000]
>java.lang.Thread.State: TIMED_WAITING (parking)
>   at sun.misc.Unsafe.park(Native Method)
>   - parking to wait for  <0x00012e7d05d8> (a 
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>   at 
> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:226)
>   at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2082)
>   at 
> java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:1090)
>   at 
> java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:807)
>   at 
> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1068)
>   at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1130)
>   at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>   at java.lang.Thread.run(Thread.java:724)
>Locked ownable synchronizers:
>   - None
> "metrics-meter-tick-thread-1" daemon prio=5 tid=0x7fb3e31ef800 nid=0x5e03 
> waiting on condition [0x00013a98c000]
>java.lang.Thread.State: WAITING (parking)
>   at sun.misc.Unsafe.park(Native Method)
>   - parking to wait for  <0x00012e7d05d8> (a 
> java.util.concurrent.locks

[jira] [Closed] (KAFKA-1323) log.dirs server property no longer supports relative directories

2014-05-27 Thread Joel Koshy (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1323?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Joel Koshy closed KAFKA-1323.
-


> log.dirs server property no longer supports relative directories
> 
>
> Key: KAFKA-1323
> URL: https://issues.apache.org/jira/browse/KAFKA-1323
> Project: Kafka
>  Issue Type: Bug
>Reporter: Joel Koshy
>Priority: Blocker
> Fix For: 0.8.1.1
>
> Attachments: KAFKA-1323.patch, KAFKA-1323.patch, KAFKA-1323.patch, 
> KAFKA-1323.patch, KAFKA-1323_2014-03-27_16:17:26.patch, 
> KAFKA-1323_2014-03-31_11:57:37.patch, KAFKA-1323_2014-04-02_12:17:57.patch, 
> KAFKA-1323_2014-04-14_16:13:07.patch, KAFKA-1323_2014-04-16_11:53:39.patch, 
> KAFKA-1323_2014-04-16_12:05:40.patch, kafka-1323-trunk-test-failures.png
>
>
> This seems to have been caused by KAFKA-1315 - we now don't support relative 
> directories.
> Steps to reproduce:
> * Set a relative directory for log.dirs. E.g., {{log.dirs=data/kafka-logs}}
> * Bring up the broker and produce some messages: 
> {{./bin/kafka-producer-perf-test.sh --broker-list localhost:9092 --messages 
> 1000 --topic test}}



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Closed] (KAFKA-1310) Zookeeper timeout causes deadlock in Controller

2014-05-27 Thread Joel Koshy (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1310?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Joel Koshy closed KAFKA-1310.
-


> Zookeeper timeout causes deadlock in Controller
> ---
>
> Key: KAFKA-1310
> URL: https://issues.apache.org/jira/browse/KAFKA-1310
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8.1
>Reporter: Fedor Korotkiy
>Assignee: Neha Narkhede
>Priority: Blocker
> Fix For: 0.8.1.1
>
>
> Steps to reproduce:
> 1. Checkout and build 0.8.1 branch from github:
> git clone g...@github.com:apache/kafka.git && cd kafka && git checkout 
> origin/0.8.1 && ./gradlew jar
> 2. Start zookeeper server:
> ./bin/zookeeper-server-start.sh config/zookeeper.properties
> 3. Start kafka server:
> ./bin/kafka-server-start.sh config/server.properties
> 4. Suspend zookeeper process for 10 seconds (ctrl-Z, then %1).
> 5. And kafka hasn't been re-registered in zookeeper.
> ./bin/zookeeper-shell.sh
> ls /brokers/ids
> >> []
> Root cause of the problem seems to be the deadlock between DeleteTopicsThread 
> and SessionExpirationListener in KafkaController.
> 1. DeleteTopicsThread acquires controllerLock and await()-s on 
> deleteTopicsCond in awaitTopicDeletionNotification()
> 2. SessionExpirationListener fires. It acquires controllerLock and tries to 
> shutdown deleteTopicManager(in onControllerResignation()). This interrupts 
> DeleteTopicsThread.
> 3. DeleteTopicsThread can't return from deleteTopicsCond.await() because 
> controllerLock is taken. We got a deadlock.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Closed] (KAFKA-1315) log.dirs property in KafkaServer intolerant of trailing slash

2014-05-27 Thread Joel Koshy (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1315?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Joel Koshy closed KAFKA-1315.
-


> log.dirs property in KafkaServer intolerant of trailing slash
> -
>
> Key: KAFKA-1315
> URL: https://issues.apache.org/jira/browse/KAFKA-1315
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8.0, 0.8.1
>Reporter: Brent Bradbury
>Assignee: Timothy Chen
>Priority: Blocker
>  Labels: newbie
> Fix For: 0.8.1.1
>
> Attachments: KAFKA-1315.patch, KAFKA-1315_2014-03-20_13:28:06.patch, 
> KAFKA-1315_2014-03-20_14:03:34.patch
>
>
> A trailing slash in log.dirs causes a java.util.NoSuchElementException on the 
> producer and a kafka.common.NotLeaderForPartitionException on the consumer. 
> Per this thread: 
> http://mail-archives.apache.org/mod_mbox/kafka-users/201307.mbox/%3ccafbh0q18placokcby8+jg6ef3n8+ysgkfqghw4yb2up18h-...@mail.gmail.com%3E
> This is because we populate the key in
> ReplicaManager.highWatermarkCheckpoints using the "dirs" config, but look
> up the key using log.dir.getParent. So, if you have a trailing slash in the
> config, they won't match. This seems a bug that we should fix. Could you
> file a jira?
> Thanks,
> Jun
> Still occuring for me, using org.apache.kafka:kafka_2.10:0.8.1



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Closed] (KAFKA-1380) 0.8.1.1 release candidate

2014-05-27 Thread Joel Koshy (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1380?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Joel Koshy closed KAFKA-1380.
-


> 0.8.1.1 release candidate
> -
>
> Key: KAFKA-1380
> URL: https://issues.apache.org/jira/browse/KAFKA-1380
> Project: Kafka
>  Issue Type: Task
>Reporter: Joel Koshy
>Assignee: Joel Koshy
> Attachments: KAFKA-1380.patch, KAFKA-1380.patch, KAFKA-1380.patch, 
> KAFKA-1380.patch, KAFKA-1380.patch, KAFKA-1380_2014-04-21_14:47:27.patch, 
> KAFKA-1380_2014-04-21_14:48:56.patch
>
>
> Filing an umbrella tracker to list all dependencies for the 0.8.1.1 bug-fix 
> release.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Closed] (KAFKA-1362) Publish sources and javadoc jars

2014-05-27 Thread Joel Koshy (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1362?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Joel Koshy closed KAFKA-1362.
-


> Publish sources and javadoc jars
> 
>
> Key: KAFKA-1362
> URL: https://issues.apache.org/jira/browse/KAFKA-1362
> Project: Kafka
>  Issue Type: Bug
>  Components: packaging
>Affects Versions: 0.8.1
>Reporter: Stevo Slavic
>Assignee: Joel Koshy
>  Labels: build
> Fix For: 0.8.1.1
>
> Attachments: KAFKA-1362.patch
>
>
> Currently just binaries jars get published on Maven Central (see 
> http://repo1.maven.org/maven2/org/apache/kafka/kafka_2.10/0.8.1/ ). Please 
> also publish sources and javadoc jars.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Closed] (KAFKA-1365) Second Manual preferred replica leader election command always fails

2014-05-27 Thread Joel Koshy (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1365?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Joel Koshy closed KAFKA-1365.
-


> Second Manual preferred replica leader election command always fails
> 
>
> Key: KAFKA-1365
> URL: https://issues.apache.org/jira/browse/KAFKA-1365
> Project: Kafka
>  Issue Type: Bug
>  Components: controller, tools
>Affects Versions: 0.8.1
>Reporter: Ryan Berdeen
>Assignee: Neha Narkhede
>Priority: Critical
> Fix For: 0.8.1.1
>
> Attachments: KAFKA-1365.patch
>
>
> After running kafka-preferred-replica-election.sh once, a second run will 
> fail with "Preferred replica leader election currently in progress for ...".
> The /admin/preferred_replica_election key is never deleted from ZooKeeper, 
> because the "isTriggeredByAutoRebalance" parameter to 
> onPreferredReplicaElection 
> (https://github.com/apache/kafka/blob/0ffec142a991849833d9767be07e895428ccaea1/core/src/main/scala/kafka/controller/KafkaController.scala#L614)
>  is used incorrectly. In the automatic case 
> (https://github.com/apache/kafka/blob/0ffec142a991849833d9767be07e895428ccaea1/core/src/main/scala/kafka/controller/KafkaController.scala#L1119),
>  it is set to false. In the manual case 
> (https://github.com/apache/kafka/blob/0ffec142a991849833d9767be07e895428ccaea1/core/src/main/scala/kafka/controller/KafkaController.scala#L1266)
>  the parameter is not passed, so it defaults to true.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Closed] (KAFKA-1358) Broker throws exception when reconnecting to zookeeper

2014-05-27 Thread Joel Koshy (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1358?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Joel Koshy closed KAFKA-1358.
-


> Broker throws exception when reconnecting to zookeeper
> --
>
> Key: KAFKA-1358
> URL: https://issues.apache.org/jira/browse/KAFKA-1358
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8.1
>Reporter: Timothy Chen
>Assignee: Timothy Chen
> Fix For: 0.8.1.1
>
> Attachments: KAFKA-1358.patch
>
>
> A non-controller broker currently if zk session expires and re-established 
> calls onControllerResignation even though it may not be the controller.
> The result is that the broker gets exception like this: 
> java.lang.NullPointerException
>   at 
> kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply$mcV$sp(KafkaController.scala:340)
>   at 
> kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply(KafkaController.scala:337)
>   at 
> kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply(KafkaController.scala:337)
>   at kafka.utils.Utils$.inLock(Utils.scala:538)
>   at 
> kafka.controller.KafkaController.onControllerResignation(KafkaController.scala:337)
>   at 
> kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply$mcZ$sp(KafkaController.scala:1068)
>   at 
> kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply(KafkaController.scala:1067)
>   at 
> kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply(KafkaController.scala:1067)
>   at kafka.utils.Utils$.inLock(Utils.scala:538)
>   at 
> kafka.controller.KafkaController$SessionExpirationListener.handleNewSession(KafkaController.scala:1067)
>   at org.I0Itec.zkclient.ZkClient$4.run(ZkClient.java:472)
>   at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71)



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Closed] (KAFKA-1398) Topic config changes can be lost and cause fatal exceptions on broker restarts

2014-05-27 Thread Joel Koshy (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1398?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Joel Koshy closed KAFKA-1398.
-


> Topic config changes can be lost and cause fatal exceptions on broker restarts
> --
>
> Key: KAFKA-1398
> URL: https://issues.apache.org/jira/browse/KAFKA-1398
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8.1
>Reporter: Joel Koshy
>Assignee: Jay Kreps
>Priority: Critical
> Fix For: 0.8.1.1
>
> Attachments: KAFKA-1398.patch, KAFKA-1398.patch, KAFKA-1398.patch, 
> KAFKA-1398_2014-04-18_13:03:03.patch
>
>
> Our topic config cleanup policy seems to be broken. When a broker is
> bounced and starting up:
> 1 - Read all the children of the config change path
> 2 - For each, if the change id is greater than the last executed change,
>   then extract the topic information.
> 3 - If there is a log for that topic on this broker, then apply the change.
>   However, if there is no log, then delete the config change.
> In step 3, a delete triggers a child change watch firing on all the other
> brokers. The other brokers currently take all the children of the config
> path but will ignore those config changes that are less than the last
> executed change. At least one issue here is that if a broker does not have
> partitions for a topic then the lastExecutedChange is not updated (for
> that topic).
> Consider this scenario:
> - Three brokers 0, 1, 2
> - Topic A has partitions only assigned to broker 0
> - Topic B has partitions only assigned to broker 1
> - Topic C has partitions only assigned to broker 2
> - Change 0: topic A
> - Change 1: topic B
> - Change 2: topic C
> - lastExecutedChange on broker 0 is 0
> - lastExecutedChange on broker 1 is 1
> - lastExecutedChange on broker 2 is 2
> - Bounce broker 1
> - The above bounce will cause Change 0 and Change 2 to get deleted.
> - Watch fires on broker 0 and 1
> - Broker 0 will try and read the topic corresponding to change 1 (since its
>   lastExecutedChange is 0) and then change 2. That read will fail:
> 2014/04/15 19:35:34.236 INFO [TopicConfigManager] [main] [kafka-server] [] 
> Processed topic config change 25 for topic xyz, setting new config to
>  {retention.ms=360, segment.ms=360}.
> 2014/04/15 19:35:34.238 FATAL [KafkaServerStartable] [main] [kafka-server] [] 
> Fatal error during KafkaServerStable startup. Prepare to shutdown
> org.I0Itec.zkclient.exception.ZkNoNodeException: 
> org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = 
> NoNode for /config/changes/config_change_26
> at 
> org.I0Itec.zkclient.exception.ZkException.create(ZkException.java:47)
> at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:685)
> at org.I0Itec.zkclient.ZkClient.readData(ZkClient.java:766)
> at org.I0Itec.zkclient.ZkClient.readData(ZkClient.java:761)
> at kafka.utils.ZkUtils$.readData(ZkUtils.scala:467)
> at 
> kafka.server.TopicConfigManager$$anonfun$kafka$server$TopicConfigManager$$processConfigChanges$2.apply(TopicConfigManager.scala:97)
> at 
> kafka.server.TopicConfigManager$$anonfun$kafka$server$TopicConfigManager$$processConfigChanges$2.apply(TopicConfigManager.scala:93)
> at 
> scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:57)
> at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:43)
> at 
> kafka.server.TopicConfigManager.kafka$server$TopicConfigManager$$processConfigChanges(TopicConfigManager.scala:93)
> at 
> kafka.server.TopicConfigManager.processAllConfigChanges(TopicConfigManager.scala:81)
> at 
> kafka.server.TopicConfigManager.startup(TopicConfigManager.scala:72)
> at kafka.server.KafkaServer.startup(KafkaServer.scala:104)
> at 
> kafka.server.KafkaServerStartable.startup(KafkaServerStartable.scala:34)
> ...
> Caused by: org.apache.zookeeper.KeeperException$NoNodeException: 
> KeeperErrorCode = NoNode for /config/changes/config_change_26
> at 
> org.apache.zookeeper.KeeperException.create(KeeperException.java:102)
> at 
> org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
> at org.apache.zookeeper.ZooKeeper.getData(ZooKeeper.java:927)
> at org.apache.zookeeper.ZooKeeper.getData(ZooKeeper.java:956)
> at org.I0Itec.zkclient.ZkConnection.readData(ZkConnection.java:103)
> at org.I0Itec.zkclient.ZkClient$9.call(ZkClient.java:770)
> at org.I0Itec.zkclient.ZkClient$9.call(ZkClient.java:766)
> at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
> ... 39 more
> Another issue is that there are two logging statements with incorrect
> qualifiers which makes things a li

[jira] [Closed] (KAFKA-1289) Misc. nitpicks in log cleaner

2014-05-27 Thread Joel Koshy (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1289?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Joel Koshy closed KAFKA-1289.
-


> Misc. nitpicks in log cleaner
> -
>
> Key: KAFKA-1289
> URL: https://issues.apache.org/jira/browse/KAFKA-1289
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8.1
>Reporter: Jay Kreps
> Fix For: 0.8.1
>
> Attachments: KAFKA-1289-v1.patch
>
>
> There are a couple of minor annoyances in the log cleaner in 0.8.1. Since 
> this is one of the major features it would be nice to address these.
> Problems:
> 1. Logging is no longer going to the kafka-cleaner.log 
> 2. Shutdown when the log cleaner is enabled is very slow
> 3. TestLogCleaner uses obsolete properties for the producer and consumer
> In addition I want to change the configuration from "dedupe" to "compact" as 
> we don't use the terminology dedupe anywhere else and I think it is less 
> intuitive.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Closed] (KAFKA-1319) kafka jar doesn't depend on metrics-annotation any more

2014-05-27 Thread Joel Koshy (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1319?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Joel Koshy closed KAFKA-1319.
-


> kafka jar doesn't depend on metrics-annotation any more
> ---
>
> Key: KAFKA-1319
> URL: https://issues.apache.org/jira/browse/KAFKA-1319
> Project: Kafka
>  Issue Type: Bug
>Reporter: Jun Rao
>Assignee: Jun Rao
> Fix For: 0.8.1, 0.8.2
>
> Attachments: KAFKA-1319.patch
>
>




--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Resolved] (KAFKA-1380) 0.8.1.1 release candidate

2014-05-27 Thread Joel Koshy (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1380?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Joel Koshy resolved KAFKA-1380.
---

Resolution: Fixed

> 0.8.1.1 release candidate
> -
>
> Key: KAFKA-1380
> URL: https://issues.apache.org/jira/browse/KAFKA-1380
> Project: Kafka
>  Issue Type: Task
>Reporter: Joel Koshy
>Assignee: Joel Koshy
> Attachments: KAFKA-1380.patch, KAFKA-1380.patch, KAFKA-1380.patch, 
> KAFKA-1380.patch, KAFKA-1380.patch, KAFKA-1380_2014-04-21_14:47:27.patch, 
> KAFKA-1380_2014-04-21_14:48:56.patch
>
>
> Filing an umbrella tracker to list all dependencies for the 0.8.1.1 bug-fix 
> release.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Closed] (KAFKA-1288) add enclosing dir in release tar gz

2014-05-27 Thread Joel Koshy (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1288?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Joel Koshy closed KAFKA-1288.
-


> add enclosing dir in release tar gz
> ---
>
> Key: KAFKA-1288
> URL: https://issues.apache.org/jira/browse/KAFKA-1288
> Project: Kafka
>  Issue Type: Bug
>  Components: packaging
>Affects Versions: 0.8.1
>Reporter: Jun Rao
>Assignee: Jun Rao
> Fix For: 0.8.1
>
> Attachments: KAFKA-1288.patch
>
>




--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-1438) Migrate kafka client tools

2014-05-27 Thread Sriharsha Chintalapani (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1438?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14010100#comment-14010100
 ] 

Sriharsha Chintalapani commented on KAFKA-1438:
---

Updated reviewboard https://reviews.apache.org/r/21865/diff/
 against branch origin/trunk

> Migrate kafka client tools
> --
>
> Key: KAFKA-1438
> URL: https://issues.apache.org/jira/browse/KAFKA-1438
> Project: Kafka
>  Issue Type: Bug
>Reporter: Guozhang Wang
>  Labels: newbie, tools, usability
> Fix For: 0.9.0
>
> Attachments: KAFKA-1438.patch, KAFKA-1438_2014-05-27_11:45:29.patch
>
>
> Currently the console/perf client tools scatter across different packages 
> since they are really used as tools:
> 1. Move Consumer/ProducerPerformance and SimpleConsumerPerformance to tools 
> and remove the perf sub-project.
> 2. Move ConsoleConsumer from kafka.consumer to kafka.tools.
> 3. Move other consumer related tools from kafka.consumer to kafka.tools.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Updated] (KAFKA-1438) Migrate kafka client tools

2014-05-27 Thread Sriharsha Chintalapani (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1438?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Sriharsha Chintalapani updated KAFKA-1438:
--

Attachment: KAFKA-1438_2014-05-27_11:45:29.patch

> Migrate kafka client tools
> --
>
> Key: KAFKA-1438
> URL: https://issues.apache.org/jira/browse/KAFKA-1438
> Project: Kafka
>  Issue Type: Bug
>Reporter: Guozhang Wang
>  Labels: newbie, tools, usability
> Fix For: 0.9.0
>
> Attachments: KAFKA-1438.patch, KAFKA-1438_2014-05-27_11:45:29.patch
>
>
> Currently the console/perf client tools scatter across different packages 
> since they are really used as tools:
> 1. Move Consumer/ProducerPerformance and SimpleConsumerPerformance to tools 
> and remove the perf sub-project.
> 2. Move ConsoleConsumer from kafka.consumer to kafka.tools.
> 3. Move other consumer related tools from kafka.consumer to kafka.tools.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


Re: Review Request 21865: Patch for KAFKA-1438

2014-05-27 Thread Sriharsha Chintalapani

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

(Updated May 27, 2014, 6:45 p.m.)


Review request for kafka.


Bugs: KAFKA-1438
https://issues.apache.org/jira/browse/KAFKA-1438


Repository: kafka


Description
---

KAFKA-1438. Migrate Kafka Client tools.


Diffs (updated)
-

  bin/kafka-console-consumer.sh e410dde271eea996416f7a67561e45b78f8e8a69 
  bin/kafka-consumer-perf-test.sh 4ed3ed9fb12edf00a82c06c4e59270a60df43ecd 
  bin/kafka-producer-perf-test.sh b4efc29d76ff8f53a310f812b9ed3de6875465b5 
  bin/kafka-run-class.sh d2fc8c0e413769bb89ada99790b90ad616044f28 
  bin/kafka-simple-consumer-perf-test.sh 
2d3e3d3be21065a4d446b9f8ed2e9d31c0685456 
  bin/windows/kafka-console-consumer.bat 
94b20a4d1d2d043b4c4280d00b807d1d27089ac1 
  bin/windows/kafka-run-class.bat f4d2904a3320ae756ed4171f6e99566f1f0cf963 
  config/tools-log4j.properties 52f07c96019b4083fc78f62cfb0a81080327e847 
  core/src/main/scala/kafka/consumer/ConsoleConsumer.scala 
24c92877cc7df20b82b7a438d39120b1caa3924d 
  core/src/main/scala/kafka/server/OffsetManager.scala 
54176283e5f199d9db08e2181e7ddfbb9c801fab 
  perf/config/log4j.properties 542b7391550b2a8bb091a38d08583bea9b3111b6 
  perf/src/main/scala/kafka/perf/ConsumerPerformance.scala 
4dde4687bada1202eaf54be2b06c2156e7726b23 
  perf/src/main/scala/kafka/perf/PerfConfig.scala 
a8fc6b9ec81c49c73c35172ddc5a67c6a96c799a 
  perf/src/main/scala/kafka/perf/ProducerPerformance.scala 
00fa90bb286900fb33a05f55e3c34c2cbb5620a1 
  perf/src/main/scala/kafka/perf/SimpleConsumerPerformance.scala 
c52ada0a30ae49c8d576703859b9d49348f62feb 

Diff: https://reviews.apache.org/r/21865/diff/


Testing
---


Thanks,

Sriharsha Chintalapani



[jira] [Closed] (KAFKA-1355) Reduce/optimize update metadata requests sent during leader election

2014-05-27 Thread Joel Koshy (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1355?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Joel Koshy closed KAFKA-1355.
-


> Reduce/optimize update metadata requests sent during leader election
> 
>
> Key: KAFKA-1355
> URL: https://issues.apache.org/jira/browse/KAFKA-1355
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8.1
>Reporter: Joel Koshy
>Priority: Critical
> Fix For: 0.8.1.1
>
> Attachments: KAFKA-1355.patch, KAFKA-1355_2014-04-04_13:48:34.patch, 
> KAFKA-1355_2014-04-04_13:51:22.patch, KAFKA-1355_2014-04-17_14:48:57.patch
>
>
> This is part of the investigation into slow shutdowns in 0.8.1. While
> logging contributes to bulk of the regression, this one also adds
> quite a bit of overhead:
> In addLeaderAndIsrRequest (called for every partition that is led by the
> broker being shut down) we also add an UpdateMetadataRequest - each call to
> addUpdateMetadataRequests does two traversals over _all_ (global)
> partitions. I think it should be straightforward to optimize this a bit.
> Marking as critical, since it is not as big an overhead as the logging.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


Re: Review Request 21865: Patch for KAFKA-1438

2014-05-27 Thread Sriharsha Chintalapani


> On May 27, 2014, 5:26 p.m., Neha Narkhede wrote:
> > We also want to move the *Performance out of perf into tools and remove the 
> > perf subproject
> 
> Sriharsha Chintalapani wrote:
> The diff contains changes necessary for moving perf classes under tools.
> 
> Neha Narkhede wrote:
> I applied the patch. It failed to apply on the .bat file and didn't get 
> rid of the perf subproject. Would you mind uploading an updated patch?

sorry not sure what went wrong there. Tested new patch against the trunk it did 
apply without any issues. Please check the new patch. I'll make changes for 
removing perf related log.


- Sriharsha


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


On May 27, 2014, 6:45 p.m., Sriharsha Chintalapani wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/21865/
> ---
> 
> (Updated May 27, 2014, 6:45 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1438
> https://issues.apache.org/jira/browse/KAFKA-1438
> 
> 
> Repository: kafka
> 
> 
> Description
> ---
> 
> KAFKA-1438. Migrate Kafka Client tools.
> 
> 
> Diffs
> -
> 
>   bin/kafka-console-consumer.sh e410dde271eea996416f7a67561e45b78f8e8a69 
>   bin/kafka-consumer-perf-test.sh 4ed3ed9fb12edf00a82c06c4e59270a60df43ecd 
>   bin/kafka-producer-perf-test.sh b4efc29d76ff8f53a310f812b9ed3de6875465b5 
>   bin/kafka-run-class.sh d2fc8c0e413769bb89ada99790b90ad616044f28 
>   bin/kafka-simple-consumer-perf-test.sh 
> 2d3e3d3be21065a4d446b9f8ed2e9d31c0685456 
>   bin/windows/kafka-console-consumer.bat 
> 94b20a4d1d2d043b4c4280d00b807d1d27089ac1 
>   bin/windows/kafka-run-class.bat f4d2904a3320ae756ed4171f6e99566f1f0cf963 
>   config/tools-log4j.properties 52f07c96019b4083fc78f62cfb0a81080327e847 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala 
> 24c92877cc7df20b82b7a438d39120b1caa3924d 
>   core/src/main/scala/kafka/server/OffsetManager.scala 
> 54176283e5f199d9db08e2181e7ddfbb9c801fab 
>   perf/config/log4j.properties 542b7391550b2a8bb091a38d08583bea9b3111b6 
>   perf/src/main/scala/kafka/perf/ConsumerPerformance.scala 
> 4dde4687bada1202eaf54be2b06c2156e7726b23 
>   perf/src/main/scala/kafka/perf/PerfConfig.scala 
> a8fc6b9ec81c49c73c35172ddc5a67c6a96c799a 
>   perf/src/main/scala/kafka/perf/ProducerPerformance.scala 
> 00fa90bb286900fb33a05f55e3c34c2cbb5620a1 
>   perf/src/main/scala/kafka/perf/SimpleConsumerPerformance.scala 
> c52ada0a30ae49c8d576703859b9d49348f62feb 
> 
> Diff: https://reviews.apache.org/r/21865/diff/
> 
> 
> Testing
> ---
> 
> 
> Thanks,
> 
> Sriharsha Chintalapani
> 
>



Review Request 21937: KAFKA-1316 WIP refactor of sender.

2014-05-27 Thread Jay Kreps

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

Review request for kafka.


Bugs: KAFKA-1316
https://issues.apache.org/jira/browse/KAFKA-1316


Repository: kafka


Description
---

KAFKA-1316 WIP.


Diffs
-

  clients/src/main/java/org/apache/kafka/clients/ClientRequest.java 
PRE-CREATION 
  clients/src/main/java/org/apache/kafka/clients/ClientResponse.java 
PRE-CREATION 
  clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java 
PRE-CREATION 
  clients/src/main/java/org/apache/kafka/clients/ConnectionState.java 
PRE-CREATION 
  clients/src/main/java/org/apache/kafka/clients/InFlightRequests.java 
PRE-CREATION 
  clients/src/main/java/org/apache/kafka/clients/NetworkClient.java 
PRE-CREATION 
  clients/src/main/java/org/apache/kafka/clients/NodeConnectionState.java 
PRE-CREATION 
  clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 
f1def508153b7361b8720dbcc939e06b0e7e45d3 
  clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java 
3e83ae0fefb7578cccbd8dc0cd9b28f22e0951b2 

Diff: https://reviews.apache.org/r/21937/diff/


Testing
---


Thanks,

Jay Kreps



[jira] [Commented] (KAFKA-1316) Refactor Sender

2014-05-27 Thread Jay Kreps (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1316?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14010153#comment-14010153
 ] 

Jay Kreps commented on KAFKA-1316:
--

Created reviewboard https://reviews.apache.org/r/21937/
 against branch trunk

> Refactor Sender
> ---
>
> Key: KAFKA-1316
> URL: https://issues.apache.org/jira/browse/KAFKA-1316
> Project: Kafka
>  Issue Type: Sub-task
>  Components: producer 
>Reporter: Jay Kreps
>Assignee: Jay Kreps
> Attachments: KAFKA-1316.patch
>
>
> Currently most of the logic of the producer I/O thread is in Sender.java.
> However we will need to do a fair number of similar things in the new 
> consumer. Specifically:
>  - Track in-flight requests
>  - Fetch metadata
>  - Manage connection lifecycle
> It may be possible to refactor some of this into a helper class that can be 
> shared with the consumer. This will require some detailed thought.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


Re: Review Request 21865: Patch for KAFKA-1438

2014-05-27 Thread Sriharsha Chintalapani

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

(Updated May 27, 2014, 7:15 p.m.)


Review request for kafka.


Bugs: KAFKA-1438
https://issues.apache.org/jira/browse/KAFKA-1438


Repository: kafka


Description (updated)
---

KAKFA-1438. Migrate Kafka Client tools.


Diffs (updated)
-

  bin/kafka-console-consumer.sh e410dde271eea996416f7a67561e45b78f8e8a69 
  bin/kafka-consumer-perf-test.sh 4ed3ed9fb12edf00a82c06c4e59270a60df43ecd 
  bin/kafka-producer-perf-test.sh b4efc29d76ff8f53a310f812b9ed3de6875465b5 
  bin/kafka-run-class.sh d2fc8c0e413769bb89ada99790b90ad616044f28 
  bin/kafka-simple-consumer-perf-test.sh 
2d3e3d3be21065a4d446b9f8ed2e9d31c0685456 
  bin/windows/kafka-console-consumer.bat 
94b20a4d1d2d043b4c4280d00b807d1d27089ac1 
  bin/windows/kafka-run-class.bat f4d2904a3320ae756ed4171f6e99566f1f0cf963 
  config/tools-log4j.properties 52f07c96019b4083fc78f62cfb0a81080327e847 
  core/src/main/scala/kafka/consumer/ConsoleConsumer.scala 
24c92877cc7df20b82b7a438d39120b1caa3924d 
  core/src/main/scala/kafka/server/OffsetManager.scala 
54176283e5f199d9db08e2181e7ddfbb9c801fab 
  perf/config/log4j.properties 542b7391550b2a8bb091a38d08583bea9b3111b6 
  perf/src/main/scala/kafka/perf/ConsumerPerformance.scala 
4dde4687bada1202eaf54be2b06c2156e7726b23 
  perf/src/main/scala/kafka/perf/PerfConfig.scala 
a8fc6b9ec81c49c73c35172ddc5a67c6a96c799a 
  perf/src/main/scala/kafka/perf/ProducerPerformance.scala 
00fa90bb286900fb33a05f55e3c34c2cbb5620a1 
  perf/src/main/scala/kafka/perf/SimpleConsumerPerformance.scala 
c52ada0a30ae49c8d576703859b9d49348f62feb 

Diff: https://reviews.apache.org/r/21865/diff/


Testing
---


Thanks,

Sriharsha Chintalapani



[jira] [Updated] (KAFKA-1316) Refactor Sender

2014-05-27 Thread Jay Kreps (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1316?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jay Kreps updated KAFKA-1316:
-

Attachment: KAFKA-1316.patch

> Refactor Sender
> ---
>
> Key: KAFKA-1316
> URL: https://issues.apache.org/jira/browse/KAFKA-1316
> Project: Kafka
>  Issue Type: Sub-task
>  Components: producer 
>Reporter: Jay Kreps
>Assignee: Jay Kreps
> Attachments: KAFKA-1316.patch
>
>
> Currently most of the logic of the producer I/O thread is in Sender.java.
> However we will need to do a fair number of similar things in the new 
> consumer. Specifically:
>  - Track in-flight requests
>  - Fetch metadata
>  - Manage connection lifecycle
> It may be possible to refactor some of this into a helper class that can be 
> shared with the consumer. This will require some detailed thought.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-1438) Migrate kafka client tools

2014-05-27 Thread Sriharsha Chintalapani (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1438?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14010155#comment-14010155
 ] 

Sriharsha Chintalapani commented on KAFKA-1438:
---

Updated reviewboard https://reviews.apache.org/r/21865/diff/
 against branch origin/trunk

> Migrate kafka client tools
> --
>
> Key: KAFKA-1438
> URL: https://issues.apache.org/jira/browse/KAFKA-1438
> Project: Kafka
>  Issue Type: Bug
>Reporter: Guozhang Wang
>  Labels: newbie, tools, usability
> Fix For: 0.9.0
>
> Attachments: KAFKA-1438.patch, KAFKA-1438_2014-05-27_11:45:29.patch, 
> KAFKA-1438_2014-05-27_12:16:00.patch
>
>
> Currently the console/perf client tools scatter across different packages 
> since they are really used as tools:
> 1. Move Consumer/ProducerPerformance and SimpleConsumerPerformance to tools 
> and remove the perf sub-project.
> 2. Move ConsoleConsumer from kafka.consumer to kafka.tools.
> 3. Move other consumer related tools from kafka.consumer to kafka.tools.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-1316) Refactor Sender

2014-05-27 Thread Jay Kreps (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1316?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14010156#comment-14010156
 ] 

Jay Kreps commented on KAFKA-1316:
--

I posted a not-totally-working wip so people could see what I was talking about.

> Refactor Sender
> ---
>
> Key: KAFKA-1316
> URL: https://issues.apache.org/jira/browse/KAFKA-1316
> Project: Kafka
>  Issue Type: Sub-task
>  Components: producer 
>Reporter: Jay Kreps
>Assignee: Jay Kreps
> Attachments: KAFKA-1316.patch
>
>
> Currently most of the logic of the producer I/O thread is in Sender.java.
> However we will need to do a fair number of similar things in the new 
> consumer. Specifically:
>  - Track in-flight requests
>  - Fetch metadata
>  - Manage connection lifecycle
> It may be possible to refactor some of this into a helper class that can be 
> shared with the consumer. This will require some detailed thought.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Updated] (KAFKA-1438) Migrate kafka client tools

2014-05-27 Thread Sriharsha Chintalapani (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1438?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Sriharsha Chintalapani updated KAFKA-1438:
--

Attachment: KAFKA-1438_2014-05-27_12:16:00.patch

> Migrate kafka client tools
> --
>
> Key: KAFKA-1438
> URL: https://issues.apache.org/jira/browse/KAFKA-1438
> Project: Kafka
>  Issue Type: Bug
>Reporter: Guozhang Wang
>  Labels: newbie, tools, usability
> Fix For: 0.9.0
>
> Attachments: KAFKA-1438.patch, KAFKA-1438_2014-05-27_11:45:29.patch, 
> KAFKA-1438_2014-05-27_12:16:00.patch
>
>
> Currently the console/perf client tools scatter across different packages 
> since they are really used as tools:
> 1. Move Consumer/ProducerPerformance and SimpleConsumerPerformance to tools 
> and remove the perf sub-project.
> 2. Move ConsoleConsumer from kafka.consumer to kafka.tools.
> 3. Move other consumer related tools from kafka.consumer to kafka.tools.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Created] (KAFKA-1471) Add Producer Unit Tests for LZ4 and LZ4HC compression

2014-05-27 Thread James Oliver (JIRA)
James Oliver created KAFKA-1471:
---

 Summary: Add Producer Unit Tests for LZ4 and LZ4HC compression
 Key: KAFKA-1471
 URL: https://issues.apache.org/jira/browse/KAFKA-1471
 Project: Kafka
  Issue Type: Sub-task
Reporter: James Oliver
 Attachments: KAFKA-1471.patch





--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-1471) Add Producer Unit Tests for LZ4 and LZ4HC compression

2014-05-27 Thread James Oliver (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1471?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14010167#comment-14010167
 ] 

James Oliver commented on KAFKA-1471:
-

Created reviewboard  against branch trunk

> Add Producer Unit Tests for LZ4 and LZ4HC compression
> -
>
> Key: KAFKA-1471
> URL: https://issues.apache.org/jira/browse/KAFKA-1471
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: James Oliver
> Fix For: 0.8.2
>
> Attachments: KAFKA-1471.patch
>
>




--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Updated] (KAFKA-1471) Add Producer Unit Tests for LZ4 and LZ4HC compression

2014-05-27 Thread James Oliver (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1471?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

James Oliver updated KAFKA-1471:


Attachment: KAFKA-1471.patch

> Add Producer Unit Tests for LZ4 and LZ4HC compression
> -
>
> Key: KAFKA-1471
> URL: https://issues.apache.org/jira/browse/KAFKA-1471
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: James Oliver
> Fix For: 0.8.2
>
> Attachments: KAFKA-1471.patch
>
>




--
This message was sent by Atlassian JIRA
(v6.2#6252)


Review Request 21938: Patch for KAFKA-1471

2014-05-27 Thread James Oliver

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

Review request for kafka.


Bugs: KAFKA-1471
https://issues.apache.org/jira/browse/KAFKA-1471


Repository: kafka


Description
---

Add producer unit tests for LZ4 and LZ4HC compression codecs


Diffs
-

  clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 
bc4074ec841650eff48dd8aafc1e6c712d8eae7d 
  clients/src/main/java/org/apache/kafka/common/record/Compressor.java 
0fa6dd2d6aad3dbe33c9c05406931caae4d8ecf5 
  core/src/main/scala/kafka/producer/ConsoleProducer.scala 
a2af988d99a94a20291d6a2dc9bec73197f1b756 
  core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala 
17e2c6e9dfd789acb4b6db37c780c862667e4e11 

Diff: https://reviews.apache.org/r/21938/diff/


Testing
---


Thanks,

James Oliver



[jira] [Commented] (KAFKA-1471) Add Producer Unit Tests for LZ4 and LZ4HC compression

2014-05-27 Thread James Oliver (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1471?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14010168#comment-14010168
 ] 

James Oliver commented on KAFKA-1471:
-

Created reviewboard https://reviews.apache.org/r/21938/diff/
 against branch trunk

> Add Producer Unit Tests for LZ4 and LZ4HC compression
> -
>
> Key: KAFKA-1471
> URL: https://issues.apache.org/jira/browse/KAFKA-1471
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: James Oliver
> Fix For: 0.8.2
>
> Attachments: KAFKA-1471.patch, KAFKA-1471.patch
>
>




--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Updated] (KAFKA-1471) Add Producer Unit Tests for LZ4 and LZ4HC compression

2014-05-27 Thread James Oliver (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1471?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

James Oliver updated KAFKA-1471:


Attachment: KAFKA-1471.patch

> Add Producer Unit Tests for LZ4 and LZ4HC compression
> -
>
> Key: KAFKA-1471
> URL: https://issues.apache.org/jira/browse/KAFKA-1471
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: James Oliver
> Fix For: 0.8.2
>
> Attachments: KAFKA-1471.patch, KAFKA-1471.patch
>
>




--
This message was sent by Atlassian JIRA
(v6.2#6252)


Re: Review Request 21865: Patch for KAFKA-1438

2014-05-27 Thread Neha Narkhede

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


I get the following error while applying the patch-

patching file bin/windows/kafka-run-class.bat
Hunk #1 FAILED at 36.
Hunk #2 FAILED at 121.
2 out of 2 hunks FAILED -- saving rejects to file 
bin/windows/kafka-run-class.bat.rej

- Neha Narkhede


On May 27, 2014, 7:15 p.m., Sriharsha Chintalapani wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/21865/
> ---
> 
> (Updated May 27, 2014, 7:15 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1438
> https://issues.apache.org/jira/browse/KAFKA-1438
> 
> 
> Repository: kafka
> 
> 
> Description
> ---
> 
> KAKFA-1438. Migrate Kafka Client tools.
> 
> 
> Diffs
> -
> 
>   bin/kafka-console-consumer.sh e410dde271eea996416f7a67561e45b78f8e8a69 
>   bin/kafka-consumer-perf-test.sh 4ed3ed9fb12edf00a82c06c4e59270a60df43ecd 
>   bin/kafka-producer-perf-test.sh b4efc29d76ff8f53a310f812b9ed3de6875465b5 
>   bin/kafka-run-class.sh d2fc8c0e413769bb89ada99790b90ad616044f28 
>   bin/kafka-simple-consumer-perf-test.sh 
> 2d3e3d3be21065a4d446b9f8ed2e9d31c0685456 
>   bin/windows/kafka-console-consumer.bat 
> 94b20a4d1d2d043b4c4280d00b807d1d27089ac1 
>   bin/windows/kafka-run-class.bat f4d2904a3320ae756ed4171f6e99566f1f0cf963 
>   config/tools-log4j.properties 52f07c96019b4083fc78f62cfb0a81080327e847 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala 
> 24c92877cc7df20b82b7a438d39120b1caa3924d 
>   core/src/main/scala/kafka/server/OffsetManager.scala 
> 54176283e5f199d9db08e2181e7ddfbb9c801fab 
>   perf/config/log4j.properties 542b7391550b2a8bb091a38d08583bea9b3111b6 
>   perf/src/main/scala/kafka/perf/ConsumerPerformance.scala 
> 4dde4687bada1202eaf54be2b06c2156e7726b23 
>   perf/src/main/scala/kafka/perf/PerfConfig.scala 
> a8fc6b9ec81c49c73c35172ddc5a67c6a96c799a 
>   perf/src/main/scala/kafka/perf/ProducerPerformance.scala 
> 00fa90bb286900fb33a05f55e3c34c2cbb5620a1 
>   perf/src/main/scala/kafka/perf/SimpleConsumerPerformance.scala 
> c52ada0a30ae49c8d576703859b9d49348f62feb 
> 
> Diff: https://reviews.apache.org/r/21865/diff/
> 
> 
> Testing
> ---
> 
> 
> Thanks,
> 
> Sriharsha Chintalapani
> 
>



[jira] [Resolved] (KAFKA-1459) kafka.tools.ConsumerOffsetChecker throws NoNodeException

2014-05-27 Thread Neha Narkhede (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1459?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Neha Narkhede resolved KAFKA-1459.
--

Resolution: Fixed
  Assignee: Sriharsha Chintalapani

Thanks for the patch, pushed to trunk

> kafka.tools.ConsumerOffsetChecker throws NoNodeException
> 
>
> Key: KAFKA-1459
> URL: https://issues.apache.org/jira/browse/KAFKA-1459
> Project: Kafka
>  Issue Type: Bug
>  Components: tools
>Affects Versions: 0.8.1.1
>Reporter: Scott Clasen
>Assignee: Sriharsha Chintalapani
>  Labels: newbie
> Attachments: KAFKA-1459.patch
>
>
> When using the kafka.tools.ConsumerOffsetChecker to check offsets for 
> consumers that are doing manual offset management, and offsets for some but 
> not all partitions have been stored, the offset checker will throw a no node 
> exception.  It should probably return 0 for partitions that dont have an 
> offset recorded yet.
> In this case I was using github.com/pinterest/secor, which may read thousands 
> or millions of messages from a partition before committing an offset.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Closed] (KAFKA-1459) kafka.tools.ConsumerOffsetChecker throws NoNodeException

2014-05-27 Thread Neha Narkhede (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1459?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Neha Narkhede closed KAFKA-1459.



> kafka.tools.ConsumerOffsetChecker throws NoNodeException
> 
>
> Key: KAFKA-1459
> URL: https://issues.apache.org/jira/browse/KAFKA-1459
> Project: Kafka
>  Issue Type: Bug
>  Components: tools
>Affects Versions: 0.8.1.1
>Reporter: Scott Clasen
>Assignee: Sriharsha Chintalapani
>  Labels: newbie
> Attachments: KAFKA-1459.patch
>
>
> When using the kafka.tools.ConsumerOffsetChecker to check offsets for 
> consumers that are doing manual offset management, and offsets for some but 
> not all partitions have been stored, the offset checker will throw a no node 
> exception.  It should probably return 0 for partitions that dont have an 
> offset recorded yet.
> In this case I was using github.com/pinterest/secor, which may read thousands 
> or millions of messages from a partition before committing an offset.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


Re: Review Request 21616: Patch for KAFKA-924

2014-05-27 Thread Neha Narkhede

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



core/src/main/scala/kafka/consumer/ConsoleConsumer.scala


This should be consumer.config for consistency



core/src/main/scala/kafka/consumer/ConsoleConsumer.scala


This will not change the value of groupIdPassed.


- Neha Narkhede


On May 23, 2014, 9:47 p.m., Sriharsha Chintalapani wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/21616/
> ---
> 
> (Updated May 23, 2014, 9:47 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-924
> https://issues.apache.org/jira/browse/KAFKA-924
> 
> 
> Repository: kafka
> 
> 
> Description
> ---
> 
> KAFKA-924. Specify console consumer properties via a single --property 
> command line parameter.
> 
> 
> KAFKA-924. Specify console consumer properties via a single --property 
> command line parameter.
> 
> 
> Diffs
> -
> 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala 
> 24c92877cc7df20b82b7a438d39120b1caa3924d 
> 
> Diff: https://reviews.apache.org/r/21616/diff/
> 
> 
> Testing
> ---
> 
> 
> Thanks,
> 
> Sriharsha Chintalapani
> 
>



Jenkins build is back to normal : Kafka-trunk #191

2014-05-27 Thread Apache Jenkins Server
See 



Re: Review Request 21899: Patch for KAFKA-1382

2014-05-27 Thread Neha Narkhede

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



core/src/main/scala/kafka/utils/ZkUtils.scala


This is a generic API for conditional updates. Logic specific to Kafka 
shouldn't go here. Let's move it to a utils class. We have utils for Zk, API, 
clients but not for replication. Let's create a ReplicationUtils and add a 
common updateIsr utility. Controller and server can use that. Let's file a JIRA 
to refactor some existing utils related to replication to go to ReplicationUtils



core/src/main/scala/kafka/utils/ZkUtils.scala


ZkUtils.parseLeaderAndIsr does this. 


- Neha Narkhede


On May 26, 2014, 5:08 a.m., Sriharsha Chintalapani wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/21899/
> ---
> 
> (Updated May 26, 2014, 5:08 a.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1382
> https://issues.apache.org/jira/browse/KAFKA-1382
> 
> 
> Repository: kafka
> 
> 
> Description
> ---
> 
> KAFKA-1382. Update zkVersion on partition state update failures.
> 
> 
> Diffs
> -
> 
>   core/src/main/scala/kafka/utils/ZkUtils.scala 
> fcbe269b6057b45793ea95f357890d5d6922e8d4 
> 
> Diff: https://reviews.apache.org/r/21899/diff/
> 
> 
> Testing
> ---
> 
> 
> Thanks,
> 
> Sriharsha Chintalapani
> 
>



Re: Review Request 21899: Patch for KAFKA-1382

2014-05-27 Thread Sriharsha Chintalapani


> On May 27, 2014, 9:10 p.m., Neha Narkhede wrote:
> > core/src/main/scala/kafka/utils/ZkUtils.scala, line 423
> > 
> >
> > ZkUtils.parseLeaderAndIsr does this.

I was trying to get LeaderIsrAndControllerEpoch for passed in string and also 
for zookeeper data. But I think i can reuse the existing method. 


- Sriharsha


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


On May 26, 2014, 5:08 a.m., Sriharsha Chintalapani wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/21899/
> ---
> 
> (Updated May 26, 2014, 5:08 a.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1382
> https://issues.apache.org/jira/browse/KAFKA-1382
> 
> 
> Repository: kafka
> 
> 
> Description
> ---
> 
> KAFKA-1382. Update zkVersion on partition state update failures.
> 
> 
> Diffs
> -
> 
>   core/src/main/scala/kafka/utils/ZkUtils.scala 
> fcbe269b6057b45793ea95f357890d5d6922e8d4 
> 
> Diff: https://reviews.apache.org/r/21899/diff/
> 
> 
> Testing
> ---
> 
> 
> Thanks,
> 
> Sriharsha Chintalapani
> 
>



[jira] [Updated] (KAFKA-997) Provide a strict verification mode when reading configuration properties

2014-05-27 Thread Sam Meder (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-997?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Sam Meder updated KAFKA-997:


Resolution: Won't Fix
Status: Resolved  (was: Patch Available)

> Provide a strict verification mode when reading configuration properties
> 
>
> Key: KAFKA-997
> URL: https://issues.apache.org/jira/browse/KAFKA-997
> Project: Kafka
>  Issue Type: Improvement
>  Components: config
>Reporter: Sam Meder
>Assignee: Sam Meder
>Priority: Minor
> Fix For: 0.8.2
>
> Attachments: strict-verification-2.patch
>
>
> This ticket is based on the discussion in KAFKA-943. It introduces a new 
> property that makes the config  system throw an exception when it encounters 
> unrecognized properties. (instead of a simple warn-level log statement). This 
> new property defaults to false.
> Hopefully this will result in fewer instance of out-of-date configuration. 



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-997) Provide a strict verification mode when reading configuration properties

2014-05-27 Thread Sam Meder (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-997?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14010310#comment-14010310
 ] 

Sam Meder commented on KAFKA-997:
-

Looked at this a bit more a while ago and am just going to close the issue. The 
config system currently needs to be extensible (for e.g. metrics system 
properties) so making strict verification work without clear namespacing and 
extension rules is impossible. We're instead taking the approach of creating 
internal constants for all the Kafka properties, so we can manage and detect 
change that way.

> Provide a strict verification mode when reading configuration properties
> 
>
> Key: KAFKA-997
> URL: https://issues.apache.org/jira/browse/KAFKA-997
> Project: Kafka
>  Issue Type: Improvement
>  Components: config
>Reporter: Sam Meder
>Assignee: Sam Meder
>Priority: Minor
> Fix For: 0.8.2
>
> Attachments: strict-verification-2.patch
>
>
> This ticket is based on the discussion in KAFKA-943. It introduces a new 
> property that makes the config  system throw an exception when it encounters 
> unrecognized properties. (instead of a simple warn-level log statement). This 
> new property defaults to false.
> Hopefully this will result in fewer instance of out-of-date configuration. 



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-1316) Refactor Sender

2014-05-27 Thread Neha Narkhede (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1316?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14010333#comment-14010333
 ] 

Neha Narkhede commented on KAFKA-1316:
--

[~jkreps] Thanks for uploading the draft patch. Didn't do an indepth review 
since this is WIP, but have a few high level comments -
1. It's probably worth adding a blocking API for poll since almost all consumer 
coordination would want that
2. There is an issue with implementing heartbeats using the current 
Sender/NetworkClient. If a user configures a higher fetch timeout, it will not 
be able to heartbeat during that time. One option (that is pretty hacky) is to 
create a separate connection between the consumer coordinator and the consumer. 
Thoughts?

> Refactor Sender
> ---
>
> Key: KAFKA-1316
> URL: https://issues.apache.org/jira/browse/KAFKA-1316
> Project: Kafka
>  Issue Type: Sub-task
>  Components: producer 
>Reporter: Jay Kreps
>Assignee: Jay Kreps
> Attachments: KAFKA-1316.patch
>
>
> Currently most of the logic of the producer I/O thread is in Sender.java.
> However we will need to do a fair number of similar things in the new 
> consumer. Specifically:
>  - Track in-flight requests
>  - Fetch metadata
>  - Manage connection lifecycle
> It may be possible to refactor some of this into a helper class that can be 
> shared with the consumer. This will require some detailed thought.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Comment Edited] (KAFKA-1316) Refactor Sender

2014-05-27 Thread Neha Narkhede (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1316?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14010333#comment-14010333
 ] 

Neha Narkhede edited comment on KAFKA-1316 at 5/27/14 9:44 PM:
---

[~jkreps] Thanks for uploading the draft patch. Didn't do an indepth review 
since this is WIP, but have a few high level comments -
1. It's probably worth adding a blocking API for NetworkClient.poll since 
almost all consumer coordination would want that
2. There is an issue with implementing heartbeats using the current 
Sender/NetworkClient. If a user configures a higher fetch timeout, it will not 
be able to heartbeat during that time. One option (that is pretty hacky) is to 
create a separate connection between the consumer coordinator and the consumer. 
Thoughts?


was (Author: nehanarkhede):
[~jkreps] Thanks for uploading the draft patch. Didn't do an indepth review 
since this is WIP, but have a few high level comments -
1. It's probably worth adding a blocking API for poll since almost all consumer 
coordination would want that
2. There is an issue with implementing heartbeats using the current 
Sender/NetworkClient. If a user configures a higher fetch timeout, it will not 
be able to heartbeat during that time. One option (that is pretty hacky) is to 
create a separate connection between the consumer coordinator and the consumer. 
Thoughts?

> Refactor Sender
> ---
>
> Key: KAFKA-1316
> URL: https://issues.apache.org/jira/browse/KAFKA-1316
> Project: Kafka
>  Issue Type: Sub-task
>  Components: producer 
>Reporter: Jay Kreps
>Assignee: Jay Kreps
> Attachments: KAFKA-1316.patch
>
>
> Currently most of the logic of the producer I/O thread is in Sender.java.
> However we will need to do a fair number of similar things in the new 
> consumer. Specifically:
>  - Track in-flight requests
>  - Fetch metadata
>  - Manage connection lifecycle
> It may be possible to refactor some of this into a helper class that can be 
> shared with the consumer. This will require some detailed thought.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


Re: Review Request 21616: Patch for KAFKA-924

2014-05-27 Thread Sriharsha Chintalapani

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

(Updated May 27, 2014, 9:51 p.m.)


Review request for kafka.


Bugs: KAFKA-924
https://issues.apache.org/jira/browse/KAFKA-924


Repository: kafka


Description (updated)
---

KAFKA-924. Specify console consumer properties via a single --property command 
line parameter.


KAFKA-924. Specify console consumer properties via a single --property command 
line parameter.


KAFKA-924. Specify console consumer properties via a single --property command 
line parameter.


Diffs (updated)
-

  core/src/main/scala/kafka/consumer/ConsoleConsumer.scala 
24c92877cc7df20b82b7a438d39120b1caa3924d 

Diff: https://reviews.apache.org/r/21616/diff/


Testing
---


Thanks,

Sriharsha Chintalapani



[jira] [Updated] (KAFKA-924) Specify console consumer properties via a single --property command line parameter

2014-05-27 Thread Sriharsha Chintalapani (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-924?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Sriharsha Chintalapani updated KAFKA-924:
-

Attachment: KAFKA-924_2014-05-27_14:51:31.patch

> Specify console consumer properties via a single --property command line 
> parameter
> --
>
> Key: KAFKA-924
> URL: https://issues.apache.org/jira/browse/KAFKA-924
> Project: Kafka
>  Issue Type: Bug
>Reporter: Swapnil Ghike
>  Labels: newbie
> Fix For: 0.8.2
>
> Attachments: KAFKA-924.patch, KAFKA-924_2014-05-23_14:47:30.patch, 
> KAFKA-924_2014-05-27_14:51:31.patch
>
>
> Quoting Neha from KAFKA-917:
> I think the right way to add access to all consumer properties is to specify 
> it through a single --property command line parameter that takes in 
> "key1=value1,key2=value2,..." list. That will make sure we don't have to keep 
> changing console consumer as well add/remove config options on the consumer. 
> Some configs make sense to be top level for console consumer though. Things 
> like topic, from-beginning, groupid etc. Rest can be specified through the 
> "property" parameter. 



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-924) Specify console consumer properties via a single --property command line parameter

2014-05-27 Thread Sriharsha Chintalapani (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-924?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14010354#comment-14010354
 ] 

Sriharsha Chintalapani commented on KAFKA-924:
--

Updated reviewboard https://reviews.apache.org/r/21616/diff/
 against branch origin/trunk

> Specify console consumer properties via a single --property command line 
> parameter
> --
>
> Key: KAFKA-924
> URL: https://issues.apache.org/jira/browse/KAFKA-924
> Project: Kafka
>  Issue Type: Bug
>Reporter: Swapnil Ghike
>  Labels: newbie
> Fix For: 0.8.2
>
> Attachments: KAFKA-924.patch, KAFKA-924_2014-05-23_14:47:30.patch, 
> KAFKA-924_2014-05-27_14:51:31.patch
>
>
> Quoting Neha from KAFKA-917:
> I think the right way to add access to all consumer properties is to specify 
> it through a single --property command line parameter that takes in 
> "key1=value1,key2=value2,..." list. That will make sure we don't have to keep 
> changing console consumer as well add/remove config options on the consumer. 
> Some configs make sense to be top level for console consumer though. Things 
> like topic, from-beginning, groupid etc. Rest can be specified through the 
> "property" parameter. 



--
This message was sent by Atlassian JIRA
(v6.2#6252)


Re: Review Request 21616: Patch for KAFKA-924

2014-05-27 Thread Neha Narkhede

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

Ship it!


Ship It!

- Neha Narkhede


On May 27, 2014, 9:51 p.m., Sriharsha Chintalapani wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/21616/
> ---
> 
> (Updated May 27, 2014, 9:51 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-924
> https://issues.apache.org/jira/browse/KAFKA-924
> 
> 
> Repository: kafka
> 
> 
> Description
> ---
> 
> KAFKA-924. Specify console consumer properties via a single --property 
> command line parameter.
> 
> 
> KAFKA-924. Specify console consumer properties via a single --property 
> command line parameter.
> 
> 
> KAFKA-924. Specify console consumer properties via a single --property 
> command line parameter.
> 
> 
> Diffs
> -
> 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala 
> 24c92877cc7df20b82b7a438d39120b1caa3924d 
> 
> Diff: https://reviews.apache.org/r/21616/diff/
> 
> 
> Testing
> ---
> 
> 
> Thanks,
> 
> Sriharsha Chintalapani
> 
>



[jira] [Closed] (KAFKA-924) Specify console consumer properties via a single --property command line parameter

2014-05-27 Thread Neha Narkhede (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-924?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Neha Narkhede closed KAFKA-924.
---


> Specify console consumer properties via a single --property command line 
> parameter
> --
>
> Key: KAFKA-924
> URL: https://issues.apache.org/jira/browse/KAFKA-924
> Project: Kafka
>  Issue Type: Bug
>Reporter: Swapnil Ghike
>Assignee: Sriharsha Chintalapani
>  Labels: newbie
> Fix For: 0.8.2
>
> Attachments: KAFKA-924.patch, KAFKA-924_2014-05-23_14:47:30.patch, 
> KAFKA-924_2014-05-27_14:51:31.patch
>
>
> Quoting Neha from KAFKA-917:
> I think the right way to add access to all consumer properties is to specify 
> it through a single --property command line parameter that takes in 
> "key1=value1,key2=value2,..." list. That will make sure we don't have to keep 
> changing console consumer as well add/remove config options on the consumer. 
> Some configs make sense to be top level for console consumer though. Things 
> like topic, from-beginning, groupid etc. Rest can be specified through the 
> "property" parameter. 



--
This message was sent by Atlassian JIRA
(v6.2#6252)


Re: Review Request 21865: Patch for KAFKA-1438

2014-05-27 Thread Sriharsha Chintalapani


> On May 27, 2014, 8:40 p.m., Neha Narkhede wrote:
> > I get the following error while applying the patch-
> > 
> > patching file bin/windows/kafka-run-class.bat
> > Hunk #1 FAILED at 36.
> > Hunk #2 FAILED at 121.
> > 2 out of 2 hunks FAILED -- saving rejects to file 
> > bin/windows/kafka-run-class.bat.rej

whats your git version. I am using git 1.8.5.2 on OS X 10.9.2. Probably issue 
with the older version of git but I'll test it again.
Thanks.


- Sriharsha


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


On May 27, 2014, 7:15 p.m., Sriharsha Chintalapani wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/21865/
> ---
> 
> (Updated May 27, 2014, 7:15 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1438
> https://issues.apache.org/jira/browse/KAFKA-1438
> 
> 
> Repository: kafka
> 
> 
> Description
> ---
> 
> KAKFA-1438. Migrate Kafka Client tools.
> 
> 
> Diffs
> -
> 
>   bin/kafka-console-consumer.sh e410dde271eea996416f7a67561e45b78f8e8a69 
>   bin/kafka-consumer-perf-test.sh 4ed3ed9fb12edf00a82c06c4e59270a60df43ecd 
>   bin/kafka-producer-perf-test.sh b4efc29d76ff8f53a310f812b9ed3de6875465b5 
>   bin/kafka-run-class.sh d2fc8c0e413769bb89ada99790b90ad616044f28 
>   bin/kafka-simple-consumer-perf-test.sh 
> 2d3e3d3be21065a4d446b9f8ed2e9d31c0685456 
>   bin/windows/kafka-console-consumer.bat 
> 94b20a4d1d2d043b4c4280d00b807d1d27089ac1 
>   bin/windows/kafka-run-class.bat f4d2904a3320ae756ed4171f6e99566f1f0cf963 
>   config/tools-log4j.properties 52f07c96019b4083fc78f62cfb0a81080327e847 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala 
> 24c92877cc7df20b82b7a438d39120b1caa3924d 
>   core/src/main/scala/kafka/server/OffsetManager.scala 
> 54176283e5f199d9db08e2181e7ddfbb9c801fab 
>   perf/config/log4j.properties 542b7391550b2a8bb091a38d08583bea9b3111b6 
>   perf/src/main/scala/kafka/perf/ConsumerPerformance.scala 
> 4dde4687bada1202eaf54be2b06c2156e7726b23 
>   perf/src/main/scala/kafka/perf/PerfConfig.scala 
> a8fc6b9ec81c49c73c35172ddc5a67c6a96c799a 
>   perf/src/main/scala/kafka/perf/ProducerPerformance.scala 
> 00fa90bb286900fb33a05f55e3c34c2cbb5620a1 
>   perf/src/main/scala/kafka/perf/SimpleConsumerPerformance.scala 
> c52ada0a30ae49c8d576703859b9d49348f62feb 
> 
> Diff: https://reviews.apache.org/r/21865/diff/
> 
> 
> Testing
> ---
> 
> 
> Thanks,
> 
> Sriharsha Chintalapani
> 
>



[jira] [Resolved] (KAFKA-924) Specify console consumer properties via a single --property command line parameter

2014-05-27 Thread Neha Narkhede (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-924?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Neha Narkhede resolved KAFKA-924.
-

Resolution: Fixed
  Assignee: Sriharsha Chintalapani

Thanks for the patches, pushed to trunk

> Specify console consumer properties via a single --property command line 
> parameter
> --
>
> Key: KAFKA-924
> URL: https://issues.apache.org/jira/browse/KAFKA-924
> Project: Kafka
>  Issue Type: Bug
>Reporter: Swapnil Ghike
>Assignee: Sriharsha Chintalapani
>  Labels: newbie
> Fix For: 0.8.2
>
> Attachments: KAFKA-924.patch, KAFKA-924_2014-05-23_14:47:30.patch, 
> KAFKA-924_2014-05-27_14:51:31.patch
>
>
> Quoting Neha from KAFKA-917:
> I think the right way to add access to all consumer properties is to specify 
> it through a single --property command line parameter that takes in 
> "key1=value1,key2=value2,..." list. That will make sure we don't have to keep 
> changing console consumer as well add/remove config options on the consumer. 
> Some configs make sense to be top level for console consumer though. Things 
> like topic, from-beginning, groupid etc. Rest can be specified through the 
> "property" parameter. 



--
This message was sent by Atlassian JIRA
(v6.2#6252)


Re: Review Request 21865: Patch for KAFKA-1438

2014-05-27 Thread Neha Narkhede


> On May 27, 2014, 8:40 p.m., Neha Narkhede wrote:
> > I get the following error while applying the patch-
> > 
> > patching file bin/windows/kafka-run-class.bat
> > Hunk #1 FAILED at 36.
> > Hunk #2 FAILED at 121.
> > 2 out of 2 hunks FAILED -- saving rejects to file 
> > bin/windows/kafka-run-class.bat.rej
> 
> Sriharsha Chintalapani wrote:
> whats your git version. I am using git 1.8.5.2 on OS X 10.9.2. Probably 
> issue with the older version of git but I'll test it again.
> Thanks.

This is using patch.
patch -p1 -i kafka-1438.patch


- Neha


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


On May 27, 2014, 7:15 p.m., Sriharsha Chintalapani wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/21865/
> ---
> 
> (Updated May 27, 2014, 7:15 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1438
> https://issues.apache.org/jira/browse/KAFKA-1438
> 
> 
> Repository: kafka
> 
> 
> Description
> ---
> 
> KAKFA-1438. Migrate Kafka Client tools.
> 
> 
> Diffs
> -
> 
>   bin/kafka-console-consumer.sh e410dde271eea996416f7a67561e45b78f8e8a69 
>   bin/kafka-consumer-perf-test.sh 4ed3ed9fb12edf00a82c06c4e59270a60df43ecd 
>   bin/kafka-producer-perf-test.sh b4efc29d76ff8f53a310f812b9ed3de6875465b5 
>   bin/kafka-run-class.sh d2fc8c0e413769bb89ada99790b90ad616044f28 
>   bin/kafka-simple-consumer-perf-test.sh 
> 2d3e3d3be21065a4d446b9f8ed2e9d31c0685456 
>   bin/windows/kafka-console-consumer.bat 
> 94b20a4d1d2d043b4c4280d00b807d1d27089ac1 
>   bin/windows/kafka-run-class.bat f4d2904a3320ae756ed4171f6e99566f1f0cf963 
>   config/tools-log4j.properties 52f07c96019b4083fc78f62cfb0a81080327e847 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala 
> 24c92877cc7df20b82b7a438d39120b1caa3924d 
>   core/src/main/scala/kafka/server/OffsetManager.scala 
> 54176283e5f199d9db08e2181e7ddfbb9c801fab 
>   perf/config/log4j.properties 542b7391550b2a8bb091a38d08583bea9b3111b6 
>   perf/src/main/scala/kafka/perf/ConsumerPerformance.scala 
> 4dde4687bada1202eaf54be2b06c2156e7726b23 
>   perf/src/main/scala/kafka/perf/PerfConfig.scala 
> a8fc6b9ec81c49c73c35172ddc5a67c6a96c799a 
>   perf/src/main/scala/kafka/perf/ProducerPerformance.scala 
> 00fa90bb286900fb33a05f55e3c34c2cbb5620a1 
>   perf/src/main/scala/kafka/perf/SimpleConsumerPerformance.scala 
> c52ada0a30ae49c8d576703859b9d49348f62feb 
> 
> Diff: https://reviews.apache.org/r/21865/diff/
> 
> 
> Testing
> ---
> 
> 
> Thanks,
> 
> Sriharsha Chintalapani
> 
>



[jira] [Created] (KAFKA-1472) Add the compression ratio metrics in the new producer

2014-05-27 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-1472:


 Summary: Add the compression ratio metrics in the new producer
 Key: KAFKA-1472
 URL: https://issues.apache.org/jira/browse/KAFKA-1472
 Project: Kafka
  Issue Type: Sub-task
Reporter: Guozhang Wang
 Fix For: 0.8.2


New producer's bytes throughput is based on compressed data. With the current 
implementation, it would be very easy to get the compression ratio of each 
closed batch upon drain(). It would be good to have such metric to help 
understanding the throughput metrics.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


Build failed in Jenkins: Kafka-trunk #192

2014-05-27 Thread Apache Jenkins Server
See 

Changes:

[neha.narkhede] KAFKA-1459 kafka.tools.ConsumerOffsetChecker throws 
NoNodeException; reviewed by Neha Narkhede

[neha.narkhede] KAFKA-924. Specify console consumer properties via a single 
--property command line parameter; reviewed by Neha Narkhede

--
[...truncated 992 lines...]

kafka.admin.AdminTest > testPartitionReassignmentNonOverlappingReplicas PASSED

kafka.admin.AdminTest > testReassigningNonExistingPartition PASSED

kafka.admin.AdminTest > testResumePartitionReassignmentThatWasCompleted PASSED

kafka.admin.AdminTest > testPreferredReplicaJsonData PASSED

kafka.admin.AdminTest > testBasicPreferredReplicaElection PASSED

kafka.admin.AdminTest > testShutdownBroker PASSED

kafka.admin.AdminTest > testTopicConfigChange PASSED

kafka.consumer.ConsumerIteratorTest > 
testConsumerIteratorDeduplicationDeepIterator PASSED

kafka.consumer.ConsumerIteratorTest > testConsumerIteratorDecodingFailure PASSED

kafka.consumer.ZookeeperConsumerConnectorTest > testCompression PASSED

kafka.consumer.ZookeeperConsumerConnectorTest > testBasic PASSED

kafka.consumer.ZookeeperConsumerConnectorTest > testCompressionSetConsumption 
PASSED

kafka.consumer.ZookeeperConsumerConnectorTest > testConsumerDecoder PASSED

kafka.consumer.ZookeeperConsumerConnectorTest > testLeaderSelectionForPartition 
PASSED

kafka.consumer.TopicFilterTest > testWhitelists PASSED

kafka.consumer.TopicFilterTest > testBlacklists PASSED

kafka.utils.JsonTest > testJsonEncoding PASSED

kafka.utils.UtilsTest > testSwallow PASSED

kafka.utils.UtilsTest > testCircularIterator PASSED

kafka.utils.UtilsTest > testReadBytes PASSED

kafka.utils.UtilsTest > testReplaceSuffix PASSED

kafka.utils.UtilsTest > testReadInt PASSED

kafka.utils.UtilsTest > testCsvList PASSED

kafka.utils.UtilsTest > testInLock PASSED

kafka.utils.SchedulerTest > testMockSchedulerNonPeriodicTask PASSED

kafka.utils.SchedulerTest > testMockSchedulerPeriodicTask PASSED

kafka.utils.SchedulerTest > testReentrantTaskInMockScheduler PASSED

kafka.utils.SchedulerTest > testNonPeriodicTask PASSED

kafka.utils.SchedulerTest > testPeriodicTask PASSED

kafka.utils.IteratorTemplateTest > testIterator PASSED

kafka.message.MessageCompressionTest > testSimpleCompressDecompress PASSED

kafka.message.MessageCompressionTest > testComplexCompressDecompress PASSED

kafka.message.MessageTest > testFieldValues PASSED

kafka.message.MessageTest > testChecksum PASSED

kafka.message.MessageTest > testEquality PASSED

kafka.message.MessageTest > testIsHashable PASSED

kafka.message.ByteBufferMessageSetTest > testWrittenEqualsRead PASSED

kafka.message.ByteBufferMessageSetTest > testIteratorIsConsistent PASSED

kafka.message.ByteBufferMessageSetTest > testSizeInBytes PASSED

kafka.message.ByteBufferMessageSetTest > testWriteTo PASSED

kafka.message.ByteBufferMessageSetTest > testEquals PASSED

kafka.message.ByteBufferMessageSetTest > testIterator PASSED

kafka.message.ByteBufferMessageSetTest > testValidBytes PASSED

kafka.message.ByteBufferMessageSetTest > testValidBytesWithCompression PASSED

kafka.message.ByteBufferMessageSetTest > testOffsetAssignment PASSED

kafka.integration.UncleanLeaderElectionTest > testUncleanLeaderElectionEnabled 
PASSED

kafka.integration.UncleanLeaderElectionTest > testUncleanLeaderElectionDisabled 
PASSED

kafka.integration.UncleanLeaderElectionTest > 
testUncleanLeaderElectionEnabledByTopicOverride PASSED

kafka.integration.UncleanLeaderElectionTest > 
testCleanLeaderElectionDisabledByTopicOverride PASSED

kafka.integration.UncleanLeaderElectionTest > 
testUncleanLeaderElectionInvalidTopicOverride PASSED

kafka.integration.FetcherTest > testFetcher PASSED

kafka.integration.PrimitiveApiTest > testFetchRequestCanProperlySerialize PASSED

kafka.integration.PrimitiveApiTest > testEmptyFetchRequest PASSED

kafka.integration.PrimitiveApiTest > testDefaultEncoderProducerAndFetch PASSED

kafka.integration.PrimitiveApiTest > 
testDefaultEncoderProducerAndFetchWithCompression PASSED

kafka.integration.PrimitiveApiTest > testProduceAndMultiFetch PASSED

kafka.integration.PrimitiveApiTest > testMultiProduce PASSED

kafka.integration.PrimitiveApiTest > testConsumerEmptyTopic PASSED

kafka.integration.PrimitiveApiTest > testPipelinedProduceRequests PASSED

kafka.integration.AutoOffsetResetTest > testResetToEarliestWhenOffsetTooHigh 
PASSED

kafka.integration.AutoOffsetResetTest > testResetToEarliestWhenOffsetTooLow 
PASSED

kafka.integration.AutoOffsetResetTest > testResetToLatestWhenOffsetTooHigh 
PASSED

kafka.integration.AutoOffsetResetTest > testResetToLatestWhenOffsetTooLow PASSED

kafka.integration.TopicMetadataTest > testAutoCreateTopic PASSED

kafka.integration.TopicMetadataTest > testTopicMetadataRequest PASSED

kafka.integration.TopicMetadataTest > testBasicTopicMetadata PASSED

kafka.integration.TopicMetadataTest > testGetAllTopicMetadata PASSED

kafka.integration.R

[jira] [Comment Edited] (KAFKA-1316) Refactor Sender

2014-05-27 Thread Jay Kreps (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1316?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14010498#comment-14010498
 ] 

Jay Kreps edited comment on KAFKA-1316 at 5/27/14 11:38 PM:


For (2) I think there are two solutions
1. Change the server semantics to allow processing multiple requests at the 
same time and out of order. 
2. Use two connections

Quick discussion:
1. Allowing out-of-order requests complicates things in the clients a bit as 
you can no longer reason that the Nth response is for the Nth request you made. 
It also isn't clear what we would even guarantee on the server side. The two 
things that we have to do is handle produce requests in order and produce back 
pressure when two much data is sent. Backpressure means the socket server needs 
to stop reading requests, but to make that decision it needs to have parsed the 
request and know it is a produce request...

2. Using two connections might work. It is a bit hacky. The consumer would need 
to create a Node object for the host-port of the current co-ordinator and then 
things would work from there on (I think). The node id would need to be some 
negative number or something. I'm not really sure if there is a clean 
generalization of this.


was (Author: jkreps):
For (2) I think there are two solutions
1. Change the server semantics to allow processing multiple requests at the 
same time and out of order. 
2. Use two connections

Quick discussion:
1. Allowing out-of-order requests complicates things in the clients a bit as 
you can no longer reason that the Nth response is for the Nth request you made. 
It also isn't clear what we would even guarantee on the server side. The two 
things that we have to do is handle produce requests in order and produce back 
pressure when two much data is sent. Backpressure means the socket server needs 
to stop reading requests, but to make that decision it needs to have parsed the 
request and know it is a produce request...

2. Using two connections might work. It is a bit hacky. The consumer would need 
to create a Node object for the host-port of the current co-ordinator and then 
things would work from there on (I think).

> Refactor Sender
> ---
>
> Key: KAFKA-1316
> URL: https://issues.apache.org/jira/browse/KAFKA-1316
> Project: Kafka
>  Issue Type: Sub-task
>  Components: producer 
>Reporter: Jay Kreps
>Assignee: Jay Kreps
> Attachments: KAFKA-1316.patch
>
>
> Currently most of the logic of the producer I/O thread is in Sender.java.
> However we will need to do a fair number of similar things in the new 
> consumer. Specifically:
>  - Track in-flight requests
>  - Fetch metadata
>  - Manage connection lifecycle
> It may be possible to refactor some of this into a helper class that can be 
> shared with the consumer. This will require some detailed thought.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-1316) Refactor Sender

2014-05-27 Thread Jay Kreps (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1316?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14010498#comment-14010498
 ] 

Jay Kreps commented on KAFKA-1316:
--

For (2) I think there are two solutions
1. Change the server semantics to allow processing multiple requests at the 
same time and out of order. 
2. Use two connections

Quick discussion:
1. Allowing out-of-order requests complicates things in the clients a bit as 
you can no longer reason that the Nth response is for the Nth request you made. 
It also isn't clear what we would even guarantee on the server side. The two 
things that we have to do is handle produce requests in order and produce back 
pressure when two much data is sent. Backpressure means the socket server needs 
to stop reading requests, but to make that decision it needs to have parsed the 
request and know it is a produce request...

2. Using two connections might work. It is a bit hacky. The consumer would need 
to create a Node object for the host-port of the current co-ordinator and then 
things would work from there on (I think).

> Refactor Sender
> ---
>
> Key: KAFKA-1316
> URL: https://issues.apache.org/jira/browse/KAFKA-1316
> Project: Kafka
>  Issue Type: Sub-task
>  Components: producer 
>Reporter: Jay Kreps
>Assignee: Jay Kreps
> Attachments: KAFKA-1316.patch
>
>
> Currently most of the logic of the producer I/O thread is in Sender.java.
> However we will need to do a fair number of similar things in the new 
> consumer. Specifically:
>  - Track in-flight requests
>  - Fetch metadata
>  - Manage connection lifecycle
> It may be possible to refactor some of this into a helper class that can be 
> shared with the consumer. This will require some detailed thought.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


Re: Review Request 21883: Patch for KAFKA-1459

2014-05-27 Thread Guozhang Wang

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



core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala


"0" could still be a valid offset since our offsets start at 0. Could you 
use -1 instead?


- Guozhang Wang


On May 24, 2014, 2:05 a.m., Sriharsha Chintalapani wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/21883/
> ---
> 
> (Updated May 24, 2014, 2:05 a.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1459
> https://issues.apache.org/jira/browse/KAFKA-1459
> 
> 
> Repository: kafka
> 
> 
> Description
> ---
> 
> KAFKA-1459. kafka.tools.ConsumerOffsetChecker throws NoNodeException.
> 
> 
> Diffs
> -
> 
>   core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala 
> 88f824fa5afe7964efc569bc0732fd40a55c5c93 
> 
> Diff: https://reviews.apache.org/r/21883/diff/
> 
> 
> Testing
> ---
> 
> 
> Thanks,
> 
> Sriharsha Chintalapani
> 
>



Re: Review Request 21865: Patch for KAFKA-1438

2014-05-27 Thread Sriharsha Chintalapani

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

(Updated May 28, 2014, 12:08 a.m.)


Review request for kafka.


Bugs: KAFKA-1438
https://issues.apache.org/jira/browse/KAFKA-1438


Repository: kafka


Description (updated)
---

KAFKA-1438. Migrate Kafka Client tools.


Diffs (updated)
-

  bin/kafka-console-consumer.sh e410dde271eea996416f7a67561e45b78f8e8a69 
  bin/kafka-console-producer.sh cd8ce62fe51e3ad64908146f727f73fc6e35b35e 
  bin/kafka-consumer-perf-test.sh 4ed3ed9fb12edf00a82c06c4e59270a60df43ecd 
  bin/kafka-producer-perf-test.sh b4efc29d76ff8f53a310f812b9ed3de6875465b5 
  bin/kafka-run-class.sh d2fc8c0e413769bb89ada99790b90ad616044f28 
  bin/kafka-simple-consumer-perf-test.sh 
2d3e3d3be21065a4d446b9f8ed2e9d31c0685456 
  bin/windows/kafka-console-consumer.bat 
94b20a4d1d2d043b4c4280d00b807d1d27089ac1 
  bin/windows/kafka-console-producer.bat 
b116e64cc4d8b19e0038f3dd1ff41bc5eb47abab 
  bin/windows/kafka-run-class.bat f4d2904a3320ae756ed4171f6e99566f1f0cf963 
  core/src/main/scala/kafka/consumer/ConsoleConsumer.scala 
1a16c691683dda0c53f316e3c4797ea38e776574 
  core/src/main/scala/kafka/producer/ConsoleProducer.scala 
a2af988d99a94a20291d6a2dc9bec73197f1b756 
  core/src/main/scala/kafka/server/OffsetManager.scala 
54176283e5f199d9db08e2181e7ddfbb9c801fab 
  perf/config/log4j.properties 542b7391550b2a8bb091a38d08583bea9b3111b6 
  perf/src/main/scala/kafka/perf/ConsumerPerformance.scala 
4dde4687bada1202eaf54be2b06c2156e7726b23 
  perf/src/main/scala/kafka/perf/PerfConfig.scala 
a8fc6b9ec81c49c73c35172ddc5a67c6a96c799a 
  perf/src/main/scala/kafka/perf/ProducerPerformance.scala 
00fa90bb286900fb33a05f55e3c34c2cbb5620a1 
  perf/src/main/scala/kafka/perf/SimpleConsumerPerformance.scala 
c52ada0a30ae49c8d576703859b9d49348f62feb 

Diff: https://reviews.apache.org/r/21865/diff/


Testing
---


Thanks,

Sriharsha Chintalapani



[jira] [Updated] (KAFKA-1438) Migrate kafka client tools

2014-05-27 Thread Sriharsha Chintalapani (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1438?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Sriharsha Chintalapani updated KAFKA-1438:
--

Attachment: KAFKA-1438_2014-05-27_17:08:59.patch

> Migrate kafka client tools
> --
>
> Key: KAFKA-1438
> URL: https://issues.apache.org/jira/browse/KAFKA-1438
> Project: Kafka
>  Issue Type: Bug
>Reporter: Guozhang Wang
>  Labels: newbie, tools, usability
> Fix For: 0.9.0
>
> Attachments: KAFKA-1438.patch, KAFKA-1438_2014-05-27_11:45:29.patch, 
> KAFKA-1438_2014-05-27_12:16:00.patch, KAFKA-1438_2014-05-27_17:08:59.patch
>
>
> Currently the console/perf client tools scatter across different packages 
> since they are really used as tools:
> 1. Move Consumer/ProducerPerformance and SimpleConsumerPerformance to tools 
> and remove the perf sub-project.
> 2. Move ConsoleConsumer from kafka.consumer to kafka.tools.
> 3. Move other consumer related tools from kafka.consumer to kafka.tools.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-1438) Migrate kafka client tools

2014-05-27 Thread Sriharsha Chintalapani (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1438?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14010538#comment-14010538
 ] 

Sriharsha Chintalapani commented on KAFKA-1438:
---

Updated reviewboard https://reviews.apache.org/r/21865/diff/
 against branch origin/trunk

> Migrate kafka client tools
> --
>
> Key: KAFKA-1438
> URL: https://issues.apache.org/jira/browse/KAFKA-1438
> Project: Kafka
>  Issue Type: Bug
>Reporter: Guozhang Wang
>  Labels: newbie, tools, usability
> Fix For: 0.9.0
>
> Attachments: KAFKA-1438.patch, KAFKA-1438_2014-05-27_11:45:29.patch, 
> KAFKA-1438_2014-05-27_12:16:00.patch, KAFKA-1438_2014-05-27_17:08:59.patch
>
>
> Currently the console/perf client tools scatter across different packages 
> since they are really used as tools:
> 1. Move Consumer/ProducerPerformance and SimpleConsumerPerformance to tools 
> and remove the perf sub-project.
> 2. Move ConsoleConsumer from kafka.consumer to kafka.tools.
> 3. Move other consumer related tools from kafka.consumer to kafka.tools.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


Re: Review Request 21865: Patch for KAFKA-1438

2014-05-27 Thread Sriharsha Chintalapani


> On May 27, 2014, 8:40 p.m., Neha Narkhede wrote:
> > I get the following error while applying the patch-
> > 
> > patching file bin/windows/kafka-run-class.bat
> > Hunk #1 FAILED at 36.
> > Hunk #2 FAILED at 121.
> > 2 out of 2 hunks FAILED -- saving rejects to file 
> > bin/windows/kafka-run-class.bat.rej
> 
> Sriharsha Chintalapani wrote:
> whats your git version. I am using git 1.8.5.2 on OS X 10.9.2. Probably 
> issue with the older version of git but I'll test it again.
> Thanks.
> 
> Neha Narkhede wrote:
> This is using patch.
> patch -p1 -i kafka-1438.patch

I tested the new patch it applies cleanly against the trunk.
MessageFormatter in ConsoleConsumer.scala should go into kafka.consumer is it ok
to add this in its own file or should it go under existing file. please advise.
There are few admin commands kafka.admin.TopicCommand, 
PreferredReplicaLeaderElectionCommand
ReassignPartitionsCommand should they also be moved under tools.


- Sriharsha


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


On May 28, 2014, 12:08 a.m., Sriharsha Chintalapani wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/21865/
> ---
> 
> (Updated May 28, 2014, 12:08 a.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1438
> https://issues.apache.org/jira/browse/KAFKA-1438
> 
> 
> Repository: kafka
> 
> 
> Description
> ---
> 
> KAFKA-1438. Migrate Kafka Client tools.
> 
> 
> Diffs
> -
> 
>   bin/kafka-console-consumer.sh e410dde271eea996416f7a67561e45b78f8e8a69 
>   bin/kafka-console-producer.sh cd8ce62fe51e3ad64908146f727f73fc6e35b35e 
>   bin/kafka-consumer-perf-test.sh 4ed3ed9fb12edf00a82c06c4e59270a60df43ecd 
>   bin/kafka-producer-perf-test.sh b4efc29d76ff8f53a310f812b9ed3de6875465b5 
>   bin/kafka-run-class.sh d2fc8c0e413769bb89ada99790b90ad616044f28 
>   bin/kafka-simple-consumer-perf-test.sh 
> 2d3e3d3be21065a4d446b9f8ed2e9d31c0685456 
>   bin/windows/kafka-console-consumer.bat 
> 94b20a4d1d2d043b4c4280d00b807d1d27089ac1 
>   bin/windows/kafka-console-producer.bat 
> b116e64cc4d8b19e0038f3dd1ff41bc5eb47abab 
>   bin/windows/kafka-run-class.bat f4d2904a3320ae756ed4171f6e99566f1f0cf963 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala 
> 1a16c691683dda0c53f316e3c4797ea38e776574 
>   core/src/main/scala/kafka/producer/ConsoleProducer.scala 
> a2af988d99a94a20291d6a2dc9bec73197f1b756 
>   core/src/main/scala/kafka/server/OffsetManager.scala 
> 54176283e5f199d9db08e2181e7ddfbb9c801fab 
>   perf/config/log4j.properties 542b7391550b2a8bb091a38d08583bea9b3111b6 
>   perf/src/main/scala/kafka/perf/ConsumerPerformance.scala 
> 4dde4687bada1202eaf54be2b06c2156e7726b23 
>   perf/src/main/scala/kafka/perf/PerfConfig.scala 
> a8fc6b9ec81c49c73c35172ddc5a67c6a96c799a 
>   perf/src/main/scala/kafka/perf/ProducerPerformance.scala 
> 00fa90bb286900fb33a05f55e3c34c2cbb5620a1 
>   perf/src/main/scala/kafka/perf/SimpleConsumerPerformance.scala 
> c52ada0a30ae49c8d576703859b9d49348f62feb 
> 
> Diff: https://reviews.apache.org/r/21865/diff/
> 
> 
> Testing
> ---
> 
> 
> Thanks,
> 
> Sriharsha Chintalapani
> 
>



Re: Review Request 21616: Patch for KAFKA-924

2014-05-27 Thread Guozhang Wang

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

Ship it!


Ship It!

- Guozhang Wang


On May 27, 2014, 9:51 p.m., Sriharsha Chintalapani wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/21616/
> ---
> 
> (Updated May 27, 2014, 9:51 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-924
> https://issues.apache.org/jira/browse/KAFKA-924
> 
> 
> Repository: kafka
> 
> 
> Description
> ---
> 
> KAFKA-924. Specify console consumer properties via a single --property 
> command line parameter.
> 
> 
> KAFKA-924. Specify console consumer properties via a single --property 
> command line parameter.
> 
> 
> KAFKA-924. Specify console consumer properties via a single --property 
> command line parameter.
> 
> 
> Diffs
> -
> 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala 
> 24c92877cc7df20b82b7a438d39120b1caa3924d 
> 
> Diff: https://reviews.apache.org/r/21616/diff/
> 
> 
> Testing
> ---
> 
> 
> Thanks,
> 
> Sriharsha Chintalapani
> 
>



[jira] [Updated] (KAFKA-1438) Migrate kafka client tools

2014-05-27 Thread Guozhang Wang (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1438?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Guozhang Wang updated KAFKA-1438:
-

Description: 
Currently the console/perf client tools scatter across different packages, we'd 
better to:

1. Move Consumer/ProducerPerformance and SimpleConsumerPerformance to tools and 
remove the perf sub-project.
2. Move ConsoleConsumer from kafka.consumer to kafka.tools.
3. Move other consumer related tools from kafka.consumer to kafka.tools.

  was:
Currently the console/perf client tools scatter across different packages since 
they are really used as tools:

1. Move Consumer/ProducerPerformance and SimpleConsumerPerformance to tools and 
remove the perf sub-project.
2. Move ConsoleConsumer from kafka.consumer to kafka.tools.
3. Move other consumer related tools from kafka.consumer to kafka.tools.


> Migrate kafka client tools
> --
>
> Key: KAFKA-1438
> URL: https://issues.apache.org/jira/browse/KAFKA-1438
> Project: Kafka
>  Issue Type: Bug
>Reporter: Guozhang Wang
>  Labels: newbie, tools, usability
> Fix For: 0.9.0
>
> Attachments: KAFKA-1438.patch, KAFKA-1438_2014-05-27_11:45:29.patch, 
> KAFKA-1438_2014-05-27_12:16:00.patch, KAFKA-1438_2014-05-27_17:08:59.patch
>
>
> Currently the console/perf client tools scatter across different packages, 
> we'd better to:
> 1. Move Consumer/ProducerPerformance and SimpleConsumerPerformance to tools 
> and remove the perf sub-project.
> 2. Move ConsoleConsumer from kafka.consumer to kafka.tools.
> 3. Move other consumer related tools from kafka.consumer to kafka.tools.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


Re: Review Request 21865: Patch for KAFKA-1438

2014-05-27 Thread Guozhang Wang

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


I get the same error as Neha on kafka-run-class.sh, could you double check if 
there is anything wrong with that diff?

- Guozhang Wang


On May 28, 2014, 12:08 a.m., Sriharsha Chintalapani wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/21865/
> ---
> 
> (Updated May 28, 2014, 12:08 a.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1438
> https://issues.apache.org/jira/browse/KAFKA-1438
> 
> 
> Repository: kafka
> 
> 
> Description
> ---
> 
> KAFKA-1438. Migrate Kafka Client tools.
> 
> 
> Diffs
> -
> 
>   bin/kafka-console-consumer.sh e410dde271eea996416f7a67561e45b78f8e8a69 
>   bin/kafka-console-producer.sh cd8ce62fe51e3ad64908146f727f73fc6e35b35e 
>   bin/kafka-consumer-perf-test.sh 4ed3ed9fb12edf00a82c06c4e59270a60df43ecd 
>   bin/kafka-producer-perf-test.sh b4efc29d76ff8f53a310f812b9ed3de6875465b5 
>   bin/kafka-run-class.sh d2fc8c0e413769bb89ada99790b90ad616044f28 
>   bin/kafka-simple-consumer-perf-test.sh 
> 2d3e3d3be21065a4d446b9f8ed2e9d31c0685456 
>   bin/windows/kafka-console-consumer.bat 
> 94b20a4d1d2d043b4c4280d00b807d1d27089ac1 
>   bin/windows/kafka-console-producer.bat 
> b116e64cc4d8b19e0038f3dd1ff41bc5eb47abab 
>   bin/windows/kafka-run-class.bat f4d2904a3320ae756ed4171f6e99566f1f0cf963 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala 
> 1a16c691683dda0c53f316e3c4797ea38e776574 
>   core/src/main/scala/kafka/producer/ConsoleProducer.scala 
> a2af988d99a94a20291d6a2dc9bec73197f1b756 
>   core/src/main/scala/kafka/server/OffsetManager.scala 
> 54176283e5f199d9db08e2181e7ddfbb9c801fab 
>   perf/config/log4j.properties 542b7391550b2a8bb091a38d08583bea9b3111b6 
>   perf/src/main/scala/kafka/perf/ConsumerPerformance.scala 
> 4dde4687bada1202eaf54be2b06c2156e7726b23 
>   perf/src/main/scala/kafka/perf/PerfConfig.scala 
> a8fc6b9ec81c49c73c35172ddc5a67c6a96c799a 
>   perf/src/main/scala/kafka/perf/ProducerPerformance.scala 
> 00fa90bb286900fb33a05f55e3c34c2cbb5620a1 
>   perf/src/main/scala/kafka/perf/SimpleConsumerPerformance.scala 
> c52ada0a30ae49c8d576703859b9d49348f62feb 
> 
> Diff: https://reviews.apache.org/r/21865/diff/
> 
> 
> Testing
> ---
> 
> 
> Thanks,
> 
> Sriharsha Chintalapani
> 
>



Re: Review Request 21663: Patch for KAFKA-1456

2014-05-27 Thread James Oliver
Yeah I meant to get to that Friday...

Created subtask KAFKA-1471 to address this
https://reviews.apache.org/r/21938/


On Mon, May 26, 2014 at 8:40 AM, Joe Stein  wrote:

>This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/21663/
>
> On May 23rd, 2014, 5:28 p.m. UTC, *Jun Rao* wrote:
>
> Thanks for the patch. Could we add the new codec in ProducerCompressionTest 
> and MessageTest too?
>
>  Sorry I missed this comment about ProducerCompressionTest the code is in 
> MessageTest MessageCompressionTest and yes should be in 
> ProducerCompressionTest too
>
> Jimi can you add code for the ProducerCompressionTest please? just do a pull 
> on trunk and a new patch on a new ticket for that I think would be ok
>
>
> - Joe
>
> On May 20th, 2014, 6:24 a.m. UTC, James Oliver wrote:
>   Review request for kafka.
> By James Oliver.
>
> *Updated May 20, 2014, 6:24 a.m.*
>  *Bugs: * KAFKA-1432 ,
> KAFKA-1456 
>  *Repository: * kafka
> Description
>
> Fix LZ4HC case object typo
>
>
> Revert unnecessary config changes, change server log folder to existing 
> directory
>
>
> Add test cases for LZ4 and LZ4HC message compression
>
>
> Add LZ4 and LZ4HC codec wrappers to client
>
>   Diffs
>
>- build.gradle (b3bbd775366e29abb960e11b0b613b3a85ac4747)
>- clients/src/main/java/org/apache/kafka/common/record/CompressionType.java
>(c557e44b68bc201c7c7e8a31e9eb2f74c359762b)
>- clients/src/main/java/org/apache/kafka/common/record/Compressor.java
>(6ae3d06ac6d27e4ecee3f5b4a18e2615d0581442)
>- clients/src/main/java/org/apache/kafka/common/record/Record.java
>(ce1177ef3c851572a3e3268a7a075e7db56fc3f7)
>- config/producer.properties (52a76114f5d0968ca0ff01b3ab1a45450c0a1e33)
>- core/src/main/scala/kafka/message/CompressionCodec.scala
>(8762a790af2ab7f60a5e4e19955c7eb7ad7bc749)
>- core/src/main/scala/kafka/message/CompressionFactory.scala
>(ca833ee317975548b3e0e91f010af36bce2af09b)
>- core/src/main/scala/kafka/message/Message.scala
>(52c082f5213ba48a39bf0d17e4ced2714276f0e0)
>- core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala
>(ed22931f24fed3530d374b56495a5e4bc89c72f2)
>- system_test/producer_perf/bin/run-compression-test-all.sh
>(PRE-CREATION)
>- system_test/producer_perf/config/server.properties
>(9f8a633ab2f92862e7e7bcc40d699daf00c69e23)
>
> View Diff 
>


Re: Review Request 21878: KAFKA-1468: Misc. improvements to test scripts from benchmarking

2014-05-27 Thread Jun Rao

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



clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java


Is target_throughput in bytes/sec?



clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java


Not sure that I follow how the sleeping logic works. Could you add some 
comments?



clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java


Should we call this OnCompletionCallback? OnCompeletion.onCompletion() is a 
bit confusing.



clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java


Should we handle the error case here?


- Jun Rao


On May 23, 2014, 9:08 p.m., Jay Kreps wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/21878/
> ---
> 
> (Updated May 23, 2014, 9:08 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1468
> https://issues.apache.org/jira/browse/KAFKA-1468
> 
> 
> Repository: kafka
> 
> 
> Description
> ---
> 
> KAFKA-1468 Misc. improvements from benchmarking.
> 
> 
> Diffs
> -
> 
>   clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 
> f1def508153b7361b8720dbcc939e06b0e7e45d3 
>   clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 
> bc4074ec841650eff48dd8aafc1e6c712d8eae7d 
>   
> clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java 
> 3e83ae0fefb7578cccbd8dc0cd9b28f22e0951b2 
>   
> clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java 
> eb18739d181662e2dee5dd6a23e0181f4c740139 
>   clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java 
> a2b77226f8c58caf632a0f4665bd4e4cd93e643d 
>   core/src/main/scala/kafka/server/KafkaConfig.scala 
> c7508d5568ec6b94c47d1839aa8695e2dbb76b97 
>   core/src/main/scala/kafka/server/RequestPurgatory.scala 
> c064c5c4cf1191335572da8a2caf5f95dce902c1 
>   core/src/main/scala/kafka/tools/TestEndToEndLatency.scala 
> 37a9ec27abfd0acd6c01f11d987dd719bc9138a5 
> 
> Diff: https://reviews.apache.org/r/21878/diff/
> 
> 
> Testing
> ---
> 
> 
> Thanks,
> 
> Jay Kreps
> 
>



Re: Review Request 21883: Patch for KAFKA-1459

2014-05-27 Thread Sriharsha Chintalapani

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

(Updated May 28, 2014, 3:19 a.m.)


Review request for kafka.


Bugs: KAFKA-1459
https://issues.apache.org/jira/browse/KAFKA-1459


Repository: kafka


Description
---

KAFKA-1459. kafka.tools.ConsumerOffsetChecker throws NoNodeException.


Diffs (updated)
-

  core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala 
ae83e4d5fa26ae600b51062282c5dd355c9b6618 

Diff: https://reviews.apache.org/r/21883/diff/


Testing
---


Thanks,

Sriharsha Chintalapani



[jira] [Commented] (KAFKA-1459) kafka.tools.ConsumerOffsetChecker throws NoNodeException

2014-05-27 Thread Sriharsha Chintalapani (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1459?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14010710#comment-14010710
 ] 

Sriharsha Chintalapani commented on KAFKA-1459:
---

Updated reviewboard https://reviews.apache.org/r/21883/diff/
 against branch origin/trunk

> kafka.tools.ConsumerOffsetChecker throws NoNodeException
> 
>
> Key: KAFKA-1459
> URL: https://issues.apache.org/jira/browse/KAFKA-1459
> Project: Kafka
>  Issue Type: Bug
>  Components: tools
>Affects Versions: 0.8.1.1
>Reporter: Scott Clasen
>Assignee: Sriharsha Chintalapani
>  Labels: newbie
> Attachments: KAFKA-1459.patch, KAFKA-1459_2014-05-27_20:19:55.patch
>
>
> When using the kafka.tools.ConsumerOffsetChecker to check offsets for 
> consumers that are doing manual offset management, and offsets for some but 
> not all partitions have been stored, the offset checker will throw a no node 
> exception.  It should probably return 0 for partitions that dont have an 
> offset recorded yet.
> In this case I was using github.com/pinterest/secor, which may read thousands 
> or millions of messages from a partition before committing an offset.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Updated] (KAFKA-1459) kafka.tools.ConsumerOffsetChecker throws NoNodeException

2014-05-27 Thread Sriharsha Chintalapani (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1459?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Sriharsha Chintalapani updated KAFKA-1459:
--

Attachment: KAFKA-1459_2014-05-27_20:19:55.patch

> kafka.tools.ConsumerOffsetChecker throws NoNodeException
> 
>
> Key: KAFKA-1459
> URL: https://issues.apache.org/jira/browse/KAFKA-1459
> Project: Kafka
>  Issue Type: Bug
>  Components: tools
>Affects Versions: 0.8.1.1
>Reporter: Scott Clasen
>Assignee: Sriharsha Chintalapani
>  Labels: newbie
> Attachments: KAFKA-1459.patch, KAFKA-1459_2014-05-27_20:19:55.patch
>
>
> When using the kafka.tools.ConsumerOffsetChecker to check offsets for 
> consumers that are doing manual offset management, and offsets for some but 
> not all partitions have been stored, the offset checker will throw a no node 
> exception.  It should probably return 0 for partitions that dont have an 
> offset recorded yet.
> In this case I was using github.com/pinterest/secor, which may read thousands 
> or millions of messages from a partition before committing an offset.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


Kafka meetup during Hadoop Summit

2014-05-27 Thread Neha Narkhede
Hi,

We are hosting a Kafka meetup during Hadoop Summit at LinkedIn on June 3rd.
Agenda includes the following talks followed by networking.

Operating Kafka at Scale by Clark Haskins and Todd Palino@LinkedIn
Secure Kafka by Rajasekar Elango@Salesforce
Samza - Stream processing at scale on Kafka by Chris Riccomini@LinkedIn

You can RSVP for the meetup here - http://meetu.ps/2mVPQR

Thanks,
Neha


[jira] [Resolved] (KAFKA-1470) Quickstart documentation contains a minor typo

2014-05-27 Thread Jun Rao (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1470?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jun Rao resolved KAFKA-1470.


Resolution: Fixed
  Assignee: Baris Can Daylik

Thanks for the patch. Committed.

> Quickstart documentation contains a minor typo
> --
>
> Key: KAFKA-1470
> URL: https://issues.apache.org/jira/browse/KAFKA-1470
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8.1.1
>Reporter: Baris Can Daylik
>Assignee: Baris Can Daylik
>Priority: Trivial
> Attachments: fix_typo_on_quickstart.diff
>
>
> Attaching the patch.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


Re: Review Request 21883: Patch for KAFKA-1459

2014-05-27 Thread Guozhang Wang

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

Ship it!


Neha, would you like to check this in as a follow-up?

- Guozhang Wang


On May 28, 2014, 3:19 a.m., Sriharsha Chintalapani wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/21883/
> ---
> 
> (Updated May 28, 2014, 3:19 a.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1459
> https://issues.apache.org/jira/browse/KAFKA-1459
> 
> 
> Repository: kafka
> 
> 
> Description
> ---
> 
> KAFKA-1459. kafka.tools.ConsumerOffsetChecker throws NoNodeException.
> 
> 
> Diffs
> -
> 
>   core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala 
> ae83e4d5fa26ae600b51062282c5dd355c9b6618 
> 
> Diff: https://reviews.apache.org/r/21883/diff/
> 
> 
> Testing
> ---
> 
> 
> Thanks,
> 
> Sriharsha Chintalapani
> 
>