Re: Review Request 34641: Patch for KAFKA-2214

2015-05-26 Thread Manikumar Reddy O
> On May 26, 2015, 7:08 a.m., Michael Noll wrote: > > core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala, line 81 > > > > > > Should we also consider reassignments that are in-progress as "errors"? > >

[jira] [Commented] (KAFKA-2222) Write "Input/output error" did not result in broker shutdown

2015-05-26 Thread Jason Rosenberg (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14560417#comment-14560417 ] Jason Rosenberg commented on KAFKA-: [~guozhang] I don't think so. The file s

[jira] [Comment Edited] (KAFKA-2222) Write "Input/output error" did not result in broker shutdown

2015-05-26 Thread Jason Rosenberg (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14560387#comment-14560387 ] Jason Rosenberg edited comment on KAFKA- at 5/27/15 5:05 AM: ---

[jira] [Resolved] (KAFKA-2201) Open file handle leak

2015-05-26 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2201?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar Reddy resolved KAFKA-2201. Resolution: Not A Problem > Open file handle leak > - > >

[jira] [Commented] (KAFKA-2222) Write "Input/output error" did not result in broker shutdown

2015-05-26 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14560396#comment-14560396 ] Guozhang Wang commented on KAFKA-: -- Hi [~jbrosenberg] is this related to KAFKA-18

[jira] [Commented] (KAFKA-2222) Write "Input/output error" did not result in broker shutdown

2015-05-26 Thread Jason Rosenberg (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14560387#comment-14560387 ] Jason Rosenberg commented on KAFKA-: Interestingly, after restarting the broke

[jira] [Created] (KAFKA-2222) Write "Input/output error" did not result in broker shutdown

2015-05-26 Thread Jason Rosenberg (JIRA)
Jason Rosenberg created KAFKA-: -- Summary: Write "Input/output error" did not result in broker shutdown Key: KAFKA- URL: https://issues.apache.org/jira/browse/KAFKA- Project: Kafka

[jira] [Updated] (KAFKA-2221) Log the entire cause which caused a reconnect in the SimpleConsumer

2015-05-26 Thread jaikiran pai (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2221?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] jaikiran pai updated KAFKA-2221: Attachment: KAFKA-2221.patch > Log the entire cause which caused a reconnect in the SimpleConsumer >

[jira] [Updated] (KAFKA-2221) Log the entire cause which caused a reconnect in the SimpleConsumer

2015-05-26 Thread jaikiran pai (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2221?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] jaikiran pai updated KAFKA-2221: Status: Patch Available (was: Open) > Log the entire cause which caused a reconnect in the SimpleCo

[jira] [Commented] (KAFKA-2221) Log the entire cause which caused a reconnect in the SimpleConsumer

2015-05-26 Thread jaikiran pai (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2221?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14560354#comment-14560354 ] jaikiran pai commented on KAFKA-2221: - Created reviewboard https://reviews.apache.org/

Review Request 34697: Patch for KAFKA-2221

2015-05-26 Thread Jaikiran Pai
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34697/ --- Review request for kafka. Bugs: KAFKA-2221 https://issues.apache.org/jira/b

[jira] [Created] (KAFKA-2221) Log the entire cause which caused a reconnect in the SimpleConsumer

2015-05-26 Thread jaikiran pai (JIRA)
jaikiran pai created KAFKA-2221: --- Summary: Log the entire cause which caused a reconnect in the SimpleConsumer Key: KAFKA-2221 URL: https://issues.apache.org/jira/browse/KAFKA-2221 Project: Kafka

[jira] [Updated] (KAFKA-2220) Improvement: Could we support rewind by time ?

2015-05-26 Thread Li Junjun (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2220?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Li Junjun updated KAFKA-2220: - Attachment: screenshot.png > Improvement: Could we support rewind by time ? > --

[jira] [Created] (KAFKA-2220) Improvement: Could we support rewind by time ?

2015-05-26 Thread Li Junjun (JIRA)
Li Junjun created KAFKA-2220: Summary: Improvement: Could we support rewind by time ? Key: KAFKA-2220 URL: https://issues.apache.org/jira/browse/KAFKA-2220 Project: Kafka Issue Type: Improvemen

Re: Review Request 34450: Fix KAFKA-2017

2015-05-26 Thread Jun Rao
> On May 21, 2015, 12:16 a.m., Jun Rao wrote: > > core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala, lines > > 102-106 > > > > > > Another way to do this is to only load from ZK on the becoming leader >

[DISCUSSION] Can we move the ack timeout in ProducerRequest to broker?

2015-05-26 Thread Jiangjie Qin
Hi, I am updating the wiki for KIP-19 and wondering why we have a replication timeout on producer side and in producer request? >From what I understand this is a server side setting and the reasons we need >this replication timeout is because we want to control the purgatory size. If >that is

[jira] [Comment Edited] (KAFKA-1737) Document required ZkSerializer for ZkClient used with AdminUtils

2015-05-26 Thread Vivek Madani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1737?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14560014#comment-14560014 ] Vivek Madani edited comment on KAFKA-1737 at 5/26/15 10:38 PM: -

[jira] [Commented] (KAFKA-1737) Document required ZkSerializer for ZkClient used with AdminUtils

2015-05-26 Thread Vivek Madani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1737?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14560014#comment-14560014 ] Vivek Madani commented on KAFKA-1737: - Thanks Guozhang. Did you also get a chance to l

[jira] [Comment Edited] (KAFKA-1737) Document required ZkSerializer for ZkClient used with AdminUtils

2015-05-26 Thread Vivek Madani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1737?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14560014#comment-14560014 ] Vivek Madani edited comment on KAFKA-1737 at 5/26/15 10:37 PM: -

Re: Kafka KIP hangout May 26

2015-05-26 Thread Neha Narkhede
Thanks for sending out the KIP notes, Ashish! On Tue, May 26, 2015 at 11:51 AM, Ashish Singh wrote: > Below are my notes. Feel free to add/ modify the content. > Kafka KIP discussion (May 26, 2015) KIP-12 (sasl/ssl authentication): > status check > > Not much discussion this time. > KIP-21 (conf

[jira] [Commented] (KAFKA-2154) MetadataResponse is Empty on a Fresh Cluster

2015-05-26 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2154?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14559899#comment-14559899 ] Jason Gustafson commented on KAFKA-2154: This seems like a duplicate of KAFKA-1867

Re: Review Request 34608: Patch for KAFKA-2217

2015-05-26 Thread Ewen Cheslack-Postava
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34608/#review85245 --- I noticed one minor issue with a metrics sensor, but otherwise LGTM

RE: [VOTE] KIP-21 Dynamic Configuration

2015-05-26 Thread Aditya Auradkar
Hey everyone, Completed the changes to KIP-4. After today's hangout, there doesn't appear to be anything remaining to discuss on this KIP. Please vote so we can formally close this. Thanks, Aditya From: Aditya Auradkar Sent: Thursday, May 21, 2015 11:26

Re: Review Request 34415: Patch for KAFKA-2195

2015-05-26 Thread Jun Rao
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34415/#review85223 --- Thanks for the patch. Just one more comment blow. clients/src/main

RE: [DISCUSS] KIP-4 - Command line and centralized administrative operations (Thread 2)

2015-05-26 Thread Aditya Auradkar
Andryii, I made a few edits to this document as discussed in the KIP-21 thread. https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations With these changes. the only difference between TopicMetadataResponse_V1 and V0 is the removal of the

Re: Review Request 34608: Patch for KAFKA-2217

2015-05-26 Thread Jason Gustafson
> On May 24, 2015, 7:16 p.m., Ewen Cheslack-Postava wrote: > > clients/src/main/java/org/apache/kafka/common/network/Selector.java, line > > 226 > > > > > > One drawback to this is that we're now constantly reallocat

[jira] [Commented] (KAFKA-2217) Refactor Client Selectable Interface for Better Concurrency Options

2015-05-26 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2217?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14559739#comment-14559739 ] Jason Gustafson commented on KAFKA-2217: Updated reviewboard https://reviews.apach

[jira] [Updated] (KAFKA-2217) Refactor Client Selectable Interface for Better Concurrency Options

2015-05-26 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2217?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-2217: --- Attachment: KAFKA-2217_2015-05-26_12:57:29.patch > Refactor Client Selectable Interface for Be

Re: Review Request 34608: Patch for KAFKA-2217

2015-05-26 Thread Jason Gustafson
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34608/ --- (Updated May 26, 2015, 7:58 p.m.) Review request for kafka. Bugs: KAFKA-2217

[jira] [Commented] (KAFKA-2168) New consumer poll() can block other calls like position(), commit(), and close() indefinitely

2015-05-26 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2168?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14559650#comment-14559650 ] Ewen Cheslack-Postava commented on KAFKA-2168: -- Some reasons you might want t

[jira] [Commented] (KAFKA-2217) Refactor Client Selectable Interface for Better Concurrency Options

2015-05-26 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2217?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14559646#comment-14559646 ] Jason Gustafson commented on KAFKA-2217: KAFKA-2168 was the main reason. Committin

[DISCUSSION] Partition Selection and Coordination By Brokers for Producers

2015-05-26 Thread Bhavesh Mistry
Hi Kafka Dev Team, I am sorry I am new to process of discussion and/or KIP. So, I had commented other email voting chain. Please do let me know correct process for collecting and staring discussion with Kafka Dev Group. Here is original message: I have had experience with both producer and co

Re: Review Request 33049: Patch for KAFKA-2084

2015-05-26 Thread Aditya Auradkar
> On May 12, 2015, 7:38 p.m., Dong Lin wrote: > > clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java, line 117 > > > > > > metric.value(timeMs), which translates to Rate.measure(config, timeMs), > > may

Re: Review Request 33049: Patch for KAFKA-2084

2015-05-26 Thread Aditya Auradkar
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33049/ --- (Updated May 26, 2015, 6:53 p.m.) Review request for kafka, Joel Koshy and Jun

Re: Kafka KIP hangout May 26

2015-05-26 Thread Ashish Singh
Below are my notes. Feel free to add/ modify the content. Kafka KIP discussion (May 26, 2015) KIP-12 (sasl/ssl authentication): status check Not much discussion this time. KIP-21 (configuration management) - Aditya will make changes to KIP-4 based on last time’s discussion. - Andrei is OK w

[jira] [Updated] (KAFKA-2084) byte rate metrics per client ID (producer and consumer)

2015-05-26 Thread Aditya A Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2084?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aditya A Auradkar updated KAFKA-2084: - Attachment: KAFKA-2084_2015-05-26_11:50:50.patch > byte rate metrics per client ID (produc

[jira] [Commented] (KAFKA-2084) byte rate metrics per client ID (producer and consumer)

2015-05-26 Thread Aditya A Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2084?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14559613#comment-14559613 ] Aditya A Auradkar commented on KAFKA-2084: -- Updated reviewboard https://reviews.a

Re: Review Request 33049: Patch for KAFKA-2084

2015-05-26 Thread Aditya Auradkar
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33049/ --- (Updated May 26, 2015, 6:50 p.m.) Review request for kafka, Joel Koshy and Jun

[jira] [Commented] (KAFKA-2217) Refactor Client Selectable Interface for Better Concurrency Options

2015-05-26 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2217?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14559567#comment-14559567 ] Jay Kreps commented on KAFKA-2217: -- This adds a lot of allocation per select call. Is the

[jira] [Commented] (KAFKA-2168) New consumer poll() can block other calls like position(), commit(), and close() indefinitely

2015-05-26 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2168?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14559563#comment-14559563 ] Jay Kreps commented on KAFKA-2168: -- This is a pretty big redesign of the consumer. The si

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

2015-05-26 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1944?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14559488#comment-14559488 ] Ashish K Singh commented on KAFKA-1944: --- [~omkreddy] thanks for the info. It's on my

[jira] [Commented] (KAFKA-2213) Log cleaner should write compacted messages using configured compression type

2015-05-26 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2213?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14559422#comment-14559422 ] Joel Koshy commented on KAFKA-2213: --- Case A: Yes. i.e., if broker compression.type is 'p

Re: Review Request 34608: Patch for KAFKA-2217

2015-05-26 Thread Jun Rao
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34608/#review85192 --- Thanks for the patch. Looks good. Just a couple of minor comments be

Re: [KIP-DISCUSSION] KIP-22 Expose a Partitioner interface in the new producer

2015-05-26 Thread Sriharsha Chintalapani
Hi Bhavesh,            I realized I didn’t send any vote closed message on this thread. Although the changes you are proposing related to partitioning but the change proposed in this KIP is minor one and will benefit the users . I appreciate if you can open up another thread for the discussion r

[jira] [Work started] (KAFKA-2107) Script to generate waiting-for-review report

2015-05-26 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2107?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-2107 started by Manikumar Reddy. -- > Script to generate waiting-for-review report >

[jira] [Updated] (KAFKA-2107) Script to generate waiting-for-review report

2015-05-26 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2107?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar Reddy updated KAFKA-2107: --- Status: Patch Available (was: In Progress) > Script to generate waiting-for-review report > -

Re: Review Request 34608: Patch for KAFKA-2217

2015-05-26 Thread Jason Gustafson
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34608/ --- (Updated May 26, 2015, 4:38 p.m.) Review request for kafka. Bugs: KAFKA-2217

[jira] [Commented] (KAFKA-2217) Refactor Client Selectable Interface for Better Concurrency Options

2015-05-26 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2217?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14559339#comment-14559339 ] Jason Gustafson commented on KAFKA-2217: Updated reviewboard https://reviews.apach

[jira] [Updated] (KAFKA-2217) Refactor Client Selectable Interface for Better Concurrency Options

2015-05-26 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2217?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-2217: --- Attachment: KAFKA-2217_2015-05-26_09:37:29.patch > Refactor Client Selectable Interface for Be

Re: [KIP-DISCUSSION] KIP-22 Expose a Partitioner interface in the new producer

2015-05-26 Thread Bhavesh Mistry
Hi All, This might be too late about partitioning strategy and use cases to cover. I have had experience with both producer and consumer side. I have different use case on this partition selection strategy. Problem: We have heterogeneous environment of producers (by that I mean we have n

Re: Review Request 33065: Patch for KAFKA-1928

2015-05-26 Thread Jun Rao
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33065/#review85180 --- Ship it! Thanks for the latest patch. +1 other than the following m

Kafka KIP hangout May 26

2015-05-26 Thread Jun Rao
Hi, Everyone, We will have a KIP hangout at 11 PST on May 26. The following is the agenda. If you want to attend and is not on the invite, please let me know. Agenda: KIP-12 (sasl/ssl authentication): status check KIP-21 (configuration management) KIP-19 (Add a request timeout to NetworkClient) K

Re: Review Request 33620: Patch for KAFKA-1690

2015-05-26 Thread Michael Herstine
- Michael --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review83993 --- On May 21, 2015, 5:37 p.m., Sriharsha Chintalapani wrot

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

2015-05-26 Thread Adamos Loizou (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1419?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14559172#comment-14559172 ] Adamos Loizou commented on KAFKA-1419: -- Hello, is there any ETA on including the 2.11

Re: Kafka server validating incoming messages

2015-05-26 Thread Gwen Shapira
I think it makes sense to take this reasonable precaution and check that the entire message was parsed successfully at the server side. On Tue, May 26, 2015 at 4:31 AM, Grant Henke wrote: > Bumping this message again to get some input before opening a Jira. > > On Thu, May 21, 2015 at 11:31 AM

[jira] [Comment Edited] (KAFKA-2201) Open file handle leak

2015-05-26 Thread Albert Visagie (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2201?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14558885#comment-14558885 ] Albert Visagie edited comment on KAFKA-2201 at 5/26/15 8:53 AM:

[jira] [Commented] (KAFKA-2201) Open file handle leak

2015-05-26 Thread Albert Visagie (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2201?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14558885#comment-14558885 ] Albert Visagie commented on KAFKA-2201: --- It is indeed. It would appear that the only

Re: Review Request 34641: Patch for KAFKA-2214

2015-05-26 Thread Michael Noll
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34641/#review85154 --- core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala