[jira] [Commented] (KAFKA-1646) Improve consumer read performance for Windows

2014-10-07 Thread xueqiang wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1646?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14162987#comment-14162987 ] xueqiang wang commented on KAFKA-1646: -- Yes, if there are many deleting and creating,

[jira] [Commented] (KAFKA-1555) provide strong consistency with reasonable availability

2014-10-07 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1555?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14162942#comment-14162942 ] Joel Koshy commented on KAFKA-1555: --- +1 (took a quick look at it) We could improve the r

Re: Review Request 25886: KAFKA-1555: provide strong consistency with reasonable availability

2014-10-07 Thread Joel Koshy
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/25886/#review55745 --- clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java

Re: Review Request 25886: KAFKA-1555: provide strong consistency with reasonable availability

2014-10-07 Thread Gwen Shapira
> On Oct. 8, 2014, midnight, Jun Rao wrote: > > core/src/main/scala/kafka/cluster/Partition.scala, line 287 > > > > > > The error code should be NotEnoughReplicasAfterAppendCode. Good catch. - Gwen ---

Re: Review Request 25886: KAFKA-1555: provide strong consistency with reasonable availability

2014-10-07 Thread Gwen Shapira
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/25886/ --- (Updated Oct. 8, 2014, 1:46 a.m.) Review request for kafka. Changes ---

[jira] [Commented] (KAFKA-1670) Corrupt log files for segment.bytes values close to Int.MaxInt

2014-10-07 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1670?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14162918#comment-14162918 ] Sriharsha Chintalapani commented on KAFKA-1670: --- Updated reviewboard https:/

[jira] [Updated] (KAFKA-1670) Corrupt log files for segment.bytes values close to Int.MaxInt

2014-10-07 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1670?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sriharsha Chintalapani updated KAFKA-1670: -- Attachment: KAFKA-1670_2014-10-07_18:39:31.patch > Corrupt log files for segment

Re: Review Request 26346: Patch for KAFKA-1670

2014-10-07 Thread Sriharsha Chintalapani
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/26346/ --- (Updated Oct. 8, 2014, 1:39 a.m.) Review request for kafka. Bugs: KAFKA-1670

Re: Review Request 26346: Patch for KAFKA-1670

2014-10-07 Thread Jun Rao
> On Oct. 7, 2014, 11:03 p.m., Jun Rao wrote: > > core/src/test/scala/unit/kafka/log/LogTest.scala, line 242 > > > > > > By increasing the segment size to 100, does the log still roll on every > > message as indicated

[jira] [Updated] (KAFKA-1644) Inherit FetchResponse from RequestOrResponse

2014-10-07 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1644?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1644: --- Resolution: Fixed Fix Version/s: 0.8.3 Status: Resolved (was: Patch Available) That for the

Re: Review Request 26346: Patch for KAFKA-1670

2014-10-07 Thread Sriharsha Chintalapani
> On Oct. 7, 2014, 11:03 p.m., Jun Rao wrote: > > core/src/test/scala/unit/kafka/log/LogTest.scala, line 242 > > > > > > By increasing the segment size to 100, does the log still roll on every > > message as indicated

Re: Kafka AWS deployment + UI console

2014-10-07 Thread Jude K
We use http://quantifind.com/KafkaOffsetMonitor/... On Tue, Oct 7, 2014 at 8:49 PM, Gwen Shapira wrote: > I'm using Hue's ZooKeeper app: > http://gethue.com/new-zookeeper-browser-app/ > > This UI looks very cute, but I didn't try it yet: > https://github.com/claudemamo/kafka-web-console > > Gwen

Re: Kafka AWS deployment + UI console

2014-10-07 Thread Gwen Shapira
I'm using Hue's ZooKeeper app: http://gethue.com/new-zookeeper-browser-app/ This UI looks very cute, but I didn't try it yet: https://github.com/claudemamo/kafka-web-console Gwen On Tue, Oct 7, 2014 at 5:08 PM, Shafaq wrote: > We are going to deploy Kafka in Production and also monitor it via c

Re: Security JIRAS

2014-10-07 Thread Gwen Shapira
I think we need to add: * Authentication of Kafka brokers with a secured ZooKeeper * Kafka should be able to generate delegation tokens for MapReduce / Spark / Yarn jobs. * Extend systest framework to allow testing secured kafka Gwen On Tue, Oct 7, 2014 at 5:15 PM, Jay Kreps wrote: > Hey guys,

Re: Review Request 26346: Patch for KAFKA-1670

2014-10-07 Thread Jun Rao
> On Oct. 7, 2014, 11:03 p.m., Jun Rao wrote: > > core/src/test/scala/unit/kafka/log/LogTest.scala, line 242 > > > > > > By increasing the segment size to 100, does the log still roll on every > > message as indicated

Security JIRAS

2014-10-07 Thread Jay Kreps
Hey guys, As promised, I added a tree of JIRAs for the stuff in the security wiki ( https://cwiki.apache.org/confluence/display/KAFKA/Security): https://issues.apache.org/jira/browse/KAFKA-1682 I tried to break it into reasonably standalone pieces. I think many of the tickets could actually be d

[jira] [Commented] (KAFKA-1684) Implement TLS/SSL authentication

2014-10-07 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1684?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14162839#comment-14162839 ] Jay Kreps commented on KAFKA-1684: -- We may be able to get most of the implementation for

[jira] [Created] (KAFKA-1688) Add authorization interface and naive implementation

2014-10-07 Thread Jay Kreps (JIRA)
Jay Kreps created KAFKA-1688: Summary: Add authorization interface and naive implementation Key: KAFKA-1688 URL: https://issues.apache.org/jira/browse/KAFKA-1688 Project: Kafka Issue Type: Sub-ta

Kafka AWS deployment + UI console

2014-10-07 Thread Shafaq
We are going to deploy Kafka in Production and also monitor it via console. (e.g. State of partitions in Broker- leader and slaves, state of consumers ) Is there out-of-the-box solution? What is the best and efficient way of deployment and monitoring Has someone tried this- looks promising http:

[jira] [Commented] (KAFKA-1555) provide strong consistency with reasonable availability

2014-10-07 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1555?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14162833#comment-14162833 ] Jun Rao commented on KAFKA-1555: The patch that Gwen provided (using a min.isr topic level

[jira] [Created] (KAFKA-1687) SASL tests

2014-10-07 Thread Jay Kreps (JIRA)
Jay Kreps created KAFKA-1687: Summary: SASL tests Key: KAFKA-1687 URL: https://issues.apache.org/jira/browse/KAFKA-1687 Project: Kafka Issue Type: Sub-task Reporter: Jay Kreps We ne

[jira] [Created] (KAFKA-1686) Implement SASL/Kerberos

2014-10-07 Thread Jay Kreps (JIRA)
Jay Kreps created KAFKA-1686: Summary: Implement SASL/Kerberos Key: KAFKA-1686 URL: https://issues.apache.org/jira/browse/KAFKA-1686 Project: Kafka Issue Type: Sub-task Affects Versions: 0.9.

Re: Review Request 25886: KAFKA-1555: provide strong consistency with reasonable availability

2014-10-07 Thread Jun Rao
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/25886/#review55730 --- Looks good. Just some minor comments below. clients/src/main/java/

[jira] [Created] (KAFKA-1685) Implement TLS/SSL tests

2014-10-07 Thread Jay Kreps (JIRA)
Jay Kreps created KAFKA-1685: Summary: Implement TLS/SSL tests Key: KAFKA-1685 URL: https://issues.apache.org/jira/browse/KAFKA-1685 Project: Kafka Issue Type: Sub-task Affects Versions: 0.9.

[jira] [Updated] (KAFKA-1684) Implement TLS/SSL authentication

2014-10-07 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1684?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jay Kreps updated KAFKA-1684: - Summary: Implement TLS/SSL authentication (was: Implement SSL authentication) > Implement TLS/SSL authen

[jira] [Created] (KAFKA-1684) Implement SSL authentication

2014-10-07 Thread Jay Kreps (JIRA)
Jay Kreps created KAFKA-1684: Summary: Implement SSL authentication Key: KAFKA-1684 URL: https://issues.apache.org/jira/browse/KAFKA-1684 Project: Kafka Issue Type: Sub-task Affects Versions:

[jira] [Created] (KAFKA-1683) Implement a "session" concept in the socket server

2014-10-07 Thread Jay Kreps (JIRA)
Jay Kreps created KAFKA-1683: Summary: Implement a "session" concept in the socket server Key: KAFKA-1683 URL: https://issues.apache.org/jira/browse/KAFKA-1683 Project: Kafka Issue Type: Sub-task

[jira] [Updated] (KAFKA-1682) Security for Kafka

2014-10-07 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1682?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jay Kreps updated KAFKA-1682: - Description: Parent ticket for security. Wiki and discussion is here: https://cwiki.apache.org/confluence/

[jira] [Created] (KAFKA-1682) Security for Kafka

2014-10-07 Thread Jay Kreps (JIRA)
Jay Kreps created KAFKA-1682: Summary: Security for Kafka Key: KAFKA-1682 URL: https://issues.apache.org/jira/browse/KAFKA-1682 Project: Kafka Issue Type: New Feature Affects Versions: 0.9.0

Re: Review Request 26346: Patch for KAFKA-1670

2014-10-07 Thread Sriharsha Chintalapani
> On Oct. 7, 2014, 11:03 p.m., Jun Rao wrote: > > core/src/main/scala/kafka/common/MessageSetSizeTooLargeException.scala, > > line 20 > > > > > > We need to add this exception to ErrorMapping and Errors. We also need

Re: Review Request 26346: Patch for KAFKA-1670

2014-10-07 Thread Jun Rao
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/26346/#review55720 --- core/src/main/scala/kafka/common/MessageSetSizeTooLargeException.sc

Re: Java New Producer IO Thread Name

2014-10-07 Thread Jun Rao
That sounds reasonable. Could you file a jira to track this? Thanks, Jun On Tue, Oct 7, 2014 at 7:47 AM, Bhavesh Mistry wrote: > Hi Kafka Dev Team, > > Since we have multiple instance of producers within one JVM, it would be > good idea to name network IO thread name by associate with client.

Re: Review Request 26291: Patch for KAFKA-1648

2014-10-07 Thread Joel Koshy
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/26291/#review55702 --- core/src/main/scala/kafka/consumer/PartitionAssignor.scala

[jira] [Commented] (KAFKA-1670) Corrupt log files for segment.bytes values close to Int.MaxInt

2014-10-07 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1670?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14162507#comment-14162507 ] Sriharsha Chintalapani commented on KAFKA-1670: --- Updated reviewboard https:/

[jira] [Updated] (KAFKA-1670) Corrupt log files for segment.bytes values close to Int.MaxInt

2014-10-07 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1670?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sriharsha Chintalapani updated KAFKA-1670: -- Attachment: KAFKA-1670_2014-10-07_13:49:10.patch > Corrupt log files for segment

Re: Review Request 26346: Patch for KAFKA-1670

2014-10-07 Thread Sriharsha Chintalapani
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/26346/ --- (Updated Oct. 7, 2014, 8:49 p.m.) Review request for kafka. Bugs: KAFKA-1670

Re: Review Request 26346: Patch for KAFKA-1670

2014-10-07 Thread Sriharsha Chintalapani
> On Oct. 7, 2014, 12:42 a.m., Jay Kreps wrote: > > core/src/main/scala/kafka/log/Log.scala, line 502 > > > > > > It is a bit subtle that you are checking for overflow this way. What we > > mean to check is just that

Re: Review Request 26346: Patch for KAFKA-1670

2014-10-07 Thread Sriharsha Chintalapani
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/26346/ --- (Updated Oct. 7, 2014, 8:39 p.m.) Review request for kafka. Bugs: KAFKA-1670

[jira] [Commented] (KAFKA-1670) Corrupt log files for segment.bytes values close to Int.MaxInt

2014-10-07 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1670?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14162486#comment-14162486 ] Sriharsha Chintalapani commented on KAFKA-1670: --- Updated reviewboard https:/

[jira] [Updated] (KAFKA-1670) Corrupt log files for segment.bytes values close to Int.MaxInt

2014-10-07 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1670?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sriharsha Chintalapani updated KAFKA-1670: -- Attachment: KAFKA-1670_2014-10-07_13:39:13.patch > Corrupt log files for segment

Re: Review Request 26390: Fix KAFKA-1641

2014-10-07 Thread Joel Koshy
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/26390/#review55696 --- core/src/main/scala/kafka/log/LogCleanerManager.scala

[jira] [Commented] (KAFKA-1499) Broker-side compression configuration

2014-10-07 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1499?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14162336#comment-14162336 ] Joel Koshy commented on KAFKA-1499: --- Talked to Jay offline and here is a summary of what

[jira] [Comment Edited] (KAFKA-1646) Improve consumer read performance for Windows

2014-10-07 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1646?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14162106#comment-14162106 ] Jay Kreps edited comment on KAFKA-1646 at 10/7/14 6:23 PM: --- Ah,

Re: Review Request 25995: Patch for KAFKA-1650

2014-10-07 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/25995/#review55612 --- core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala

Re: [Java New Producer Kafka Trunk ] Need a State Check API Method

2014-10-07 Thread Jay Kreps
Hey Bhavesh, But isn't the problem here that you are trying to send messages after closing the producer? I think what I am saying is that since calling close is something the user initiates we don't need an api for this--you can keep track of this yourself, right? -Jay On Mon, Oct 6, 2014 at 11

[jira] [Commented] (KAFKA-1646) Improve consumer read performance for Windows

2014-10-07 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1646?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14162106#comment-14162106 ] Jay Kreps commented on KAFKA-1646: -- Ah, you are saying Windows does a worse job of preall

[jira] [Updated] (KAFKA-1681) Newly elected KafkaController might not start deletion of pending topics

2014-10-07 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1681?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Neha Narkhede updated KAFKA-1681: - Fix Version/s: 0.8.2 > Newly elected KafkaController might not start deletion of pending topics >

Re: No process listening on advertised.port

2014-10-07 Thread Biju N
KAFKA-1092 (https://issues.apache.org/jira/browse/KAFKA-1092) provided the answer to the query. Thanks all. On Tue, Oct 7, 2014 at 8:26 AM, Biju N wrote: > Hello There, >I have the following in my server.properties on a two node kafka test > cluster > > …. > port=6667 > host.name=f-bcpc-vm3.

Java New Producer IO Thread Name

2014-10-07 Thread Bhavesh Mistry
Hi Kafka Dev Team, Since we have multiple instance of producers within one JVM, it would be good idea to name network IO thread name by associate with client.id configuration. kafka-producer-network-thread + client.tId Thanks, Bhavesh

[jira] [Created] (KAFKA-1681) Newly elected KafkaController might not start deletion of pending topics

2014-10-07 Thread Sriharsha Chintalapani (JIRA)
Sriharsha Chintalapani created KAFKA-1681: - Summary: Newly elected KafkaController might not start deletion of pending topics Key: KAFKA-1681 URL: https://issues.apache.org/jira/browse/KAFKA-1681

[jira] [Commented] (KAFKA-1663) Controller unable to shutdown after a soft failure

2014-10-07 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1663?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14161942#comment-14161942 ] Sriharsha Chintalapani commented on KAFKA-1663: --- [~nehanarkhede] Both TopicD

[jira] [Reopened] (KAFKA-1663) Controller unable to shutdown after a soft failure

2014-10-07 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1663?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Neha Narkhede reopened KAFKA-1663: -- [~sriharsha], while talking to Jun, realized that there may have been a regression introduced by th

[jira] [Commented] (KAFKA-1646) Improve consumer read performance for Windows

2014-10-07 Thread xueqiang wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1646?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14161857#comment-14161857 ] xueqiang wang commented on KAFKA-1646: -- This issue is caused by the discrete blocks o

No process listening on advertised.port

2014-10-07 Thread Biju N
Hello There, I have the following in my server.properties on a two node kafka test cluster …. port=6667 host.name=f-bcpc-vm3.bcpc.example.com advertised.host.name=f-bcpc-vm3.bcpc.example.com advertised.port=9092 … When I bring up Kafka, there is no process listening on port 9092 but Kafka list

[jira] [Updated] (KAFKA-1644) Inherit FetchResponse from RequestOrResponse

2014-10-07 Thread Anton Karamanov (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1644?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Anton Karamanov updated KAFKA-1644: --- Attachment: 0003-KAFKA-1644-Inherit-FetchResponse-from-RequestOrRespo.patch Indeed. Here's upd