Re: [VOTE] 4.0.1 RC2

2025-09-23 Thread Christo Lolov
, and a patch is already > available. Do you want to include the patch in 4.0.1, or should I do the > cherry-pick after 4.0.1 is released? > > Best, > Chia-Ping > > > On 2025/09/11 10:09:00 Christo Lolov wrote: > > Hello Kafka users, developers and client-developers, >

[jira] [Resolved] (KAFKA-8202) StackOverflowError on producer when splitting batches

2025-09-20 Thread Christo Lolov (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8202?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Christo Lolov resolved KAFKA-8202. -- Resolution: Fixed > StackOverflowError on producer when splitting batc

[jira] [Resolved] (KAFKA-8350) Splitting batches should consider topic-level message size

2025-09-11 Thread Christo Lolov (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8350?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Christo Lolov resolved KAFKA-8350. -- Resolution: Fixed > Splitting batches should consider topic-level message s

[VOTE] 4.0.1 RC2

2025-09-11 Thread Christo Lolov
Hello Kafka users, developers and client-developers, This is the third candidate for release of Apache Kafka 4.0.1. Since this is a patch version it contains the bug fixes outlined in the release notes below! Release notes for the 4.0.1 release: https://dist.apache.org/repos/dist/dev/kafka/4.0.1

Re: Apache Kafka 4.2 -- Release Manager

2025-09-09 Thread Christo Lolov
Hello! I know I haven't completed the 4.0.1 release yet, but I am quite keen on trying to be the release manager for a minor release in the future. Obviously, I am happy for anybody else with more experience to give this a go first, I just wanted to put my hand up in case we lack other volunteers.

Re: [VOTE] 4.0.1 RC1

2025-09-09 Thread Christo Lolov
t; > - https://github.com/apache/kafka/pull/20470 > - https://github.com/apache/kafka/pull/20484 > - https://github.com/apache/kafka-site/pull/721 > - https://github.com/apache/kafka/pull/20506 > > > -Matthias > > > On 9/3/25 9:13 AM, Christo Lolov wrote: > > Heya M

Re: [VOTE] 4.0.1 RC1

2025-09-03 Thread Christo Lolov
e should get a new > RC? Getting a new RC might be simplest (even if the bug is not a > regression), instead of doing a 4.0.2 bug-fix release as follow up. > > Thoughts? > > Thanks a lot, >-Matthias > > > On 9/3/25 8:14 AM, Christo Lolov wrote: > > Hello! > >

Re: [VOTE] 4.0.1 RC1

2025-09-03 Thread Christo Lolov
ionality > - Verified signatures and checksums of the downloaded artifacts > > +1 (non-binding) > > Thanks for the release! > Hong-Yi Chen > > Christo Lolov 於 2025年8月19日 週二 下午11:43寫道: > > > Hello Kafka users, developers and client-developers, > > > > This is the

Re: [VOTE] 4.0.1 RC1

2025-08-29 Thread Christo Lolov
ve missed the steps > > in > > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=34840886#ReleaseProcess-CreateGraalVMBasedNativeApacheKafkaDockerArtifact(Forversions > > >=3.8.0) > > > > Thanks, > > Mickael > > > > On Tue, Aug 19, 2025 a

Re: [VOTE] 4.1.0 RC2

2025-08-21 Thread Christo Lolov
Hello! I carried out the following checks: * Built Kafka from source using JDK 24 * Ran unit tests * Validated signatures and checksums * Ran the quickstart tutorial * Tested disabling and reenabling of ELR * Tested ELR defaulting to enabled after an upgrade from a 4.0.0 cluster to 4.1.0 * Tested

[VOTE] 4.0.1 RC1

2025-08-19 Thread Christo Lolov
Hello Kafka users, developers and client-developers, This is the second candidate for release of Apache Kafka 4.0.1. Since this is a patch version it contains the bug fixes outlined in the release notes below! Release notes for the 4.0.1 release: https://dist.apache.org/repos/dist/dev/kafka/4.0.

Re: [VOTE] 4.0.1 RC0

2025-08-11 Thread Christo Lolov
you take a look at https://github.com/apache/kafka/pull/20273 and > > let us know what you think? > > > > Thanks. > > > > -Matthias > > > > On 7/25/25 1:51 AM, Christo Lolov wrote: > > > Hello Kafka users, developers and client-developers, > &

Re: [VOTE] 4.1.0 RC1

2025-07-30 Thread Christo Lolov
Heya! I tested creating a cluster using the binary tgz. Furthermore, I went through the enabling, producing, consuming, and disabling of queues. Lastly, I carried out the same steps on both Docker images. I went through the JavaDocs and clicked on links to confirm they work. I chose artifacts at

[VOTE] 4.0.1 RC0

2025-07-25 Thread Christo Lolov
Hello Kafka users, developers and client-developers, This is the first candidate for release of Apache Kafka 4.0.1. Since this is a patch version it contains the bug fixes outlined in the release notes below! Release notes for the 4.0.1 release: https://dist.apache.org/repos/dist/dev/kafka/4.0.1

Re: [DISCUSS] Apache Kafka 4.0.1 release

2025-07-23 Thread Christo Lolov
Hello, The 23rd is now drawing to an end. I am slightly behind schedule, but will be continuing with creating the artifacts tomorrow! Best, Christo On Mon, 30 Jun 2025 at 12:31, Christo Lolov wrote: > Hello! > > I have volunteered to be the release manager for the 4.0.1 release. &

[DISCUSS] Apache Kafka 4.0.1 release

2025-06-30 Thread Christo Lolov
Hello! I have volunteered to be the release manager for the 4.0.1 release. I have put together the release plan at https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+4.0.1 The first release candidate will be cut on the 23rd of July (~3 weeks from the date of this email). Feel free to

Re: 4.0.1 bug-fix release?

2025-06-26 Thread Christo Lolov
Hello! I would like to volunteer as a release manager for 4.0.1! I haven't done this before so I will most likely reach out for help if accepted :) Best, Christo On Wed, 25 Jun 2025 at 16:48, Josep Prat wrote: > But of course, Vivekanand, you are more than welcome to help out testing > and val

Re: [VOTE] KIP-1123: Rack-aware partitioning for Kafka Producer

2025-05-14 Thread Christo Lolov
Hello! Apologies for coming late to the party. I think this is a great idea and don't see why we can't try it out in practice! +1 (binding) Best, Christo On Tue, 29 Apr 2025 at 15:16, Andrew Schofield < andrew_schofield_j...@outlook.com> wrote: > Thanks for the KIP, Ivan. > > +1 (binding) > >

Re: [DISCUSS] KIP-1176: Tiered Storage for Active Log Segment

2025-05-06 Thread Christo Lolov
Hello! It is great to see another proposal on the same topic, but optimising for different scenarios, so thanks a lot for the effort put in this! I have a few questions and statements in no particular order. If you use acks=-1 (acks=all) then an acknowledgement can only be sent to the producer i

Re: [DISCUSS] KIP-1150 Diskless Topics

2025-04-22 Thread Christo Lolov
Hello! I want to start with saying that this is a big and impressive undertaking and I am really excited to see its progression! I am posting my initial comments in this thread, but they span a few of the child KIPs. Let me know which questions you would like to move elsewhere. I understand that y

Re: [ANNOUNCE] New committer: TengYao Chi

2025-04-11 Thread Christo Lolov
Congratulations! On Thu, 10 Apr 2025 at 18:22, David Arthur wrote: > Congrats!! > > David Arthur > > > On Thu, Apr 10, 2025 at 12:43 TengYao Chi wrote: > > > Thank you very much everyone!! > > > > I truly appreciate all your reviews, guidance, and discussions on my code > > contributions. > > Y

[jira] [Resolved] (KAFKA-18694) Migrate suitable classes to recorods in coordinator-common module

2025-02-05 Thread Christo Lolov (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18694?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Christo Lolov resolved KAFKA-18694. --- Resolution: Fixed > Migrate suitable classes to recorods in coordinator-common mod

[jira] [Resolved] (KAFKA-18705) Move ConfigRepository to metadata module

2025-02-05 Thread Christo Lolov (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18705?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Christo Lolov resolved KAFKA-18705. --- Resolution: Fixed > Move ConfigRepository to metadata mod

[jira] [Resolved] (KAFKA-18698) Migrate suitable classes to records in server and server-common modules

2025-02-05 Thread Christo Lolov (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18698?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Christo Lolov resolved KAFKA-18698. --- Resolution: Fixed > Migrate suitable classes to records in server and server-common modu

[jira] [Resolved] (KAFKA-18545) Remove Zookeeper logic from LogManager

2025-02-03 Thread Christo Lolov (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18545?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Christo Lolov resolved KAFKA-18545. --- Resolution: Fixed > Remove Zookeeper logic from LogMana

[jira] [Resolved] (KAFKA-18685) Cleanup DynamicLogConfig constructor

2025-02-03 Thread Christo Lolov (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18685?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Christo Lolov resolved KAFKA-18685. --- Resolution: Fixed > Cleanup DynamicLogConfig construc

Re: [DISCUSS] KIP-1126: Serialize changes to Kafka with a build queue

2025-01-24 Thread Christo Lolov
Heya David, Thanks a lot for this! Building on top of Chris's questions - does including only a subset of the stages in the merge queue mean that while subsequent stages (i.e. tests) can run in parallel-ish across PRs they will contain trunk + set of PRs ahead of this one which passed the stage i

[jira] [Resolved] (KAFKA-18598) Remove ControllerMetadataMetrics zk related Metrics

2025-01-22 Thread Christo Lolov (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18598?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Christo Lolov resolved KAFKA-18598. --- Resolution: Fixed The PR has been reviewed and merged both in trunk and in 4.0 > Rem

[jira] [Resolved] (KAFKA-18595) Remove AuthorizerUtils#sessionToRequestContext

2025-01-22 Thread Christo Lolov (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18595?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Christo Lolov resolved KAFKA-18595. --- Resolution: Fixed The PR has been reviewed and merged both in trunk and 4.0 > Rem

Re: [ANNOUNCE] New committer: Andrew Schofield

2024-12-02 Thread Christo Lolov
Congratulations Andrew, well deserved! On Mon, 2 Dec 2024 at 16:36, Josep Prat wrote: > Congrats Andrew! > > > On Mon, Dec 2, 2024 at 5:33 PM Mickael Maison > wrote: > > > Congratulations Andrew! > > > > Thanks, > > Mickael > > > > On Mon, Dec 2, 2024 at 5:27 PM Kuan Po Tseng > wrote: > > > >

Re: [VOTE] KIP-1058: Txn consumer exerts pressure on remote storage when reading non-txn topic

2024-11-08 Thread Christo Lolov
Heya, Apologies for the delay. You have my +1 vote as well! I am excited to see this making it in 4.0 Best, Christo On Wed, 6 Nov 2024 at 09:43, Kamal Chandraprakash < kamal.chandraprak...@gmail.com> wrote: > Hi all, > > The KIP is passed with 3 (+1) binding votes. Thanks everyone! > > Binding

Re: [VOTE] KIP-1105: Make remote log manager thread-pool configs dynamic

2024-11-08 Thread Christo Lolov
Heya, I have ran a few times into issues which required adjustment of the thread pools and having to carry out a rolling restart was cumbersome. +1 (binding) for the change! Best, Christo On Thu, 7 Nov 2024 at 16:17, Satish Duggana wrote: > +1 (binding) > > On Thu, 7 Nov 2024 at 13:55, Federi

Re: [DISCUSS] KIP-1058: Txn consumer exerts pressure on remote storage when reading non-txn topic

2024-10-04 Thread Christo Lolov
Heya, Apologies for the delay. I have been thinking about this problem recently as well and while I believe storing a boolean in the metadata is good, I think we can do better by introducing a new method to the RLMM along the lines of Optional nextRemoteLogSegmentMetadataWithTxnIndex(TopicIdParti

Re: [ANNOUNCE] New committer: Kamal Chandraprakash

2024-10-04 Thread Christo Lolov
Many congratulations Kamal! Very well deserved! On Tue, 1 Oct 2024 at 08:21, Jorge Esteban Quilcate Otoya < quilcate.jo...@gmail.com> wrote: > Congrats Kamal!! > > On Tue 1. Oct 2024 at 7.54, Kamal Chandraprakash < > kamal.chandraprak...@gmail.com> wrote: > > > Thank you all for your warm wishes!

Re: New release branch 3.9

2024-09-20 Thread Christo Lolov
Hello, I have filed https://issues.apache.org/jira/browse/KAFKA-17584 as a blocker. While it has not been introduced by a KIP, I think this has durability implications and needs to be addressed immediately. Let me know if you disagree! Best, Christo On Thu, 19 Sept 2024 at 16:01, José Armando G

[jira] [Created] (KAFKA-17584) Changing dynamic configurations resets retention

2024-09-20 Thread Christo Lolov (Jira)
Christo Lolov created KAFKA-17584: - Summary: Changing dynamic configurations resets retention Key: KAFKA-17584 URL: https://issues.apache.org/jira/browse/KAFKA-17584 Project: Kafka Issue

[jira] [Resolved] (KAFKA-17579) Dynamic LogCleaner configurations are not picked up upon restart

2024-09-20 Thread Christo Lolov (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17579?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Christo Lolov resolved KAFKA-17579. --- Resolution: Fixed > Dynamic LogCleaner configurations are not picked up upon rest

[jira] [Created] (KAFKA-17579) Dynamic LogCleaner configurations are not picked up upon restart

2024-09-19 Thread Christo Lolov (Jira)
Christo Lolov created KAFKA-17579: - Summary: Dynamic LogCleaner configurations are not picked up upon restart Key: KAFKA-17579 URL: https://issues.apache.org/jira/browse/KAFKA-17579 Project: Kafka

Re: [VOTE] KIP-1075: Introduce delayed remote list offsets purgatory to make LIST_OFFSETS async

2024-08-28 Thread Christo Lolov
Hello! Apologies for the late arrival. I read through the KIP and the discussion thread and I have no concerns with the proposed approach. This is a great improvement! +1 from me. Best, Christo On Fri, 23 Aug 2024 at 02:50, Luke Chen wrote: > +1 from me. > > Thanks. > > On Thu, Aug 22, 2024 a

Re: [VOTE] KIP-950: Tiered Storage Disablement

2024-07-25 Thread Christo Lolov
ture. However, I > am > >> a bit sceptical on the ZK part of the story. The 3.8 release is supposed > >> to > >> be the last one supporting ZK so I don't really see how we could bring > it > >> to ZK, knowing that we don't plan to do a 3.9 release (cur

Re: [DISCUSS] KIP-950: Tiered Storage Disablement

2024-07-16 Thread Christo Lolov
ood if you could add that into the KIP. > > > > Otherwise, LGTM. > > > > Thank you. > > Luke > > > > On Mon, May 13, 2024 at 11:55 PM Christo Lolov > > wrote: > > > > > Heya! > > > > > > re Kamal - Okay, I believe I un

[jira] [Created] (KAFKA-17031) Make configurations public

2024-06-24 Thread Christo Lolov (Jira)
Christo Lolov created KAFKA-17031: - Summary: Make configurations public Key: KAFKA-17031 URL: https://issues.apache.org/jira/browse/KAFKA-17031 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-16857) Zookeeper - Add new ZNodes

2024-05-28 Thread Christo Lolov (Jira)
Christo Lolov created KAFKA-16857: - Summary: Zookeeper - Add new ZNodes Key: KAFKA-16857 URL: https://issues.apache.org/jira/browse/KAFKA-16857 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-16856) Zookeeper - Add new exception

2024-05-28 Thread Christo Lolov (Jira)
Christo Lolov created KAFKA-16856: - Summary: Zookeeper - Add new exception Key: KAFKA-16856 URL: https://issues.apache.org/jira/browse/KAFKA-16856 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-16855) KRaft - Wire replaying a TopicRecord

2024-05-28 Thread Christo Lolov (Jira)
Christo Lolov created KAFKA-16855: - Summary: KRaft - Wire replaying a TopicRecord Key: KAFKA-16855 URL: https://issues.apache.org/jira/browse/KAFKA-16855 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-16854) Zookeeper - Add v5 of StopReplica

2024-05-28 Thread Christo Lolov (Jira)
Christo Lolov created KAFKA-16854: - Summary: Zookeeper - Add v5 of StopReplica Key: KAFKA-16854 URL: https://issues.apache.org/jira/browse/KAFKA-16854 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-16853) Split RemoteLogManagerScheduledThreadPool

2024-05-28 Thread Christo Lolov (Jira)
Christo Lolov created KAFKA-16853: - Summary: Split RemoteLogManagerScheduledThreadPool Key: KAFKA-16853 URL: https://issues.apache.org/jira/browse/KAFKA-16853 Project: Kafka Issue Type: Sub

[jira] [Created] (KAFKA-16851) Add remote.log.disable.policy

2024-05-28 Thread Christo Lolov (Jira)
Christo Lolov created KAFKA-16851: - Summary: Add remote.log.disable.policy Key: KAFKA-16851 URL: https://issues.apache.org/jira/browse/KAFKA-16851 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-16852) Add *.thread.pool.size

2024-05-28 Thread Christo Lolov (Jira)
Christo Lolov created KAFKA-16852: - Summary: Add *.thread.pool.size Key: KAFKA-16852 URL: https://issues.apache.org/jira/browse/KAFKA-16852 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-16850) KRaft - Add v2 of TopicRecord

2024-05-28 Thread Christo Lolov (Jira)
Christo Lolov created KAFKA-16850: - Summary: KRaft - Add v2 of TopicRecord Key: KAFKA-16850 URL: https://issues.apache.org/jira/browse/KAFKA-16850 Project: Kafka Issue Type: Sub-task

Re: [VOTE] KIP-950: Tiered Storage Disablement

2024-05-24 Thread Christo Lolov
Hello! I am closing this vote as ACCEPTED with 3 binding +1 (Luke, Chia-Ping and Satish) and 1 non-binding +1 (Kamal) - thank you for the reviews! Realistically, I don't think I have the bandwidth to get this in 3.8.0. Due to this, I will mark tentatively the Zookeeper part for 3.9 if the communi

[jira] [Created] (KAFKA-16790) Calls to RemoteLogManager are made before it is configured

2024-05-17 Thread Christo Lolov (Jira)
Christo Lolov created KAFKA-16790: - Summary: Calls to RemoteLogManager are made before it is configured Key: KAFKA-16790 URL: https://issues.apache.org/jira/browse/KAFKA-16790 Project: Kafka

[VOTE] KIP-950: Tiered Storage Disablement

2024-05-14 Thread Christo Lolov
Heya! I would like to start a vote on KIP-950: Tiered Storage Disablement in order to catch the last Kafka release targeting Zookeeper - https://cwiki.apache.org/confluence/display/KAFKA/KIP-950%3A++Tiered+Storage+Disablement Best, Christo

Re: [DISCUSS] KIP-950: Tiered Storage Disablement

2024-05-13 Thread Christo Lolov
ink infinite retries is a good idea since if a broker is down > forever, this request will never complete. > You mentioned the existing topic deletion is using the similar pattern, how > does it handle this issue? > > Thanks. > Luke > > On Thu, May 9, 2024 at 9:21 PM Christo Lol

Re: [DISCUSS] KIP-950: Tiered Storage Disablement

2024-05-09 Thread Christo Lolov
ller, not listening for > changes in > >>>> ZK. > >>>> > >>>> 3. In the KRaft handler steps, you said: > >>>> "The controller also updates the Topic metadata to increment the > >>>> tiered_epoch and update the tiere

Re: [VOTE] KIP-1018: Introduce max remote fetch timeout config

2024-05-09 Thread Christo Lolov
Heya Kamal, Thanks for the KIP and the answers in the discussion! +1 from me :) Best, Christo On Thu, 9 May 2024 at 11:11, Federico Valeri wrote: > +1 non binding > > Thanks > > On Thu, May 9, 2024 at 12:05 PM Luke Chen wrote: > > > > Hi Kamal, > > > > Thanks for the KIP! > > +1 from me. > >

Re: [DISCUSS] KIP-1018: Introduce max remote fetch timeout config

2024-04-29 Thread Christo Lolov
Heya! Is it difficult to instead add the metric at kafka.network:type=RequestMetrics,name=TieredStorageMs (or some other name=*)? Alternatively, if it is difficult to add it there, is it possible to add 2 metrics, one at the RequestMetrics level (even if it is total-time-ms - (all other times)) an

Re: [DISCUSS] KIP-1018: Introduce max remote fetch timeout config

2024-04-27 Thread Christo Lolov
Heya Kamal, I quite like the proposal and would support it! However, today I don't think we have a metric which shows the latency of fetch requests which are served from remote, am I wrong? I looked at both https://github.com/clolov/kafka/blob/trunk/core/src/main/scala/kafka/network/RequestChanne

Re: [VOTE] KIP-1023: Follower fetch from tiered offset

2024-04-26 Thread Christo Lolov
Heya Abhijeet, Thanks a lot for pushing this forward, especially with the explanation of EARLIEST_PENDING_UPLOAD_OFFSET_TIMESTAMP! +1 from me :) Best, Christo On Fri, 26 Apr 2024 at 12:50, Luke Chen wrote: > Hi Abhijeet, > > Thanks for the KIP. > +1 from me. > > Thanks. > Luke > > On Fri, Apr

Re: [ANNOUNCE] New committer: Igor Soarez

2024-04-25 Thread Christo Lolov
Congratulations Igor :) ! On Thu, 25 Apr 2024 at 17:07, Igor Soarez wrote: > Thanks everyone, I'm very honoured to join! > > -- > Igor >

Re: [VOTE] KIP-1037: Allow WriteTxnMarkers API with Alter Cluster Permission

2024-04-22 Thread Christo Lolov
Heya Nikhil, Thanks for the proposal, as mentioned before it makes sense to me! +1 (binding) Best, Christo On Sat, 20 Apr 2024 at 00:25, Justine Olshan wrote: > Hey Nikhil, > > I meant to comment on the discussion thread, but my draft took so long, you > opened the vote. > > Regardless, I jus

Re: [DISCUSS] KIP-950: Tiered Storage Disablement

2024-04-19 Thread Christo Lolov
would have to use already existing methods Let me know if there are any new comments or I have missed something! Best, Christo On Mon, 15 Apr 2024 at 12:40, Christo Lolov wrote: > Heya Doguscan, > > I believe that the state of the world after this KIP will be the following: > >

Re: [DISCUSS] KIP-950: Tiered Storage Disablement

2024-04-15 Thread Christo Lolov
on from DisableRemoteTopic >> within the StopReplicas API which will then disappear in KRaft world as it >> is already scheduled for deprecation. What do you think? >> >> Sounds good to me. >> >> Thanks. >> Luke >> >> On Tue, Apr 9, 2024 at 6:46 PM

Re: [DISCUSS] KIP-1037: Allow WriteTxnMarkers API with Alter Cluster Permission

2024-04-15 Thread Christo Lolov
Heya Nikhil, Thank you for raising this KIP! Your proposal makes sense to me. In essence you are saying that the permission required by WriteTxnMarkers should be the same as for CreateAcls and DeleteAcls, which is reasonable. If we trust an administrator to assign the correct permissions then we

Re: [ANNOUNCE] New Kafka PMC Member: Greg Harris

2024-04-15 Thread Christo Lolov
Congratulations, Greg :) On Mon, 15 Apr 2024 at 07:34, Zhisheng Zhang <31791909...@gmail.com> wrote: > Congratulations Greg! > > > Manikumar 于2024年4月15日周一 13:49写道: > > > Congratulations, Greg. > > > > On Mon, Apr 15, 2024 at 11:18 AM Bruno Cadonna > wrote: > > > > > > Congratulations, Greg! > >

Re: [DISCUSS] KIP-950: Tiered Storage Disablement

2024-04-09 Thread Christo Lolov
tarting doing unexpected upload/fetch tasks, right? > I agree we should make sure in ZK mode, all brokers are completed the > stopPartitions before moving to "Disabled" state because ZK node watcher is > working in a separate thread. But not sure about KRaft mode. > > Thanks

[jira] [Created] (KAFKA-16480) ListOffsets change should have an associated API/IBP version update

2024-04-05 Thread Christo Lolov (Jira)
Christo Lolov created KAFKA-16480: - Summary: ListOffsets change should have an associated API/IBP version update Key: KAFKA-16480 URL: https://issues.apache.org/jira/browse/KAFKA-16480 Project: Kafka

Re: [DISCUSS] KIP-1023: Follower fetch from tiered offset

2024-04-05 Thread Christo Lolov
> > > Hi Christo, > > > > Thanks for reviewing the KIP. > > > > The follower needs the earliest-pending-upload-offset (and the > > corresponding leader epoch) from the leader. > > This is the first offset the follower will have locally. > > > &

Re: [DISCUSS] KIP-950: Tiered Storage Disablement

2024-03-29 Thread Christo Lolov
ending Controller -> Broker RPCs. In fact, KIP-631 deprecated these > > RPCs. > > > > > > > > I will come up with some recommendations on how we could improve that > > > one but until then, @Luke please feel free to review the KIP. > > > > > > > > @Satish, if we wan

Re: [DISCUSS] KIP-1023: Follower fetch from tiered offset

2024-03-29 Thread Christo Lolov
Heya! First of all, thank you very much for the proposal, you have explained the problem you want solved very well - I think a faster bootstrap of an empty replica is definitely an improvement! For my understanding, which concrete offset do you want the leader to give back to a follower - earlies

Re: [ANNOUNCE] New committer: Christo Lolov

2024-03-26 Thread Christo Lolov
Thank you everyone! It wouldn't have been possible without quite a lot of reviews and extremely helpful inputs from you and the rest of the community! I am looking forward to working more closely with you going forward :) On Tue, 26 Mar 2024 at 14:31, Kirk True wrote: > Congratulations Christo!

Re: [DISCUSS] KIP-950: Tiered Storage Disablement

2024-03-18 Thread Christo Lolov
Duggana > wrote: > > > Hi Christo, > > Thanks for volunteering to contribute to the KIP discussion. I suggest > > considering this KIP for both ZK and KRaft as it will be helpful for > > this feature to be available in 3.8.0 running with ZK clusters. > > > > Thank

Re: [DISCUSS] KIP-950: Tiered Storage Disablement

2024-01-17 Thread Christo Lolov
Hello! I volunteer to get this KIP moving forward and implemented in Apache Kafka 3.8. I have caught up with Mehari offline and we have agreed that given Apache Kafka 4.0 being around the corner we would like to propose this feature only for KRaft clusters. Any and all reviews and comments are w

[jira] [Created] (KAFKA-16154) Make broker changes to return an offset for LATEST_TIERED_TIMESTAMP

2024-01-17 Thread Christo Lolov (Jira)
Christo Lolov created KAFKA-16154: - Summary: Make broker changes to return an offset for LATEST_TIERED_TIMESTAMP Key: KAFKA-16154 URL: https://issues.apache.org/jira/browse/KAFKA-16154 Project: Kafka

Re: [VOTE] KIP-1005: Expose EarliestLocalOffset and TieredOffset

2024-01-17 Thread Christo Lolov
11 Jan 2024 at 17:52, Divij Vaidya > > > > wrote: > > > > > > > > > > +1 (binding) > > > > > > > > > > Divij Vaidya > > > > > > > > > > > > > > > > > > > > On Tue, Dec 26, 2023

[jira] [Resolved] (KAFKA-15734) KRaft support in BaseConsumerTest

2024-01-16 Thread Christo Lolov (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15734?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Christo Lolov resolved KAFKA-15734. --- Resolution: Fixed > KRaft support in BaseConsumerT

Re: [DISCUSS] KIP-1005: Add EarliestLocalOffset to GetOffsetShell

2024-01-15 Thread Christo Lolov
, if the timestamp greater than recently > committed record timestamp is given.]" > > WDYT? > > Thanks. > Luke > > On Mon, Jan 15, 2024 at 6:46 PM Christo Lolov > wrote: > > > Heya Luke, > > > > Thank for the question! I have expanded in the KIP -

Re: [DISCUSS] KIP-1005: Add EarliestLocalOffset to GetOffsetShell

2024-01-15 Thread Christo Lolov
gt; Thank you for making the change Christo. It looks good to me. > > > > -- > > Divij Vaidya > > > > > > > > On Thu, Jan 11, 2024 at 11:19 AM Christo Lolov > > wrote: > > > > > Thank you Divij! > > > > > > I hav

Re: [DISCUSS] KIP-1005: Add EarliestLocalOffset to GetOffsetShell

2024-01-11 Thread Christo Lolov
; > -- > Divij Vaidya > > > > On Tue, Nov 21, 2023 at 2:19 PM Christo Lolov > wrote: > > > Heya! > > > > Thanks a lot for this. I have updated the KIP to include exposing the > > tiered-offset as well. Let me know whether the Public Interfaces section > >

[VOTE] KIP-1005: Expose EarliestLocalOffset and TieredOffset

2023-12-21 Thread Christo Lolov
Heya all! KIP-1005 ( https://cwiki.apache.org/confluence/display/KAFKA/KIP-1005%3A+Expose+EarliestLocalOffset+and+TieredOffset) has been open for around a month with no further comments - I would like to start a voting round on it! Best, Christo

Re: [VOTE] KIP-1007: Introduce Remote Storage Not Ready Exception

2023-12-21 Thread Christo Lolov
Heya Kamal, The proposed change makes sense to me as it will be a more explicit behaviour than what Kafka does today - I am happy with it! +1 (non-binding) from me Best, Christo On Tue, 12 Dec 2023 at 09:01, Kamal Chandraprakash < kamal.chandraprak...@gmail.com> wrote: > Hi, > > I would like t

[jira] [Created] (KAFKA-16002) Implement RemoteCopyLagSegments, RemoteDeleteLagBytes and RemoteDeleteLagSegments

2023-12-13 Thread Christo Lolov (Jira)
Christo Lolov created KAFKA-16002: - Summary: Implement RemoteCopyLagSegments, RemoteDeleteLagBytes and RemoteDeleteLagSegments Key: KAFKA-16002 URL: https://issues.apache.org/jira/browse/KAFKA-16002

Re: [VOTE] KIP-963: Additional metrics in Tiered Storage

2023-11-23 Thread Christo Lolov
Kamal Chandraprakash > > wrote: > > > > > > +1 (non-binding). Thanks for the KIP! > > > > > > On Tue, Nov 21, 2023, 03:04 Divij Vaidya > > wrote: > > > > > > > + 1 (binding) > > > > > > >

[jira] [Created] (KAFKA-15883) Implement RemoteCopyLagBytes

2023-11-22 Thread Christo Lolov (Jira)
Christo Lolov created KAFKA-15883: - Summary: Implement RemoteCopyLagBytes Key: KAFKA-15883 URL: https://issues.apache.org/jira/browse/KAFKA-15883 Project: Kafka Issue Type: Sub-task

Re: [DISCUSS] KIP-1005: Add EarliestLocalOffset to GetOffsetShell

2023-11-21 Thread Christo Lolov
> highest-copied-remote-offset via > > GetOffsetShell tool? This will be useful during the debugging session. > > > > > > On Mon, Nov 20, 2023 at 5:38 PM Christo Lolov > > wrote: > > > > > Hello all! > > > > > > I would like to start

[DISCUSS] KIP-1005: Add EarliestLocalOffset to GetOffsetShell

2023-11-20 Thread Christo Lolov
Hello all! I would like to start a discussion for https://cwiki.apache.org/confluence/display/KAFKA/KIP-1005%3A+Add+EarliestLocalOffset+to+GetOffsetShell . A new offset called local log start offset was introduced as part of KIP-405: Kafka Tiered Storage. KIP-1005 aims to expose this offset by ch

[VOTE] KIP-963: Additional metrics in Tiered Storage

2023-11-20 Thread Christo Lolov
Hello all! Now that the discussion for KIP-963 has winded down, I would like to open it for a vote targeting 3.7.0 as the release. You can find the current version of the KIP at https://cwiki.apache.org/confluence/display/KAFKA/KIP-963%3A+Additional+metrics+in+Tiered+Storage Best, Christo

Re: [DISCUSS] KIP-963: Upload and delete lag metrics in Tiered Storage

2023-11-17 Thread Christo Lolov
ake those fine grained remote fetch flow sequence > metrics separately later? > > Thanks, > Satish. > > On Tue, 14 Nov 2023 at 22:07, Christo Lolov > wrote: > > > > Heya everyone, > > > > Apologies for the delay in my response and thank you very much for all

Re: [DISCUSS] KIP-963: Upload and delete lag metrics in Tiered Storage

2023-11-14 Thread Christo Lolov
I find the record-based lag less useful as records > > > can't > > > > > be relied as a stable unit of measure. So, if we can keep bytes- > and > > > > > segment-based lag, LGTM. > > > > > 3.4. Agree, these metrics should be on the broker

Re: [DISCUSS] KIP-963: Upload and delete lag metrics in Tiered Storage

2023-10-30 Thread Christo Lolov
other internals) to have some metrics attached -- maybe if you > could shared scenarios where this metrics would be useful would be helpful. > > 7. On the metrics naming: not sure the `Total*` prefix is really needed or > adds meaning. When I found it useful is when there are related metr

Re: [DISCUSS] KIP-963: Upload and delete lag metrics in Tiered Storage

2023-10-24 Thread Christo Lolov
metrics and their granularity! Best, Christo On Fri, 13 Oct 2023 at 10:14, Christo Lolov wrote: > Heya Gantigmaa, > > Apologies for the (very) late reply! > > Now that 3.6 has been released and reviewers have a bit more time I will > be picking up this KIP again. I am more

[jira] [Created] (KAFKA-15660) File-based Tiered Storage should delete folders upon topic deletion

2023-10-20 Thread Christo Lolov (Jira)
Christo Lolov created KAFKA-15660: - Summary: File-based Tiered Storage should delete folders upon topic deletion Key: KAFKA-15660 URL: https://issues.apache.org/jira/browse/KAFKA-15660 Project: Kafka

Re: [DISCUSS] KIP-963: Upload and delete lag metrics in Tiered Storage

2023-10-13 Thread Christo Lolov
ose metrics > were not mentioned in the original KIP-405 or KIP-930. Do you think these > would make sense to be added to this KIP and get included in the > discussion? > > Regards, > Gantigmaa > > On Wed, Aug 9, 2023 at 1:53 PM Christo Lolov > wrote: > > > Heya

[jira] [Resolved] (KAFKA-15385) Replace EasyMock with Mockito for AbstractStreamTest

2023-10-10 Thread Christo Lolov (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15385?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Christo Lolov resolved KAFKA-15385. --- Resolution: Duplicate Closing this in favour of https://issues.apache.org/jira/browse/KAFKA

[jira] [Resolved] (KAFKA-15382) Replace EasyMock with Mockito for KStreamTransformValuesTest

2023-10-10 Thread Christo Lolov (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15382?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Christo Lolov resolved KAFKA-15382. --- Resolution: Duplicate Closing this ticket in favour of https://issues.apache.org/jira

[jira] [Resolved] (KAFKA-15383) Replace EasyMock with Mockito for KTableImplTest

2023-10-10 Thread Christo Lolov (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15383?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Christo Lolov resolved KAFKA-15383. --- Resolution: Duplicate Closing this ticket in favour of https://issues.apache.org/jira

[jira] [Resolved] (KAFKA-15384) Replace EasyMock with Mockito for KTableTransformValuesTest

2023-10-10 Thread Christo Lolov (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15384?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Christo Lolov resolved KAFKA-15384. --- Resolution: Fixed Closing this ticket in favour of https://issues.apache.org/jira/browse

Re: [VOTE] 3.6.0 RC0

2023-09-19 Thread Christo Lolov
Heya, I have compiled and ran the test target successfully for the 3.6.0-rc0 branch on: Java 11, Scala 2.13 - ARM Java 17, Scala 2.13 - ARM Java 20, Scala 2.18 - ARM Java 11, Scala 2.13 - Intel x86 Java 17, Scala 2.13 - Intel x86 Java 20, Scala 2.13 - Intel x86 I will update the Zookeeper KIP ti

[jira] [Resolved] (KAFKA-15399) Enable OffloadAndConsumeFromLeader test

2023-09-01 Thread Christo Lolov (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15399?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Christo Lolov resolved KAFKA-15399. --- Resolution: Fixed > Enable OffloadAndConsumeFromLeader t

[jira] [Created] (KAFKA-15427) Integration tests in TS test harness detect resource leaks

2023-09-01 Thread Christo Lolov (Jira)
Christo Lolov created KAFKA-15427: - Summary: Integration tests in TS test harness detect resource leaks Key: KAFKA-15427 URL: https://issues.apache.org/jira/browse/KAFKA-15427 Project: Kafka

  1   2   >