Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-08-27 Thread Lucas Wang
Thanks for the comments, Joel. I addressed all but the last one, where Jun also shared a comment in the Vote thread to change it to "controller.listener.name". I actually feel CONTROLLER is better since it's a well defined concept in Kafka, while it's easier to confuse people with CONTROL since in

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-08-24 Thread Joel Koshy
I had some offline discussions with Lucas on this KIP. While it is much more work than the original proposals, separating the control plane entirely removes any interference with the data plane as summarized under the rejected alternatives section. Just a few minor comments: - Can you update t

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-08-22 Thread Eno Thereska
Ok thanks, if you guys are seeing this at LinkedIn then the motivation makes more sense. Eno On Tue, Aug 21, 2018 at 5:39 PM, Becket Qin wrote: > Hi Eno, > > Thanks for the comments. This KIP is not really about improving the > performance in general. It is about ensuring the cluster state can

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-08-21 Thread Lucas Wang
Hi Eno, I fully agree with Becket here. If the motivation section makes sense, and we know we can get burnt by this problem, then the exact numbers (which vary case by case according to the config settings and traffic pattern) are no longer as important. Thanks, Lucas On Tue, Aug 21, 2018 at 9:

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-08-21 Thread Lucas Wang
Thanks Becket. Following the convention of KIP-103 makes sense. I've updated the KIP with your proposed changes. Please take another look. Lucas On Mon, Aug 20, 2018 at 7:29 AM Becket Qin wrote: > Hi Lucas, > > In KIP-103, we introduced a convention to define and look up the listeners. > So it

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-08-21 Thread Becket Qin
Hi Eno, Thanks for the comments. This KIP is not really about improving the performance in general. It is about ensuring the cluster state can still be updated quickly even if the brokers are under heavy load. We have seen quite often that it took dozens of seconds for a broker to process the

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-08-20 Thread Eno Thereska
Hi folks, I looked at the previous numbers that Lucas provided (thanks!) but it's still not clear to me whether the performance benefits justify the added complexity. I'm looking for some intuition here (a graph would be great but not required): for a small/medium/large cluster, what are the expec

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-08-20 Thread Becket Qin
Hi Lucas, In KIP-103, we introduced a convention to define and look up the listeners. So it would be good if the later KIPs can follow the same convention. >From what I understand, the advertised.listeners is actually designed for our purpose, i.e. providing a list of listeners that can be used i

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-08-16 Thread Lucas Wang
Thanks for the review, Becket. (1) After comparing the two approaches, I still feel the current writeup is a little better. a. The current writeup asks for an explicit endpoint while reusing the existing "inter.broker.listener.name" with the exactly same semantic, and your proposed change asks for

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-08-16 Thread Becket Qin
Hi Lucas, Thanks for updating the wiki. The updated motivation description looks good to me. One additional benefit of having a separate port for controller messages is that we can protect the control plane with something like IP table. Reading the proposed change a bit more, I found it is a litt

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-08-13 Thread Lucas Wang
@Becket Makes sense. I've updated the KIP by adding the following paragraph to the motivation section > Today there is no separate between controller requests and regular data > plane requests. Specifically (1) a controller in a cluster uses the same > advertised endpoints to connect to brokers a

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-08-13 Thread Becket Qin
Hi Lucas, Thanks for the explanation. It might be a nitpick, but it seems better to mention in the motivation part that today the client requests and controller requests are not only sharing the same queue, but also a bunch of things else, so that we can avoid asking people to read the rejected al

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-08-09 Thread Lucas Wang
@Becket, I've asked for review by Jun and Joel in the vote thread. Regarding the separate thread and port, I did talk about it in the rejected alternative design 1. Please let me know if you'd like more elaboration or moving it to the motivation, etc. Thanks, Lucas On Wed, Aug 8, 2018 at 3:59 PM

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-08-08 Thread Becket Qin
Hi Lucas, Yes, a separate Jira is OK. Since the proposal has significantly changed since the initial vote started. We probably should let the others who have already voted know and ensure they are happy with the updated proposal. Also, it seems the motivation part of the KIP wiki is still just ta

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-08-07 Thread Lucas Wang
Hi Becket, Thanks for the review. The current write up in the KIP won’t change the ordering behavior. Are you ok with addressing that as a separate independent issue (I’ll create a separate ticket for it)? If so, can you please give me a +1 on the vote thread? Thanks, Lucas On Tue, Aug 7, 2018 a

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-08-07 Thread Becket Qin
Thanks for the updated KIP wiki, Lucas. Looks good to me overall. It might be an implementation detail, but do we still plan to use the correlation id to ensure the request processing order? Thanks, Jiangjie (Becket) Qin On Tue, Jul 31, 2018 at 3:39 AM, Lucas Wang wrote: > Thanks for your rev

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-30 Thread Lucas Wang
Thanks for your review, Dong. Ack that these configs will have a bigger impact for users. On the other hand, I would argue that the request queue becoming full may or may not be a rare scenario. How often the request queue gets full depends on the request incoming rate, the request processing rate

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-27 Thread Dong Lin
Hey Lucas, Thanks for the update. The current KIP propose new broker configs "listeners.for.controller" and "advertised.listeners.for.controller". This is going to be a big change since listeners are among the most important configs that every user needs to change. According to the rejected alter

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-27 Thread Lucas Wang
A kind reminder for review of this KIP. Thank you very much! Lucas On Wed, Jul 25, 2018 at 10:23 PM, Lucas Wang wrote: > Hi All, > > I've updated the KIP by adding the dedicated endpoints for controller > connections, > and pinning threads for controller requests. > Also I've updated the title

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-25 Thread Lucas Wang
Hi All, I've updated the KIP by adding the dedicated endpoints for controller connections, and pinning threads for controller requests. Also I've updated the title of this KIP. Please take a look and let me know your feedback. Thanks a lot for your time! Lucas On Tue, Jul 24, 2018 at 10:19 AM, M

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-24 Thread Mayuresh Gharat
Hi Lucas, I agree, if we want to go forward with a separate controller plane and data plane and completely isolate them, having a separate port for controller with a separate Acceptor and a Processor sounds ideal to me. Thanks, Mayuresh On Mon, Jul 23, 2018 at 11:04 PM Becket Qin wrote: > Hi

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-23 Thread Becket Qin
Hi Lucas, Yes, I agree that a dedicated end to end control flow would be ideal. Thanks, Jiangjie (Becket) Qin On Tue, Jul 24, 2018 at 1:05 PM, Lucas Wang wrote: > Thanks for the comment, Becket. > So far, we've been trying to avoid making any request handler thread > special. > But if we were

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-23 Thread Lucas Wang
Thanks for the comment, Becket. So far, we've been trying to avoid making any request handler thread special. But if we were to follow that path in order to make the two planes more isolated, what do you think about also having a dedicated processor thread, and dedicated port for the controller? T

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-23 Thread Becket Qin
Personally I am not fond of the dequeue approach simply because it is against the basic idea of isolating the controller plane and data plane. With a single dequeue, theoretically speaking the controller requests can starve the clients requests. I would prefer the approach with a separate controlle

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-23 Thread Lucas Wang
Sure, I can summarize the usage of correlation id. But before I do that, it seems the same out-of-order processing can also happen to Produce requests sent by producers, following the same example you described earlier. If that's the case, I think this probably deserves a separate doc and design in

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-23 Thread Dong Lin
Hey Lucas, Could you update the KIP if you are confident with the approach which uses correlation id? The idea around correlation id is kind of scattered across multiple emails. It will be useful if other reviews can read the KIP to understand the latest proposal. Thanks, Dong On Mon, Jul 23, 20

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-23 Thread Mayuresh Gharat
I like the idea of the dequeue implementation by Lucas. This will help us avoid additional queue for controller and additional configs in Kafka. Thanks, Mayuresh On Sun, Jul 22, 2018 at 2:58 AM Becket Qin wrote: > Hi Jun, > > The usage of correlation ID might still be useful to address the cas

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-22 Thread Becket Qin
Hi Jun, The usage of correlation ID might still be useful to address the cases that the controller epoch and leader epoch check are not sufficient to guarantee correct behavior. For example, if the controller sends a LeaderAndIsrRequest followed by a StopReplicaRequest, and the broker processes

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-21 Thread Jun Rao
Hmm, since we already use controller epoch and leader epoch for properly caching the latest partition state, do we really need correlation id for ordering the controller requests? Thanks, Jun On Fri, Jul 20, 2018 at 2:18 PM, Becket Qin wrote: > Lucas and Mayuresh, > > Good idea. The correlatio

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-19 Thread Becket Qin
Lucas and Mayuresh, Good idea. The correlation id should work. In the ControllerChannelManager, a request will be resent until a response is received. So if the controller to broker connection disconnects after controller sends R1_a, but before the response of R1_a is received, a disconnection ma

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-19 Thread Jun Rao
I agree that there is no strong ordering when there are more than one socket connections. Currently, we rely on controllerEpoch and leaderEpoch to ensure that the receiving broker picks up the latest state for each partition. One potential issue with the dequeue approach is that if the queue is fu

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-19 Thread Mayuresh Gharat
Yea, the correlationId is only set to 0 in the NetworkClient constructor. Since we reuse the same NetworkClient between Controller and the broker, a disconnection should not cause it to reset to 0, in which case it can be used to reject obsolete requests. Thanks, Mayuresh On Thu, Jul 19, 2018 at

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-19 Thread Lucas Wang
@Dong, Great example and explanation, thanks! @All Regarding the example given by Dong, it seems even if we use a queue, and a dedicated controller request handling thread, the same result can still happen because R1_a will be sent on one connection, and R1_b & R2 will be sent on a different conne

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-19 Thread Mayuresh Gharat
Actually nvm, correlationId is reset in case of connection loss, I think. Thanks, Mayuresh On Thu, Jul 19, 2018 at 11:11 AM Mayuresh Gharat wrote: > I agree with Dong that out-of-order processing can happen with having 2 > separate queues as well and it can even happen today. > Can we use the

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-19 Thread Mayuresh Gharat
I agree with Dong that out-of-order processing can happen with having 2 separate queues as well and it can even happen today. Can we use the correlationId in the request from the controller to the broker to handle ordering ? Thanks, Mayuresh On Thu, Jul 19, 2018 at 6:41 AM Becket Qin wrote: >

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-19 Thread Becket Qin
Good point, Joel. I agree that a dedicated controller request handling thread would be a better isolation. It also solves the reordering issue. On Thu, Jul 19, 2018 at 2:23 PM, Joel Koshy wrote: > Good example. I think this scenario can occur in the current code as well > but with even lower pro

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-18 Thread Joel Koshy
Good example. I think this scenario can occur in the current code as well but with even lower probability given that there are other non-controller requests interleaved. It is still sketchy though and I think a safer approach would be separate queues and pinning controller request handling to one h

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-18 Thread Dong Lin
Hey Becket, I think you are right that there may be out-of-order processing. However, it seems that out-of-order processing may also happen even if we use a separate queue. Here is the example: - Controller sends R1 and got disconnected before receiving response. Then it reconnects and sends R2.

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-18 Thread Dong Lin
Hey Lucas, Let me present a more specific sequence of events to show why out-of-order processing can happen. Can you see if it makes sense? Say broker has 12 request handler threads and there are many large ProduceRequest in the queue. Each ProduceRequest takes 200 ms to be processed. Let's also

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-18 Thread Lucas Wang
Hi Dong, Sure. Regarding the 2nd case you mentioned "- If the controller has not received response for R1 before it is disconnected, it will re-send R1 followed by R2 after it is re-connected to the broker." with the max inflight request set to 1, after the connection is re-established, the contr

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-18 Thread Dong Lin
Hey Lucas, I think for now we can probably discuss based on the existing Kafka's design where controller to a broker is hard coded to be 1. It looks like Becket has provided a good example in which requests from the same controller can be processed out of order. Thanks, Dong On Wed, Jul 18, 2018

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-18 Thread Dong Lin
Hey Becket, Sorry I misunderstood your example. I thought you mean requests from different controller are re-ordered. I think you have provided a very good example and it should be safer to still use two queues. Let me clarify the example a bit more below. - If the controller has received respon

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-18 Thread Lucas Wang
@Becket and Dong, I think currently the ordering guarantee is achieved because the max inflight request from the controller to a broker is hard coded to be 1. If let's hypothetically say the max inflight requests is > 1, then I think Dong is right to say that even the separate queue cannot guarant

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-18 Thread Dong Lin
Hey Becket, It seems that the requests from the old controller will be discarded due to old controller epoch. It is not clear whether this is a problem. And if this out-of-order processing of controller requests is a problem, it seems like an existing problem which also applies to the multi-queue

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-18 Thread Becket Qin
Hi Mayuresh/Joel, Using the request channel as a dequeue was bright up some time ago when we initially thinking of prioritizing the request. The concern was that the controller requests are supposed to be processed in order. If we can ensure that there is one controller request in the request chan

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-18 Thread Joel Koshy
@Mayuresh - I like your idea. It appears to be a simpler less invasive alternative and it should work. Jun/Becket/others, do you see any pitfalls with this approach? On Wed, Jul 18, 2018 at 12:03 PM, Lucas Wang wrote: > @Mayuresh, > That's a very interesting idea that I haven't thought before. >

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-18 Thread Lucas Wang
@Mayuresh, That's a very interesting idea that I haven't thought before. It seems to solve our problem at hand pretty well, and also avoids the need to have a new size metric and capacity config for the controller request queue. In fact, if we were to adopt this design, there is no public interface

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-18 Thread Mayuresh Gharat
Hi Lucas, Seems like the main intent here is to prioritize the controller request over any other requests. In that case, we can change the request queue to a dequeue, where you always insert the normal requests (produce, consume,..etc) to the end of the dequeue, but if its a controller request, yo

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-18 Thread Becket Qin
Hey Joel, Thank for the detail explanation. I agree the current design makes sense. My confusion is about whether the new config for the controller queue capacity is necessary. I cannot think of a case in which users would change it. Thanks, Jiangjie (Becket) Qin On Wed, Jul 18, 2018 at 6:00 PM

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-18 Thread Becket Qin
Hi Lucas, I guess my question can be rephrased to "do we expect user to ever change the controller request queue capacity"? If we agree that 20 is already a very generous default number and we do not expect user to change it, is it still necessary to expose this as a config? Thanks, Jiangjie (Be

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-17 Thread Lucas Wang
@Becket 1. Thanks for the comment. You are right that normally there should be just one controller request because of muting, and I had NOT intended to say there would be many enqueued controller requests. I went through the KIP again, and I'm not sure which part conveys that info. I'd be happy to

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-17 Thread Mayuresh Gharat
Hi Lucas, Thanks for the KIP. I am trying to understand why you think "The memory consumption can rise given the total number of queued requests can go up to 2x" in the impact section. Normally the requests from controller to a Broker are not high volume, right ? Thanks, Mayuresh On Tue, Jul 1

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-17 Thread Joel Koshy
Hey Becket - good point. Lucas and I were talking about this offline last week. It is true that there is only one request in flight for processing. However, there may be more during a controller failover but it should not be very high - basically the maximum number of controller failures that can o

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-17 Thread Becket Qin
Thanks for the KIP, Lucas. Separating the control plane from the data plane makes a lot of sense. In the KIP you mentioned that the controller request queue may have many requests in it. Will this be a common case? The controller requests still goes through the SocketServer. The SocketServer will

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-12 Thread Dong Lin
Thanks for the update Lucas. I think the motivation section is intuitive. It will be good to learn more about the comments from other reviewers. On Thu, Jul 12, 2018 at 9:48 PM, Lucas Wang wrote: > Hi Dong, > > I've updated the motivation section of the KIP by explaining the cases that > would

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-12 Thread Lucas Wang
Hi Dong, I've updated the motivation section of the KIP by explaining the cases that would have user impacts. Please take a look at let me know your comments. Thanks, Lucas On Mon, Jul 9, 2018 at 5:53 PM, Lucas Wang wrote: > Hi Dong, > > The simulation of disk being slow is merely for me to ea

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-09 Thread Lucas Wang
Hi Dong, The simulation of disk being slow is merely for me to easily construct a testing scenario with a backlog of produce requests. In production, other than the disk being slow, a backlog of produce requests may also be caused by high produce QPS. In that case, we may not want to kill the brok

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-03 Thread Dong Lin
Hey Jun, Thanks much for the comments. It is good point. So the feature may be useful for JBOD use-case. I have one question below. Hey Lucas, Do you think this feature is also useful for non-JBOD setup or it is only useful for the JBOD setup? It may be useful to understand this. When the broke

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-03 Thread Lucas Wang
Thanks for the insightful comment, Jun. @Dong, Since both of the two comments in your previous email are about the benefits of this KIP and whether it's useful, in light of Jun's last comment, do you agree that this KIP can be beneficial in the case mentioned by Jun? Please let me know, thanks! R

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-03 Thread Jun Rao
Hi, Lucas, Dong, If all disks on a broker are slow, one probably should just kill the broker. In that case, this KIP may not help. If only one of the disks on a broker is slow, one may want to fail that disk and move the leaders on that disk to other brokers. In that case, being able to process th

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-02 Thread Dong Lin
Hey Lucas, Thanks for the reply. Some follow up questions below. Regarding 1, if each ProduceRequest covers 20 partitions that are randomly distributed across all partitions, then each ProduceRequest will likely cover some partitions for which the broker is still leader after it quickly processes

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-02 Thread Lucas Wang
Hi Dong, Thanks for your valuable comments. Please see my reply below. 1. The Google doc showed only 1 partition. Now let's consider a more common scenario where broker0 is the leader of many partitions. And let's say for some reason its IO becomes slow. The number of leader partitions on broker0

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-01 Thread Dong Lin
Hey Lucas, Thanks much for the detailed documentation of the experiment. Initially I also think having a separate queue for controller requests is useful because, as you mentioned in the summary section of the Google doc, controller requests are generally more important than data requests and we

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-06-29 Thread Lucas Wang
Hi Eno, Sorry for the delay in getting the experiment results. Here is a link to the positive impact achieved by implementing the proposed change: https://docs.google.com/document/d/1ge2jjp5aPTBber6zaIT9AdhWFWUENJ3JO6Zyu4f9tgQ/edit?usp=sharing Please take a look when you have time and let me know

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-06-26 Thread Harsha
Thanks for the pointer. Will take a look might suit our requirements better. Thanks, Harsha On Mon, Jun 25th, 2018 at 2:52 PM, Lucas Wang wrote: > > > > Hi Harsha, > > If I understand correctly, the replication quota mechanism proposed in > KIP-73 can be helpful in that scenario. > Have you

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-06-25 Thread Lucas Wang
Hi Harsha, If I understand correctly, the replication quota mechanism proposed in KIP-73 can be helpful in that scenario. Have you tried it out? Thanks, Lucas On Sun, Jun 24, 2018 at 8:28 AM, Harsha wrote: > Hi Lucas, > One more question, any thoughts on making this configurable >

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-06-24 Thread Harsha
Hi Lucas,              One more question, any thoughts on making this configurable and also allowing subset of data requests to be prioritized. For example ,we notice in our cluster when we take out a broker and bring new one it will try to become follower and have lot of fetch requests to other

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-06-22 Thread Lucas Wang
Hi Eno, Sorry for the delayed response. - I haven't implemented the feature yet, so no experimental results so far. And I plan to test in out in the following days. - You are absolutely right that the priority queue does not completely prevent data requests being processed ahead of controller req

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-06-20 Thread Eno Thereska
Hi Lucas, Sorry for the delay, just had a look at this. A couple of questions: - did you notice any positive change after implementing this KIP? I'm wondering if you have any experimental results that show the benefit of the two queues. - priority is usually not sufficient in addressing the probl

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-06-15 Thread Ted Yu
Change looks good. Thanks On Fri, Jun 15, 2018 at 8:42 AM, Lucas Wang wrote: > Hi Ted, > > Thanks for the suggestion. I've updated the KIP. Please take another look. > > Lucas > > On Thu, Jun 14, 2018 at 6:34 PM, Ted Yu wrote: > > > Currently in KafkaConfig.scala : > > > > val QueuedMaxReque

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-06-15 Thread Lucas Wang
Hi Ted, Thanks for the suggestion. I've updated the KIP. Please take another look. Lucas On Thu, Jun 14, 2018 at 6:34 PM, Ted Yu wrote: > Currently in KafkaConfig.scala : > > val QueuedMaxRequests = 500 > > It would be good if you can include the default value for this new config > in the KI

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-06-14 Thread Ted Yu
Currently in KafkaConfig.scala : val QueuedMaxRequests = 500 It would be good if you can include the default value for this new config in the KIP. Thanks On Thu, Jun 14, 2018 at 4:28 PM, Lucas Wang wrote: > Hi Ted, Dong > > I've updated the KIP by adding a new config, instead of reusing the

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-06-14 Thread Lucas Wang
Hi Ted, Dong I've updated the KIP by adding a new config, instead of reusing the existing one. Please take another look when you have time. Thanks a lot! Lucas On Thu, Jun 14, 2018 at 2:33 PM, Ted Yu wrote: > bq. that's a waste of resource if control request rate is low > > I don't know if co

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-06-14 Thread Ted Yu
bq. that's a waste of resource if control request rate is low I don't know if control request rate can get to 100,000, likely not. Then using the same bound as that for data requests seems high. On Wed, Jun 13, 2018 at 10:13 PM, Lucas Wang wrote: > Hi Ted, > > Thanks for taking a look at this

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-06-13 Thread Lucas Wang
Hi Ted, Thanks for taking a look at this KIP. Let's say today the setting of "queued.max.requests" in cluster A is 1000, while the setting in cluster B is 100,000. The 100 times difference might have indicated that machines in cluster B have larger memory. By reusing the "queued.max.requests", th

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-06-13 Thread Ted Yu
Lucas: Under Rejected Alternatives, #2, can you elaborate a bit more on why the separate config has bigger impact ? Thanks On Wed, Jun 13, 2018 at 2:00 PM, Dong Lin wrote: > Hey Luca, > > Thanks for the KIP. Looks good overall. Some comments below: > > - We usually specify the full mbean for th

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-06-13 Thread Lucas Wang
Thanks for the comments, Dong. I've updated the KIP and addressed your 3 comments. Please take another look when you get a chance. Lucas On Wed, Jun 13, 2018 at 2:00 PM, Dong Lin wrote: > Hey Luca, > > Thanks for the KIP. Looks good overall. Some comments below: > > - We usually specify the fu

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-06-13 Thread Dong Lin
Hey Luca, Thanks for the KIP. Looks good overall. Some comments below: - We usually specify the full mbean for the new metrics in the KIP. Can you specify it in the Public Interface section similar to KIP-237

[DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-06-13 Thread Lucas Wang
Hi Kafka experts, I created KIP-291 to add a separate queue for controller requests: https://cwiki.apache.org/confluence/display/KAFKA/KIP-291%3A+Have+separate+queues+for+control+requests+and+data+requests Can you please take a look and let me know your feedback? Thanks a lot for your time! Rega