Re: Review Request 31568: Patch for KAFKA-1989

2015-03-17 Thread Guozhang Wang
> On March 17, 2015, 8:56 p.m., Guozhang Wang wrote: > > core/src/main/scala/kafka/server/DelayedOperation.scala, line 116 > > > > > > We need to make tickMs and wheelSize configurable. > > Yasuhiro Matsuda wrote: >

[jira] [Commented] (KAFKA-1997) Refactor Mirror Maker

2015-03-17 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1997?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14366588#comment-14366588 ] Jiangjie Qin commented on KAFKA-1997: - Created reviewboard https://reviews.apache.org/

[jira] [Updated] (KAFKA-1997) Refactor Mirror Maker

2015-03-17 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1997?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiangjie Qin updated KAFKA-1997: Attachment: KAFKA-1997.patch > Refactor Mirror Maker > - > > Key

Review Request 32193: Patch for KAFKA-1997

2015-03-17 Thread Jiangjie Qin
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/32193/ --- Review request for kafka. Bugs: KAFKA-1997 https://issues.apache.org/jira/b

Re: [Discussion] Using Client Requests and Responses in Server

2015-03-17 Thread Neha Narkhede
> > How about keeping those server-side for now, since there's no duplication? > We can move them over in a follow-up jira. +1. The work involved in this refactoring is pretty sizable. I would be ok with splitting it in 2 JIRAs - one that converts the duplicated ones over first and another that c

[jira] [Commented] (KAFKA-1997) Refactor Mirror Maker

2015-03-17 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1997?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14366549#comment-14366549 ] Jiangjie Qin commented on KAFKA-1997: - [~xiaotao183] Yes, that's not right. I'll submi

[jira] [Comment Edited] (KAFKA-1997) Refactor Mirror Maker

2015-03-17 Thread TAO XIAO (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1997?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14366543#comment-14366543 ] TAO XIAO edited comment on KAFKA-1997 at 3/18/15 2:52 AM: -- [~beck

[jira] [Commented] (KAFKA-1997) Refactor Mirror Maker

2015-03-17 Thread TAO XIAO (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1997?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14366543#comment-14366543 ] TAO XIAO commented on KAFKA-1997: - [~becket_qin] I think I found a bug in MirrorMaker.scal

[jira] [Commented] (KAFKA-1688) Add authorization interface and naive implementation

2015-03-17 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1688?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14366490#comment-14366490 ] Gwen Shapira commented on KAFKA-1688: - Got it. That does make sense. > Add authorizat

Re: [Discussion] Using Client Requests and Responses in Server

2015-03-17 Thread Gwen Shapira
Never mind, it is implemented. Its just called LIST_OFFSETS. On Tue, Mar 17, 2015 at 6:27 PM, Gwen Shapira wrote: > Thanks Jiangjie! > > Another, more questionable missing-in-action: > > Client API had offset fetch request and offset commit requests. I > understand those :) > The Server API als

Re: [Discussion] Using Client Requests and Responses in Server

2015-03-17 Thread Gwen Shapira
Thanks Jiangjie! Another, more questionable missing-in-action: Client API had offset fetch request and offset commit requests. I understand those :) The Server API also has OffsetRequest, which returns offset corresponding to a specific time stamp. Its used by consumers. This sounds like somethi

Re: [Discussion] Using Client Requests and Responses in Server

2015-03-17 Thread Jiangjie Qin
I think those two requests are only used by controller to broker communication. Not sure if client side will need them in KIP-4, unlikely I guess. Jiangjie (Becket) Qin On 3/17/15, 6:08 PM, "Gwen Shapira" wrote: >Hi, > >I'm starting this thread for the various questions I run into while >refact

Re: [Discussion] Using Client Requests and Responses in Server

2015-03-17 Thread Gwen Shapira
Same for UPDATE_METADATA. Also, missing in client side. For good reasons :) How about keeping those server-side for now, since there's no duplication? We can move them over in a follow-up jira. Gwen On Tue, Mar 17, 2015 at 6:08 PM, Gwen Shapira wrote: > Hi, > > I'm starting this thread for the

[Discussion] Using Client Requests and Responses in Server

2015-03-17 Thread Gwen Shapira
Hi, I'm starting this thread for the various questions I run into while refactoring the server to use client requests and responses. Help is appreciated :) First question: LEADER_AND_ISR request and STOP_REPLICA request are unimplemented in the client. Do we want to implement them as part of th

Re: Review Request 31568: Patch for KAFKA-1989

2015-03-17 Thread Yasuhiro Matsuda
> On March 17, 2015, 8:56 p.m., Guozhang Wang wrote: > > core/src/main/scala/kafka/utils/timer/Timer.scala, line 68 > > > > > > I think bucket.flush(reinsurt) will always fail on all the items since > > their expiratio

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

2015-03-17 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1566?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14366377#comment-14366377 ] Sriharsha Chintalapani commented on KAFKA-1566: --- [~nehanarkhede] patch appli

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

2015-03-17 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_2015-03-17_17:19:23.patch > Kafka environment configurati

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

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

Re: Review Request 29724: Patch for KAFKA-1566

2015-03-17 Thread Sriharsha Chintalapani
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29724/ --- (Updated March 18, 2015, 12:19 a.m.) Review request for kafka. Bugs: KAFKA-15

[jira] [Commented] (KAFKA-2028) Unable to start the ZK instance after myid file was missing and had to recreate it.

2015-03-17 Thread InduR (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2028?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14366362#comment-14366362 ] InduR commented on KAFKA-2028: -- The Data and log files are also mapped to be on tmp; # A com

[jira] [Commented] (KAFKA-2028) Unable to start the ZK instance after myid file was missing and had to recreate it.

2015-03-17 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2028?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14366349#comment-14366349 ] Joe Stein commented on KAFKA-2028: -- the issue is like related to using the /tmp directory

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

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

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

2015-03-17 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_2015-03-17_17:01:38.patch > Kafka environment configurati

Re: Review Request 29724: Patch for KAFKA-1566

2015-03-17 Thread Sriharsha Chintalapani
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29724/ --- (Updated March 18, 2015, 12:01 a.m.) Review request for kafka. Bugs: KAFKA-15

Re: Broker is not aware of new partitions assigned

2015-03-17 Thread Allen Wang
I am not sure if Kafka version difference is the issue as later adding partitions for other topics works. Is there any compatibility issues on protocol level in 0.8.2.1 vs. 0.8.1.1? Also restarting the controller seems to fix the problem. On Tue, Mar 17, 2015 at 4:08 PM, Mayuresh Gharat wrote:

[jira] [Updated] (KAFKA-2028) Unable to start the ZK instance after myid file was missing and had to recreate it.

2015-03-17 Thread InduR (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2028?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] InduR updated KAFKA-2028: - Description: Created a Dev 3 node cluster environment in Jan and the environment has been up and running without

[jira] [Commented] (KAFKA-1461) Replica fetcher thread does not implement any back-off behavior

2015-03-17 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1461?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14366286#comment-14366286 ] Sriharsha Chintalapani commented on KAFKA-1461: --- [~guozhang] updated the PR

Re: Broker is not aware of new partitions assigned

2015-03-17 Thread Mayuresh Gharat
Probably you can try restarting the controller and have same version for the controller and the brokers. BTW, was there any specific reason you are running 2 different versions for the controller and other brokers? Thanks, Mayuresh On Tue, Mar 17, 2015 at 4:02 PM, Allen Wang wrote: > Yes, the

[jira] [Commented] (KAFKA-1461) Replica fetcher thread does not implement any back-off behavior

2015-03-17 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1461?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14366262#comment-14366262 ] Sriharsha Chintalapani commented on KAFKA-1461: --- Updated reviewboard https:/

[jira] [Updated] (KAFKA-1461) Replica fetcher thread does not implement any back-off behavior

2015-03-17 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1461?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sriharsha Chintalapani updated KAFKA-1461: -- Attachment: KAFKA-1461_2015-03-17_16:03:33.patch > Replica fetcher thread does n

Re: Broker is not aware of new partitions assigned

2015-03-17 Thread Allen Wang
Yes, the watcher is still alive. The log in the controller indicates that it observed the changes. On Tue, Mar 17, 2015 at 2:05 PM, Mayuresh Gharat wrote: > I think the way reassignment works is asynchronous. Changes are made to > zookeeper but those changes get reflected only when controller w

Re: Review Request 31366: Patch for KAFKA-1461

2015-03-17 Thread Sriharsha Chintalapani
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31366/ --- (Updated March 17, 2015, 11:03 p.m.) Review request for kafka. Bugs: KAFKA-14

[jira] [Commented] (KAFKA-2028) Unable to start the ZK instance after myid file was missing and had to recreate it.

2015-03-17 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2028?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14366250#comment-14366250 ] Joe Stein commented on KAFKA-2028: -- can you post your zookeeper.properties > Unable to s

Re: Review Request 30809: Patch for KAFKA-1888

2015-03-17 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/30809/#review76835 --- Also, I think we can move ValidatingProducer/Consumer and BoostrapCo

Re: Review Request 30809: Patch for KAFKA-1888

2015-03-17 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/30809/#review76103 --- There some coding convention suggestions that may not be comprehensi

[jira] [Updated] (KAFKA-2016) RollingBounceTest takes long

2015-03-17 Thread Ted Malaska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2016?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ted Malaska updated KAFKA-2016: --- Reviewer: Jun Rao Labels: newbie (was: ) Affects Version/s: 0.8.2.1

[jira] [Updated] (KAFKA-2016) RollingBounceTest takes long

2015-03-17 Thread Ted Malaska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2016?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ted Malaska updated KAFKA-2016: --- Attachment: KAFKA-2016-1.patch Modified unit test to not wait 5000 but 100 milliseconds between retrie

[jira] [Commented] (KAFKA-2016) RollingBounceTest takes long

2015-03-17 Thread Ted Malaska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2016?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14366174#comment-14366174 ] Ted Malaska commented on KAFKA-2016: Nope my bad that was my mistake. They all get se

Build failed in Jenkins: KafkaPreCommit #39

2015-03-17 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-1997; Follow-up to add the shutdown hook before starting the consumers; reviewed by Guozhang Wang -- [...truncated 2041 lines...] kafka.utils.SchedulerTest > testPerio

Re: Review Request 31967: Patch for KAFKA-1546

2015-03-17 Thread Aditya Auradkar
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31967/ --- (Updated March 17, 2015, 9:46 p.m.) Review request for kafka. Bugs: KAFKA-154

[jira] [Commented] (KAFKA-1546) Automate replica lag tuning

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

[jira] [Updated] (KAFKA-1546) Automate replica lag tuning

2015-03-17 Thread Aditya A Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1546?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aditya A Auradkar updated KAFKA-1546: - Attachment: KAFKA-1546_2015-03-17_14:46:10.patch > Automate replica lag tuning > -

[jira] [Commented] (KAFKA-1997) Refactor Mirror Maker

2015-03-17 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1997?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14366141#comment-14366141 ] Guozhang Wang commented on KAFKA-1997: -- Commit the follow-up patch to trunk. > Refac

[jira] [Commented] (KAFKA-2016) RollingBounceTest takes long

2015-03-17 Thread Ted Malaska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2016?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14366134#comment-14366134 ] Ted Malaska commented on KAFKA-2016: OK I got it. So controlled.shutdown.retry.backof

Re: Review Request 31706: Patch for KAFKA-1997

2015-03-17 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31706/#review76815 --- Ship it! Ship It! - Guozhang Wang On March 17, 2015, 8:47 p.m.,

[jira] [Issue Comment Deleted] (KAFKA-1860) File system errors are not detected unless Kafka tries to write

2015-03-17 Thread Mayuresh Gharat (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1860?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mayuresh Gharat updated KAFKA-1860: --- Comment: was deleted (was: We need to reproduce and test the fix on our kafka server mp) > Fi

Re: Broker is not aware of new partitions assigned

2015-03-17 Thread Mayuresh Gharat
I think the way reassignment works is asynchronous. Changes are made to zookeeper but those changes get reflected only when controller watcher fires for the respective zookeeper path. Is your watcher still alive? Thanks, Mayuresh On Tue, Mar 17, 2015 at 1:29 PM, Allen Wang wrote: > Looking a b

[jira] [Commented] (KAFKA-1860) File system errors are not detected unless Kafka tries to write

2015-03-17 Thread Mayuresh Gharat (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1860?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14366088#comment-14366088 ] Mayuresh Gharat commented on KAFKA-1860: We need to reproduce and test the fix on

[jira] [Closed] (KAFKA-2023) git clone kafka repository requires https

2015-03-17 Thread Anatoli Fomenko (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2023?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Anatoli Fomenko closed KAFKA-2023. -- Verified in the doc repo and on the site. Thank you. > git clone kafka repository requires https >

Re: Review Request 32172: Patch for KAFKA-1860

2015-03-17 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/32172/#review76809 --- LGTM, let's try to reproduce the issue and see if it works as expect

Re: Review Request 31568: Patch for KAFKA-1989

2015-03-17 Thread Sriharsha Chintalapani
> On March 17, 2015, 8:56 p.m., Guozhang Wang wrote: > > We can probably remove DelayedItem if it is not referenced by anyone any > > more. I am using DelayedItem for KAFKA-1461. - Sriharsha --- This is an automatically generated e-mai

[jira] [Updated] (KAFKA-1860) File system errors are not detected unless Kafka tries to write

2015-03-17 Thread Mayuresh Gharat (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1860?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mayuresh Gharat updated KAFKA-1860: --- Status: Patch Available (was: Open) > File system errors are not detected unless Kafka tries

[jira] [Commented] (KAFKA-1860) File system errors are not detected unless Kafka tries to write

2015-03-17 Thread Mayuresh Gharat (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1860?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14366069#comment-14366069 ] Mayuresh Gharat commented on KAFKA-1860: Created reviewboard https://reviews.apach

Re: Review Request 31568: Patch for KAFKA-1989

2015-03-17 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31568/#review76459 --- We can probably remove DelayedItem if it is not referenced by anyone

Review Request 32172: Patch for KAFKA-1860

2015-03-17 Thread Mayuresh Gharat
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/32172/ --- Review request for kafka. Bugs: KAFKA-1860 https://issues.apache.org/jira/b

[jira] [Updated] (KAFKA-1860) File system errors are not detected unless Kafka tries to write

2015-03-17 Thread Mayuresh Gharat (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1860?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mayuresh Gharat updated KAFKA-1860: --- Attachment: KAFKA-1860.patch > File system errors are not detected unless Kafka tries to write

Re: Review Request 31706: Patch for KAFKA-1997

2015-03-17 Thread Jiangjie Qin
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31706/ --- (Updated March 17, 2015, 8:47 p.m.) Review request for kafka. Bugs: KAFKA-199

[jira] [Updated] (KAFKA-1997) Refactor Mirror Maker

2015-03-17 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1997?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiangjie Qin updated KAFKA-1997: Attachment: KAFKA-1997_2015-03-17_13:47:01.patch > Refactor Mirror Maker > - > >

[jira] [Commented] (KAFKA-1997) Refactor Mirror Maker

2015-03-17 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1997?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14366052#comment-14366052 ] Jiangjie Qin commented on KAFKA-1997: - Updated reviewboard https://reviews.apache.org/

[jira] [Created] (KAFKA-2028) Unable to start the ZK instance after myid file was missing and had to recreate it.

2015-03-17 Thread InduR (JIRA)
InduR created KAFKA-2028: Summary: Unable to start the ZK instance after myid file was missing and had to recreate it. Key: KAFKA-2028 URL: https://issues.apache.org/jira/browse/KAFKA-2028 Project: Kafka

Re: Broker is not aware of new partitions assigned

2015-03-17 Thread Allen Wang
Looking a bit more into controller log, it seems that when the partition assignment is changed in ZooKeeper, the controller has quite a lot exceptions communicating with new brokers where the partitions are assigned. One thing to note is that the new brokers have Kafka version 0.8.2.1 and the contr

[jira] [Commented] (KAFKA-1688) Add authorization interface and naive implementation

2015-03-17 Thread Parth Brahmbhatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1688?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14366010#comment-14366010 ] Parth Brahmbhatt commented on KAFKA-1688: - I am looking for a common abstraction f

[jira] [Assigned] (KAFKA-1912) Create a simple request re-routing facility

2015-03-17 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1912?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sriharsha Chintalapani reassigned KAFKA-1912: - Assignee: Sriharsha Chintalapani > Create a simple request re-routing fac

[jira] [Assigned] (KAFKA-2007) update offsetrequest for more useful information we have on broker about partition

2015-03-17 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2007?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sriharsha Chintalapani reassigned KAFKA-2007: - Assignee: Sriharsha Chintalapani > update offsetrequest for more useful i

[jira] [Updated] (KAFKA-1912) Create a simple request re-routing facility

2015-03-17 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1912?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sriharsha Chintalapani updated KAFKA-1912: -- Assignee: (was: Sriharsha Chintalapani) > Create a simple request re-routing

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

2015-03-17 Thread Andrii Biletskyi
Guys, Thanks for a great discussion! Here are the actions points: 1. Q: Get rid of all scala requests objects, use java protocol definitions. A: Gwen kindly took that (KAFKA-1927). It's important to speed up review procedure there since this ticket blocks other important changes. 2.

[jira] [Commented] (KAFKA-1688) Add authorization interface and naive implementation

2015-03-17 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1688?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14365929#comment-14365929 ] Gwen Shapira commented on KAFKA-1688: - We are moving all Requests and Responses to Jav

[jira] [Commented] (KAFKA-2016) RollingBounceTest takes long

2015-03-17 Thread Ted Malaska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2016?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14365893#comment-14365893 ] Ted Malaska commented on KAFKA-2016: I would love to help out on this issue if that is

Re: Review Request 31967: Patch for KAFKA-1546

2015-03-17 Thread Aditya Auradkar
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31967/#review76781 --- core/src/main/scala/kafka/server/ReplicaManager.scala

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

2015-03-17 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1926?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-1926: Assignee: Tong Li > Replace kafka.utils.Utils with o.a.k.common.utils.Utils > --

[jira] [Commented] (KAFKA-1982) change kafka.examples.Producer to use the new java producer

2015-03-17 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1982?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14365881#comment-14365881 ] Gwen Shapira commented on KAFKA-1982: - Pinging [~junrao] I think its ready for commit

[jira] [Commented] (KAFKA-1961) Looks like its possible to delete _consumer_offsets topic

2015-03-17 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1961?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14365875#comment-14365875 ] Gwen Shapira commented on KAFKA-1961: - ping [~nehanarkhede] :) I think this is ready

[jira] [Commented] (KAFKA-1688) Add authorization interface and naive implementation

2015-03-17 Thread Parth Brahmbhatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1688?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14365860#comment-14365860 ] Parth Brahmbhatt commented on KAFKA-1688: - Another thing I wanted to bring up is h

[jira] [Created] (KAFKA-2027) kafka never notifies the zookeeper client when a partition moved with due to an auto-rebalance (when auto.leader.rebalance.enable=true)

2015-03-17 Thread Sampath Reddy Lambu (JIRA)
Sampath Reddy Lambu created KAFKA-2027: -- Summary: kafka never notifies the zookeeper client when a partition moved with due to an auto-rebalance (when auto.leader.rebalance.enable=true) Key: KAFKA-2027 URL:

[jira] [Updated] (KAFKA-2026) Logging of unused options always shows null for the value and is misleading if the option is used by serializers

2015-03-17 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2026?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-2026: - Description: This is a really simple issue. When AbstractConfig logs unused messag

[jira] [Created] (KAFKA-2026) Logging of unused options always shows null for the value and is misleading if the option is used by serializers

2015-03-17 Thread Ewen Cheslack-Postava (JIRA)
Ewen Cheslack-Postava created KAFKA-2026: Summary: Logging of unused options always shows null for the value and is misleading if the option is used by serializers Key: KAFKA-2026 URL: https://issues.apach

[jira] [Commented] (KAFKA-1912) Create a simple request re-routing facility

2015-03-17 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1912?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14365766#comment-14365766 ] Jay Kreps commented on KAFKA-1912: -- I think this is easier than I thought. We have an as

[jira] [Updated] (KAFKA-1912) Create a simple request re-routing facility

2015-03-17 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1912?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joe Stein updated KAFKA-1912: - Fix Version/s: 0.8.3 > Create a simple request re-routing facility > -

[jira] [Updated] (KAFKA-1927) Replace requests in kafka.api with requests in org.apache.kafka.common.requests

2015-03-17 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1927?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joe Stein updated KAFKA-1927: - Fix Version/s: 0.8.3 > Replace requests in kafka.api with requests in > org.apache.kafka.common.requests

[jira] [Assigned] (KAFKA-1927) Replace requests in kafka.api with requests in org.apache.kafka.common.requests

2015-03-17 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1927?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joe Stein reassigned KAFKA-1927: Assignee: Gwen Shapira > Replace requests in kafka.api with requests in > org.apache.kafka.common.

Broker is not aware of new partitions assigned

2015-03-17 Thread Allen Wang
Hello, I developed a tool to add partitions and assign new partitions to a set of brokers in one operation by utilizing the API AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(). It worked well in most cases. However, in one case, I found that the brokers are not aware of new partitions

Re: [DISCUSSION] KIP-15 close(timeout) for producer

2015-03-17 Thread Jiangjie Qin
Hi Jun, Yes, as Guozhang said, the main reason we set a flag is because close(0) is expected to be called by sender thread itself. If we want to maintain the semantic meaning of close(), one alternative is to have an abort() method does the same thing as close(0) except cleanup. And in close(timeo

Re: [KIP-DISCUSSION] KIP-13 Quotas

2015-03-17 Thread Steven Wu
Jay, let's say an app produces to 10 different topics. one of the topic is sent from a library. due to whatever condition/bug, this lib starts to send messages over the quota. if we go with the delayed response approach, it will cause the whole shared RecordAccumulator buffer to be filled up. that

Re: [KIP-DISCUSSION] KIP-13 Quotas

2015-03-17 Thread Jay Kreps
Hey Guozhang, Cool, I think we are mostly on the same page. I think I agree with what you are saying, the error code thing is basically a matter of taste. It would be possible to put all kinds of things as error codes and it would be possible to devise a scheme for clients to deal with it--for exa

Re: [KIP-DISCUSSION] KIP-13 Quotas

2015-03-17 Thread Jay Kreps
Hey Steven, It is true that hitting the quota will cause back-pressure on the producer. But the solution is simple, a producer that wants to avoid this should stay under its quota. In other words this is a contract between the cluster and the client, with each side having something to uphold. Quit

Re: [KIP-DISCUSSION] KIP-13 Quotas

2015-03-17 Thread Joel Koshy
Yes I think there are two policy issues but I don't think they are separate/mutually exclusive for the purposes of this discussion - the first being what should the broker do if quota is violated and second, what should it return to the client (error code or status code). (The separate discussion u

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

2015-03-17 Thread Andrii Biletskyi
Joel, You are right, I removed ClusterMetadata because we have partially what we need in TopicMetadata. Also, as Jay pointed out earlier, we would like to have "orthogonal" API, but at the same time we need to be backward compatible. But I like your idea and even have some other arguments for thi

Re: [KIP-DISCUSSION] KIP-13 Quotas

2015-03-17 Thread Steven Wu
Ewen, I see your point regarding the shared buffer. yes, a bad/slow broker could potentially consume up all buffer. On the other hand, I do like the batching behavior of shared RecordAccumulator buffer. On Tue, Mar 17, 2015 at 8:25 AM, Guozhang Wang wrote: > Ewen, > > 1. I think we are on the sa

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

2015-03-17 Thread Joel Koshy
I may be missing some context but hopefully this will also be covered today: I thought the earlier proposal where there was an explicit ClusterMetadata request was clearer and explicit. During the course of this thread I think the conclusion was that the main need was for controller information and

Re: [KIP-DISCUSSION] KIP-13 Quotas

2015-03-17 Thread Guozhang Wang
Ewen, 1. I think we are on the same page as per "malicious clients", that it should not be the target of either approach. I was just trying to separate the discussion from "what if user just keep retrying" and maybe I was not clear. 2. I was not advocating option A on the wiki, in my previous ema

Re: [VOTE] KIP-7 Security - IP Filtering

2015-03-17 Thread Jeff Holoman
bump On Tue, Mar 3, 2015 at 8:12 PM, Jeff Holoman wrote: > Guozhang, > > The way the patch is implemented, the check is done in the acceptor thread > accept() method of the Socket Server, just before connectionQuotas. > > Thanks > > Jeff > > On Tue, Mar 3, 2015 at 7:59 PM, Guozhang Wang wrote:

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

2015-03-17 Thread Andrii Biletskyi
Jun, 101. Okay, if you say that such use case is important. I also think using clientId for these purposes is fine - if we already have this field as part of all Wire protocol messages, why not use that. I will update KIP-4 page if nobody has other ideas (which may come up during the call today).

[jira] [Commented] (KAFKA-2025) In multi-consumer setup - explicit commit, commits on all partitions

2015-03-17 Thread Pradeep G (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2025?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14365011#comment-14365011 ] Pradeep G commented on KAFKA-2025: -- Workaround available at --> http://grokbase.com/t/ka

[jira] [Updated] (KAFKA-2025) In multi-consumer setup - explicit commit, commits on all partitions

2015-03-17 Thread Pradeep G (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2025?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Pradeep G updated KAFKA-2025: - Priority: Critical (was: Blocker) > In multi-consumer setup - explicit commit, commits on all partitions

[jira] [Updated] (KAFKA-2025) In multi-consumer setup - explicit commit, commits on all partitions

2015-03-17 Thread Pradeep G (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2025?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Pradeep G updated KAFKA-2025: - Description: In a setup where there are two consumers C1 & C2 belonging to consumer group CG, two partiti

[jira] [Created] (KAFKA-2025) In multi-consumer setup - explicit commit, commits on all partitions

2015-03-17 Thread Pradeep G (JIRA)
Pradeep G created KAFKA-2025: Summary: In multi-consumer setup - explicit commit, commits on all partitions Key: KAFKA-2025 URL: https://issues.apache.org/jira/browse/KAFKA-2025 Project: Kafka I

Re: [KIP-DISCUSSION] KIP-13 Quotas

2015-03-17 Thread Ewen Cheslack-Postava
Steven - that's a reasonable concern. I think I've mentioned the same sort of issue in the issues about the new producer's RecordAccumulator not timing out sends, e.g. in https://issues.apache.org/jira/browse/KAFKA-1788. The shared buffer causes problems if one broker isn't available for awhile sin