Re: Review Request 29590: 1. Removed defaults for serializer/deserializer. 2. Converted type cast exception to serialization exception in the producer. 3. Added string ser/deser. 4. Moved the isKey fl

2015-01-08 Thread Manikumar Reddy O
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29590/#review67194 --- clients/src/main/java/org/apache/kafka/common/serialization/StringS

[jira] [Updated] (KAFKA-1845) KafkaConfig should use ConfigDef

2015-01-08 Thread Andrii Biletskyi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1845?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andrii Biletskyi updated KAFKA-1845: Issue Type: Sub-task (was: Improvement) Parent: KAFKA-1694 > KafkaConfig should use

[jira] [Commented] (KAFKA-1845) KafkaConfig should use ConfigDef

2015-01-08 Thread Andrii Biletskyi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1845?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14269348#comment-14269348 ] Andrii Biletskyi commented on KAFKA-1845: - I converted it to sub-task and put unde

[jira] [Created] (KAFKA-1847) Update Readme to reflect changes in gradle wrapper

2015-01-08 Thread Martin Schinz (JIRA)
Martin Schinz created KAFKA-1847: Summary: Update Readme to reflect changes in gradle wrapper Key: KAFKA-1847 URL: https://issues.apache.org/jira/browse/KAFKA-1847 Project: Kafka Issue Type:

[jira] [Commented] (KAFKA-1847) Update Readme to reflect changes in gradle wrapper

2015-01-08 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1847?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14269478#comment-14269478 ] Joe Stein commented on KAFKA-1847: -- Hi [~thinktainer] this is already updated https://gi

Re: Review Request 29523: Patch for KAFKA-1723

2015-01-08 Thread Manikumar Reddy O
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29523/ --- (Updated Jan. 8, 2015, 4:13 p.m.) Review request for kafka. Bugs: KAFKA-1723

[jira] [Commented] (KAFKA-1723) make the metrics name in new producer more standard

2015-01-08 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1723?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14269489#comment-14269489 ] Manikumar Reddy commented on KAFKA-1723: Updated reviewboard https://reviews.apach

[jira] [Updated] (KAFKA-1723) make the metrics name in new producer more standard

2015-01-08 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1723?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar Reddy updated KAFKA-1723: --- Attachment: KAFKA-1723_2015-01-08_21:41:13.patch > make the metrics name in new producer more

[jira] [Commented] (KAFKA-1847) Update Readme to reflect changes in gradle wrapper

2015-01-08 Thread Martin Schinz (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1847?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14269512#comment-14269512 ] Martin Schinz commented on KAFKA-1847: -- Sorry, I must have missed that when reading i

[jira] [Resolved] (KAFKA-1847) Update Readme to reflect changes in gradle wrapper

2015-01-08 Thread Martin Schinz (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1847?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Martin Schinz resolved KAFKA-1847. -- Resolution: Invalid > Update Readme to reflect changes in gradle wrapper > -

Re: Review Request 29523: Patch for KAFKA-1723

2015-01-08 Thread Manikumar Reddy O
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29523/ --- (Updated Jan. 8, 2015, 4:34 p.m.) Review request for kafka. Bugs: KAFKA-1723

[jira] [Commented] (KAFKA-1723) make the metrics name in new producer more standard

2015-01-08 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1723?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14269513#comment-14269513 ] Manikumar Reddy commented on KAFKA-1723: Updated reviewboard https://reviews.apach

[jira] [Updated] (KAFKA-1723) make the metrics name in new producer more standard

2015-01-08 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1723?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar Reddy updated KAFKA-1723: --- Attachment: KAFKA-1723_2015-01-08_22:02:22.patch > make the metrics name in new producer more

Review Request 29714: Patch for KAFKA-1810

2015-01-08 Thread Jeff Holoman
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29714/ --- Review request for kafka. Bugs: KAFKA-1810 https://issues.apache.org/jira/b

[jira] [Updated] (KAFKA-1810) Add IP Filtering / Whitelists-Blacklists

2015-01-08 Thread Jeff Holoman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1810?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jeff Holoman updated KAFKA-1810: Status: Patch Available (was: Open) > Add IP Filtering / Whitelists-Blacklists > -

[jira] [Commented] (KAFKA-1810) Add IP Filtering / Whitelists-Blacklists

2015-01-08 Thread Jeff Holoman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1810?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14269567#comment-14269567 ] Jeff Holoman commented on KAFKA-1810: - Created reviewboard https://reviews.apache.org/

[jira] [Updated] (KAFKA-1810) Add IP Filtering / Whitelists-Blacklists

2015-01-08 Thread Jeff Holoman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1810?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jeff Holoman updated KAFKA-1810: Attachment: KAFKA-1810.patch > Add IP Filtering / Whitelists-Blacklists > -

[jira] [Commented] (KAFKA-1810) Add IP Filtering / Whitelists-Blacklists

2015-01-08 Thread Jeff Holoman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1810?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14269579#comment-14269579 ] Jeff Holoman commented on KAFKA-1810: - This patch is a first pass at implementing IP F

[jira] [Commented] (KAFKA-1723) make the metrics name in new producer more standard

2015-01-08 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1723?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14269585#comment-14269585 ] Manikumar Reddy commented on KAFKA-1723: 1. Added MetricName object and related ch

unable to sign up for kafka email distribution

2015-01-08 Thread Padgett, Ben
Unable to confirm myself on kafka email list. Mail is getting kicked back from apache email servers according to our tech department. Thanks!

Re: unable to sign up for kafka email distribution

2015-01-08 Thread Guozhang Wang
Hi Ben, Your email is received on the mailing list, the kicking-back may be a false alarm. Guozhang On Thu, Jan 8, 2015 at 9:22 AM, Padgett, Ben wrote: > Unable to confirm myself on kafka email list. Mail is getting kicked back > from apache email servers according to our tech department. > >

[jira] [Created] (KAFKA-1848) Checking shutdown during each iteration of ZookeeperConsumerConnector

2015-01-08 Thread Aditya Auradkar (JIRA)
Aditya Auradkar created KAFKA-1848: -- Summary: Checking shutdown during each iteration of ZookeeperConsumerConnector Key: KAFKA-1848 URL: https://issues.apache.org/jira/browse/KAFKA-1848 Project: Kafk

[jira] [Commented] (KAFKA-1848) Checking shutdown during each iteration of ZookeeperConsumerConnector

2015-01-08 Thread Aditya Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1848?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14269841#comment-14269841 ] Aditya Auradkar commented on KAFKA-1848: I've asked Guozhang to assign the ticket

[jira] [Commented] (KAFKA-1848) Checking shutdown during each iteration of ZookeeperConsumerConnector

2015-01-08 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1848?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14269866#comment-14269866 ] Guozhang Wang commented on KAFKA-1848: -- Aditya, I have added you to the contributor l

Re: unable to sign up for kafka email distribution

2015-01-08 Thread Padgett, Ben
I haven’t gotten any emails from the email list however. When I joined the Apache Cassandra list I got very frequent emails. Was trying to subscribe at this email dev-subscr...@kafka.apache.org. A Thanks! From: Guozhang Wang mailto:wangg...@gmail.com>>

Re: Review Request 29714: Patch for KAFKA-1810

2015-01-08 Thread Jeff Holoman
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29714/ --- (Updated Jan. 8, 2015, 7:14 p.m.) Review request for kafka. Bugs: KAFKA-1810

[jira] [Updated] (KAFKA-1848) Checking shutdown during each iteration of ZookeeperConsumerConnector

2015-01-08 Thread Aditya Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1848?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aditya Auradkar updated KAFKA-1848: --- Assignee: Aditya Auradkar > Checking shutdown during each iteration of ZookeeperConsumerConnec

Review Request 29724: Patch for KAFKA-1566

2015-01-08 Thread Sriharsha Chintalapani
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29724/ --- Review request for kafka. Bugs: KAFKA-1566 https://issues.apache.org/jira/b

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

2015-01-08 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1566?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sriharsha Chintalapani updated KAFKA-1566: -- Attachment: KAFKA-1566.patch > Kafka environment configuration (kafka-env.sh) >

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

2015-01-08 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1566?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14270026#comment-14270026 ] Sriharsha Chintalapani commented on KAFKA-1566: --- Created reviewboard https:/

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

2015-01-08 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1566?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sriharsha Chintalapani updated KAFKA-1566: -- Assignee: Sriharsha Chintalapani (was: Cosmin Lehene) Status: Patch Availa

Re: Review Request 28769: Patch for KAFKA-1809

2015-01-08 Thread Gwen Shapira
> On Jan. 7, 2015, 3:01 a.m., Jun Rao wrote: > > Thanks for the patch. It's a lot work! A few general comments below, in > > addition to the more detailed comments. > > > > 1. Formatting: add a space after comma in function signature and function > > calls. So instead of > > def foo(a: A,

Re: Review Request 29523: Patch for KAFKA-1723

2015-01-08 Thread Jun Rao
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29523/#review67246 --- Thanks for the patch. Looks good overall. A few minor comments below

[jira] [Commented] (KAFKA-1723) make the metrics name in new producer more standard

2015-01-08 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1723?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14270066#comment-14270066 ] Jun Rao commented on KAFKA-1723: Thanks for the patch. 1. Yes, I think this is needed. 2

Re: Review Request 29523: Patch for KAFKA-1723

2015-01-08 Thread Jay Kreps
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29523/#review67309 --- clients/src/main/java/org/apache/kafka/common/metrics/MetricName.ja

Re: Review Request 29523: Patch for KAFKA-1723

2015-01-08 Thread Jay Kreps
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29523/#review67314 --- clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java <

[jira] [Commented] (KAFKA-1723) make the metrics name in new producer more standard

2015-01-08 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1723?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14270163#comment-14270163 ] Jay Kreps commented on KAFKA-1723: -- 2. I think you are saying that sensor names are still

Review Request 29728: Patch for KAFKA-1848

2015-01-08 Thread Aditya Auradkar
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29728/ --- Review request for kafka. Bugs: KAFKA-1848 https://issues.apache.org/jira/b

Re: Review Request 29728: Patch for KAFKA-1848

2015-01-08 Thread Aditya Auradkar
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29728/ --- (Updated Jan. 8, 2015, 10:49 p.m.) Review request for kafka. Bugs: KAFKA-1848

Re: Review Request 29692: Patch for kafka-1841

2015-01-08 Thread Jun Rao
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29692/ --- (Updated Jan. 8, 2015, 11:07 p.m.) Review request for kafka. Bugs: kafka-1841

[jira] [Updated] (KAFKA-1841) OffsetCommitRequest API - timestamp field is not versioned

2015-01-08 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1841?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1841: --- Attachment: kafka-1841_2015-01-08_15:07:57.patch > OffsetCommitRequest API - timestamp field is not versioned

[jira] [Commented] (KAFKA-1841) OffsetCommitRequest API - timestamp field is not versioned

2015-01-08 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1841?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14270187#comment-14270187 ] Jun Rao commented on KAFKA-1841: Updated reviewboard https://reviews.apache.org/r/29692/di

[jira] [Commented] (KAFKA-1841) OffsetCommitRequest API - timestamp field is not versioned

2015-01-08 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1841?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14270222#comment-14270222 ] Jun Rao commented on KAFKA-1841: This is actually a bit tricky to fix. To make this really

Review Request 29738: Patch for kafka-1797

2015-01-08 Thread Jun Rao
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29738/ --- Review request for kafka. Bugs: kafka-1797 https://issues.apache.org/jira/b

[jira] [Commented] (KAFKA-1797) add the serializer/deserializer api to the new java client

2015-01-08 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1797?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14270349#comment-14270349 ] Jun Rao commented on KAFKA-1797: Created reviewboard https://reviews.apache.org/r/29738/di

[jira] [Updated] (KAFKA-1797) add the serializer/deserializer api to the new java client

2015-01-08 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1797?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1797: --- Attachment: kafka-1797.patch > add the serializer/deserializer api to the new java client > --

Re: Review Request 29590: 1. Removed defaults for serializer/deserializer. 2. Converted type cast exception to serialization exception in the producer. 3. Added string ser/deser. 4. Moved the isKey fl

2015-01-08 Thread Jun Rao
> On Jan. 8, 2015, 12:07 p.m., Manikumar Reddy O wrote: > > clients/src/main/java/org/apache/kafka/common/serialization/StringSerializer.java, > > line 34 > > > > > > we should support null values right?. This is requi

[jira] [Created] (KAFKA-1849) Utils.readBytes() should support null ByteBuffer

2015-01-08 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-1849: -- Summary: Utils.readBytes() should support null ByteBuffer Key: KAFKA-1849 URL: https://issues.apache.org/jira/browse/KAFKA-1849 Project: Kafka Issue Type: Bug

Re: Review Request 28769: Patch for KAFKA-1809

2015-01-08 Thread Gwen Shapira
> On Jan. 7, 2015, 3:01 a.m., Jun Rao wrote: > > core/src/main/scala/kafka/cluster/Broker.scala, lines 36-52 > > > > > > It would be good to document the format for both v1 and v2. > > > > Also, would it be cl

[jira] [Commented] (KAFKA-1849) Utils.readBytes() should support null ByteBuffer

2015-01-08 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1849?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14270385#comment-14270385 ] Jay Kreps commented on KAFKA-1849: -- Not sure if that is the right place for it. I think U

[jira] [Created] (KAFKA-1850) Failed reassignment leads to additional replica

2015-01-08 Thread Alex Tian (JIRA)
Alex Tian created KAFKA-1850: Summary: Failed reassignment leads to additional replica Key: KAFKA-1850 URL: https://issues.apache.org/jira/browse/KAFKA-1850 Project: Kafka Issue Type: Bug

[jira] [Commented] (KAFKA-1850) Failed reassignment leads to additional replica

2015-01-08 Thread Alex Tian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1850?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14270430#comment-14270430 ] Alex Tian commented on KAFKA-1850: -- Maybe my reassignment plan is not the best choice: At

[jira] [Updated] (KAFKA-1850) Failed reassignment leads to additional replica

2015-01-08 Thread Alex Tian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1850?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alex Tian updated KAFKA-1850: - Description: When I start a topic reassignment (Totally 36 partitions) in my Kafka Cluster, 24 partitions

[jira] [Updated] (KAFKA-1850) Failed reassignment leads to additional replica

2015-01-08 Thread Alex Tian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1850?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alex Tian updated KAFKA-1850: - Description: When I start a topic reassignment (Totally 36 partitions) in my Kafka Cluster, 24 partitions

[jira] [Updated] (KAFKA-1850) Failed reassignment leads to additional replica

2015-01-08 Thread Alex Tian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1850?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alex Tian updated KAFKA-1850: - Description: When I start a topic reassignment (Totally 36 partitions) in my Kafka Cluster, 24 partitions

[jira] [Updated] (KAFKA-1850) Failed reassignment leads to additional replica

2015-01-08 Thread Alex Tian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1850?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alex Tian updated KAFKA-1850: - Description: When I start a topic reassignment (Totally 36 partitions) in my Kafka Cluster, 24 partitions

[jira] [Updated] (KAFKA-1850) Failed reassignment leads to additional replica

2015-01-08 Thread Alex Tian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1850?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alex Tian updated KAFKA-1850: - Description: When I start a topic reassignment (Totally 36 partitions) in my Kafka Cluster, 24 partitions

[jira] [Updated] (KAFKA-1850) Failed reassignment leads to additional replica

2015-01-08 Thread Alex Tian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1850?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alex Tian updated KAFKA-1850: - Description: When I start a topic reassignment (Totally 36 partitions) in my Kafka Cluster, 24 partitions

[jira] [Commented] (KAFKA-1849) Utils.readBytes() should support null ByteBuffer

2015-01-08 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1849?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14270447#comment-14270447 ] Jun Rao commented on KAFKA-1849: The issue is that MessageAndMetadata uses Utils.readBytes

[jira] [Commented] (KAFKA-1850) Failed reassignment leads to additional replica

2015-01-08 Thread Alex Tian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1850?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14270451#comment-14270451 ] Alex Tian commented on KAFKA-1850: -- Sorry again for too long change history here, maybe I

[jira] [Created] (KAFKA-1851) OffsetFetchRequest returns extra partitions when input only contains unknown partitions

2015-01-08 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-1851: -- Summary: OffsetFetchRequest returns extra partitions when input only contains unknown partitions Key: KAFKA-1851 URL: https://issues.apache.org/jira/browse/KAFKA-1851 Project: Ka

Review Request 29751: Patch for kafka-1851

2015-01-08 Thread Jun Rao
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29751/ --- Review request for kafka. Bugs: kafka-1851 https://issues.apache.org/jira/b

[jira] [Commented] (KAFKA-1851) OffsetFetchRequest returns extra partitions when input only contains unknown partitions

2015-01-08 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1851?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14270464#comment-14270464 ] Jun Rao commented on KAFKA-1851: Created reviewboard https://reviews.apache.org/r/29751/di

[jira] [Updated] (KAFKA-1851) OffsetFetchRequest returns extra partitions when input only contains unknown partitions

2015-01-08 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1851?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1851: --- Status: Patch Available (was: Open) > OffsetFetchRequest returns extra partitions when input only contains un

[jira] [Updated] (KAFKA-1851) OffsetFetchRequest returns extra partitions when input only contains unknown partitions

2015-01-08 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1851?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1851: --- Attachment: kafka-1851.patch > OffsetFetchRequest returns extra partitions when input only contains unknown >

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

2015-01-08 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-1852: -- Summary: OffsetCommitRequest can commit offset on unknown topic Key: KAFKA-1852 URL: https://issues.apache.org/jira/browse/KAFKA-1852 Project: Kafka Issue Type: Bug

[jira] [Resolved] (KAFKA-1849) Utils.readBytes() should support null ByteBuffer

2015-01-08 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1849?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-1849. Resolution: Not a Problem Ok, we actually handle null explicitly in MessageAndMetadata. So, this is not an

Review Request 29752: Patch for KAFKA-1836

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

[jira] [Updated] (KAFKA-1836) metadata.fetch.timeout.ms set to zero blocks forever

2015-01-08 Thread jaikiran pai (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1836?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] jaikiran pai updated KAFKA-1836: Attachment: KAFKA-1836-new-patch.patch > metadata.fetch.timeout.ms set to zero blocks forever >

[jira] [Commented] (KAFKA-1836) metadata.fetch.timeout.ms set to zero blocks forever

2015-01-08 Thread jaikiran pai (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1836?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14270525#comment-14270525 ] jaikiran pai commented on KAFKA-1836: - Thanks Ewen for your feedback. I've taken into

[DISCUSS] 0.8.2-beta2 release

2015-01-08 Thread Joe Stein
Hi, I was thinking that once all the blockers are committed for 0.8.2 that we should release beta2. Thoughts? /*** Joe Stein Founder, Principal Consultant Big Data Open Source Security LLC http://www.stealth.ly Twitter: @allthingshadoop

[jira] [Commented] (KAFKA-1850) Failed reassignment leads to additional replica

2015-01-08 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1850?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14270551#comment-14270551 ] Joe Stein commented on KAFKA-1850: -- An attachment will make it easier for folks to look a

Re: Review Request 29738: Patch for kafka-1797

2015-01-08 Thread Manikumar Reddy O
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29738/#review67374 --- Ship it! Ship It! - Manikumar Reddy O On Jan. 9, 2015, 1:22 a.m.

Re: Review Request 29724: Patch for KAFKA-1566

2015-01-08 Thread Manikumar Reddy O
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29724/#review67376 --- Ship it! Ship It! - Manikumar Reddy O On Jan. 8, 2015, 8:47 p.m.

[jira] [Created] (KAFKA-1853) Unsuccesful suffix rename attempt of LogSegment can leak files and also leave the LogSegment in an invalid state

2015-01-08 Thread jaikiran pai (JIRA)
jaikiran pai created KAFKA-1853: --- Summary: Unsuccesful suffix rename attempt of LogSegment can leak files and also leave the LogSegment in an invalid state Key: KAFKA-1853 URL: https://issues.apache.org/jira/browse/

[jira] [Commented] (KAFKA-1853) Unsuccesful suffix rename attempt of LogSegment can leak files and also leave the LogSegment in an invalid state

2015-01-08 Thread jaikiran pai (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1853?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14270627#comment-14270627 ] jaikiran pai commented on KAFKA-1853: - I've a patch for this which I'll submit in a wh

[jira] [Updated] (KAFKA-1853) Unsuccesful suffix rename attempt of LogSegment can leak files and also leave the LogSegment in an invalid state

2015-01-08 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1853?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joe Stein updated KAFKA-1853: - Fix Version/s: 0.8.3 > Unsuccesful suffix rename attempt of LogSegment can leak files and also leave > th

Review Request 29755: Patch for KAFKA-1853

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

[jira] [Updated] (KAFKA-1853) Unsuccesful suffix rename attempt of LogSegment can leak files and also leave the LogSegment in an invalid state

2015-01-08 Thread jaikiran pai (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1853?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] jaikiran pai updated KAFKA-1853: Status: Patch Available (was: Open) Created reviewboard https://reviews.apache.org/r/29755/diff aga

Re: Review Request 29755: Patch for KAFKA-1853

2015-01-08 Thread Jaikiran Pai
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29755/#review67382 --- This is my very first attempt at Scala, so feel free to let me know

Re: Follow-up On Important Issues for 0.8.2

2015-01-08 Thread Bhavesh Mistry
Adding User Community to see if any one knows behavior of Producer for issue #1) and status of 2). Thanks, Bhavesh On Fri, Jan 2, 2015 at 12:37 PM, Bhavesh Mistry wrote: > Hi Kafka Dev Team, > > I am following-up with you guys regarding New (Java) Producer behavior in > event of network or fi

[jira] [Updated] (KAFKA-1853) Unsuccessful suffix rename attempt of LogSegment can leak files and also leave the LogSegment in an invalid state

2015-01-08 Thread jaikiran pai (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1853?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] jaikiran pai updated KAFKA-1853: Summary: Unsuccessful suffix rename attempt of LogSegment can leak files and also leave the LogSegme

[jira] [Created] (KAFKA-1854) Allow the JIRA username and password to be prompted during patch submission

2015-01-08 Thread jaikiran pai (JIRA)
jaikiran pai created KAFKA-1854: --- Summary: Allow the JIRA username and password to be prompted during patch submission Key: KAFKA-1854 URL: https://issues.apache.org/jira/browse/KAFKA-1854 Project: Kafk

[jira] [Commented] (KAFKA-1854) Allow the JIRA username and password to be prompted during patch submission

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

[jira] [Updated] (KAFKA-1854) Allow the JIRA username and password to be prompted during patch submission

2015-01-08 Thread jaikiran pai (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1854?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] jaikiran pai updated KAFKA-1854: Attachment: KAFKA-1854.patch > Allow the JIRA username and password to be prompted during patch subm

Review Request 29756: Patch for KAFKA-1854

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

Re: Review Request 29756: Patch for KAFKA-1854

2015-01-08 Thread Jaikiran Pai
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29756/#review67388 --- This one needs a minor change which I'm going to submit as an update