Fwd: [DISCUSS] KIP-12 - Kafka Sasl/Kerberos implementation

2015-02-11 Thread Harsha
Hi, Here is the initial proposal for sasl/kerberos implementation for kafka https://cwiki.apache.org/confluence/x/YI4WAw and JIRA https://issues.apache.org/jira/browse/KAFKA-1686. I am currently working on prototype which will add more details to the KIP. Just opening the thread to say

[jira] [Updated] (KAFKA-1866) LogStartOffset gauge throws exceptions after log.delete()

2015-02-11 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1866?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sriharsha Chintalapani updated KAFKA-1866: -- Attachment: KAFKA-1866_2015-02-11_09:25:33.patch LogStartOffset gauge throws

[jira] [Commented] (KAFKA-1757) Can not delete Topic index on Windows

2015-02-11 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1757?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14316454#comment-14316454 ] Sriharsha Chintalapani commented on KAFKA-1757: --- [~junrao] Can you please

[jira] [Commented] (KAFKA-1866) LogStartOffset gauge throws exceptions after log.delete()

2015-02-11 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1866?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14316572#comment-14316572 ] Sriharsha Chintalapani commented on KAFKA-1866: --- Updated reviewboard

Re: [KIP-DISCUSSION] Mirror Maker Enhancement

2015-02-11 Thread Joel Koshy
Hi Jay, The data channels are actually a big part of the complexity of the zero data loss design, though, right? Because then you need some reverse channel to flow the acks back to the consumer based on where you are versus just acking what you have read and written (as in the code snippet I

[DISCUSS] KIP-12 - Kafka Sasl/Kerberos implementation

2015-02-11 Thread Harsha
Hi, Here is the initial proposal for sasl/kerberos implementation for kafka https://cwiki.apache.org/confluence/x/YI4WAw and JIRA https://issues.apache.org/jira/browse/KAFKA-1686. I am currently working on prototype which will add more details to the KIP. Just opening the thread to say

[jira] [Commented] (KAFKA-1566) Kafka environment configuration (kafka-env.sh)

2015-02-11 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1566?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14316455#comment-14316455 ] Sriharsha Chintalapani commented on KAFKA-1566: --- [~jkreps] [~nehanarkhede]

[jira] [Commented] (KAFKA-1852) OffsetCommitRequest can commit offset on unknown topic

2015-02-11 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1852?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14316447#comment-14316447 ] Sriharsha Chintalapani commented on KAFKA-1852: --- [~jjkoshy] pinging for a

[jira] [Commented] (KAFKA-1374) LogCleaner (compaction) does not support compressed topics

2015-02-11 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1374?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14316526#comment-14316526 ] Jay Kreps commented on KAFKA-1374: -- Hey guys, the test kafka.tools.TestLogCleaning is a

Re: [KIP-DISCUSSION] Mirror Maker Enhancement

2015-02-11 Thread Jay Kreps
Guozhang, I agree with 1-3, I do think what I was proposing was simpler but perhaps there are gaps in that? Hey Joel--Here was a sketch of what I was proposing. I do think this get's rid of manual offset tracking, especially doing so across threads with dedicated commit threads, which I think is

Re: Review Request 30084: Patch for KAFKA-1866

2015-02-11 Thread Sriharsha Chintalapani
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/30084/ --- (Updated Feb. 11, 2015, 5:25 p.m.) Review request for kafka. Bugs:

[jira] [Commented] (KAFKA-1852) OffsetCommitRequest can commit offset on unknown topic

2015-02-11 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1852?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14316739#comment-14316739 ] Joel Koshy commented on KAFKA-1852: --- Thanks for the ping - will take a look.

[jira] [Updated] (KAFKA-1943) Producer request failure rate should not include MessageSetSizeTooLarge and MessageSizeTooLargeException

2015-02-11 Thread Aditya Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1943?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aditya Auradkar updated KAFKA-1943: --- Issue Type: Sub-task (was: Bug) Parent: KAFKA-1946 Producer request failure rate

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

2015-02-11 Thread Chi Hoang
For the Sample usage section, please consider https://github.com/airbnb/kafkat. We find that tool to be very easy to use, and extremely useful for our administration tasks. Chi On Mon, Feb 9, 2015 at 9:03 AM, Guozhang Wang wangg...@gmail.com wrote: I feel the benefits of lowering the

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

2015-02-11 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1646?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14316747#comment-14316747 ] Jay Kreps commented on KAFKA-1646: -- Hey [~waldenchen] yeah I think what you are saying is

[jira] [Created] (KAFKA-1946) Improve BrokerTopicMetrics reporting

2015-02-11 Thread Aditya Auradkar (JIRA)
Aditya Auradkar created KAFKA-1946: -- Summary: Improve BrokerTopicMetrics reporting Key: KAFKA-1946 URL: https://issues.apache.org/jira/browse/KAFKA-1946 Project: Kafka Issue Type:

Re: [DISCUSS] KIP-12 - Kafka Sasl/Kerberos implementation

2015-02-11 Thread Joe Stein
Thanks Harsha, looks good so far. How were you thinking of running the KerberosTicketManager as a standalone process or like controller or is it a layer of code that does the plumbing pieces everywhere? ~ Joestein On Wed, Feb 11, 2015 at 12:18 PM, Harsha ka...@harsha.io wrote: Hi, Here is

[jira] [Commented] (KAFKA-1374) LogCleaner (compaction) does not support compressed topics

2015-02-11 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1374?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14316735#comment-14316735 ] Jay Kreps commented on KAFKA-1374: -- Great! LogCleaner (compaction) does not support

[jira] [Updated] (KAFKA-1914) Count TotalProduceRequestRate and TotalFetchRequestRate in BrokerTopicMetrics

2015-02-11 Thread Aditya Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1914?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aditya Auradkar updated KAFKA-1914: --- Issue Type: Sub-task (was: Bug) Parent: KAFKA-1946 Count TotalProduceRequestRate

[jira] [Commented] (KAFKA-1419) cross build for scala 2.11

2015-02-11 Thread Helena Edelson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1419?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14316760#comment-14316760 ] Helena Edelson commented on KAFKA-1419: --- This ticket says the cross build is

[jira] [Commented] (KAFKA-1926) Replace kafka.utils.Utils with o.a.k.common.utils.Utils

2015-02-11 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1926?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14316759#comment-14316759 ] Jay Kreps commented on KAFKA-1926: -- At a high level this looks good. The main thrust of

[jira] [Commented] (KAFKA-1926) Replace kafka.utils.Utils with o.a.k.common.utils.Utils

2015-02-11 Thread Tong Li (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1926?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14316830#comment-14316830 ] Tong Li commented on KAFKA-1926: [~jkreps] Jay, really appreciate your quick review

Re: [DISCUSSION] KIP-2: Refactor Brokers to Allow Multiple Endpoints

2015-02-11 Thread Jun Rao
+1 for proposed changes in 1 and 2. 1. The impact is that if someone uses SimpleConsumer and references Broker explicitly, the application needs code change to compile with 0.8.3. Since SimpleConsumer is not widely used, breaking the API in SimpleConsumer but maintaining overall code cleanness

[jira] [Updated] (KAFKA-1938) [doc] Quick start example should reference appropriate Kafka version

2015-02-11 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1938?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar Reddy updated KAFKA-1938: --- Attachment: remove-081-references.patch Attaching a patch which removes 0.8.1 references form

[jira] [Commented] (KAFKA-1374) LogCleaner (compaction) does not support compressed topics

2015-02-11 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1374?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14316691#comment-14316691 ] Manikumar Reddy commented on KAFKA-1374: yes, we are using TestLogCleaning tool

[jira] [Assigned] (KAFKA-1887) controller error message on shutting the last broker

2015-02-11 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1887?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sriharsha Chintalapani reassigned KAFKA-1887: - Assignee: Sriharsha Chintalapani controller error message on shutting

[jira] [Updated] (KAFKA-1920) Add a metric to count client side errors in BrokerTopicMetrics

2015-02-11 Thread Aditya Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1920?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aditya Auradkar updated KAFKA-1920: --- Issue Type: Sub-task (was: Improvement) Parent: KAFKA-1946 Add a metric to count

[jira] [Commented] (KAFKA-1374) LogCleaner (compaction) does not support compressed topics

2015-02-11 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1374?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14316714#comment-14316714 ] Joel Koshy commented on KAFKA-1374: --- I can review this next week. However, as far as

[jira] [Updated] (KAFKA-1936) Track offset commit requests separately from produce requests

2015-02-11 Thread Aditya Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1936?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aditya Auradkar updated KAFKA-1936: --- Issue Type: Sub-task (was: Bug) Parent: KAFKA-1946 Track offset commit requests

[jira] [Resolved] (KAFKA-1945) MetaData Response - Broker hostname is wrong

2015-02-11 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1945?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joel Koshy resolved KAFKA-1945. --- Resolution: Invalid MetaData Response - Broker hostname is wrong

Re: Build failed in Jenkins: Kafka-trunk #389

2015-02-11 Thread Joe Stein
I ran into this issue locally too. I left it running overnight and it was stuck when I got back to it. This is where I had to kill the terminal at this morning. kafka.admin.TopicCommandTest testConfigPreservationAcrossPartitionAlteration PASSED kafka.api.ApiUtilsTest testShortStringASCII

[jira] [Commented] (KAFKA-1944) Rename LogCleaner and related classes to LogCompactor

2015-02-11 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1944?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14316222#comment-14316222 ] Joel Koshy commented on KAFKA-1944: --- Sure - I think this can wait until that is in.

[jira] [Commented] (KAFKA-1926) Replace kafka.utils.Utils with o.a.k.common.utils.Utils

2015-02-11 Thread Tong Li (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1926?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14316380#comment-14316380 ] Tong Li commented on KAFKA-1926: [~harsha_ch]Yeah, will do that. Thanks. Replace

[jira] [Commented] (KAFKA-1926) Replace kafka.utils.Utils with o.a.k.common.utils.Utils

2015-02-11 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1926?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14316319#comment-14316319 ] Sriharsha Chintalapani commented on KAFKA-1926: --- [~tongli] you should make

[jira] [Commented] (KAFKA-1944) Rename LogCleaner and related classes to LogCompactor

2015-02-11 Thread Ashish Kumar Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1944?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14316951#comment-14316951 ] Ashish Kumar Singh commented on KAFKA-1944: --- Sure. Rename LogCleaner and

Re: [KIP-DISCUSSION] Mirror Maker Enhancement

2015-02-11 Thread Jay Kreps
Cool, I agree with all that. I agree about the need for a rebalancing callback. Totally agree about record handler. It would be great to see if a prototype of this is workable. Thanks guys! -Jay On Wed, Feb 11, 2015 at 12:36 PM, Joel Koshy jjkosh...@gmail.com wrote: Hey Jay, Guozhang,

Re: [KIP-DISCUSSION] Mirror Maker Enhancement

2015-02-11 Thread Joel Koshy
Hey Jay, Guozhang, Becket and I got together to discuss this and we think: - It seems that your proposal based on the new consumer and flush call should work. - We would likely need to call the poll with a timeout that matches the offset commit interval in order to deal with low volume

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

2015-02-11 Thread Jay Kreps
Hey Andrii, To answer your earlier question we just really can't be adding any more scala protocol objects. These things are super hard to maintain because they hand code the byte parsing and don't have good versioning support. Since we are already planning on converting we definitely don't want

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

2015-02-11 Thread Andrii Biletskyi
Hey Jay, I would like to continue this discussion as it seem there is no progress here. First of all, could you please explain what did you mean in 2? How exactly are we going to migrate to the new java protocol definitions. And why it's a blocker for centralized CLI? I agree with you, this

Re: Review Request 29468: Patch for KAFKA-1805

2015-02-11 Thread Gwen Shapira
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29468/#review72032 --- Can you add unit tests? - Gwen Shapira On Feb. 11, 2015, 10:34

Re: Review Request 29468: Patch for KAFKA-1805

2015-02-11 Thread Parth Brahmbhatt
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29468/ --- (Updated Feb. 11, 2015, 10:34 p.m.) Review request for kafka. Bugs:

[jira] [Commented] (KAFKA-1805) Kafka ProducerRecord should implement equals

2015-02-11 Thread Parth Brahmbhatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1805?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14317118#comment-14317118 ] Parth Brahmbhatt commented on KAFKA-1805: - Updated reviewboard

[jira] [Updated] (KAFKA-1805) Kafka ProducerRecord should implement equals

2015-02-11 Thread Parth Brahmbhatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1805?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Parth Brahmbhatt updated KAFKA-1805: Attachment: KAFKA-1805_2015-02-11_14:34:28.patch Kafka ProducerRecord should implement

[jira] [Commented] (KAFKA-1805) Kafka ProducerRecord should implement equals

2015-02-11 Thread Parth Brahmbhatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1805?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14317108#comment-14317108 ] Parth Brahmbhatt commented on KAFKA-1805: - Updated reviewboard

Re: Review Request 29468: Patch for KAFKA-1805

2015-02-11 Thread Parth Brahmbhatt
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29468/ --- (Updated Feb. 11, 2015, 10:30 p.m.) Review request for kafka. Bugs:

[jira] [Updated] (KAFKA-1805) Kafka ProducerRecord should implement equals

2015-02-11 Thread Parth Brahmbhatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1805?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Parth Brahmbhatt updated KAFKA-1805: Attachment: KAFKA-1805_2015-02-11_14:30:14.patch Kafka ProducerRecord should implement

Re: Review Request 29468: Patch for KAFKA-1805

2015-02-11 Thread Parth Brahmbhatt
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29468/ --- (Updated Feb. 11, 2015, 10:37 p.m.) Review request for kafka. Bugs:

Re: Review Request 29468: Patch for KAFKA-1805

2015-02-11 Thread Parth Brahmbhatt
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29468/#review72031 ---

Re: Review Request 29468: Patch for KAFKA-1805

2015-02-11 Thread Parth Brahmbhatt
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29468/ --- (Updated Feb. 11, 2015, 10:53 p.m.) Review request for kafka. Bugs:

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

2015-02-11 Thread Andrii Biletskyi
Jay, Thanks for answering. You understood correctly, most of my comments were related to your point 1) - about well thought-out apis. Also, yes, as I understood we would like to introduce a single unified CLI tool with centralized server-side request handling for lots of existing ones (incl.

Re: Review Request 29468: Patch for KAFKA-1805

2015-02-11 Thread Parth Brahmbhatt
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29468/ --- (Updated Feb. 11, 2015, 10:49 p.m.) Review request for kafka. Bugs:

Re: [DISCUSSION] KAFKA-1697 - make NotEnoughReplicasAfterAppend a non-retriable exception

2015-02-11 Thread Guozhang Wang
If people have agreed upon this semantic: quote if you set retries 0 you are saying I accept duplicates but want to ensure my stuff gets written, if you set retries = 0 you are saying I can't abide duplicates and am willing to tolerate loss. So Retryable for us means retry may succeed. \quote

Re: [DISCUSSION] KAFKA-1697 - make NotEnoughReplicasAfterAppend a non-retriable exception

2015-02-11 Thread Gwen Shapira
Makes sense to me. Thanks for the very detailed clarification, Jay :) Will leave NotEnoughReplicasAfterAppend as retriable. Gwen On Wed, Feb 11, 2015 at 4:18 PM, Guozhang Wang wangg...@gmail.com wrote: If people have agreed upon this semantic: quote if you set retries 0 you are saying I

Re: [DISCUSSION] KIP-2: Refactor Brokers to Allow Multiple Endpoints

2015-02-11 Thread Joel Koshy
The description that Jun gave for (2) was the detail I was looking for - Gwen can you update the KIP with that for completeness/clarity? I'm +1 as well overall. However, I think it would be good if we also get an ack from someone who is more experienced on the operations side (say, Todd) to

Re: [DISCUSSION] KIP-2: Refactor Brokers to Allow Multiple Endpoints

2015-02-11 Thread Gwen Shapira
Added Jun's notes to the KIP (Thanks for explaining so clearly, Jun. I was clearly struggling with this...) and removed the reference to use.new.wire.protocol. On Wed, Feb 11, 2015 at 4:19 PM, Joel Koshy jjkosh...@gmail.com wrote: The description that Jun gave for (2) was the detail I was

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

2015-02-11 Thread Jay Kreps
Yeah I totally agree that we don't want to just have one do admin stuff command that has the union of all parameters. What I am saying is that command line tools are one client of the administrative apis, but these will be used in a number of scenarios so they should make logical sense even in

Re: [DISCUSSION] KIP-2: Refactor Brokers to Allow Multiple Endpoints

2015-02-11 Thread Gwen Shapira
Jun, I'm not sure we should default wire.protocol.version to the previous version. This will make fresh installs a bit weird :) I think we should default to the new version and assume that when I'm upgrading a broker, I'm re-using an existing configuration file. This way, if I'm upgrading

[jira] [Updated] (KAFKA-1805) Kafka ProducerRecord should implement equals

2015-02-11 Thread Parth Brahmbhatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1805?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Parth Brahmbhatt updated KAFKA-1805: Attachment: KAFKA-1805_2015-02-11_14:37:41.patch Kafka ProducerRecord should implement

[jira] [Commented] (KAFKA-1805) Kafka ProducerRecord should implement equals

2015-02-11 Thread Parth Brahmbhatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1805?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14317124#comment-14317124 ] Parth Brahmbhatt commented on KAFKA-1805: - Updated reviewboard

[jira] [Commented] (KAFKA-1805) Kafka ProducerRecord should implement equals

2015-02-11 Thread Parth Brahmbhatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1805?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14317123#comment-14317123 ] Parth Brahmbhatt commented on KAFKA-1805: - Updated reviewboard

Re: Review Request 29468: Patch for KAFKA-1805

2015-02-11 Thread Parth Brahmbhatt
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29468/ --- (Updated Feb. 11, 2015, 10:37 p.m.) Review request for kafka. Bugs:

[jira] [Updated] (KAFKA-1805) Kafka ProducerRecord should implement equals

2015-02-11 Thread Parth Brahmbhatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1805?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Parth Brahmbhatt updated KAFKA-1805: Attachment: KAFKA-1805_2015-02-11_14:37:09.patch Kafka ProducerRecord should implement

[jira] [Updated] (KAFKA-1805) Kafka ProducerRecord should implement equals

2015-02-11 Thread Parth Brahmbhatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1805?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Parth Brahmbhatt updated KAFKA-1805: Attachment: KAFKA-1805_2015-02-11_14:49:10.patch Kafka ProducerRecord should implement

Re: Review Request 29468: Patch for KAFKA-1805

2015-02-11 Thread Parth Brahmbhatt
On Feb. 11, 2015, 10:35 p.m., Gwen Shapira wrote: Can you add unit tests? Added uni tests. - Parth --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29468/#review72032

[jira] [Commented] (KAFKA-1805) Kafka ProducerRecord should implement equals

2015-02-11 Thread Parth Brahmbhatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1805?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14317145#comment-14317145 ] Parth Brahmbhatt commented on KAFKA-1805: - Updated reviewboard

[jira] [Commented] (KAFKA-1660) Ability to call close() with a timeout on the Java Kafka Producer.

2015-02-11 Thread Parth Brahmbhatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1660?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14317171#comment-14317171 ] Parth Brahmbhatt commented on KAFKA-1660: - I am not aware of any clean way to

Re: [DISCUSS] KIP-12 - Kafka Sasl/Kerberos implementation

2015-02-11 Thread Gwen Shapira
Looks good. Thanks for working on this. One note, the Channel implementation from SSL only works on Java7 and up. Since we are still supporting Java 6, I'm working on a lighter wrapper that will be a composite on SocketChannel but will not extend it. Perhaps you'll want to use that. Looking

Re: Review Request 29647: Patch for KAFKA-1697

2015-02-11 Thread Gwen Shapira
On Feb. 11, 2015, 1:48 p.m., Joel Koshy wrote: clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.java, line 23 https://reviews.apache.org/r/29647/diff/3/?file=860600#file860600line23 Pending discussion. Gwen Shapira wrote: changing

[jira] [Commented] (KAFKA-1660) Ability to call close() with a timeout on the Java Kafka Producer.

2015-02-11 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1660?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14317335#comment-14317335 ] Jay Kreps commented on KAFKA-1660: -- Yeah I'm not proposing calling thread.stop(), we

Re: [DISCUSSION] KAFKA-1697 - make NotEnoughReplicasAfterAppend a non-retriable exception

2015-02-11 Thread Jay Kreps
Yeah, hey Joel, that was a super long winded way of saying let's leave it at Retriable. I agree there is another concept which is non duplicate producing but when we do the idempotence stuff then all things will have that property so it may be okay just to leave it for now since network errors

Re: Review Request 29647: Patch for KAFKA-1697

2015-02-11 Thread Gwen Shapira
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29647/ --- (Updated Feb. 12, 2015, 2:45 a.m.) Review request for kafka. Bugs:

[jira] [Updated] (KAFKA-1697) remove code related to ack1 on the broker

2015-02-11 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-1697: Attachment: KAFKA-1697_2015-02-11_18:45:42.patch remove code related to ack1 on the broker

[jira] [Commented] (KAFKA-1697) remove code related to ack1 on the broker

2015-02-11 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14317471#comment-14317471 ] Gwen Shapira commented on KAFKA-1697: - Updated reviewboard

[jira] [Commented] (KAFKA-1947) can't explicitly set replica-assignment when add partitions

2015-02-11 Thread Honghai Chen (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14317513#comment-14317513 ] Honghai Chen commented on KAFKA-1947: - After fix the command line, the command still

[jira] [Created] (KAFKA-1947) can't explicitly set replica-assignment when add partitions

2015-02-11 Thread Honghai Chen (JIRA)
Honghai Chen created KAFKA-1947: --- Summary: can't explicitly set replica-assignment when add partitions Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka

[jira] [Commented] (KAFKA-1938) [doc] Quick start example should reference appropriate Kafka version

2015-02-11 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1938?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14317390#comment-14317390 ] Gwen Shapira commented on KAFKA-1938: - +1 LGTM (non-binding) [doc] Quick start

Re: Review Request 28769: Patch for KAFKA-1809

2015-02-11 Thread Gwen Shapira
On Feb. 11, 2015, 9:45 a.m., Jun Rao wrote: Thanks for the new patch. Some more comments. 1. We should think through whether we need to add security protocol to existing tools like SimleConsumerShell and UpdateOffsetsInZk. 2. There are unused imports. 3. The patch needs rebase.

Re: [DISCUSSION] KIP-2: Refactor Brokers to Allow Multiple Endpoints

2015-02-11 Thread Todd Palino
Thanks, Gwen. This looks good to me as far as the wire protocol versioning goes. I agree with you on defaulting to the new wire protocol version for new installs. I think it will also need to be very clear (to general installer of Kafka, and not just developers) in documentation when the wire

Re: [KIP-DISCUSSION] Mirror Maker Enhancement

2015-02-11 Thread Neha Narkhede
Thanks for the explanation, Joel! Would love to see the results of the throughput experiment and I'm a +1 on everything else, including the rebalance callback and record handler. -Neha On Wed, Feb 11, 2015 at 1:13 PM, Jay Kreps jay.kr...@gmail.com wrote: Cool, I agree with all that. I agree

[jira] [Updated] (KAFKA-1697) remove code related to ack1 on the broker

2015-02-11 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-1697: Attachment: KAFKA-1697_2015-02-11_18:47:53.patch remove code related to ack1 on the broker

Re: Review Request 29647: Patch for KAFKA-1697

2015-02-11 Thread Gwen Shapira
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29647/ --- (Updated Feb. 12, 2015, 2:47 a.m.) Review request for kafka. Bugs:

[jira] [Commented] (KAFKA-1697) remove code related to ack1 on the broker

2015-02-11 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14317475#comment-14317475 ] Gwen Shapira commented on KAFKA-1697: - Updated reviewboard

Re: Review Request 29647: Patch for KAFKA-1697

2015-02-11 Thread Gwen Shapira
On Feb. 11, 2015, 1:48 p.m., Joel Koshy wrote: core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala, line 96 https://reviews.apache.org/r/29647/diff/3/?file=860605#file860605line96 Do we need this here? IMO, its a good idea to have this in any test that starts new threads

Re: [DISCUSSION] KAFKA-1697 - make NotEnoughReplicasAfterAppend a non-retriable exception

2015-02-11 Thread Neha Narkhede
As for me, my preference is to leave it as retriable since it is clear that the produce may succeed on a retry (and may introduce a duplicate). I agree that idempotence will bring full closure to this though. +1 On Wed, Feb 11, 2015 at 5:24 PM, Jay Kreps jay.kr...@gmail.com wrote: Yeah,

[jira] [Updated] (KAFKA-1938) [doc] Quick start example should reference appropriate Kafka version

2015-02-11 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1938?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-1938: Assignee: Manikumar Reddy [doc] Quick start example should reference appropriate Kafka version

Re: Review Request 29647: Patch for KAFKA-1697

2015-02-11 Thread Gwen Shapira
On Feb. 11, 2015, 1:48 p.m., Joel Koshy wrote: clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.java, line 23 https://reviews.apache.org/r/29647/diff/3/?file=860600#file860600line23 Pending discussion. Gwen Shapira wrote: changing

[jira] [Commented] (KAFKA-1948) kafka.api.consumerTests are hanging

2015-02-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1948?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14317722#comment-14317722 ] Guozhang Wang commented on KAFKA-1948: -- Gwen, Just ran locally a few times and I do

[jira] [Commented] (KAFKA-1948) kafka.api.consumerTests are hanging

2015-02-11 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1948?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14317713#comment-14317713 ] Gwen Shapira commented on KAFKA-1948: - [~guozhang] [~onurkaraman] - since you worked

[jira] [Created] (KAFKA-1948) kafka.api.consumerTests are hanging

2015-02-11 Thread Gwen Shapira (JIRA)
Gwen Shapira created KAFKA-1948: --- Summary: kafka.api.consumerTests are hanging Key: KAFKA-1948 URL: https://issues.apache.org/jira/browse/KAFKA-1948 Project: Kafka Issue Type: Bug

[jira] [Commented] (KAFKA-1887) controller error message on shutting the last broker

2015-02-11 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1887?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14317695#comment-14317695 ] Gwen Shapira commented on KAFKA-1887: - [~harsha_ch] I think the idea was simply to

Build failed in Jenkins: KafkaPreCommit #3

2015-02-11 Thread Apache Jenkins Server
See https://builds.apache.org/job/KafkaPreCommit/3/ -- [...truncated 901 lines...] kafka.log.LogManagerTest testCheckpointRecoveryPoints PASSED kafka.log.LogManagerTest testRecoveryDirectoryMappingWithTrailingSlash PASSED kafka.log.LogManagerTest

Re: Review Request 29647: Patch for KAFKA-1697

2015-02-11 Thread Gwen Shapira
On Feb. 12, 2015, 4:20 a.m., Joe Stein wrote: core/src/main/scala/kafka/server/ReplicaManager.scala, line 307 https://reviews.apache.org/r/29647/diff/5/?file=861501#file861501line307 Could we change this to a match case? requiredAcks match { case 0 = {} case 1 =

[jira] [Updated] (KAFKA-1945) MetaData Response - Broker hostname is wrong

2015-02-11 Thread saravana kumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1945?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] saravana kumar updated KAFKA-1945: -- Description: I use python-kafka's SimpleConsumer to listen to a topic in kafka broker. Kafka

Re: Kafka New(Java) Producer Connection reset by peer error and LB

2015-02-11 Thread Ewen Cheslack-Postava
Agree with Jay. It's unfortunate that this gets logged because in this case it's just noise, but this is an exception that can happen both in potentially bad cases (remote peer closed connection forcibly with outstanding unprocessed data) or in normal cases that aren't problematic (TCP connection

Re: Review Request 28769: Patch for KAFKA-1809

2015-02-11 Thread Jun Rao
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/28769/#review71916 --- Thanks for the new patch. Some more comments. 1. We should think

[jira] [Resolved] (KAFKA-1377) transient unit test failure in LogOffsetTest

2015-02-11 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1377?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar Reddy resolved KAFKA-1377. Resolution: Fixed now i am not getting this exception..so closing the issue. transient

[jira] [Created] (KAFKA-1945) MetaData Response - Broker hostname is wrong

2015-02-11 Thread saravana kumar (JIRA)
saravana kumar created KAFKA-1945: - Summary: MetaData Response - Broker hostname is wrong Key: KAFKA-1945 URL: https://issues.apache.org/jira/browse/KAFKA-1945 Project: Kafka Issue Type: Bug

[jira] [Commented] (KAFKA-1945) MetaData Response - Broker hostname is wrong

2015-02-11 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1945?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14315841#comment-14315841 ] Manikumar Reddy commented on KAFKA-1945: You can set host.name ,

Re: [DISCUSSION] KAFKA-1697 - make NotEnoughReplicasAfterAppend a non-retriable exception

2015-02-11 Thread Joel Koshy
Thanks for the comments - however, it is not clear to me what your preference is on making NotEnoughReplicasAfterAppend retriable vs non-retriable. As for me, my preference is to leave it as retriable since it is clear that the produce may succeed on a retry (and may introduce a duplicate). I

Re: Review Request 29647: Patch for KAFKA-1697

2015-02-11 Thread Joel Koshy
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29647/#review71960 ---

  1   2   >