JIRA username: soarez
GitHub username: soarez
Looking to assign https://issues.apache.org/jira/browse/KAFKA-8526 to myself
--
Igor Soarez
Hello,
Looking to get some reviews for KAFKA-8526 - Broker may select a failed dir for
new replica even in the presence of other live dirs
https://github.com/apache/kafka/pull/6969
https://issues.apache.org/jira/browse/KAFKA-8526
Thanks,
--
Igor Soarez
Hi everyone,
Following a face-to-face discussion with Ron and Colin,
I have just made further improvements to this KIP:
1. Every log directory gets a random UUID assigned, even if just one
log dir is configured in the Broker.
2. All online log directories are registered, even if just one if
Hi Ismael,
I believe I have addressed all concerns.
Please have a look, and consider a vote on this KIP.
Thank you,
--
Igor
Hi Mickael,
Thanks for voting, and for pointing out the mistake.
I've corrected it in the KIP now.
The proposed name is "QueuedReplicaToDirAssignments".
Best,
--
Igor
Hi Ziming,
Thank you for having a look and taking the time to vote.
I have already opened some PRs, see:
https://issues.apache.org/jira/browse/KAFKA-14127
Best,
--
Igor
Hi Ron,
Thank you for having a look a this KIP.
Indeed, the log directory UUID should always be generated
and loaded. I've have corrected the wording in the KIP to clarify.
It is a bit of a pain to replace the field, but I agree that is
the best approach for the same reason you pointed out.
I h
Hi Ron,
Thanks for drilling down on this. I think the KIP isn't really clear here,
and the metadata caching section you quoted needs clarification.
The "hosting broker's latest registration" refers to the previous,
not the current registration. The registrations are only compared by
the controlle
023 at 10:04 AM Ron Dagostino wrote:
> >
> > Ok, great, that makes sense, Igor. Thanks. +1 (binding) on the KIP from
> > me.
> >
> > Ron
> >
> > > On Sep 13, 2023, at 11:58 AM, Igor Soarez
> > > wrote:
> > >
> > > Hi R
Hi everyone,
I believe we can close this voting thread now, as there
were three +1 binding votes from Ziming, Mickael and Ron.
With that, this vote passes.
Thanks to everyone who participated in reviewing,
and/or taking the time to vote on this KIP!
Best,
--
Igor
Hi Ron,
I think we can generalize the deconfigured directory scenario
in your last question to address this situation too.
When handling a broker registration request, the controller
can check if OfflineLogDirs=false and any UUIDs are missing
in OnlineLogDirs, compared with the previous registrat
Hi everyone,
After a conversation with Colin McCabe and Proven Provenzano yesterday,
we decided that the benefits outweigh the concerns with the overhead
of associating a directory UUID to every replica in the metadata
partition records.
i.e. We prefer to always associate the log dir UUID even whe
Hi everyone,
Earlier today Colin, Ron, Proven and I had a chat about this work.
We discussed several aspects which I’d like to share here.
## A new reserved UUID
We'll reserve a third UUID to indicate an unspecified dir,
but one that is known to be selected. As opposed to the
default UNKNOWN_DIR
Hi David,
Thanks for shedding light on migration goals, makes sense.
Your preference for option a) makes it even more attractive.
We'll keep that as the preferred approach, thanks for the advice.
> One question with this approach is how the KRaft controller learns about
> the multiple log directo
Hi Colin,
> I would call #2 LOST. It was assigned in the past, but we don't know where.
> I see that you called this OFFLINE). This is not really normal...
> it should happen only when we're migrating from ZK mode to KRaft mode,
> or going from an older KRaft release with multiple directories to a
Hi all,
I think at least one of those is my fault, apologies.
I'll try to make sure all my tests are passing from now on.
It doesn't help that GitHub always shows that the tests have failed,
even when they have not. I suspect this is because Jenkins always
marks the builds as unstable, even when
y is
offline is now more of an open question. It's unclear if this will
actually be necessary.
Please share if you have any thoughts.
Best,
--
Igor
On Tue, Oct 10, 2023, at 5:28 AM, Igor Soarez wrote:
> Hi Colin,
>
> Thanks for the renaming suggestions. UNASSIGNED is better then
Hi Omnia, Hi Claude,
Thanks for putting this KIP together.
This is an important unresolved issue in Kafka,
which I have witnessed several times in production.
Please see my questions below:
10 Given the goal is to prevent OOMs, do we also need to
limit the number of KafkaPrincipals in use?
11.
Hi Claude,
Thanks for writing this KIP. This issue seems particularly
thorny, and I appreciate everyone's effort to address this.
I want to share my concern with the KIP's proposal of the
use of memory mapped files – mmap is Java's achilles heel,
Kafka should make less use of it, not more.
The J
Hi Claude,
MappedByteBuffer is the fastest, but allows for this global pause.
RandomAccessFile or FileChannel (without .map) will be slower, but involve a
syscall for IO. Because threads get marked safe before entering a syscall, any
delays in the operation affect just that thread, not the whol
Hi Justine,
I'm sorry this release is delayed. A few new blockers have come up and we're
working through them.
Here's the release plan:
https://cwiki.apache.org/confluence/display/KAFKA/Release+plan+3.7.1
Best,
--
Igor
3.7 branch:
Docker Build Test Pipeline:
https://github.com/apache/kafka/actions/runs/9455339546
/**
Thanks,
--
Igor Soarez
Now attaching the container build reports, which I seem to have forgotten to
include before.
--
Igor
kafka/test:test (alpine 3.19.1)
===
Total: 0 (HIGH: 0, CRITICAL: 0)
Hi Edoardo,
It is late, but not too late. I have cherry-picked your change
to the 3.7 branch and I'll build a second release candidate.
If you could have a look at the first RC, please let me know if
you spot any issues with it that can be avoided in the next RC.
Thanks,
--
Igor
**
Thanks,
--
Igor Soarez
Hi Luke, I was just waiting for "please vote by" date in the first email.
Hi all,
Thank you all for having a look at this RC and voting.
I'm now closing the vote.
The vote passes with:
- 3 +1 binding votes from Mickael, Luke and Justine
- 1 +1 non-binding votes from Jakub
- 0 -1 votes
I'll mov
Zhang, Chia-Ping Tsai, Chris
Egerton, Colin Patrick McCabe, David Arthur, David Jacot, Divij Vaidya, Dmitry
Werner, Edoardo Comar, flashmouse, Florin Akermann, Gantigmaa Selenge, Gaurav
Narula, Greg Harris, Igor Soarez, ilyazr, Ismael Juma, Jason Gustafson, Jeff
Kim, jiangyuan, Joel Hamill, John Yu
A correction to a funny mistake in my previous message:
That's *61 contributors* to this release, not 1086!
--
Igor Soarez
On Mon, Jul 1, 2024, at 11:25 AM, Igor Soarez wrote:
> The Apache Kafka community is pleased to announce the release for Apache
> Kafka 3.7.1
>
>
Hi Manikumar,
Thanks for pointing this out. The apache/kafka:latest has now been updated from
apache/kafka:3.7.1.
Best,
--
Igor
Hi all,
I had left a comment on the JIRA earlier, and thanks to Mickael I was made
aware of this thread, so I'll share it here too:
I agree it makes sense to make this information available through RPCs,
but the Quorum is probably not the right place to represent registered (but
inactive
Hi Josep,
Thanks for running this release.
+1 non-binding. I performed the following checks:
- Verified signatures and checksums
- Built from source
- Basic workload against a KRaft cluster using the artifacts for both Scala
versions
- Browsed Javadoc
- Checked LICENSE
Best,
--
Igor
Hi Josep,
Thanks for running this release.
+1 non-binding.
I performed the following checks:
- Verified signatures and checksums
- Built from source
- Basic workload against a KRaft cluster using the artifacts for Scala 2.13
- Browsed Javadoc
- Checked LICENSE
- Ran and cecked Docker image con
Hi Josep,
Thank you once again for running this release.
I noticed that site-docs/upgrade.html in kafka_2.13-3.8.0-site-docs.tgz
does not include the corrections from
https://github.com/apache/kafka-site/pull/614
namely the replacement of version 3.6 with 3.7 in the upgrade notes header,
and the
Hi Josep,
That makes sense to me, thanks for clarifying.
+1 non-binding from me then.
--
Igor
My understanding was that the reason for the shorter cycle
to the 3.9 release was based on the assumption that KIP-1012
would be ready soon, so we could get to 4.0 quicker.
If we can't move to 4.0 sooner, what's to gain with an early 3.9?
--
Igor
Hi Omnia,
Thanks for this KIP, this is a valuable improvement, and
apologies for my late feedback on this.
IS1: Without an indication of how many pages or items are left,
it is impossible for the client to calculate progress.
Having the cursor in the response just indicates whether there
are more
t; API. Would it make sense to authorize these requests as other inter-broker
> protocol calls are usually authorized, that is ClusterAction on Cluster
> resource?
>
> Thanks,
> Viktor
>
> On Tue, Nov 28, 2023 at 4:18 PM Igor Soarez wrote:
>
> > Hi everyone,
> &g
Hi Omnia,
Thanks for this KIP.
11. These seem to me to be small misspellings, please double-check:
s/MM2 main features/MM2's main features
s/syncing consumer group offset/syncing consumer group offsets
s/relays/relies
s/recored's offset/recorded offsets
s/clusters without need for/clusters withou
Hi Omnia,
Thanks for your answers, and I see you've updated the KIP so thanks for the
changes too.
+1 (binding), thanks for the KIP
--
Igor
Hi everyone,
I'd like to volunteer to be the release manager for a 3.7.1 release.
Please keep in mind, this would be my first release, so I might have some
questions,
and it might also take me a bit longer to work through the release process.
So I'm thinking a good target would be toward the end
Thanks everyone, I'm very honoured to join!
--
Igor
Congratulations Lianet!
--
Igor
Congratulations Josep!
--
Igor
Congratulations Jeff!
--
Igor
JIRA first? Create a KIP first?
Since the existing policy makes little sense IMO, should it be changed
straightwaway or should we have an option to activate the correct behavior and
keep the existing policy as default?
--
Igor Soarez
Apologies, never mind this. I realize now I've confused Log objects with Topic
Partitions.
So since Log size is constant, number of logs in the log directory is actually
a good criteria.
--
Igor Soarez
On Thu, May 7, 2020, at 4:41 PM, Igor Soarez wrote:
>
> When running Kafka wi
Hi Jason,
Apologies for the delay in this reply.
Thank you for having having a look at this KIP and sharing your suggestions.
> 1. (nit): Instead of "storage id," maybe we could call it "directory id"?
> It seems a little clear since each log dir gets a unique id.
I agree, "directory id" is a m
Hi Ziming,
I'm sorry it took me a while to reply.
Thank you for having a look at this KIP and providing feedback.
> 1. We have a version field in meta.properties, currently it’s 1, and we can
> set it to 2 in this KIP, and we can give an example of server.properties and
> it’s corresponding meta
Hi Pere,
Thanks for the KIP. I think this would be a useful addition.
Best,
--
Igor
On Thu, Jan 14, 2021, at 12:11 PM, Rajini Sivaram wrote:
> Hi Pere,
>
> Thanks for the KIP, it is a useful metric. Btw, in the `Public Interfaces`
> section, can we just document the metric names since the code
4 to 3.5 it will not be empty. And maybe we can also
> remove `LogDirectoriesOfflineRequest` you proposed in this KIP.
>
> --
> Best,
> Ziming
>
>
>> On Aug 18, 2022, at 11:24 PM, Igor Soarez wrote:
>>
>> Hi Ziming,
>>
>> I'm sorry
Thanks Jose and David for running this patch release. Congratulations to all!
I don't see the tag or the usual commit sequence in the 3.3 branch for this
release. I'd expect a `3.3.1` and a commit moving the version to
3.3.2-SNAPSHOT. The latest commit in the 3.3 branch still has
`version=3.3.1
Hi Colin,
> I agree controlled shutdown is tricky to get just right. I suppose this is a
> case where the RPCs we send out are not purely "fire and forget"; we have to
> listen for the response. But that can be done in an event-based way.
> Controlled shutdown is also probably the last thing we
Hi David,
Thanks for the KIP, this is very exciting!
How does JBOD relate to this work? KRaft mode doesn't yet support configuring
brokers with multiple log directories. If the brokers in the existing cluster
are configured with multiple log dirs, does the migration imply that the
existing bro
Hello,
There’s now a proposal to address ZK to KRaft migration — KIP-866 — but
currently it doesn't address JBOD so I've decided to update this proposal to
address that migration scenario.
So given that:
- When migrating from a ZK cluster running JBOD to KRaft, brokers registering
in KRaft mo
controller need to
> keep track of the on- and off-line directories anyway. i.e. We could add
> LogDirsOfflined and LogDirsOnlined fields to both request and response and
> have the broker keep including a log dir in requests until acknowledged in
> the response, but otherwise they'
Hi David,
Zookeeper mode writes meta.properties with version=0. KRaft mode requires
version=1 in meta.properties.
Will a manual step be required to update meta.properties or will brokers
somehow update meta.properties files to version 1?
Thanks,
--
Igor
Hi Jun,
Thank you for reviewing the KIP. Please find my replies to
your comments below.
10. Thanks for pointing out this typo; it has been corrected.
11. I agree that the additional delay in switching to the
future replica is undesirable, however I see a couple of
issues if we forward the requ
Hi Jun,
Thank you for having another look.
11. That is correct. I have updated the KIP in an attempt to make this clearer.
I think the goal should be to try to minimize the chance that a log directory
may happen while the metadata is incorrect about the log directory assignment,
but also have a f
Hi Tom,
Thank you for having another look.
20. Upon a downgrade to a Kafka version that runs the current
"version == 1" assertion, then yes — a downgrade would not be possible
without first updating (manually) the meta.properties files back
to the previous version.
We could prevent this issue if
Hi Tom,
Thank you for having another look.
20. That is a good point.
Thinking about your suggestion:
How would this look like in a non-JBOD KRraft cluster upgrade to JBOD mode?
Upgrading to version that includes the JBOD support patch would automatically
update meta.properties to include the n
Hi Jun,
Thank you for your comments and questions.
30. Thank you for pointing this out. The isNew flag is not available
in KRaft mode. The broker can consider the metadata records:
If, and only if, the logdir assigned is Uuid.ZERO then the replica can
be considered new.
Being able to determine i
Hi David,
Thank you for your suggestions and for having a look at this KIP.
1. Yes, that should be OK. I have updated the section
"Migrating a cluster in ZK mode running with JBOD" to reflect this.
2. I've updated the motivation section to state that.
Best,
--
Igor
Can someone review this pull request please?
https://github.com/apache/kafka/pull/9577
It has been open for quite some time, and it seems pinging folks directly on
GitHub has not worked.
Thanks,
--
Igor
Hi Colin,
It's very exciting to see KRaft ready for production!
I see the value of marking it ready for production even with the current
missing features.
However, I'm worried about marking ZK mode as deprecated before these missing
features are ready. It might be hard to estimate right now h
Hi Mickael,
Thanks for writing this KIP. This would be a very useful improvement!
--
Igor
On Thu, Apr 7, 2022, at 10:16 AM, Mickael Maison wrote:
> Hi,
>
> I wrote a small KIP to expose the total and usable space of logdirs
> via the DescribeLogDirs API:
> https://cwiki.apache.org/confluence/dis
Hi Matthew,
Thanks for submitting this KIP.
This is a useful improvement.
+1 non binding
Best,
--
Igor
On Fri, Apr 22, 2022, at 12:01 PM, Tom Bentley wrote:
> Hi Matthew,
>
> Thanks for the KIP, +1 (binding).
>
> Kind regards,
>
> Tom
>
> On Thu, 14 Apr 2022 at 12:15, Matthew de Detrich
> wrot
I think I have a fix for this in https://github.com/apache/kafka/pull/12174
Please take a look
--
Igor
On Tue, May 17, 2022, at 9:00 AM, Lim Qingwei wrote:
> Thanks for all the tips, I'll look into KAFKA-13907.
>
> On Tue, May 17, 2022 at 3:40 AM deng ziming
> wrote:
>
>> You can help to solve t
Thanks for this KIP Mickael.
+1 non binding
--
Igor
On Tue, May 17, 2022, at 2:48 PM, Luke Chen wrote:
> Hi Mickael,
>
> +1 (binding) from me.
> Thanks for the KIP!
>
> Luke
>
> On Tue, May 17, 2022 at 9:30 PM Mickael Maison
> wrote:
>
>> Hi,
>>
>> I'd like to start a vote on KIP-827. It propos
Hi,
This can also be quite useful to make better use of existing functionality in
the Kafka API — moving replicas between log directories via
ALTER_REPLICA_LOG_DIRS. If usable space information is also available the
caller can make better decisions using the same API. It means a more consistent
Hi all,
I have proposal to handle JBOD disk failures in KRaft mode.
With KIP-833 KRaft is being marked production ready and ZK mode is being
deprecated but support for JBOD is still a big feature that's missing.
Please have a look and share your thoughts:
https://cwiki.apache.org/confluence/di
Hi José,
Thanks for having a look at this KIP and thanks for pointing this out, I've had
a look at KIP-856.
It's good to see there's some overlap in our proposals. we're both proposing:
- Identifying log directories with a UUID
- Extending the storage tool to ensure each log directory has a UUI
Hi José
Thanks for the KIP.
1. In the motivation section you refer to a possible scenario where "it is
possible for the cluster metadata partition to establish a leader with
inconsistency or data loss" and that "the user needs to make sure that the new
voter node has a superset of the previous
Hi all,
I’ve had to step away from work for personal reasons for a couple of months –
until mid April 2023. I don’t think I’ll be able to continue to address
feedback or update this KIP before then.
--
Igor
Hi Jun,
Thank you for sharing your questions, please find my answers below.
41. There can only be user partitions on `metadata.log.dir` if that log
dir is also listed in `log.dirs`.
`LogManager` does not specifically load contents from `metadata.log.dir`.
The broker will communicate UUIDs to th
Thank you for another review Ziming, much appreciated!
1. and 2. You are correct, it would be a big and perhaps strange difference.
Since our last exchange of emails, the proposal has changed and now it
does follow your suggestion to bump metadata.version.
The KIP mentions it under "Compatibility,
My impression is also that a lot of users run older,
out of EOL, versions of Kafka.
The final 3.x version is particularly concerning, as it will be
the last bridge to migrate away from ZK. If a big portion of users
only upgrade after its EOL period, we might only then discover an
important bug and
Hi Christo,
Thank you for your interest in this KIP.
Indeed, I'd like to open up voting ASAP.
I'm hoping there will still be a bit more feedback,
but if not I'll probably request a vote next week or so.
Do you have any concerns or suggestions regarding this KIP?
I'll have a look at your KIP too
Hi everyone,
Someone suggested at the recent Kafka Summit that it may be useful
to have a video call to discuss remaining concerns.
I'm proposing we have a video call Monday 29th May 16:30-17:00 UTC.
If you'd like to join, please reply to the thread or to me directly so
I can send you a link.
P
Hi Divij, Christo,
Thank you for pointing that out.
Let's aim instead for Monday 5th of June, at the same time – 16:30-17:00 UTC.
Please let me know if this doesn't work either.
Best,
--
Igor
Hi Alexandre,
Thank you for having a look at this KIP, and thank you for pointing this out.
I like the idea of expanding the health status of a log directory beyond
just online/offline status.
This KIP currently proposes a single logdir state transition, from
online to offline, conveyed in a lis
Hi all,
I have created a TLA+ specification for this KIP, available here:
https://github.com/soarez/kafka/blob/kip-858-tla-plus/tla/Kip858.tla
If there are no further comments I'll start
a voting thread next week.
--
Igor
Hi Christo,
Thank you for the KIP. Kafka is very sensitive to filesystem errors,
and at the first IO error the whole log directory is permanently
considered offline. It seems your proposal aims to increase the
robustness of Kafka, and that's a positive improvement.
I have some questions:
11. "In
Hi all,
We just had a video call to discuss this KIP and I just wanted
update this thread with a note on the meeting.
Attendees:
- Igor
- Christo
- Divij
- Colt
Items discussed:
- Context, motivation and overview of the proposal.
- How log directories are identified by each Broker.
- How old v
Thanks for the KIP.
Seems straightforward, LGTM.
Non binding +1.
--
Igor
Hi everyone,
We're getting closer to dropping ZooKeeper support, and JBOD
in KRaft mode is one of the outstanding big missing features.
It's been a while since there was new feedback on KIP-858 [1]
which aims to address this gap, so I'm calling for a vote.
A huge thank you to everyone who has re
Congratulations Divij!
--
Igor
Hi Colin,
Thanks for your support with getting this over the line and that’s
great re the preliminary pass! Thanks also for sharing your
thoughts, I've had a careful look at each of these and sharing my
comments below.
I agree, it is important to avoid a perf hit on non-JBOD.
I've opted for tagge
Hi Colin,
Thanks for your questions.
Please have a look at my answers below.
> In the previous email I asked, "who is responsible for assigning replicas to
> broker directories?" Can you clarify what the answer is to that? If the
> answer is the controller, there is no need for an "unknown" s
Another +1 here, also non-binding.
Thank you Omnia!
--
Igor
On Fri, Apr 30, 2021, at 3:15 PM, Ryanne Dolan wrote:
> +1 (non-binding), thanks!
>
> On Thu, Jan 21, 2021, 4:31 AM Omnia Ibrahim wrote:
>
>> Hi
>> Can I get a vote on this, please?
>>
>> Best
>> Omnia
>>
>> On Tue, Dec 15, 2020 a
Hi all,
First of all, apologies for digging up this year-old thread.
I believe that without further changes we will be losing support for a couple
of important SCRAM management scenarios after the transition to a
Zookeeper-less Kafka.
One of the scenarios is a migration of a cluster. Topics an
Maybe it would be nice if it was possible to hook into or extend Admin client
interactions, to allow for custom logic supporting use cases such as this.
Scenarios where topic/resource management is centralized are probably not
uncommon.
--
Igor
On Sat, Jun 12, 2021, at 9:42 AM, Matthew de Detri
Thanks for the KIP Mickael.
Makes sense. +1 (non-binding)
--
Igor
Hi Konstantine,
Thanks for running this release.
+1 non-binding
I ran the following checks:
- Built from source
- Ran local cluster of 3 nodes with ZooKeeper
- Created a topic, 3 partitions 3 replicas
- Ran a couple producers and consumers
- Verified all logs for any errors
--
Igor
Hi,
Could someone please grant me access in the wiki to create a KIP?
username: soarez
Thanks,
--
Igor
smime.p7s
Description: S/MIME cryptographic signature
Hi all,
I'd like to propose that we simplify the operation of KRaft servers a bit by
removing the requirement to run kafka-storage.sh for new storage directories.
Please take a look at the KIP and provide your feedback:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-785%3A+Automatic+stor
Hi Mason,
Thanks for the KIP.
There is already a KIP-785. Should this be KIP-786?
--
Igor
On Fri, Oct 22, 2021, at 9:26 PM, Mason Legere wrote:
> Hi All,
>
> I would like to start a discussion for the following KIP - aimed at
> providing some insight into when our clients are being throttled.
>
Hi all,
Bumping this thread as it’s been a while.
Looking forward to any kind of feedback, pease take a look.
I created a short PR with a possible implementation -
https://github.com/apache/kafka/pull/11549
--
Igor
> On 18 Oct 2021, at 15:11, Igor Soarez wrote:
>
> Hi all,
>
Hi Talat,
Have you considered using 10x more topics - perhaps using multiple clusters -
and avoid having to do any filtering in the clients?
--
Igor
On Tue, Nov 30, 2021, at 8:16 PM, Talat Uyarer wrote:
> Hi Eric,
>
> Thanks for your comments. My goal is apply filter without any
> serialization
successfully created.
>
> I agree that in testing, it is nice not to have to run a separate command. To
> facilitate this, we could have a bash script that allows developers to start
> up a single node cluster without running kafka-storage.sh. That might be
> helpful. I suppose a do
Hi David,
Apart from the obviously identified TODO items I coudln't find any issues. It
looks great!
--
Igor
On Wed, Dec 15, 2021, at 8:37 AM, David Jacot wrote:
> Thanks, Mickael. I will fix this.
>
> Best,
> David
>
> On Mon, Dec 13, 2021 at 3:55 PM Mickael Maison
> wrote:
>>
>> Hi David,
>
1 - 100 of 183 matches
Mail list logo