Lucas Wang created KAFKA-14424:
--
Summary: Cancellation of an ongoing replica reassignment should
have sanity checks
Key: KAFKA-14424
URL: https://issues.apache.org/jira/browse/KAFKA-14424
Project: Kafka
Lucas Wang created KAFKA-14381:
--
Summary: Support listing all partitions being reassigned in a
cluster
Key: KAFKA-14381
URL: https://issues.apache.org/jira/browse/KAFKA-14381
Project: Kafka
Lucas Wang created KAFKA-14213:
--
Summary: Reduce lock contention between
control-plane-kafka-request-handler and kafka-log-cleaner-thread
Key: KAFKA-14213
URL: https://issues.apache.org/jira/browse/KAFKA-14213
Lucas Wang created KAFKA-13815:
--
Summary: Avoid reinitialization for a replica that is being deleted
Key: KAFKA-13815
URL: https://issues.apache.org/jira/browse/KAFKA-13815
Project: Kafka
Issue
Lucas Wang created KAFKA-13797:
--
Summary: Adding metric to indicate metadata response outgoing
bytes rate
Key: KAFKA-13797
URL: https://issues.apache.org/jira/browse/KAFKA-13797
Project: Kafka
Lucas Wang created KAFKA-13188:
--
Summary: Release the memory back into MemoryPool
Key: KAFKA-13188
URL: https://issues.apache.org/jira/browse/KAFKA-13188
Project: Kafka
Issue Type: Improvement
Lucas Wang created KAFKA-12315:
--
Summary: Clearing the ZkReplicaStateMachine request batch state
upon ControllerMovedException
Key: KAFKA-12315
URL: https://issues.apache.org/jira/browse/KAFKA-12315
Lucas Wang created KAFKA-10751:
--
Summary: Generate log to help estimate messages lost during ULE
Key: KAFKA-10751
URL: https://issues.apache.org/jira/browse/KAFKA-10751
Project: Kafka
Issue
Lucas Wang created KAFKA-10734:
--
Summary: Speedup the processing of LeaderAndIsr request
Key: KAFKA-10734
URL: https://issues.apache.org/jira/browse/KAFKA-10734
Project: Kafka
Issue Type
change "whose default capacity is 20" to "whose capacity is 20"
> in the section "How are controller requests handled over the dedicated
> connections"? The use of word "default" seems to suggest that this is
> configurable.
>
> Thanks,
> Do
d to me.
>
> Thanks,
>
> Jun
>
> On Fri, Oct 5, 2018 at 1:12 PM, Lucas Wang wrote:
>
> > Thanks for the suggestion, Ismael. I like it.
> >
> > Jun,
> > I'm excited to get the +1, thanks a lot!
> > Meanwhile what do you feel ab
gt; Ismael
>
>
> On 4 Oct 2018 11:08 am, "Lucas Wang" wrote:
>
> Thanks Jun. I've changed the KIP with the suggested 2 step upgrade.
> Please take a look again when you have time.
>
> Regards,
> Lucas
>
>
> On Thu, Oct 4, 2018 at 10:06 AM Jun Rao
roker sets
> controller.listener.name. At that point, the controller listener is ready
> in every broker.
>
> Thanks,
>
> Jun
>
> On Tue, Oct 2, 2018 at 10:38 AM, Lucas Wang wrote:
>
> > Thanks for the further comments, Jun.
> >
> > 200. Currently in
. The end result should be the same
> behavior as today." Currently, the controller makes connections based on
> its local inter.broker.listener.name config without checking the target
> broker's ZK registration. For consistency, perhaps we can just follow the
> same strategy for
Hi Jun,
Sorry to bother you again. Can you please take a look at the wiki again
when you have time?
Thanks a lot!
Lucas
On Wed, Sep 19, 2018 at 3:57 PM Lucas Wang wrote:
> Hi Jun,
>
> Thanks a lot for the detailed explanation.
> I've restored the wiki to a previous vers
controller listener.
> If this is indeed something that everyone wants to set, we can make it a
> required config in a major release.
>
> Thanks,
>
> Jun
>
> On Tue, Sep 11, 2018 at 3:46 PM, Lucas Wang wrote:
>
> > @Jun Rao
> >
> > I made the recent
> 101. The KIP says that advertised.listeners and listeners will now have a
> > different default value including controller. Could you document what the
> > default value looks like?
> >
> > 102. About removing the the following configs. How does that affect the
> > upgrade
to have a separate metric for that. The controller request
> queue size may not reflect the history in a window.
>
> Jun
>
> On Wed, Sep 5, 2018 at 3:38 PM, Lucas Wang wrote:
>
> > Thanks Jun for your quick response. It looks like I forgot to click the
> > "Update
ude the new controller listener? If not, do
> we need additional metrics to measure the utilization of the io thread for
> the control plane?
>
> Jun
>
> On Mon, Aug 27, 2018 at 6:26 PM, Lucas Wang wrote:
>
> > Thanks for the comments, Jun.
> >
> > 1. I think t
proach to handle both listener.name.for.controller and
> inter.broker.listener.name consistently. To me, the former seems slightly
> better.
>
> 3. To be consistent with the existing naming, should
> listener.name.for.controller
> be controller.listener.name?
>
> Thanks,
>
>
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
Lucas Wang created KAFKA-7350:
-
Summary: Improve or remove the PreferredReplicaImbalanceCount
metric
Key: KAFKA-7350
URL: https://issues.apache.org/jira/browse/KAFKA-7350
Project: Kafka
Issue
[
https://issues.apache.org/jira/browse/KAFKA-6753?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Lucas Wang resolved KAFKA-6753.
---
Resolution: Fixed
> Speed up event processing on the control
ne change v.s
> two-lines
> >> change, I am a little confused, can you elaborate?
> >>
> >> Regarding the possibility of hurry and misreading. It is the system
> admin's
> >> responsibility to configure the right listener to ensure that different
> >>
[
https://issues.apache.org/jira/browse/KAFKA-6753?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Lucas Wang reopened KAFKA-6753:
---
> Speed up event processing on the control
r
> that we always follow the same of convention instead of doing it in
> different ways.
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
>
>
> On Fri, Aug 17, 2018 at 4:34 AM, Lucas Wang wrote:
>
> > Thanks for the review, Becket.
> >
> > (1) After comparin
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
[
https://issues.apache.org/jira/browse/KAFKA-6974?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Lucas Wang resolved KAFKA-6974.
---
Resolution: Won't Fix
> Changes the interaction between request handler threads and fetcher
id asking people to read the rejected
> alternatives.
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
>
>
>
>
>
>
> On Fri, Aug 10, 2018 at 6:23 AM, Lucas Wang wrote:
>
> > @Becket,
> >
> > I've asked for review by Jun and Joel in the vote t
understand the
> whole proposal.
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
> On Wed, Aug 8, 2018 at 12:44 PM, Lucas Wang wrote:
>
> > Hi Becket,
> >
> > Thanks for the review. The current write up in the KIP won’t change the
> > ordering behavior. Are you
tion variables
> for the concern that Jun raised, but it's an implementation detail that we
> can defer to a discussion in the PR.)
>
> On Sat, Jul 14, 2018 at 10:45 PM, Lucas Wang
> wrote:
>
> > Hi Jun,
> >
> > I agree by using the conditional variables, th
; On Tue, Jul 31, 2018 at 3:39 AM, Lucas Wang wrote:
>
> > 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 sce
recent discussion, e.g. out of order processing if we don't use a dedicated
> thread for controller request, it may be useful to explain the problem in
> the motivation section.
>
> Thanks,
> Dong
>
> On Fri, Jul 27, 2018 at 1:28 PM, Lucas Wang wrote:
>
> > A kind re
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
unds ideal to me.
>
> Thanks,
>
> Mayuresh
>
>
> On Mon, Jul 23, 2018 at 11:04 PM Becket Qin wrote:
>
> > Hi Lucas,
> >
> > Yes, I agree that a dedicated end to end control flow would be ideal.
> >
> > Thanks,
> >
> > Jiangjie (Becket) Qin
e dequeue, theoretically speaking the controller requests can
> starve the clients requests. I would prefer the approach with a separate
> controller request queue and a dedicated controller request handler thread.
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
> On Tue, Jul 24, 2018
picks up the latest state for
> > each
> > > >>> partition.
> > > >>>
> > > >>> One potential issue with the dequeue approach is that if the queue
> is
> > > >> full,
> > > >>> there is no guarantee that t
er
> >> > > request
> >> > > > inserted into the queue, the controller request order may change
> and
> >> > > cause
> >> > > > problem. For example, think about the following sequence:
> >> > > > 1. Controller s
sed out of order.
>
> Thanks,
> Dong
>
> On Wed, Jul 18, 2018 at 8:35 PM, Lucas Wang wrote:
>
> > @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
>
ay cause problem.
> >
> > Thanks,
> >
> > Jiangjie (Becket) Qin
> >
> >
> >
> > On Thu, Jul 19, 2018 at 3:23 AM, Joel Koshy wrote:
> >
> > > @Mayuresh - I like your idea. It appears to be a simpler less invasive
> > > alternati
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,
> > >
&g
Lucas Wang created KAFKA-7180:
-
Summary: In testHWCheckpointWithFailuresSingleLogSegment, wait
until server1 has joined the ISR before shutting down server2
Key: KAFKA-7180
URL: https://issues.apache.org/jira/browse
> Thanks,
> >
> > Jiangjie (Becket) Qin
> >
> >
> > On Fri, Jul 13, 2018 at 1:10 PM, Dong Lin wrote:
> >
> > > Thanks for the update Lucas.
> > >
> > > I think the motivation section is intuitive. It will be good to learn
>
Hi Jun,
I agree by using the conditional variables, there is no need to add such a
new config.
Also thanks for approving this KIP.
Lucas
Lucas Wang created KAFKA-7162:
-
Summary: Flaky unit tests caused by record creation timestamps
differ from validation time by more than timestampDiffMaxMs
Key: KAFKA-7162
URL: https://issues.apache.org/jira/browse
e sendRequest()/receiveRequest() coordinate on the lock
> and the conditions (similar to how ArrayBlockingQueue is implemented). This
> way, any new request can wake up the blocked request handling threads
> immediately.
>
> Thanks,
>
> Jun
>
>
> On Fri, Jun 29, 2018 at
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 me
away from the failed disk may still be long even with
> this KIP. What do you think?
>
> Thanks,
> Dong
>
>
> On Tue, Jul 3, 2018 at 4:38 PM, Lucas Wang wrote:
>
> > Thanks for the insightful comment, Jun.
> >
> > @Dong,
> > Since both of t
by this KIP?
> >
> > Not sure why system operator directly cares number of truncated messages.
> > Do you mean this KIP can improve average throughput or reduce message
> > duplication? It will be good to understand this.
> >
> > Thanks,
> > Dong
> &g
l 2, 2018 at 5:52 PM, Ted Yu wrote:
> For #1, I don't know what would be good approximation for M.
> Maybe use max((TO / 2) / N, M / N) as default value for poll timeout ?
>
> For #2, I don't see the picture in email :-)
> Can you use third party website ?
>
> Th
lost.
> Do you mind posting that part again ?
>
> Thanks
>
> On Fri, Jun 29, 2018 at 4:53 PM, Lucas Wang wrote:
>
> > Hi Jun,
> >
> > Thanks for your comments.
> > 1. I just replied in the discussion thread about the positive change this
> > KIP ca
Does this KIP help improve user experience only when there is issue with
> broker, e.g. significant backlog in the request queue due to slow disk as
> described in the Google doc? Or is this KIP also useful when there is no
> ongoing issue in the cluster? It might be helpful to clarify t
computed by measuring the amount of time waiting on the
> request queue. Will these 2 metrics be extended to support 2 request
> queues.
>
> Jun
>
>
> On Mon, Jun 18, 2018 at 1:04 PM, Lucas Wang wrote:
>
> > Hi All,
> >
> > I've addressed a couple of com
your feedback.
Regards,
Lucas
On Tue, Jun 26, 2018 at 9:52 AM, Harsha wrote:
> 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:
>
> >
> >
>
application/client requests.
> We are also exploring the similar solution to de-prioritize if a new
> replica comes in for fetch requests, we are ok with the replica to be
> taking time but the leaders should prioritize the client requests.
>
>
> Thanks,
> Harsha
>
> On Fri
would be
> good to know what % of the problem this KIP addresses.
>
> Thanks
> Eno
>
> On Fri, Jun 15, 2018 at 4:44 PM, Ted Yu wrote:
>
> > Change looks good.
> >
> > Thanks
> >
> > On Fri, Jun 15, 2018 at 8:42 AM, Lucas Wang
> wrote:
&g
Hi Jun, Ismael,
Can you please take a look when you get a chance? Thanks!
Lucas
On Mon, Jun 18, 2018 at 1:47 PM, Ted Yu wrote:
> +1
>
> On Mon, Jun 18, 2018 at 1:04 PM, Lucas Wang wrote:
>
> > Hi All,
> >
> > I've addressed a couple of comments in the dis
Hi All,
I've addressed a couple of comments in the discussion thread for KIP-291,
and
got no objections after making the changes. Therefore I would like to start
the voting thread.
KIP:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-291%3A+Have+separate+queues+for+control+requests+and+data
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
> > existing one.
> > Please take another look when you have
gt; 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 KIP.
&g
BytesInPerSec", rather than replacing the sensor name "
> > BytesInPerSec" with e.g. "ClientBytesInPerSec".
> >
> > - It seems that the KIP changes the semantics of the broker config
> > "queued.max.requests" because the number of total requests queued in
d by "queued.max.requests". This probably
> needs to be specified in the Public Interfaces section for discussion.
>
>
> Thanks,
> Dong
>
>
> On Wed, Jun 13, 2018 at 12:45 PM, Lucas Wang
> wrote:
>
> > Hi Kafka experts,
> >
> > I created K
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
Lucas Wang created KAFKA-7040:
-
Summary: The replica fetcher thread may truncate accepted messages
during multiple fast leadership transitions
Key: KAFKA-7040
URL: https://issues.apache.org/jira/browse/KAFKA-7040
Lucas Wang created KAFKA-6974:
-
Summary: Changes the interaction between request handler threads
and fetcher threads into an ASYNC model
Key: KAFKA-6974
URL: https://issues.apache.org/jira/browse/KAFKA-6974
Lucas Wang created KAFKA-6753:
-
Summary: Speed up event processing on the controller
Key: KAFKA-6753
URL: https://issues.apache.org/jira/browse/KAFKA-6753
Project: Kafka
Issue Type: Improvement
[
https://issues.apache.org/jira/browse/KAFKA-6652?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Lucas Wang resolved KAFKA-6652.
---
Resolution: Won't Fix
> The controller should log failed attempts to transition a re
[
https://issues.apache.org/jira/browse/KAFKA-6612?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Lucas Wang resolved KAFKA-6612.
---
Resolution: Fixed
> Added logic to prevent increasing partition counts during topic delet
Lucas Wang created KAFKA-6652:
-
Summary: The controller should log failed attempts to transition a
replica to OfflineReplica state
Key: KAFKA-6652
URL: https://issues.apache.org/jira/browse/KAFKA-6652
Lucas Wang created KAFKA-6650:
-
Summary: The controller should be able to handle a partially
deleted topic
Key: KAFKA-6650
URL: https://issues.apache.org/jira/browse/KAFKA-6650
Project: Kafka
Lucas Wang created KAFKA-6630:
-
Summary: Speed up the processing of StopReplicaResponse events on
the controller
Key: KAFKA-6630
URL: https://issues.apache.org/jira/browse/KAFKA-6630
Project: Kafka
Lucas Wang created KAFKA-6612:
-
Summary: Added logic to prevent increasing partition counts during
topic deletion
Key: KAFKA-6612
URL: https://issues.apache.org/jira/browse/KAFKA-6612
Project: Kafka
Lucas Wang created KAFKA-6481:
-
Summary: Improving performance of the function
ControllerChannelManager.addUpdateMetadataRequestForBrokers
Key: KAFKA-6481
URL: https://issues.apache.org/jira/browse/KAFKA-6481
73 matches
Mail list logo