[jira] [Commented] (KAFKA-1654) Provide a way to override server configuration from command line

2014-11-06 Thread Jarek Jarcec Cecho (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1654?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14201718#comment-14201718 ] Jarek Jarcec Cecho commented on KAFKA-1654: --- I've added second version of the pa

Re: Review Request 26474: KAFKA-1654 Provide a way to override server configuration from command line

2014-11-06 Thread Jarek Cecho
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/26474/ --- (Updated Nov. 7, 2014, 7:14 a.m.) Review request for kafka and Neha Narkhede.

[jira] [Updated] (KAFKA-1654) Provide a way to override server configuration from command line

2014-11-06 Thread Jarek Jarcec Cecho (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1654?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jarek Jarcec Cecho updated KAFKA-1654: -- Attachment: KAFKA-1654.patch > Provide a way to override server configuration from comma

[jira] [Commented] (KAFKA-391) Producer request and response classes should use maps

2014-11-06 Thread Honghai Chen (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-391?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14201702#comment-14201702 ] Honghai Chen commented on KAFKA-391: Many thanks for you help. After debugging and test

[jira] [Updated] (KAFKA-1742) ControllerContext removeTopic does not correctly update state

2014-11-06 Thread Onur Karaman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1742?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Onur Karaman updated KAFKA-1742: Attachment: KAFKA-1742.patch > ControllerContext removeTopic does not correctly update state > -

Re: Announcing Confluent

2014-11-06 Thread pankaj ojha
Best of Luck..keep rocking... On Fri, Nov 7, 2014 at 1:30 AM, Joe Brown wrote: > Best of luck!!! > > J > > On 6 Nov 2014, at 18:28, Jay Kreps wrote: > > > Hey all, > > > > I’m happy to announce that Jun Rao, Neha Narkhede and I are creating a > > company around Kafka called Confluent. We ar

Re: Review Request 27634: Patch for KAFKA-1667

2014-11-06 Thread Gwen Shapira
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/27634/#review60188 --- Awesome. Its building successfully now. core/src/main/scala/kafka/

[jira] [Created] (KAFKA-1758) corrupt recovery file prevents startup

2014-11-06 Thread Jason Rosenberg (JIRA)
Jason Rosenberg created KAFKA-1758: -- Summary: corrupt recovery file prevents startup Key: KAFKA-1758 URL: https://issues.apache.org/jira/browse/KAFKA-1758 Project: Kafka Issue Type: Bug

Jenkins build is back to normal : Kafka-trunk #324

2014-11-06 Thread Apache Jenkins Server
See

Re: Review Request 27391: Fix KAFKA-1634: Add the global retentionTime (in milis) and disable the per-offset timestamp

2014-11-06 Thread Joel Koshy
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/27391/#review60285 --- clients/src/main/java/org/apache/kafka/common/requests/OffsetCommit

[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-11-06 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14201475#comment-14201475 ] Jun Rao commented on KAFKA-1282: Nicu, I was doing some manual testing of this feature. W

[jira] [Commented] (KAFKA-1756) never allow the replica fetch size to be less than the max message size

2014-11-06 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1756?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14201449#comment-14201449 ] Gwen Shapira commented on KAFKA-1756: - I wonder if it makes sense for the brokers to p

[jira] [Updated] (KAFKA-1747) TestcaseEnv improperly shares state between instances

2014-11-06 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1747?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Neha Narkhede updated KAFKA-1747: - Component/s: system tests > TestcaseEnv improperly shares state between instances > --

[jira] [Updated] (KAFKA-1746) System tests don't handle errors well

2014-11-06 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1746?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Neha Narkhede updated KAFKA-1746: - Resolution: Fixed Status: Resolved (was: Patch Available) Thanks for the patch! Pushed to

[jira] [Updated] (KAFKA-1747) TestcaseEnv improperly shares state between instances

2014-11-06 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1747?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Neha Narkhede updated KAFKA-1747: - Fix Version/s: 0.8.3 > TestcaseEnv improperly shares state between instances > ---

[jira] [Updated] (KAFKA-1746) System tests don't handle errors well

2014-11-06 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1746?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Neha Narkhede updated KAFKA-1746: - Fix Version/s: 0.8.3 > System tests don't handle errors well > ---

[jira] [Updated] (KAFKA-1746) System tests don't handle errors well

2014-11-06 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1746?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Neha Narkhede updated KAFKA-1746: - Component/s: system tests > System tests don't handle errors well > --

[jira] [Updated] (KAFKA-1747) TestcaseEnv improperly shares state between instances

2014-11-06 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1747?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Neha Narkhede updated KAFKA-1747: - Resolution: Fixed Status: Resolved (was: Patch Available) Thanks for the patch, Ewen! Pus

[jira] [Comment Edited] (KAFKA-1744) Fetch Response contains messages prior to the requested offset

2014-11-06 Thread Evan Huus (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1744?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14201425#comment-14201425 ] Evan Huus edited comment on KAFKA-1744 at 11/7/14 2:00 AM: --- [~ju

[jira] [Resolved] (KAFKA-1744) Fetch Response contains messages prior to the requested offset

2014-11-06 Thread Evan Huus (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1744?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Evan Huus resolved KAFKA-1744. -- Resolution: Fixed > Fetch Response contains messages prior to the requested offset > ---

[jira] [Commented] (KAFKA-1744) Fetch Response contains messages prior to the requested offset

2014-11-06 Thread Evan Huus (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1744?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14201425#comment-14201425 ] Evan Huus commented on KAFKA-1744: -- [~junrao] very nice stealth edit of the spec, thank y

[jira] [Commented] (KAFKA-1744) Fetch Response contains messages prior to the requested offset

2014-11-06 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1744?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14201424#comment-14201424 ] Jun Rao commented on KAFKA-1744: Evan, I added the following explanation to the wiki. "I

[jira] [Commented] (KAFKA-1695) Authenticate connection to Zookeeper

2014-11-06 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1695?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14201419#comment-14201419 ] Neha Narkhede commented on KAFKA-1695: -- bq. I think it makes sense to open separate J

[jira] [Commented] (KAFKA-1756) never allow the replica fetch size to be less than the max message size

2014-11-06 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1756?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14201404#comment-14201404 ] Jun Rao commented on KAFKA-1756: Joe, The fix for 1) is a bit hard to do. kafka-topics ju

[jira] [Commented] (KAFKA-1744) Fetch Response contains messages prior to the requested offset

2014-11-06 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1744?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14201394#comment-14201394 ] Neha Narkhede commented on KAFKA-1744: -- The broker sends data to the consumer using z

[jira] [Commented] (KAFKA-1695) Authenticate connection to Zookeeper

2014-11-06 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1695?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14201393#comment-14201393 ] Gwen Shapira commented on KAFKA-1695: - The pull request is in! I think it makes sense

[jira] [Resolved] (KAFKA-1741) consumer get always old messages

2014-11-06 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1741?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Neha Narkhede resolved KAFKA-1741. -- Resolution: Won't Fix > consumer get always old messages > > >

[jira] [Commented] (KAFKA-1741) consumer get always old messages

2014-11-06 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1741?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14201391#comment-14201391 ] Neha Narkhede commented on KAFKA-1741: -- It seems like the format of the OffsetCommitR

[jira] [Commented] (KAFKA-1744) Fetch Response contains messages prior to the requested offset

2014-11-06 Thread Evan Huus (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1744?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14201374#comment-14201374 ] Evan Huus commented on KAFKA-1744: -- [~nehanarkhede] this was discovered in the golang con

[jira] [Commented] (KAFKA-1744) Fetch Response contains messages prior to the requested offset

2014-11-06 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1744?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14201365#comment-14201365 ] Neha Narkhede commented on KAFKA-1744: -- [~eapache] I'm assuming that you are referrin

Re: Review Request 27535: Patch for KAFKA-1747

2014-11-06 Thread Neha Narkhede
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/27535/#review60279 --- Ship it! Ship It! - Neha Narkhede On Nov. 3, 2014, 7:46 p.m., Ew

[jira] [Updated] (KAFKA-1755) Log cleaner thread should not exit on errors

2014-11-06 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1755?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Neha Narkhede updated KAFKA-1755: - Labels: newbie++ (was: ) > Log cleaner thread should not exit on errors > ---

Re: Review Request 27534: Patch for KAFKA-1746

2014-11-06 Thread Neha Narkhede
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/27534/#review60276 --- Ship it! Ship It! - Neha Narkhede On Nov. 3, 2014, 7:46 p.m., Ew

[jira] [Updated] (KAFKA-1481) Stop using dashes AND underscores as separators in MBean names

2014-11-06 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1481?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joel Koshy updated KAFKA-1481: -- Attachment: alternateLayout2.png alternateLayout1.png diff-for-alternate-

[jira] [Commented] (KAFKA-1752) add --replace-broker option

2014-11-06 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1752?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14201245#comment-14201245 ] Neha Narkhede commented on KAFKA-1752: -- +1 on [~gwenshap]'s suggestion. > add --repl

[jira] [Commented] (KAFKA-1751) handle "broker not exists" scenario

2014-11-06 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1751?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14201243#comment-14201243 ] Neha Narkhede commented on KAFKA-1751: -- [~junrao] This is something I found while usi

[jira] [Updated] (KAFKA-1634) Improve semantics of timestamp in OffsetCommitRequests and update documentation

2014-11-06 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1634?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-1634: - Attachment: KAFKA-1634_2014-11-06_15:35:46.patch > Improve semantics of timestamp in OffsetCommitR

[jira] [Commented] (KAFKA-1634) Improve semantics of timestamp in OffsetCommitRequests and update documentation

2014-11-06 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1634?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14201211#comment-14201211 ] Guozhang Wang commented on KAFKA-1634: -- Updated reviewboard https://reviews.apache.or

Re: Review Request 27391: Fix KAFKA-1634: Add the global retentionTime (in milis) and disable the per-offset timestamp

2014-11-06 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/27391/ --- (Updated Nov. 6, 2014, 11:35 p.m.) Review request for kafka. Bugs: KAFKA-1634

[jira] [Updated] (KAFKA-1753) add --decommission-broker option

2014-11-06 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1753?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Neha Narkhede updated KAFKA-1753: - Reviewer: Neha Narkhede > add --decommission-broker option > > >

[jira] [Commented] (KAFKA-1753) add --decommission-broker option

2014-11-06 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1753?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14201162#comment-14201162 ] Neha Narkhede commented on KAFKA-1753: -- +1 on decommission-broker > add --decommissi

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

2014-11-06 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1723?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14201138#comment-14201138 ] Jun Rao commented on KAFKA-1723: [~otis], let me explain that a bit more. Historically, we

[jira] [Commented] (KAFKA-1654) Provide a way to override server configuration from command line

2014-11-06 Thread Jarek Jarcec Cecho (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1654?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14201096#comment-14201096 ] Jarek Jarcec Cecho commented on KAFKA-1654: --- Looking at feedback proposal more c

[jira] [Comment Edited] (KAFKA-1654) Provide a way to override server configuration from command line

2014-11-06 Thread Jarek Jarcec Cecho (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1654?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14201096#comment-14201096 ] Jarek Jarcec Cecho edited comment on KAFKA-1654 at 11/6/14 10:42 PM: ---

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

2014-11-06 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1757?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Neha Narkhede updated KAFKA-1757: - Reviewer: Jay Kreps Assignee: (was: Jay Kreps) > Can not delete Topic index on Windows > -

Re: Announcing Confluent

2014-11-06 Thread Joe Brown
Best of luck!!! J On 6 Nov 2014, at 18:28, Jay Kreps wrote: > Hey all, > > I’m happy to announce that Jun Rao, Neha Narkhede and I are creating a > company around Kafka called Confluent. We are planning on productizing the > kind of Kafka-based real-time data platform we built out at LinkedIn.

Re: Announcing Confluent

2014-11-06 Thread Steve Morin
Jay, Neha and Jun congratz!! On Thu, Nov 6, 2014 at 11:09 AM, Bhavesh Mistry wrote: > HI Guys, > > Thanks for your awesome support. I wish you good luck !! Thanks for open > sources Kafka !! > > Thanks, > > Bhavesh > > On Thu, Nov 6, 2014 at 10:52 AM, Rajasekar Elango > wrote: > > > Congrats

[jira] [Issue Comment Deleted] (KAFKA-1754) KOYA - Kafka on YARN

2014-11-06 Thread Thomas Weise (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1754?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Thomas Weise updated KAFKA-1754: Comment: was deleted (was: [~acmurthy] Thanks for the support. Disk reservation will benefit Kafka g

[jira] [Commented] (KAFKA-1738) Partitions for topic not created after restart from forced shutdown

2014-11-06 Thread schandr (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1738?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14200748#comment-14200748 ] schandr commented on KAFKA-1738: Will apply the patch against the 0.8.2-beta and post the

[jira] [Commented] (KAFKA-1738) Partitions for topic not created after restart from forced shutdown

2014-11-06 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1738?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14200708#comment-14200708 ] Neha Narkhede commented on KAFKA-1738: -- Good catch. Thanks for following up on this,

[jira] [Updated] (KAFKA-1738) Partitions for topic not created after restart from forced shutdown

2014-11-06 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1738?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Neha Narkhede updated KAFKA-1738: - Reviewer: Neha Narkhede > Partitions for topic not created after restart from forced shutdown > --

Re: Review Request 27690: Patch for kafka-1738

2014-11-06 Thread Neha Narkhede
> On Nov. 6, 2014, 7:20 p.m., Neha Narkhede wrote: > > Ship It! Minor nit: Could you change fails to failed? - Neha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/27690/#review60215

Re: Review Request 27690: Patch for kafka-1738

2014-11-06 Thread Neha Narkhede
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/27690/#review60215 --- Ship it! Ship It! - Neha Narkhede On Nov. 6, 2014, 5:34 p.m., Ju

Re: Announcing Confluent

2014-11-06 Thread Bhavesh Mistry
HI Guys, Thanks for your awesome support. I wish you good luck !! Thanks for open sources Kafka !! Thanks, Bhavesh On Thu, Nov 6, 2014 at 10:52 AM, Rajasekar Elango wrote: > Congrats. Wish you all the very best and success. > > Thanks, > Raja. > > On Thu, Nov 6, 2014 at 1:36 PM, Niek Sande

Re: No longer supporting Java 6, if? when?

2014-11-06 Thread Jay Kreps
Yeah it is a little bit silly that people are still using Java 6. I guess this is a tradeoff--being more conservative in our java support means more people can use our software, whereas upgrading gives us developers a better experience since we aren't stuck with ancient stuff. Nonetheless I would

Re: No longer supporting Java 6, if? when?

2014-11-06 Thread Gwen Shapira
Java6 is supported on CDH4 but not CDH5. On Thu, Nov 6, 2014 at 9:54 AM, Koert Kuipers wrote: > when is java 6 dropped by the hadoop distros? > > i am still aware of many clusters that are java 6 only at the moment. > > > > On Thu, Nov 6, 2014 at 12:44 PM, Gwen Shapira > wrote: > > > +1 for dro

[jira] [Commented] (KAFKA-1738) Partitions for topic not created after restart from forced shutdown

2014-11-06 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1738?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14200650#comment-14200650 ] Jun Rao commented on KAFKA-1738: You can follow the instruction at https://cwiki.apache.o

Re: No longer supporting Java 6, if? when?

2014-11-06 Thread Koert Kuipers
when is java 6 dropped by the hadoop distros? i am still aware of many clusters that are java 6 only at the moment. On Thu, Nov 6, 2014 at 12:44 PM, Gwen Shapira wrote: > +1 for dropping Java 6 > > On Thu, Nov 6, 2014 at 9:31 AM, Steven Schlansker < > sschlans...@opentable.com > > wrote: > >

Re: Announcing Confluent

2014-11-06 Thread Rajasekar Elango
Congrats. Wish you all the very best and success. Thanks, Raja. On Thu, Nov 6, 2014 at 1:36 PM, Niek Sanders wrote: > Congrats! > > On Thu, Nov 6, 2014 at 10:28 AM, Jay Kreps wrote: > > Hey all, > > > > I’m happy to announce that Jun Rao, Neha Narkhede and I are creating a > > company around K

Re: Announcing Confluent

2014-11-06 Thread Niek Sanders
Congrats! On Thu, Nov 6, 2014 at 10:28 AM, Jay Kreps wrote: > Hey all, > > I’m happy to announce that Jun Rao, Neha Narkhede and I are creating a > company around Kafka called Confluent. We are planning on productizing the > kind of Kafka-based real-time data platform we built out at LinkedIn. We

[jira] [Commented] (KAFKA-1738) Partitions for topic not created after restart from forced shutdown

2014-11-06 Thread schandr (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1738?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14200603#comment-14200603 ] schandr commented on KAFKA-1738: GreatAnd thank you for the Patch. How should we apply

Announcing Confluent

2014-11-06 Thread Jay Kreps
Hey all, I’m happy to announce that Jun Rao, Neha Narkhede and I are creating a company around Kafka called Confluent. We are planning on productizing the kind of Kafka-based real-time data platform we built out at LinkedIn. We are doing this because we think this is a really powerful idea and we

[jira] [Commented] (KAFKA-1476) Get a list of consumer groups

2014-11-06 Thread BalajiSeshadri (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1476?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14200575#comment-14200575 ] BalajiSeshadri commented on KAFKA-1476: --- Created review board for just this JIRA. [

[jira] [Commented] (KAFKA-1476) Get a list of consumer groups

2014-11-06 Thread BalajiSeshadri (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1476?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14200570#comment-14200570 ] BalajiSeshadri commented on KAFKA-1476: --- Created reviewboard https://reviews.apache.

[jira] [Updated] (KAFKA-1476) Get a list of consumer groups

2014-11-06 Thread BalajiSeshadri (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1476?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] BalajiSeshadri updated KAFKA-1476: -- Attachment: KAFKA-1476.patch > Get a list of consumer groups > - > >

Review Request 27693: Patch for KAFKA-1476

2014-11-06 Thread Balaji Seshadri
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/27693/ --- Review request for kafka. Bugs: KAFKA-1476 https://issues.apache.org/jira/b

[jira] [Commented] (KAFKA-1476) Get a list of consumer groups

2014-11-06 Thread BalajiSeshadri (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1476?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14200556#comment-14200556 ] BalajiSeshadri commented on KAFKA-1476: --- [~nehanarkhede] or [~junrao] or [~jkreps] C

[jira] [Updated] (KAFKA-1476) Get a list of consumer groups

2014-11-06 Thread BalajiSeshadri (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1476?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] BalajiSeshadri updated KAFKA-1476: -- Status: Patch Available (was: In Progress) > Get a list of consumer groups > --

[jira] [Commented] (KAFKA-1476) Get a list of consumer groups

2014-11-06 Thread BalajiSeshadri (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1476?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14200550#comment-14200550 ] BalajiSeshadri commented on KAFKA-1476: --- Created reviewboard https://reviews.apache.

[jira] [Updated] (KAFKA-1476) Get a list of consumer groups

2014-11-06 Thread BalajiSeshadri (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1476?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] BalajiSeshadri updated KAFKA-1476: -- Attachment: KAFKA-1476.patch > Get a list of consumer groups > - > >

Review Request 27691: Patch for KAFKA-1476

2014-11-06 Thread Balaji Seshadri
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/27691/ --- Review request for kafka. Bugs: KAFKA-1476 https://issues.apache.org/jira/b

Re: No longer supporting Java 6, if? when?

2014-11-06 Thread Gwen Shapira
+1 for dropping Java 6 On Thu, Nov 6, 2014 at 9:31 AM, Steven Schlansker wrote: > Java 6 has been End of Life since Feb 2013. > Java 7 (and 8, but unfortunately that's too new still) has very compelling > features which can make development a lot easier. > > The sooner more projects drop Java 6

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

2014-11-06 Thread Otis Gospodnetic (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1723?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14200536#comment-14200536 ] Otis Gospodnetic commented on KAFKA-1723: - I don't follow this 100%, [~junrao]. T

Re: No longer supporting Java 6, if? when?

2014-11-06 Thread Steven Schlansker
Java 6 has been End of Life since Feb 2013. Java 7 (and 8, but unfortunately that's too new still) has very compelling features which can make development a lot easier. The sooner more projects drop Java 6 the better, in my opinion :) On Nov 5, 2014, at 7:45 PM, Worthy LaFollette wrote: > Mostl

[jira] [Commented] (KAFKA-1738) Partitions for topic not created after restart from forced shutdown

2014-11-06 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1738?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14200522#comment-14200522 ] Jun Rao commented on KAFKA-1738: Sri, Pradeep, Do you think you can try the patch and see

[jira] [Updated] (KAFKA-1738) Partitions for topic not created after restart from forced shutdown

2014-11-06 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1738?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1738: --- Priority: Blocker (was: Major) Affects Version/s: (was: 0.8.1.1) Fix Version/s: 0.8.2

[jira] [Commented] (KAFKA-1738) Partitions for topic not created after restart from forced shutdown

2014-11-06 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1738?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14200512#comment-14200512 ] Jun Rao commented on KAFKA-1738: Created reviewboard https://reviews.apache.org/r/27690/di

[jira] [Updated] (KAFKA-1738) Partitions for topic not created after restart from forced shutdown

2014-11-06 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1738?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1738: --- Assignee: Jun Rao Status: Patch Available (was: Open) > Partitions for topic not created after restart

[jira] [Updated] (KAFKA-1738) Partitions for topic not created after restart from forced shutdown

2014-11-06 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1738?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1738: --- Attachment: kafka-1738.patch > Partitions for topic not created after restart from forced shutdown > -

Review Request 27690: Patch for kafka-1738

2014-11-06 Thread Jun Rao
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/27690/ --- Review request for kafka. Bugs: kafka-1738 https://issues.apache.org/jira/b

[jira] [Commented] (KAFKA-1738) Partitions for topic not created after restart from forced shutdown

2014-11-06 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1738?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14200510#comment-14200510 ] Jun Rao commented on KAFKA-1738: You actually found a real bug, thanks! We exposed an exis

[jira] [Updated] (KAFKA-1743) ConsumerConnector.commitOffsets in 0.8.2 is not backward compatible

2014-11-06 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1743?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar Reddy updated KAFKA-1743: --- Assignee: Manikumar Reddy Status: Patch Available (was: Open) > ConsumerConnector.commi

[jira] [Updated] (KAFKA-1743) ConsumerConnector.commitOffsets in 0.8.2 is not backward compatible

2014-11-06 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1743?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar Reddy updated KAFKA-1743: --- Attachment: KAFKA-1743.patch > ConsumerConnector.commitOffsets in 0.8.2 is not backward compat

[jira] [Commented] (KAFKA-1743) ConsumerConnector.commitOffsets in 0.8.2 is not backward compatible

2014-11-06 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1743?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14200478#comment-14200478 ] Manikumar Reddy commented on KAFKA-1743: Created reviewboard https://reviews.apach

Review Request 27684: Patch for KAFKA-1743

2014-11-06 Thread Manikumar Reddy O
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/27684/ --- Review request for kafka. Bugs: KAFKA-1743 https://issues.apache.org/jira/b

[jira] [Commented] (KAFKA-1754) KOYA - Kafka on YARN

2014-11-06 Thread Thomas Weise (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1754?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14200457#comment-14200457 ] Thomas Weise commented on KAFKA-1754: - [~acmurthy] Thanks for the support. Disk reserv

[jira] [Commented] (KAFKA-1754) KOYA - Kafka on YARN

2014-11-06 Thread Thomas Weise (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1754?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14200455#comment-14200455 ] Thomas Weise commented on KAFKA-1754: - [~acmurthy] Thanks for the support. Disk reserv

[jira] [Commented] (KAFKA-1754) KOYA - Kafka on YARN

2014-11-06 Thread Thomas Weise (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1754?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14200409#comment-14200409 ] Thomas Weise commented on KAFKA-1754: - [~ste...@apache.org] Yes, we are running DataTo

[jira] [Commented] (KAFKA-1667) topic-level configuration not validated

2014-11-06 Thread Dmytro Kostiuchenko (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1667?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14200381#comment-14200381 ] Dmytro Kostiuchenko commented on KAFKA-1667: The main issue with the old patch

Re: Review Request 27634: Patch for KAFKA-1667

2014-11-06 Thread Dmytro Kostiuchenko
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/27634/ --- (Updated Nov. 6, 2014, 4:12 p.m.) Review request for kafka. Bugs: KAFKA-1667

[jira] [Commented] (KAFKA-1667) topic-level configuration not validated

2014-11-06 Thread Dmytro Kostiuchenko (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1667?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14200370#comment-14200370 ] Dmytro Kostiuchenko commented on KAFKA-1667: Updated reviewboard https://revie

Re: Review Request 27634: Patch for KAFKA-1667

2014-11-06 Thread Dmytro Kostiuchenko
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/27634/ --- (Updated Nov. 6, 2014, 4:10 p.m.) Review request for kafka. Bugs: KAFKA-1667

[jira] [Updated] (KAFKA-1667) topic-level configuration not validated

2014-11-06 Thread Dmytro Kostiuchenko (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1667?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dmytro Kostiuchenko updated KAFKA-1667: --- Attachment: KAFKA-1667_2014-11-06_17:10:14.patch > topic-level configuration not vali

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

2014-11-06 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-1757?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lukáš Vyhlídka updated KAFKA-1757: -- Status: Patch Available (was: Open) > Can not delete Topic index on Windows > -

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

2014-11-06 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-1757?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lukáš Vyhlídka updated KAFKA-1757: -- Attachment: lucky-v.patch A patch that looks like a fix for the issue. > Can not delete Topic i

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

2014-11-06 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-1757?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lukáš Vyhlídka updated KAFKA-1757: -- Status: Open (was: Patch Available) > Can not delete Topic index on Windows > -

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

2014-11-06 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-1757?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lukáš Vyhlídka updated KAFKA-1757: -- Fix Version/s: 0.8.2 Status: Patch Available (was: Open) > Can not delete Topic inde

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

2014-11-06 Thread JIRA
Lukáš Vyhlídka created KAFKA-1757: - Summary: Can not delete Topic index on Windows Key: KAFKA-1757 URL: https://issues.apache.org/jira/browse/KAFKA-1757 Project: Kafka Issue Type: Bug

[jira] [Comment Edited] (KAFKA-1745) Each new thread creates a PIPE and KQUEUE as open files during producer.send() and does not get cleared when the thread that creates them is cleared.

2014-11-06 Thread Vishal (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1745?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14199817#comment-14199817 ] Vishal edited comment on KAFKA-1745 at 11/6/14 12:03 PM: - [~junrao

[jira] [Comment Edited] (KAFKA-1745) Each new thread creates a PIPE and KQUEUE as open files during producer.send() and does not get cleared when the thread that creates them is cleared.

2014-11-06 Thread Vishal (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1745?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14199817#comment-14199817 ] Vishal edited comment on KAFKA-1745 at 11/6/14 12:04 PM: - [~junrao

  1   2   >