Re: [DISCUSS] Client-side Assignment for New Consumer

2015-08-11 Thread Ewen Cheslack-Postava
On Tue, Aug 11, 2015 at 11:29 PM, Jiangjie Qin wrote: > Ewen, > > Thanks for the explanation. > > For (1), I am more concerned about the failure case instead of normal case. > What if a consumer somehow was kick out of a group but is still consuming > and committing offsets? Does that mean the ne

Re: [DISCUSS] Client-side Assignment for New Consumer

2015-08-11 Thread Jiangjie Qin
Ewen, Thanks for the explanation. For (1), I am more concerned about the failure case instead of normal case. What if a consumer somehow was kick out of a group but is still consuming and committing offsets? Does that mean the new owner and old owner might potentially consuming from and committin

Re: [DISCUSS] Client-side Assignment for New Consumer

2015-08-11 Thread Ewen Cheslack-Postava
On Tue, Aug 11, 2015 at 10:15 PM, Jiangjie Qin wrote: > Hi Jason, > > Thanks for writing this up. It would be useful to generalize the group > concept. I have a few questions below. > > 1. In old consumer actually the partition assignment are done by consumers > themselves. We used zookeeper to g

Re: [DISCUSS] Client-side Assignment for New Consumer

2015-08-11 Thread Ewen Cheslack-Postava
On Tue, Aug 11, 2015 at 10:03 PM, Onur Karaman wrote: > Just to make the conversation a bit easier (I don't think we have really > established names for these modes yet), basically with the new > KafkaConsumer today there's: > - "external management", where the application figures out the group >

Re: KIP Meeting Notes 08/11/2015

2015-08-11 Thread Jiangjie Qin
Hey Guozhang, Will it be a little bit hard to keep the volunteer list up to date? Personally I would prefer to have a summery e-mail automatically sent to kafka-dev list every day for tickets with patches submitted in recent 7 days. The email can also include the reviewer for the ticket. And peopl

Re: Review Request 36548: Patch for KAFKA-2336

2015-08-11 Thread Jiangjie Qin
> On Aug. 11, 2015, 10:08 p.m., Gwen Shapira wrote: > > Ship It! > > Gwen Shapira wrote: > Jiangjie, I commited despite your concerns since this patch fixes a huge > potential issue. > > If you have an idea for an improved fix, we can tackle this in a follow > up. Thanks Gwen. I

Re: [DISCUSS] Client-side Assignment for New Consumer

2015-08-11 Thread Jiangjie Qin
Hi Jason, Thanks for writing this up. It would be useful to generalize the group concept. I have a few questions below. 1. In old consumer actually the partition assignment are done by consumers themselves. We used zookeeper to guarantee that a partition will only be consumed by one consumer thre

Re: [DISCUSS] Client-side Assignment for New Consumer

2015-08-11 Thread Onur Karaman
Just to make the conversation a bit easier (I don't think we have really established names for these modes yet), basically with the new KafkaConsumer today there's: - "external management", where the application figures out the group management and partition assignment externally - "kafka managemen

Re: Kafka Indentation

2015-08-11 Thread Neha Narkhede
Jay is in the mood for the mother of all bikeshedding exercises. Let me add the (in)famous build framework question to the mix. I think we should move to Maven :-P On Tue, Aug 11, 2015 at 9:31 PM, Jay Kreps wrote: > Ha ha, love that this thread is simultaneously an argument over code > whitespac

[jira] [Commented] (KAFKA-2367) Add Copycat runtime data API

2015-08-11 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2367?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14692903#comment-14692903 ] Ewen Cheslack-Postava commented on KAFKA-2367: -- I don't think it's useful on

[jira] [Commented] (KAFKA-2367) Add Copycat runtime data API

2015-08-11 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2367?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14692900#comment-14692900 ] James Cheng commented on KAFKA-2367: Ah, I think I understand. This would mostly be us

[jira] [Commented] (KAFKA-2367) Add Copycat runtime data API

2015-08-11 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2367?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14692893#comment-14692893 ] Ewen Cheslack-Postava commented on KAFKA-2367: -- [~wushujames] see the "Schema

Re: Kafka Indentation

2015-08-11 Thread Jay Kreps
Ha ha, love that this thread is simultaneously an argument over code whitespace AND language choice. Getting agreement here will be like the open source discussion equivalent of trying to simultaneously conquer both France and Russia. Anyone have preferences on text editors? I've always thought em

[jira] [Commented] (KAFKA-2367) Add Copycat runtime data API

2015-08-11 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2367?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14692887#comment-14692887 ] James Cheng commented on KAFKA-2367: [~ewencp], you mentioned schema migration support

[jira] [Commented] (KAFKA-2367) Add Copycat runtime data API

2015-08-11 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2367?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14692853#comment-14692853 ] Ewen Cheslack-Postava commented on KAFKA-2367: -- The runtime API should not af

Re: Copycat data API & serializers

2015-08-11 Thread Ewen Cheslack-Postava
Bumping this thread so hopefully more people see it. There is also some discussion in the corresponding JIRA: https://issues.apache.org/jira/browse/KAFKA-2367 Any feedback is useful, even if just to say you personally don't have any strong opinions on this topic. Thanks, -Ewen On Fri, Jul 31, 20

Re: Kafka Indentation

2015-08-11 Thread Grant Henke
+1 on not breaking blame -1 on 4 spaces for scala -1 on rewriting Kafka in Java +1 on upping our Scala game so I guess an accumulative of 0 for me ;) On Tue, Aug 11, 2015 at 7:37 PM, Ashish Singh wrote: > I am also a +1 on not breaking git blame. IDEs support language specific > settings in s

[jira] [Commented] (KAFKA-2398) Transient test failure for SocketServerTest - Socket closed.

2015-08-11 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2398?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14692693#comment-14692693 ] Jiangjie Qin commented on KAFKA-2398: - [~benstopford] do we have a duplicate ticket of

[jira] [Created] (KAFKA-2422) Allow copycat connector plugins to be aliased to simpler names

2015-08-11 Thread Ewen Cheslack-Postava (JIRA)
Ewen Cheslack-Postava created KAFKA-2422: Summary: Allow copycat connector plugins to be aliased to simpler names Key: KAFKA-2422 URL: https://issues.apache.org/jira/browse/KAFKA-2422 Project:

Re: KAFKA-2364 migrate docs from SVN to git

2015-08-11 Thread Gwen Shapira
Ah, there is already a JIRA in the title. Never mind :) On Tue, Aug 11, 2015 at 5:51 PM, Gwen Shapira wrote: > The vote opened 5 days ago. I believe we can conclude with 3 binding +1, 3 > non-binding +1 and no -1. > > Ismael, are you opening and JIRA and migrating? Or are we looking for a > volu

Re: KAFKA-2364 migrate docs from SVN to git

2015-08-11 Thread Gwen Shapira
The vote opened 5 days ago. I believe we can conclude with 3 binding +1, 3 non-binding +1 and no -1. Ismael, are you opening and JIRA and migrating? Or are we looking for a volunteer? On Tue, Aug 11, 2015 at 5:46 PM, Ashish Singh wrote: > +1 on same repo. > > On Tue, Aug 11, 2015 at 12:21 PM, E

Re: KAFKA-2364 migrate docs from SVN to git

2015-08-11 Thread Ashish Singh
+1 on same repo. On Tue, Aug 11, 2015 at 12:21 PM, Edward Ribeiro wrote: > +1. As soon as possible, please. :) > > On Sat, Aug 8, 2015 at 4:05 PM, Neha Narkhede wrote: > > > +1 on the same repo for code and website. It helps to keep both in sync. > > > > On Thu, Aug 6, 2015 at 1:52 PM, Grant He

Re: Review Request 33049: Patch for KAFKA-2084

2015-08-11 Thread Jun Rao
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33049/#review95035 --- core/src/main/scala/kafka/server/KafkaConfig.scala (line 418)

Re: Kafka Indentation

2015-08-11 Thread Ashish Singh
I am also a +1 on not breaking git blame. IDEs support language specific settings in same project. On Tue, Aug 11, 2015 at 5:29 PM, Gwen Shapira wrote: > +1 on not breaking git blame > > -1 on rewriting Kafka in Java > +1 on upping our Scala game (as Ismael pointed out) > > On Tue, Aug 11, 2015

Re: Review Request 33049: Patch for KAFKA-2084

2015-08-11 Thread Jun Rao
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33049/#review95033 --- Just a couple of comments below. Otherwise, LGTM. clients/src/main

Re: Kafka Indentation

2015-08-11 Thread Gwen Shapira
+1 on not breaking git blame -1 on rewriting Kafka in Java +1 on upping our Scala game (as Ismael pointed out) On Tue, Aug 11, 2015 at 5:23 PM, Jason Gustafson wrote: > Can the java code be indented without affecting the results of git blame? > If not, then I'd vote to leave it as it is. > > (A

Re: Kafka Indentation

2015-08-11 Thread Mayuresh Gharat
missed it. +1 on rewriting Kafka in Java. Thanks, Mayuresh On Tue, Aug 11, 2015 at 5:23 PM, Jason Gustafson wrote: > Can the java code be indented without affecting the results of git blame? > If not, then I'd vote to leave it as it is. > > (Also +1 on rewriting Kafka in Java) > > -Jason > > O

Re: Kafka Indentation

2015-08-11 Thread Mayuresh Gharat
+1 on consistency. Thanks, Mayuresh On Tue, Aug 11, 2015 at 5:15 PM, Aditya Auradkar < aaurad...@linkedin.com.invalid> wrote: > Bump. Anyone else have an opinion? > > Neha/Jay - You've made your thoughts clear. Any thoughts on how/if we make > any changes? > > Thanks, > Aditya > > > On Fri, Jul

Re: Kafka Indentation

2015-08-11 Thread Jason Gustafson
Can the java code be indented without affecting the results of git blame? If not, then I'd vote to leave it as it is. (Also +1 on rewriting Kafka in Java) -Jason On Tue, Aug 11, 2015 at 5:15 PM, Aditya Auradkar < aaurad...@linkedin.com.invalid> wrote: > Bump. Anyone else have an opinion? > > Ne

Re: Kafka Indentation

2015-08-11 Thread Aditya Auradkar
Bump. Anyone else have an opinion? Neha/Jay - You've made your thoughts clear. Any thoughts on how/if we make any changes? Thanks, Aditya On Fri, Jul 24, 2015 at 10:32 AM, Aditya Auradkar wrote: > I'm with Neha on this one. I don't have a strong preference on 2 vs 4 but > I do think that cons

[jira] [Commented] (KAFKA-1778) Create new re-elect controller admin function

2015-08-11 Thread Abhishek Nigam (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1778?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14692628#comment-14692628 ] Abhishek Nigam commented on KAFKA-1778: --- Thanks Guozhang, I will write it up in a ni

[jira] [Commented] (KAFKA-1387) Kafka getting stuck creating ephemeral node it has already created when two zookeeper sessions are established in a very short period of time

2015-08-11 Thread James Lent (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1387?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14692595#comment-14692595 ] James Lent commented on KAFKA-1387: --- It has been a while since I investigated this issue

Re: KIP Meeting Notes 08/11/2015

2015-08-11 Thread Guozhang Wang
Good question. I can personally think of pros and cons of having a volunteer list, most of them are pros but one con is that the list will never be comprehensive and in that sense sort of discouraging people to assign themselves as the reviewer. Without such a list, contributors would most likely

[jira] [Commented] (KAFKA-2410) Implement "Auto Topic Creation" client side and remove support from Broker side

2015-08-11 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2410?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14692400#comment-14692400 ] Grant Henke commented on KAFKA-2410: Great! Though I am concerned it overlaps with som

[jira] [Commented] (KAFKA-1778) Create new re-elect controller admin function

2015-08-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1778?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14692387#comment-14692387 ] Guozhang Wang commented on KAFKA-1778: -- Could you summarize your proposal on your 27/

[jira] [Commented] (KAFKA-2408) (new) system tests: ConsoleConsumerService occasionally fails to register consumed message

2015-08-11 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2408?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14692379#comment-14692379 ] ASF GitHub Bot commented on KAFKA-2408: --- Github user asfgit closed the pull request

[jira] [Resolved] (KAFKA-2408) (new) system tests: ConsoleConsumerService occasionally fails to register consumed message

2015-08-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2408?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-2408. -- Resolution: Fixed Issue resolved by pull request 123 [https://github.com/apache/kafka/pull/123]

[GitHub] kafka pull request: KAFKA-2408 ConsoleConsumerService direct log o...

2015-08-11 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/123 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabl

[jira] [Commented] (KAFKA-2367) Add Copycat runtime data API

2015-08-11 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2367?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14692375#comment-14692375 ] Gwen Shapira commented on KAFKA-2367: - I would prefer to use Avro as the internal Data

[jira] [Commented] (KAFKA-1507) Using GetOffsetShell against non-existent topic creates the topic unintentionally

2015-08-11 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1507?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14692373#comment-14692373 ] Sriharsha Chintalapani commented on KAFKA-1507: --- [~jkreps] Since there is in

[jira] [Commented] (KAFKA-1778) Create new re-elect controller admin function

2015-08-11 Thread Abhishek Nigam (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1778?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14692370#comment-14692370 ] Abhishek Nigam commented on KAFKA-1778: --- Hi Guozhang, I agree 100% with you. Can you

[jira] [Commented] (KAFKA-2410) Implement "Auto Topic Creation" client side and remove support from Broker side

2015-08-11 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2410?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14692368#comment-14692368 ] Sriharsha Chintalapani commented on KAFKA-2410: --- [~granthenke] This issue is

[jira] [Updated] (KAFKA-2336) Changing offsets.topic.num.partitions after the offset topic is created breaks consumer group partition assignment

2015-08-11 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2336?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-2336: Resolution: Fixed Fix Version/s: 0.8.3 Status: Resolved (was: Patch Available) +1

Re: Review Request 36548: Patch for KAFKA-2336

2015-08-11 Thread Gwen Shapira
> On Aug. 11, 2015, 10:08 p.m., Gwen Shapira wrote: > > Ship It! Jiangjie, I commited despite your concerns since this patch fixes a huge potential issue. If you have an idea for an improved fix, we can tackle this in a follow up. - Gwen

Re: Review Request 36548: Patch for KAFKA-2336

2015-08-11 Thread Gwen Shapira
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/36548/#review95012 --- Ship it! Ship It! - Gwen Shapira On Aug. 11, 2015, 3:37 p.m., Gr

[jira] [Assigned] (KAFKA-2363) ProducerSendTest.testCloseWithZeroTimeoutFromCallerThread Transient Failure

2015-08-11 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2363?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ben Stopford reassigned KAFKA-2363: --- Assignee: Ben Stopford > ProducerSendTest.testCloseWithZeroTimeoutFromCallerThread Transient

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

2015-08-11 Thread Parth Brahmbhatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1683?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14692317#comment-14692317 ] Parth Brahmbhatt commented on KAFKA-1683: - [~eugenstud] I believe this patch will

[jira] [Commented] (KAFKA-1778) Create new re-elect controller admin function

2015-08-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1778?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14692289#comment-14692289 ] Guozhang Wang commented on KAFKA-1778: -- Chiming in late here, I think we are actually

Re: Review Request 36858: Patch for KAFKA-2120

2015-08-11 Thread Mayuresh Gharat
> On Aug. 11, 2015, 8:49 p.m., Jason Gustafson wrote: > > clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java, > > line 302 > > > > > > Can we make this value greater than sessionTimeoutMs (w

[jira] [Commented] (KAFKA-2398) Transient test failure for SocketServerTest - Socket closed.

2015-08-11 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2398?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14687399#comment-14687399 ] Ben Stopford commented on KAFKA-2398: - [~ijuma] [~becket_qin] so can we close this Jir

Re: KIP Meeting Notes 08/11/2015

2015-08-11 Thread Grant Henke
> > 2. Encourage contributors to set the "reviewer" field when change JIRA > status to "patch available", and encourage volunteers assigning themselves > to "reviewers" for pending tickets. Is there somewhere that describes who to pick as a reviewer based on the patch? Would it be worth listing

Re: Review Request 36858: Patch for KAFKA-2120

2015-08-11 Thread Jason Gustafson
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/36858/#review94999 --- Ship it! LGTM (other than the minor issue below). As discussed on t

Re: Review Request 36858: Patch for KAFKA-2120

2015-08-11 Thread Jun Rao
> On Aug. 7, 2015, 12:36 a.m., Jun Rao wrote: > > clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java, > > line 223 > > > > > > Not sure if the test is needed. First, it seems th

[jira] [Commented] (KAFKA-2406) ISR propagation should be throttled to avoid overwhelming controller.

2015-08-11 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2406?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14682431#comment-14682431 ] Jiangjie Qin commented on KAFKA-2406: - [~junrao] [~ashishujjain] We discussed about KI

[DISCUSS] Client-side Assignment for New Consumer

2015-08-11 Thread Jason Gustafson
Hi Kafka Devs, One of the nagging issues in the current design of the new consumer has been the need to support a variety of assignment strategies. We've encountered this in particular in the design of copycat and the processing framework (KIP-28). From what I understand, Samza also has a number o

[jira] [Commented] (KAFKA-1387) Kafka getting stuck creating ephemeral node it has already created when two zookeeper sessions are established in a very short period of time

2015-08-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1387?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14682416#comment-14682416 ] Guozhang Wang commented on KAFKA-1387: -- [~fpj] Could you help taking a look at this i

[jira] [Updated] (KAFKA-2120) Add a request timeout to NetworkClient

2015-08-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2120?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2120: - Reviewer: Jason Gustafson [~hachikuji] assigning to you for reviews. Please feel free to re-assign

[jira] [Updated] (KAFKA-313) Add JSON/CSV output and looping options to ConsumerGroupCommand

2015-08-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-313?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-313: Status: In Progress (was: Patch Available) > Add JSON/CSV output and looping options to ConsumerGrou

Re: KAFKA-2364 migrate docs from SVN to git

2015-08-11 Thread Edward Ribeiro
+1. As soon as possible, please. :) On Sat, Aug 8, 2015 at 4:05 PM, Neha Narkhede wrote: > +1 on the same repo for code and website. It helps to keep both in sync. > > On Thu, Aug 6, 2015 at 1:52 PM, Grant Henke wrote: > > > +1 for the same repo. The closer docs can be to code the more accurate

Re: Review Request 37357: Upgrade LZ4 to version 1.3 to avoid crashing with IBM Java 7

2015-08-11 Thread Ismael Juma
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/37357/#review94979 --- Changes look good. It seems like there quite a few changes in the up

KIP Meeting Notes 08/11/2015

2015-08-11 Thread Guozhang Wang
First of all, WebEx seems working! And we will upload the recorded video later. Quick summary: KIP-26: RP-99 (https://github.com/apache/kafka/pull/99) pending for reviews. KIP-28: RP-130 (https://github.com/apache/kafka/pull/130) looking for feedbacks on: 1. API design (see o.k.a.stream.example

Re: Review Request 37357: Upgrade LZ4 to version 1.3 to avoid crashing with IBM Java 7

2015-08-11 Thread Rajini Sivaram
> On Aug. 11, 2015, 3:29 p.m., Ismael Juma wrote: > > clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockInputStream.java, > > line 177 > > > > > > Why not use `SafeUtils`? The implementation of `Unsa

[jira] [Updated] (KAFKA-2421) Upgrade LZ4 to version 1.3 to avoid crashing with IBM Java 7

2015-08-11 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2421?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram updated KAFKA-2421: -- Attachment: KAFKA-2421_2015-08-11_18:54:26.patch > Upgrade LZ4 to version 1.3 to avoid crashing

[jira] [Commented] (KAFKA-2421) Upgrade LZ4 to version 1.3 to avoid crashing with IBM Java 7

2015-08-11 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2421?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14682303#comment-14682303 ] Rajini Sivaram commented on KAFKA-2421: --- Updated reviewboard https://reviews.apache.

Re: Review Request 37357: Upgrade LZ4 to version 1.3 to avoid crashing with IBM Java 7

2015-08-11 Thread Rajini Sivaram
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/37357/ --- (Updated Aug. 11, 2015, 6:56 p.m.) Review request for kafka. Bugs: KAFKA-2421

[jira] [Commented] (KAFKA-1387) Kafka getting stuck creating ephemeral node it has already created when two zookeeper sessions are established in a very short period of time

2015-08-11 Thread Fedor Korotkiy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1387?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14682269#comment-14682269 ] Fedor Korotkiy commented on KAFKA-1387: --- Have you tried steps from issue description

[jira] [Commented] (KAFKA-1387) Kafka getting stuck creating ephemeral node it has already created when two zookeeper sessions are established in a very short period of time

2015-08-11 Thread Mayuresh Gharat (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1387?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14682263#comment-14682263 ] Mayuresh Gharat commented on KAFKA-1387: Can the person who uploaded the patch sub

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

2015-08-11 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1695?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-1695: Reviewer: Flavio Junqueira (was: Gwen Shapira) > Authenticate connection to Zookeeper > ---

[jira] [Commented] (KAFKA-1778) Create new re-elect controller admin function

2015-08-11 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1778?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14682249#comment-14682249 ] Gwen Shapira commented on KAFKA-1778: - Apparently I can't assign Reviewer if there is

[jira] [Updated] (KAFKA-2143) Replicas get ahead of leader and fail

2015-08-11 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2143?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-2143: Reviewer: Gwen Shapira > Replicas get ahead of leader and fail > ---

Re: Can someone review ticket 1778

2015-08-11 Thread Abhishek Nigam
Hi Guozhang, Can you please re-review KAFKA 1778 design. Just to provide background for this ticket. This was a sub-ticket of kafka admin commands KIP-4. The goal of this was to avoid cascading controller moves maybe during rolling broker bounce. The approaches discussed were as follows: a) Use a

[jira] [Updated] (KAFKA-2390) Seek() should take a callback.

2015-08-11 Thread Dong Lin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2390?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dong Lin updated KAFKA-2390: Status: Patch Available (was: Open) > Seek() should take a callback. > -- > >

Re: Typo on documentation

2015-08-11 Thread Edward Ribeiro
haha, no problem. :) Btw, ​I uploaded the patch yesterday. Cheers, Edward​ On Tue, Aug 11, 2015 at 2:53 PM, Guozhang Wang wrote: > You are right, I woke up from the future I guess :) > > On Mon, Aug 10, 2015 at 11:54 PM, Gwen Shapira wrote: > > > We can't create PRs for doc bugs because the do

[jira] [Commented] (KAFKA-1215) Rack-Aware replica assignment option

2015-08-11 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1215?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14682185#comment-14682185 ] ASF GitHub Bot commented on KAFKA-1215: --- GitHub user allenxwang opened a pull reques

[GitHub] kafka pull request: KAFKA-1215: Rack-Aware replica assignment opti...

2015-08-11 Thread allenxwang
GitHub user allenxwang opened a pull request: https://github.com/apache/kafka/pull/132 KAFKA-1215: Rack-Aware replica assignment option The PR tries to achieve the following: - Make rack-aware assignment and rack data structure optional as opposed to be part of the core dat

Re: Typo on documentation

2015-08-11 Thread Guozhang Wang
You are right, I woke up from the future I guess :) On Mon, Aug 10, 2015 at 11:54 PM, Gwen Shapira wrote: > We can't create PRs for doc bugs because the docs are (still) in SVN... > > On Mon, Aug 10, 2015 at 11:24 PM, Guozhang Wang > wrote: > > > Moving forward, I would suggest we just create t

Re: [DISCUSS] KIP-28 - Add a transform client for data processing

2015-08-11 Thread Guozhang Wang
Jiangjie, Thanks for the explanation, now I understands the scenario. It is one of the CEP in stream processing, in which I think the local state should be used for some sort of pattern matching. More concretely, let's say in this case we have a local state storing what have been observed. Then th

Re: [DISCUSS] KIP-28 - Add a transform client for data processing

2015-08-11 Thread Jiangjie Qin
Guozhang, By interleaved groups of message, I meant something like this: Say we have message 0,1,2,3, message 0 and 2 together completes a business logic, message 1 and 3 together completes a business logic. In that case, after user processed message 2, they cannot commit offsets because if they c

[jira] [Commented] (KAFKA-2336) Changing offsets.topic.num.partitions after the offset topic is created breaks consumer group partition assignment

2015-08-11 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2336?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14681958#comment-14681958 ] Grant Henke commented on KAFKA-2336: Updated reviewboard https://reviews.apache.org/r/

Re: Review Request 36548: Patch for KAFKA-2336

2015-08-11 Thread Grant Henke
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/36548/ --- (Updated Aug. 11, 2015, 3:37 p.m.) Review request for kafka. Bugs: KAFKA-2336

[jira] [Updated] (KAFKA-2336) Changing offsets.topic.num.partitions after the offset topic is created breaks consumer group partition assignment

2015-08-11 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2336?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Grant Henke updated KAFKA-2336: --- Attachment: KAFKA-2336_2015-08-11_10:37:41.patch > Changing offsets.topic.num.partitions after the off

Re: Review Request 37357: Upgrade LZ4 to version 1.3 to avoid crashing with IBM Java 7

2015-08-11 Thread Ismael Juma
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/37357/#review94916 --- clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockI

[GitHub] kafka pull request: MINOR: Fix hard coded strings in ProduceRespon...

2015-08-11 Thread granthenke
GitHub user granthenke opened a pull request: https://github.com/apache/kafka/pull/131 MINOR: Fix hard coded strings in ProduceResponse You can merge this pull request into a Git repository by running: $ git pull https://github.com/granthenke/kafka minor-string Alternatively

[jira] [Commented] (KAFKA-2421) Upgrade LZ4 to version 1.3 to avoid crashing with IBM Java 7

2015-08-11 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2421?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14681947#comment-14681947 ] Rajini Sivaram commented on KAFKA-2421: --- Attached patch upgrades LZ4 to version 1.3

[jira] [Updated] (KAFKA-2421) Upgrade LZ4 to version 1.3 to avoid crashing with IBM Java 7

2015-08-11 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2421?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram updated KAFKA-2421: -- Attachment: KAFKA-2421.patch > Upgrade LZ4 to version 1.3 to avoid crashing with IBM Java 7 > --

[jira] [Updated] (KAFKA-2421) Upgrade LZ4 to version 1.3 to avoid crashing with IBM Java 7

2015-08-11 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2421?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram updated KAFKA-2421: -- Status: Patch Available (was: Open) > Upgrade LZ4 to version 1.3 to avoid crashing with IBM Jav

[jira] [Commented] (KAFKA-2421) Upgrade LZ4 to version 1.3 to avoid crashing with IBM Java 7

2015-08-11 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2421?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14681931#comment-14681931 ] Rajini Sivaram commented on KAFKA-2421: --- Created reviewboard https://reviews.apache.

Review Request 37357: Upgrade LZ4 to version 1.3 to avoid crashing with IBM Java 7

2015-08-11 Thread Rajini Sivaram
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/37357/ --- Review request for kafka. Bugs: KAFKA-2421 https://issues.apache.org/jira/b

[jira] [Created] (KAFKA-2421) Upgrade LZ4 to version 1.3 to avoid crashing with IBM Java 7

2015-08-11 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-2421: - Summary: Upgrade LZ4 to version 1.3 to avoid crashing with IBM Java 7 Key: KAFKA-2421 URL: https://issues.apache.org/jira/browse/KAFKA-2421 Project: Kafka

[jira] [Commented] (KAFKA-2078) Getting Selector [WARN] Error in I/O with host java.io.EOFException

2015-08-11 Thread PC (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2078?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14681847#comment-14681847 ] PC commented on KAFKA-2078: --- Hi again, It just happened again. This time, pumped only 4 message

[jira] [Commented] (KAFKA-2078) Getting Selector [WARN] Error in I/O with host java.io.EOFException

2015-08-11 Thread PC (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2078?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14681680#comment-14681680 ] PC commented on KAFKA-2078: --- I can reproduce this bug though it appears to be a challenge to do

Re: [jira] [Commented] (KAFKA-1690) new java producer needs ssl support as a client

2015-08-11 Thread Rajini Sivaram
Harsha, The test is very timing sensitive and doesn't always go through a renegotiation. Here is the trace from a run that passed and a failed run (I added the logging to the end of SSLTransportLayer.handshake()). The successful run shows a single handshake at the start, the failed run that hangs

Re: [DISCUSS] KIP-27 - Conditional Publish

2015-08-11 Thread Ben Kirwin
This is a very nice summary of the consistency / correctness issues possible with a commit log. > (assuming it’s publishing asynchronously and in an open loop) It's perhaps already clear to folks here, but -- if you *don't* do that, and instead only send one batch of messages at a time and check