Hi,
Generally, max production throughput will be calculated as follows:
- max_requests_per_sec = 1000 / request_latency_ms * 5
(max.in.flight.requests)
- max_throughput = max_requests_per_sec * messages-per-request
So I suggest identifying which is the bottleneck first, req-latency or
too-small-
of
> partitions.
> And, finding the eligible replica manually, will increase the outage
> mitigation time
> as the producers/consumers are blocked when there are offline partitions.
>
> --
> Kamal
>
>
> On Wed, Sep 11, 2024 at 3:57 AM Haruki Okada wrote:
>
>
Hi,
Thank you for the KIP.
The motivation sounds make sense to me.
I have a few questions:
- [nits] "AlterPartitions request" in Error handling section is
"AlterPartitionReassignments request" actually, right?
- Don't we need to include cordoned information in DescribeLogDirs response
too? Some
Haruki Okada created KAFKA-17061:
Summary: KafkaController takes long time to connect to newly added
broker after registration on large cluster
Key: KAFKA-17061
URL: https://issues.apache.org/jira/browse/KAFKA
for your PR to hit the timeout and abort
> > again. If not, I can spend a minute looking at this other test a bit
> later
> > tonight
> >
> > On Fri, Jun 7, 2024 at 6:05 PM Haruki Okada wrote:
> >
> > > Opened the PR h(ttps://github.com/apache/kafka/pull/
se.
> > Let's disable for now to unblock builds, and revert later if we can't
> solve
> > it until code freeze?
> >
> > 2024年6月8日(土) 9:31 Luke Chen :
> >
> > > Hi Haruki,
> > >
> > > Thanks for identifying this bloc
:
> Hi Haruki,
>
> Thanks for identifying this blocking test.
> Could you help quickly open a PR to disable this test to unblock the CI
> build?
>
> Thanks.
> Luke
>
> On Sat, Jun 8, 2024 at 8:20 AM Haruki Okada wrote:
>
> > Hi
> >
> > I found th
Hi
I found that the hanging can be reproduced locally.
The blocking test is
"org.apache.kafka.common.security.authenticator.ClientAuthenticationFailureTest.testAdminClientWithInvalidCredentials".
It started to block after this commit (
https://github.com/apache/kafka/commit/c01279b92acefd913508958
Hi, Murali.
First, could you add the KIP-1042 to the index (
https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals)
as well so that everyone can find it easily?
I took a look at the KIP, then I have 2 questions:
1. Though the new MATCH resource pattern type may reduce the
Haruki Okada created KAFKA-16541:
Summary: Potential leader epoch checkpoint file corruption on OS
crash
Key: KAFKA-16541
URL: https://issues.apache.org/jira/browse/KAFKA-16541
Project: Kafka
Haruki Okada created KAFKA-16393:
Summary: SslTransportLayer doesn't implement write(ByteBuffer[],
int, int) correctly
Key: KAFKA-16393
URL: https://issues.apache.org/jira/browse/KAFKA-16393
Pr
Hi, Divij.
This isn't about config default value/constraint change though, I found
there's a behavior discrepancy in max.block.ms config, which may cause
breaking change if we change the behavior.
The detail is described in the ticket:
https://issues.apache.org/jira/browse/KAFKA-16372
What do you
Haruki Okada created KAFKA-16372:
Summary: max.block.ms behavior inconsistency with javadoc and the
config description
Key: KAFKA-16372
URL: https://issues.apache.org/jira/browse/KAFKA-16372
Project
Hi.
KafkaConsumer can subscribe to topics by pattern:
https://kafka.apache.org/36/javadoc/org/apache/kafka/clients/consumer/Consumer.html#subscribe(java.util.regex.Pattern)
2023年12月1日(金) 22:05 Chunlin Yang :
> Hi team,
>
> My use case is I have a central controller to manage tens of thousands of
Haruki Okada created KAFKA-15924:
Summary: Flaky test -
QuorumControllerTest.testFatalMetadataReplayErrorOnActive
Key: KAFKA-15924
URL: https://issues.apache.org/jira/browse/KAFKA-15924
Project
Haruki Okada created KAFKA-15921:
Summary: Flaky test -
SaslScramSslEndToEndAuthorizationTest.testAuthentications
Key: KAFKA-15921
URL: https://issues.apache.org/jira/browse/KAFKA-15921
Project
Haruki Okada created KAFKA-15920:
Summary: Flaky test - PlaintextConsumerTest.testCoordinatorFailover
Key: KAFKA-15920
URL: https://issues.apache.org/jira/browse/KAFKA-15920
Project: Kafka
Haruki Okada created KAFKA-15919:
Summary: Flaky test -
BrokerLifecycleManagerTest.testAlwaysSendsAccumulatedOfflineDirs
Key: KAFKA-15919
URL: https://issues.apache.org/jira/browse/KAFKA-15919
Haruki Okada created KAFKA-15918:
Summary: Flaky test - OffsetsApiIntegrationTest.
testResetSinkConnectorOffsets
Key: KAFKA-15918
URL: https://issues.apache.org/jira/browse/KAFKA-15918
Project: Kafka
Haruki Okada created KAFKA-15917:
Summary: Flaky test - OffsetsApiIntegrationTest.
testAlterSinkConnectorOffsetsZombieSinkTasks
Key: KAFKA-15917
URL: https://issues.apache.org/jira/browse/KAFKA-15917
Hi!
I found that ReplicaFetcherThreadBenchmark (and other few benchmarks) is
not working now and reported as KAFKA-15567.
I submitted a patch for that in https://github.com/apache/kafka/pull/14513.
Could anyone review the PR?
Thanks,
--
Okada Haruki
ocadar...@gmail.co
Haruki Okada created KAFKA-15567:
Summary: ReplicaFetcherThreadBenchmark is not working
Key: KAFKA-15567
URL: https://issues.apache.org/jira/browse/KAFKA-15567
Project: Kafka
Issue Type
Hi Jack,
Thank you for the great work, not only the spec but also for the
comprehensive documentation about the replication.
Actually I wrote some TLA+ spec to verify unclean leader election behavior
before so I will double-check my understanding with your complete spec :)
Thanks,
2023年9月10日(日)
Hi, Omnia.
Thanks for the KIP.
The feature sounds indeed helpful and the strategy to use bloom-filter
looks good.
I have three questions:
1. Do you have any memory-footprint estimation
for TimeControlledBloomFilter?
* If I read the KIP correctly, TimeControlledBloomFilter will be
allocated p
Haruki Okada created KAFKA-15046:
Summary: Produce performance issue under high disk load
Key: KAFKA-15046
URL: https://issues.apache.org/jira/browse/KAFKA-15046
Project: Kafka
Issue Type
Hi, Luke.
Though this proposal definitely looks interesting, as others pointed out,
the leader election implementation would be the hard part.
And I think even LEO-based-election is not safe, which could cause silent
committed-data loss easily.
Let's say we have replicas A,B,C and A is the leade
t successful metadata even when it timed
out.
So I'm not sure if it makes sense to add a new API or timeout config (for
KIP-912), considering KafkaProducer already has many timeout parameters so
it could introduce another complexity.
2023年3月9日(木) 12:18 jian fu :
> Hi Haruki Okada:
>
&g
You can just use Producer#partitionsFor
2023年3月9日(木) 11:13 jian fu :
> Hi All:
>
> If anyone can help to give some comments or suggestions for the proposal.
> Thanks in advance!
>
> Regards
> Jian
>
> jian fu 于2023年3月6日周一 17:00写道:
>
> > Hi Everyone:
> > Nice to meet you.
> >
> > I created one KI
Haruki Okada created KAFKA-14445:
Summary: Producer doesn't request metadata update on
REQUEST_TIMED_OUT
Key: KAFKA-14445
URL: https://issues.apache.org/jira/browse/KAFKA-14445
Project:
Hi.
That also sounds good to me.
TLA+ spec helps us to understand the protocol from a high-level.
Also I agree with Tim's point that the spread of familiarity with
formal methods would be beneficial in the long term.
2022年7月26日(火) 23:58 Tom Bentley :
> Hi,
>
> I noticed that TLA+ has featured i
Hi, Kafka.
We found that the race in topic-deletion procedure could cause the
preferred replica imbalance metric to be negative.
The phenomenon can easily happen when many topics are deleted at once, and
since we use the metric for monitoring, we have to restart the controller
to fix the metric ev
Hi Bruno,
Thank you for driving the release !
Can we add KIP-764 to the plan? (
https://cwiki.apache.org/confluence/display/KAFKA/KIP-764%3A+Configurable+backlog+size+for+creating+Acceptor
)
Thanks,
2022年2月16日(水) 18:22 Bruno Cadonna :
> Hi Julien,
>
> Thank you for the feedback on the release
this, so it requires analyzing the implementation itself.
> (what I actually did with log4j2) About c, logback does not provide a Kafka
> Appender so we have to provide an equivalent.
>
>
> It is why I prefer to use log4j2. How do you think?
>
>
> Thanks,
>
> Dong
Hi, Dongjin,
Sorry for interrupting the discussion.
And thank you for your hard work about KIP-653, KIP-719.
I understand that KIP-653 is already accepted so log4j2 is the choice of
the Kafka community though, I'm now feeling that logback is a better choice
here.
Reasons:
- even after "log4shel
Haruki Okada created KAFKA-13403:
Summary: KafkaServer crashes when deleting topics due to the race
in log deletion
Key: KAFKA-13403
URL: https://issues.apache.org/jira/browse/KAFKA-13403
Project
+1 (binding). Thanks for the KIP!
>
> Best,
> David
>
> On Wed, Oct 20, 2021 at 12:09 PM Haruki Okada wrote:
>
> > Currently, KIP-764 got
> > +1 binding
> > +2 non-binding
> >
> > votes. Could anyone help checking the KIP and join the vote?
> &
Currently, KIP-764 got
+1 binding
+2 non-binding
votes. Could anyone help checking the KIP and join the vote?
Thanks,
2021年10月18日(月) 18:21 Haruki Okada :
> Hi Lucas.
>
> Thanks for the vote.
> As far as I understand, it should be enough to adjust `net.core.somaxconn`
>
ented in the config documentation
> > (e.g. net.core.somaxconn)?
> >
> >
> > On Mon, Oct 18, 2021 at 4:23 PM Haruki Okada
> wrote:
> >
> >> Hi Luke.
> >>
> >> Thank you for the vote.
> >> Updated KIP to link to ServerSocket#bind java
n't think everyone is familiar with the definition of the method
> parameters.
>
> Thank you.
> Luke
>
> On Mon, Oct 18, 2021 at 3:43 PM Haruki Okada wrote:
>
> > Hi Kafka.
> >
> > Let me bump this VOTE thread for the KIP.
> > We applied proposed chan
2021年8月9日(月) 11:25 Haruki Okada :
> Thanks for your comment LI-san.
>
> Could anyone else review and vote for the KIP?
>
> I think the situation described in the KIP's motivation can happen in any
> large-scale Kafka deployment, so may be helpful for many users while the
&g
property. I think we
> should name the property socket.listen.backlog.size for better clarity on
> what the property configures. Besides that, the proposal looks good to me.
>
> David
>
> On Wed, Jul 28, 2021 at 8:09 AM Haruki Okada wrote:
>
> > Hi, Kafka.
> >
>
15:49 Xiangyuan LI :
> Hi Haruki Okada:
> i read your comment, thx for your detail explain!
> add backlog parameter is a useful suggestion, hope it could added to
> kafka.
>
> Haruki Okada 于2021年8月2日周一 上午7:43写道:
>
> > Hi, Kafka.
> >
> > I would like to
Hi, Kafka.
I would like to start a vote on KIP that makes SocketServer acceptor's
backlog size configurable.
KIP:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-764%3A+Configurable+backlog+size+for+creating+Acceptor
Discussion thread:
https://lists.apache.org/thread.html/rd77469b7de0190d6
Hi, Kafka.
Does anyone have any thoughts or suggestions about the KIP?
If there seems to be no, I would like to start a VOTE later.
Thanks,
2021年7月22日(木) 16:17 Haruki Okada :
> Hi, Kafka.
>
> I proposed KIP-764, which tries to add new KafkaConfig to adjust
> Acceptor's b
Hi, Kafka.
I proposed KIP-764, which tries to add new KafkaConfig to adjust Acceptor's
backlog size.
As described in the KIP and the ticket KAFKA-9648, currently backlog size
is fixed value (50) and it may not be enough to handle incoming connections
from massive clients.
So we would like to make
Hi Bill.
Thank you for your quick response!
2021年7月22日(木) 7:40 Bill Bejeck :
> Hi Haruki,
>
> You're all set with the wiki permissions now.
>
> Regards,
> Bill
>
> On Wed, Jul 21, 2021 at 1:43 PM Haruki Okada wrote:
>
> > Hi, Kafka.
> >
> > I
Hi, Kafka.
I would like to request permission for KIP creation under
https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
to propose a config-addition related to
https://issues.apache.org/jira/browse/KAFKA-9648 .
JIRA id: ocadaruma
email: ocadar...@gmail.com
Thanks,
--
e default auto.offset.reset setting to
> the earliest.
> The auto.offset.reset setting starts before kafka V1.0, which means, there
> are already a lot of users using it and get used to it now.
>
> Thanks.
> Luke
>
> On Fri, Feb 19, 2021 at 2:24 PM Haruki Okada wrote:
>
&g
Hi, Kafka.
Recently I noticed that splitting partition may cause message delivery loss
for consumers with auto.offset.reset=latest.
I described details in https://issues.apache.org/jira/browse/KAFKA-12261 .
Since delivery loss is undesired in most cases, I think this should be
described in Consu
Haruki Okada created KAFKA-12261:
Summary: auto.offset.reset should be earliest by default
Key: KAFKA-12261
URL: https://issues.apache.org/jira/browse/KAFKA-12261
Project: Kafka
Issue Type
Hi.
One possible solution I can imagine is to use replication throttle.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-73+Replication+Quotas
You can set lower replication quota for other brokers -> failed broker
traffic during catching-up to prevent exhausting storage throughput.
2020年12月1
Haruki Okada created KAFKA-10690:
Summary: Produce-response delay caused by lagging replica fetch
which blocks in-sync one
Key: KAFKA-10690
URL: https://issues.apache.org/jira/browse/KAFKA-10690
There are two cases that zero-copy fetch thanks to sendfile don't work.
- SSL encryption is enabled
* Need to encrypt on Kafka process before sending to client
-
https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java#L946
*
53 matches
Mail list logo