[jira] [Commented] (KAFKA-635) Producer error when trying to send not displayed unless in DEBUG logging level

2012-11-27 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-635?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13505222#comment-13505222 ] Jun Rao commented on KAFKA-635: --- Chris, Thanks for reporting this. Do you really need

Re: git wip incubator-kafka.git

2012-11-27 Thread Jun Rao
Chris, I plan to submit an infra ticket for the post graduation items. One of the subtasks will be to move our svn repo to a new location (w/o incubator). We could either reuse your ticket and ask for a new repo name or we could close your ticket and create a new one. Which one would you prefer?

Kafka 0.8 trial version

2012-11-26 Thread Jun Rao
Hi, Everyone, Thanks to people who contributed to the project, we have made significant progress in Kafka replication. Now, we'd like people to give it a try. Please check out revision 1411070 from https://svn.apache.org/repos/asf/incubator/kafka/branches/0.8 and follow the following wiki.

trunk and branches update

2012-11-26 Thread Jun Rao
Hi, Since we are getting close to the 0.8 release, I have made the following svn changes in our repository. moved 0.7 branch to 0.7.0 moved trunk to 0.7 copied 0.8 to trunk Now trunk is available for post 0.8 development. 0.8 changes will still be checked into the 0.8 branch and we will merge

[jira] [Created] (KAFKA-632) ProducerRequest should take ByteBufferMessageSet instead of MessageSet

2012-11-26 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-632: - Summary: ProducerRequest should take ByteBufferMessageSet instead of MessageSet Key: KAFKA-632 URL: https://issues.apache.org/jira/browse/KAFKA-632 Project: Kafka Issue

[jira] [Created] (KAFKA-633) AdminTest.testShutdownBroker fails

2012-11-26 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-633: - Summary: AdminTest.testShutdownBroker fails Key: KAFKA-633 URL: https://issues.apache.org/jira/browse/KAFKA-633 Project: Kafka Issue Type: Bug Components: core

[jira] [Commented] (KAFKA-532) Multiple controllers can co-exist during soft failures

2012-11-18 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-532?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13499860#comment-13499860 ] Jun Rao commented on KAFKA-532: --- Thanks for patch v5. Looks good. Just a couple of minor

[jira] [Commented] (KAFKA-532) Multiple controllers can co-exist during soft failures

2012-11-18 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-532?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13499941#comment-13499941 ] Jun Rao commented on KAFKA-532: --- 50. My feeling is that request level error code conveys

[jira] [Updated] (KAFKA-612) move shutting down of fetcher thread out of critical path

2012-11-18 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-612?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-612: -- Attachment: kafka-612-v2.patch Thanks for the review. Attach patch v2. 1.1 This is to handle the case

[jira] [Updated] (KAFKA-612) move shutting down of fetcher thread out of critical path

2012-11-18 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-612?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-612: -- Resolution: Fixed Fix Version/s: 0.8 Status: Resolved (was: Patch Available) Thanks

[jira] [Closed] (KAFKA-612) move shutting down of fetcher thread out of critical path

2012-11-18 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-612?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao closed KAFKA-612. - move shutting down of fetcher thread out of critical path

[jira] [Updated] (KAFKA-605) System Test - Log Retention Cases should wait longer before getting the common starting offset in replica log segments

2012-11-16 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-605?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-605: -- Resolution: Fixed Fix Version/s: 0.8 Status: Resolved (was: Patch Available) Thanks

[jira] [Closed] (KAFKA-605) System Test - Log Retention Cases should wait longer before getting the common starting offset in replica log segments

2012-11-16 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-605?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao closed KAFKA-605. - System Test - Log Retention Cases should wait longer before getting the common starting offset in replica log

[jira] [Commented] (KAFKA-620) UnknownHostError looking for a ZK node crashes the broker

2012-11-16 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-620?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13499043#comment-13499043 ] Jun Rao commented on KAFKA-620: --- Do you have other ZK hosts in your connection string

[jira] [Commented] (KAFKA-613) MigrationTool should disable shallow iteration in the 0.7 consumer

2012-11-16 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-613?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13499269#comment-13499269 ] Jun Rao commented on KAFKA-613: --- Thanks for patch v1. It would be good to also log a warning

[jira] [Resolved] (KAFKA-614) DumpLogSegment offset verification is incorrect for compressed messages

2012-11-16 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-614?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-614. --- Resolution: Fixed Fix Version/s: 0.8 Thanks for the patch. +1 and committed to 0.8

[jira] [Closed] (KAFKA-614) DumpLogSegment offset verification is incorrect for compressed messages

2012-11-16 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-614?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao closed KAFKA-614. - DumpLogSegment offset verification is incorrect for compressed messages

[jira] [Commented] (KAFKA-618) Deadlock between leader-finder-thread and consumer-fetcher-thread during broker failure

2012-11-16 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-618?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13499309#comment-13499309 ] Jun Rao commented on KAFKA-618: --- Thanks for the patch. +1 Deadlock between

[jira] [Resolved] (KAFKA-613) MigrationTool should disable shallow iteration in the 0.7 consumer

2012-11-16 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-613?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-613. --- Resolution: Fixed Fix Version/s: 0.8 Thanks for patch v2. +1. Committed to 0.8

[jira] [Closed] (KAFKA-613) MigrationTool should disable shallow iteration in the 0.7 consumer

2012-11-16 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-613?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao closed KAFKA-613. - MigrationTool should disable shallow iteration in the 0.7 consumer

[jira] [Created] (KAFKA-613) MigrationTool should disable shallow iteration in the 0.7 consumer

2012-11-15 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-613: - Summary: MigrationTool should disable shallow iteration in the 0.7 consumer Key: KAFKA-613 URL: https://issues.apache.org/jira/browse/KAFKA-613 Project: Kafka Issue Type

[jira] [Created] (KAFKA-614) DumpLogSegment offset verification is incorrect for compressed messages

2012-11-15 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-614: - Summary: DumpLogSegment offset verification is incorrect for compressed messages Key: KAFKA-614 URL: https://issues.apache.org/jira/browse/KAFKA-614 Project: Kafka Issue

[jira] [Commented] (KAFKA-618) Deadlock between leader-finder-thread and consumer-fetcher-thread during broker failure

2012-11-15 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-618?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13498555#comment-13498555 ] Jun Rao commented on KAFKA-618: --- This is a very good finding. The following is one way

[jira] [Created] (KAFKA-612) move shutting down of fetcher thread out of critical path

2012-11-14 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-612: - Summary: move shutting down of fetcher thread out of critical path Key: KAFKA-612 URL: https://issues.apache.org/jira/browse/KAFKA-612 Project: Kafka Issue Type: Bug

[jira] [Updated] (KAFKA-612) move shutting down of fetcher thread out of critical path

2012-11-14 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-612?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-612: -- Attachment: kafka-612.patch Attach a patch. It introduces a separate method for shutting down empty fetcher

[jira] [Updated] (KAFKA-612) move shutting down of fetcher thread out of critical path

2012-11-14 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-612?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-612: -- Status: Patch Available (was: Open) move shutting down of fetcher thread out of critical path

[jira] [Commented] (KAFKA-544) Retain key in producer and expose it in the consumer

2012-11-14 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-544?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13497680#comment-13497680 ] Jun Rao commented on KAFKA-544: --- Thanks for patch v3. Looks good overall. Some minor comments

[jira] [Commented] (KAFKA-544) Retain key in producer and expose it in the consumer

2012-11-14 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-544?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13497755#comment-13497755 ] Jun Rao commented on KAFKA-544: --- +1 on patch v5. For 30, could you add the same constructor

[jira] [Commented] (KAFKA-606) System Test Transient Failure (case 0302 GC Pause) - Log segments mismatched across replicas

2012-11-12 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-606?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13495387#comment-13495387 ] Jun Rao commented on KAFKA-606: --- There seems to be 2 issues. (1) Since we only do shallow

[jira] [Commented] (KAFKA-585) Remove custom metrics jar and replace with latest from metrics HEAD

2012-11-12 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-585?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13495462#comment-13495462 ] Jun Rao commented on KAFKA-585: --- Thanks for patch v1. +1. Just make sure that the basic

[jira] [Updated] (KAFKA-591) Add test cases to test log size retention and more

2012-11-09 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-591?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-591: -- Resolution: Fixed Fix Version/s: 0.8 Status: Resolved (was: Patch Available) Thanks

[jira] [Closed] (KAFKA-591) Add test cases to test log size retention and more

2012-11-09 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-591?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao closed KAFKA-591. - Add test cases to test log size retention and more

[jira] [Created] (KAFKA-604) Add missing metrics in 0.8

2012-11-08 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-604: - Summary: Add missing metrics in 0.8 Key: KAFKA-604 URL: https://issues.apache.org/jira/browse/KAFKA-604 Project: Kafka Issue Type: Bug Components: core

[jira] [Commented] (KAFKA-546) Fix commit() in zk consumer for compressed messages

2012-11-07 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-546?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13492552#comment-13492552 ] Jun Rao commented on KAFKA-546: --- Thanks for patch v3. A couple of more comments: 30

[jira] [Commented] (KAFKA-574) KafkaController unnecessarily reads leaderAndIsr info from ZK

2012-11-07 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-574?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13492975#comment-13492975 ] Jun Rao commented on KAFKA-574: --- Thanks for patch v4. Patch looks good and system tests pass

[jira] [Resolved] (KAFKA-546) Fix commit() in zk consumer for compressed messages

2012-11-07 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-546?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-546. --- Resolution: Fixed Fix Version/s: 0.8 Thanks for patch v5. +1. Committed to 0.8. Fix

[jira] [Closed] (KAFKA-546) Fix commit() in zk consumer for compressed messages

2012-11-07 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-546?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao closed KAFKA-546. - Fix commit() in zk consumer for compressed messages

[jira] [Commented] (KAFKA-532) Multiple controllers can co-exist during soft failures

2012-11-06 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-532?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13491566#comment-13491566 ] Jun Rao commented on KAFKA-532: --- Thanks for patch v4. A few more comments: 40

[jira] [Commented] (KAFKA-574) KafkaController unnecessarily reads leaderAndIsr info from ZK

2012-11-06 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-574?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13491642#comment-13491642 ] Jun Rao commented on KAFKA-574: --- Thanks for patch v3. Just one more comment: 30

[jira] [Commented] (KAFKA-593) Empty log index file created when it shouldn't be empty

2012-11-05 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-593?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13490770#comment-13490770 ] Jun Rao commented on KAFKA-593: --- Thanks for patch v3. Looks good. Some minor comments. 30

[jira] [Commented] (KAFKA-546) Fix commit() in zk consumer for compressed messages

2012-11-05 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-546?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13490977#comment-13490977 ] Jun Rao commented on KAFKA-546: --- Thanks for patch v2. Some comments: 20

[jira] [Resolved] (KAFKA-596) LogSegment.firstAppendTime not reset after truncate to

2012-11-05 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-596?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-596. --- Resolution: Fixed Fix Version/s: 0.8 Thanks for patch v3. +1. Committed to 0.8

[jira] [Closed] (KAFKA-596) LogSegment.firstAppendTime not reset after truncate to

2012-11-05 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-596?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao closed KAFKA-596. - LogSegment.firstAppendTime not reset after truncate

[jira] [Resolved] (KAFKA-577) extend DumpLogSegments to verify consistency btw data and index

2012-11-05 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-577?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-577. --- Resolution: Fixed Thanks for patch v6. +1. Committed to 0.8 with a minor change: using ::= on a list instead

[jira] [Commented] (KAFKA-596) LogSegment.firstAppendTime not reset after truncate to

2012-11-02 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-596?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13489473#comment-13489473 ] Jun Rao commented on KAFKA-596: --- I agree that setting firstAppendTime to None

[jira] [Commented] (KAFKA-546) Fix commit() in zk consumer for compressed messages

2012-11-02 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-546?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13489481#comment-13489481 ] Jun Rao commented on KAFKA-546: --- Can't see to apply the patch cleanly to 0.8. Could you

[jira] [Created] (KAFKA-598) decouple fetch size from max message size

2012-11-01 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-598: - Summary: decouple fetch size from max message size Key: KAFKA-598 URL: https://issues.apache.org/jira/browse/KAFKA-598 Project: Kafka Issue Type: Bug Components

[jira] [Commented] (KAFKA-574) KafkaController unnecessarily reads leaderAndIsr info from ZK

2012-11-01 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-574?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13488810#comment-13488810 ] Jun Rao commented on KAFKA-574: --- Thanks for patch v2. Just one more comment: 20

[jira] [Commented] (KAFKA-532) Multiple controllers can co-exist during soft failures

2012-11-01 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-532?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13488893#comment-13488893 ] Jun Rao commented on KAFKA-532: --- Thanks for patch v3. The overall approach seems to work

[jira] [Commented] (KAFKA-188) Support multiple data directories

2012-11-01 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-188?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13489184#comment-13489184 ] Jun Rao commented on KAFKA-188: --- +1 on patch v8. Thanks, Support multiple

[jira] [Resolved] (KAFKA-491) KafkaRequestHandler needs to handle exceptions

2012-11-01 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-491?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-491. --- Resolution: Fixed Thanks for patch v2. +1. Committed to 0.8. KafkaRequestHandler needs

[jira] [Closed] (KAFKA-491) KafkaRequestHandler needs to handle exceptions

2012-11-01 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-491?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao closed KAFKA-491. - KafkaRequestHandler needs to handle exceptions

[jira] [Commented] (KAFKA-577) extend DumpLogSegments to verify consistency btw data and index

2012-11-01 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-577?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13489255#comment-13489255 ] Jun Rao commented on KAFKA-577: --- A couple of more comments for patch v4. 40. LinkedList

[jira] [Commented] (KAFKA-593) Empty log index file created when it shouldn't be empty

2012-11-01 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-593?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13489260#comment-13489260 ] Jun Rao commented on KAFKA-593: --- Thanks for the patch. A couple of comments: 1

[jira] [Commented] (KAFKA-596) LogSegment.firstAppendTime not reset after truncate to

2012-11-01 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-596?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13489266#comment-13489266 ] Jun Rao commented on KAFKA-596: --- Thanks for the patch. A couple of comments: 1

Re: Maintainer system?

2012-11-01 Thread Jun Rao
I think that's a good idea. It will be good to have at least 2 maintainers per component. I'd encourage more people to review patches. The more patches one reviews, the more familiar he/she is with the components. Thanks, Jun On Thu, Oct 4, 2012 at 1:13 PM, Jay Kreps jay.kr...@gmail.com wrote:

[jira] [Commented] (KAFKA-593) Empty log index file created when it shouldn't be empty

2012-10-31 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-593?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13487841#comment-13487841 ] Jun Rao commented on KAFKA-593: --- Here is the issue. We rolled a new segment in the follower

[jira] [Closed] (KAFKA-594) Update System Test due to new argument --sync in ProducerPerformance

2012-10-31 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-594?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao closed KAFKA-594. - Update System Test due to new argument --sync in ProducerPerformance

[jira] [Updated] (KAFKA-586) system test configs are broken

2012-10-31 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-586?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-586: -- Resolution: Fixed Fix Version/s: 0.8 Status: Resolved (was: Patch Available) Thanks

[jira] [Closed] (KAFKA-586) system test configs are broken

2012-10-31 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-586?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao closed KAFKA-586. - system test configs are broken -- Key: KAFKA-586

[jira] [Created] (KAFKA-596) LogSegment.firstAppendTime not reset after truncate to

2012-10-31 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-596: - Summary: LogSegment.firstAppendTime not reset after truncate to Key: KAFKA-596 URL: https://issues.apache.org/jira/browse/KAFKA-596 Project: Kafka Issue Type: Bug

[jira] [Commented] (KAFKA-596) LogSegment.firstAppendTime not reset after truncate to

2012-10-31 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-596?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13487947#comment-13487947 ] Jun Rao commented on KAFKA-596: --- The fix is to set LogSegment.firstAppendTime to none if we

[jira] [Commented] (KAFKA-188) Support multiple data directories

2012-10-30 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-188?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13486921#comment-13486921 ] Jun Rao commented on KAFKA-188: --- Our system tests fail with the latest patch. python -B

[jira] [Commented] (KAFKA-593) Empty log index file created when it shouldn't be empty

2012-10-30 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-593?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13486990#comment-13486990 ] Jun Rao commented on KAFKA-593: --- Producer used sync mode. So, there is 1 message per batch

[jira] [Commented] (KAFKA-574) KafkaController unnecessarily reads leaderAndIsr info from ZK

2012-10-29 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-574?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13486091#comment-13486091 ] Jun Rao commented on KAFKA-574: --- Thanks for the patch. Some comments: 1

[jira] [Updated] (KAFKA-577) extend DumpLogSegments to verify consistency btw data and index

2012-10-29 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-577?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-577: -- Comment: was deleted (was: +1 on patch v6. Committed to 0.8.) extend DumpLogSegments to verify

[jira] [Updated] (KAFKA-267) Enhance ProducerPerformance to generate unique random Long value for payload

2012-10-29 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-267?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-267: -- Resolution: Fixed Fix Version/s: 0.8 Status: Resolved (was: Patch Available) +1 on patch v6

[jira] [Commented] (KAFKA-577) extend DumpLogSegments to verify consistency btw data and index

2012-10-29 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-577?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13486548#comment-13486548 ] Jun Rao commented on KAFKA-577: --- Thanks for patch v2. Instead of exiting on first

[jira] [Resolved] (KAFKA-575) Partition.makeFollower() reads broker info from ZK

2012-10-29 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-575?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-575. --- Resolution: Fixed Fix Version/s: 0.8 Thanks for patch v3. +1. Committed to 0.8

[jira] [Closed] (KAFKA-575) Partition.makeFollower() reads broker info from ZK

2012-10-29 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-575?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao closed KAFKA-575. - Partition.makeFollower() reads broker info from ZK

[jira] [Reopened] (KAFKA-577) extend DumpLogSegments to verify consistency btw data and index

2012-10-29 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-577?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao reopened KAFKA-577: --- extend DumpLogSegments to verify consistency btw data and index

Re: move 0.8 back to trunk

2012-10-28 Thread Jun Rao
or after we move to git ? On Fri, Oct 26, 2012 at 7:41 PM, Jun Rao jun...@gmail.com wrote: The 0.8 branch has stabilized quite a bit now. We still have to look into some corner cases and performance, but most system tests are now passing. I think it's time to bring 0.8 back to trunk and let

[jira] [Commented] (KAFKA-588) Index truncation doesn't seem to remove the last entry properly

2012-10-26 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-588?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13485020#comment-13485020 ] Jun Rao commented on KAFKA-588: --- From the log, the broker wants to truncate log to offset

[jira] [Commented] (KAFKA-267) Enhance ProducerPerformance to generate unique random Long value for payload

2012-10-26 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-267?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13485024#comment-13485024 ] Jun Rao commented on KAFKA-267: --- Thanks for patch v7. Not sure if printing messages

[jira] [Commented] (KAFKA-532) Multiple controllers can co-exist during soft failures

2012-10-26 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-532?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13485030#comment-13485030 ] Jun Rao commented on KAFKA-532: --- The purpose of controller epoc is to prevent an older

[jira] [Commented] (KAFKA-267) Enhance ProducerPerformance to generate unique random Long value for payload

2012-10-26 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-267?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13485042#comment-13485042 ] Jun Rao commented on KAFKA-267: --- What we can do is that, in system tests, tune #retries

[jira] [Commented] (KAFKA-577) extend DumpLogSegments to verify consistency btw data and index

2012-10-26 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-577?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13485306#comment-13485306 ] Jun Rao commented on KAFKA-577: --- Thanks for the patch. Some comments: 1. Could you add

[jira] [Commented] (KAFKA-581) provides windows batch script for starting Kafka/Zookeeper

2012-10-25 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-581?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13484233#comment-13484233 ] Jun Rao commented on KAFKA-581: --- Since we have quite a few scripts in bin/. I think it's

[jira] [Updated] (KAFKA-584) produce/fetch remote time metric not set correctly when num.acks = 1

2012-10-25 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-584?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-584: -- Attachment: kafka-584.patch Attach a patch. The problem is that with ack = 1, response could be sent before

[jira] [Commented] (KAFKA-584) produce/fetch remote time metric not set correctly when num.acks = 1

2012-10-25 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-584?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13484251#comment-13484251 ] Jun Rao commented on KAFKA-584: --- Also, volatile makes a long value atomic according

[jira] [Updated] (KAFKA-584) produce/fetch remote time metric not set correctly when num.acks = 1

2012-10-25 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-584?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-584: -- Resolution: Fixed Status: Resolved (was: Patch Available) Thanks for the review. Changed the logging

[jira] [Closed] (KAFKA-584) produce/fetch remote time metric not set correctly when num.acks = 1

2012-10-25 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-584?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao closed KAFKA-584. - produce/fetch remote time metric not set correctly when num.acks = 1

[jira] [Closed] (KAFKA-576) SimpleConsumer throws UnsupportedOperationException: empty.head

2012-10-24 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-576?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao closed KAFKA-576. - SimpleConsumer throws UnsupportedOperationException: empty.head

[jira] [Resolved] (KAFKA-576) SimpleConsumer throws UnsupportedOperationException: empty.head

2012-10-24 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-576?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-576. --- Resolution: Fixed Fix Version/s: 0.8 Thanks for the patch. Committed to 0.8 with the following minor

[jira] [Commented] (KAFKA-267) Enhance ProducerPerformance to generate unique random Long value for payload

2012-10-24 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-267?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13483293#comment-13483293 ] Jun Rao commented on KAFKA-267: --- Thanks for patch v5. Looks good. Some minor comments: 50

[jira] [Commented] (KAFKA-491) KafkaRequestHandler needs to handle exceptions

2012-10-24 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-491?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13483367#comment-13483367 ] Jun Rao commented on KAFKA-491: --- Thanks for patch v1. Some comments: 1. KafkaApis: 1.1

[jira] [Resolved] (KAFKA-581) provides windows batch script for starting Kafka/Zookeeper

2012-10-24 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-581?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-581. --- Resolution: Fixed Thanks for the patch. Committed to 0.8 in bin/windows/. provides windows

[jira] [Closed] (KAFKA-581) provides windows batch script for starting Kafka/Zookeeper

2012-10-24 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-581?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao closed KAFKA-581. - provides windows batch script for starting Kafka/Zookeeper

[jira] [Commented] (KAFKA-579) remove connection timeout in SyncProducer

2012-10-24 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-579?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13483399#comment-13483399 ] Jun Rao commented on KAFKA-579: --- Very good point. To address this issue, we have created

[jira] [Updated] (KAFKA-571) Add more test cases to System Test

2012-10-24 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-571?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-571: -- Resolution: Fixed Status: Resolved (was: Patch Available) Thanks for the patch. +1. Committed to 0.8

[jira] [Closed] (KAFKA-571) Add more test cases to System Test

2012-10-24 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-571?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao closed KAFKA-571. - Add more test cases to System Test -- Key: KAFKA-571

Re: [VOTE] Graduate Kafka from Incubator to TLP status

2012-10-23 Thread Jun Rao
of responsibility of the Apache Kafka Project; and be it further RESOLVED, that the persons listed immediately below be and hereby are appointed to serve as the initial members of the Apache Kafka Project: * Jun Rao junrao@... * Jay Kreps jkreps@... * Neha Narkhede

[jira] [Updated] (KAFKA-580) system test testcase_0122 under replication fails due to large # of data loss

2012-10-23 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-580?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-580: -- Resolution: Fixed Fix Version/s: 0.8 Status: Resolved (was: Patch Available) Thanks

[jira] [Commented] (KAFKA-340) Implement clean shutdown in 0.8

2012-10-23 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-340?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13482553#comment-13482553 ] Jun Rao commented on KAFKA-340: --- Thanks for patch v2. Some more comments and clarification

[jira] [Closed] (KAFKA-578) Leader finder thread in ConsumerFetcherManager needs to handle exceptions

2012-10-23 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-578?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao closed KAFKA-578. - Leader finder thread in ConsumerFetcherManager needs to handle exceptions

[jira] [Resolved] (KAFKA-578) Leader finder thread in ConsumerFetcherManager needs to handle exceptions

2012-10-23 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-578?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-578. --- Resolution: Fixed Fix Version/s: 0.8 Fixed as kafka-580. Leader finder thread

[jira] [Commented] (KAFKA-574) KafkaController unnecessarily reads leaderAndIsr info from ZK

2012-10-21 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-574?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13481068#comment-13481068 ] Jun Rao commented on KAFKA-574: --- Sure, Prashanth. Do you think that you can provide a patch

[jira] [Commented] (KAFKA-541) Use metrics CSV reporter instead of jmx tool for system tests

2012-10-19 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-541?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13480089#comment-13480089 ] Jun Rao commented on KAFKA-541: --- Thanks for the patch. Some comments: 1. I see the following

[jira] [Commented] (KAFKA-267) Enhance ProducerPerformance to generate unique random Long value for payload

2012-10-19 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-267?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13480112#comment-13480112 ] Jun Rao commented on KAFKA-267: --- Thanks for patch v4. It's better, but still messy. Some more

[jira] [Updated] (KAFKA-571) Add more test cases to System Test

2012-10-19 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-571?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-571: -- Attachment: kafka-571-SimpleConsumerShell.patch Thanks for the patch. Some comments: 1. I patched

  1   2   3   4   5   6   7   8   9   10   >