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

2025-05-19 Thread Ivan Yurchenko
t; > > > +1 binding > > > > > > > > Best, > > > > > > > > On Tue, Apr 22, 2025 at 8:56 PM Hector Geraldino (BLOOMBERG/ 919 3RD > > A) < > > > > hgerald...@bloomberg.net> wrote: > > > > > >

Re: [DISCUSS] KIP-1150 Diskless Topics

2025-05-16 Thread Ivan Yurchenko
On Tue, May 13, 2025, at 19:34, Jun Rao wrote: > > JR4. "Balance traffic among brokers and eliminate broker hotspots with > per-client granularity". Does that mean all traffic from a client is served > from a single broker? This seems to reduce the scalability from the client > perspective. We p

Re: [DISCUSS] KIP-1150 Diskless Topics

2025-05-16 Thread Ivan Yurchenko
On Tue, May 13, 2025, at 19:34, Jun Rao wrote: > JR3. "Permit multi-region active-active topics with automatic failover". > Could you elaborate on the benefit of this? Cloud providers still charge > cross region data transfer in object stores, right? First, we should have emphasized in the KIP th

Re: [DISCUSS] KIP-1150 Diskless Topics

2025-05-16 Thread Ivan Yurchenko
Hi Jun, Thank you for your questions! On Tue, May 13, 2025, at 19:34, Jun Rao wrote: > Hi, Josep, > > Thanks for the KIP. At the highlevel, the KIP is well thought through and > provides multiple benefits for Kafka in the Cloud. A few comments below. > > JR1. One of the key motivations is to el

[jira] [Created] (KAFKA-19193) Rack-aware partitioning for Kafka Producer

2025-04-23 Thread Ivan Yurchenko (Jira)
Ivan Yurchenko created KAFKA-19193: -- Summary: Rack-aware partitioning for Kafka Producer Key: KAFKA-19193 URL: https://issues.apache.org/jira/browse/KAFKA-19193 Project: Kafka Issue Type

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

2025-04-23 Thread Ivan Yurchenko
gt; > > On Mar 30, 2025, at 2:13 AM, Ivan Yurchenko wrote: > > > > Hi Federico, > > > > There's a sentence about reduced latency in the Motivation section. Do you > > think that'd be enough? > > > > Best, > > Ivan > > > &g

Re: [SPAM]Re: [DISCUSS] KIP-1150 Diskless Topics

2025-04-23 Thread Ivan Yurchenko
it'd be at least 2x higher. But I > > believe that given the major savings, many acks=1 users will be happy to > > make the tradeoff. > > > > While on the topic of cost, I hastily ran some cost calculations and found > > that the KIP should reduce replication

Re: [DISCUSS] KIP-1150 Diskless Topics

2025-04-23 Thread Ivan Yurchenko
hundred milliseconds, and here it'd be at least 2x higher. But I > > believe that given the major savings, many acks=1 users will be happy to > > make the tradeoff. > > > > While on the topic of cost, I hastily ran some cost calculations and found > > that the K

[DISCUSS] KIP-1164: Topic Based Batch Coordinator

2025-04-23 Thread Ivan Yurchenko
Hi all! We want to start the discussion thread for KIP-1164: Topic Based Batch Coordinator [1], which is a sub-KIP for KIP-1150 [2]. Let's use the main KIP-1150 discuss thread [3] for high-level questions, motivation, and general direction of the feature and this thread for discussing the batc

[DISCUSS] KIP-1163: Diskless Core

2025-04-23 Thread Ivan Yurchenko
Hi all! We want to start the discussion thread for KIP-1163: Diskless Core [1], which is a sub-KIP for KIP-1150 [2]. Let's use the main KIP-1150 discuss thread [3] for high-level questions, motivation, and general direction of the feature and this thread for particular details of implementatio

Re: [DISCUSS] KIP-1150 Diskless Topics

2025-04-19 Thread Ivan Yurchenko
Hi Ziming, > 1. Is this feature available by just a minor adjust of config or it will > intrude current code heavily, say, AutoMq is 100% compatible with Kafka and > doesn’t intrude the code heavily If we speak about the part visible to the user, we expect: 1. Minimal changes to the client co

Re: [DISCUSS] KIP-1150 Diskless Topics

2025-04-19 Thread Ivan Yurchenko
Hi Luke and all! I'll be participating in this discussion from the authors' side together with Josep and some other colleagues. > 2. "Write through to object storage, avoiding local disk usage" > While this title and the goal said no local disk usage, I'd like to make > sure is it really zero lo

[jira] [Created] (KAFKA-19161) Diskless Topics

2025-04-16 Thread Ivan Yurchenko (Jira)
Ivan Yurchenko created KAFKA-19161: -- Summary: Diskless Topics Key: KAFKA-19161 URL: https://issues.apache.org/jira/browse/KAFKA-19161 Project: Kafka Issue Type: New Feature

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

2025-04-07 Thread Ivan Yurchenko
Hi, Maybe someone else wants to have a look? Best, Ivan On Mon, Mar 31, 2025, at 14:07, Federico Valeri wrote: > +1 non binding > > On Mon, Mar 24, 2025 at 9:51 AM Frédérik Rouleau > wrote: > > > > +1 (non-binding) > > > > On Sat, Mar 22, 2025 at 12:32 PM

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

2025-03-29 Thread Ivan Yurchenko
d latency in the motivation section? > > > On Mon, Mar 10, 2025 at 9:24 PM Ivan Yurchenko wrote: > > > > Hi all, > > > > I took the pause with this KIP while Kafka 4.0 was in making to not > > distract the folks. Now let's continue the discussion! > > &

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

2025-03-22 Thread Ivan Yurchenko
Hi! As there's no more comments regarding KIP-1123 [1], I'd like to put it to a vote. Best regards, Ivan [1] https://cwiki.apache.org/confluence/display/KAFKA/KIP-1123:+Rack-aware+partitioning+for+Kafka+Producer

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

2025-03-11 Thread Ivan Yurchenko
pache-kafka-cost-calculators > > [2] WarpStream and its $220m acquisition > > https://www.linkedin.com/pulse/how-confluent-acquired-warpstream-220m-after-just-13-months-hxgyf/ > > [3] Confluent Freight > > https://www.confluent.io/blog/introducing-confluent-cloud-

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

2024-12-20 Thread Ivan Yurchenko
Hello all, I'd like to propose a new KIP to discuss: KIP-1123: Rack-aware partitioning for Kafka Producer [1]. Best, Ivan Yurchenko [1] https://cwiki.apache.org/confluence/display/KAFKA/KIP-1123%3A+Rack-aware+partitioning+for+Kafka+Producer

Re: Running Kafka from IDE

2024-10-24 Thread Ivan Yurchenko
Hi Pradyumna, It's possible. Here are some points: 1. kafka.Kafka is the class you want to run. 2. As the parameter, pass the path to your server.properties, i.e. the config file. 3. Point to the logger config as the VM properties: -Dlog4j.configuration=file:/path/to/log4j.properties. 4. You nee

Re: [ANNOUNCE] New Kafka PMC Member: Josep Prat

2024-09-07 Thread Ivan Yurchenko
Congrats! On Sat, Sep 7, 2024, at 14:38, Kamal Chandraprakash wrote: > Congrats Josep! > > On Sat, Sep 7, 2024, 13:00 Jorge Esteban Quilcate Otoya < > quilcate.jo...@gmail.com> wrote: > > > Felicidades Josep! > > > > On Sat 7. Sep 2024 at 8.52, David Jacot wrote: > > > > > Congrats! > > > > > >

[jira] [Created] (KAFKA-17494) Document message structure in Protocol Guide

2024-09-07 Thread Ivan Yurchenko (Jira)
Ivan Yurchenko created KAFKA-17494: -- Summary: Document message structure in Protocol Guide Key: KAFKA-17494 URL: https://issues.apache.org/jira/browse/KAFKA-17494 Project: Kafka Issue Type

[jira] [Created] (KAFKA-17323) Document UINT16 instead of UINT32

2024-08-12 Thread Ivan Yurchenko (Jira)
Ivan Yurchenko created KAFKA-17323: -- Summary: Document UINT16 instead of UINT32 Key: KAFKA-17323 URL: https://issues.apache.org/jira/browse/KAFKA-17323 Project: Kafka Issue Type

[jira] [Created] (KAFKA-17322) Document required request and response header versions

2024-08-12 Thread Ivan Yurchenko (Jira)
Ivan Yurchenko created KAFKA-17322: -- Summary: Document required request and response header versions Key: KAFKA-17322 URL: https://issues.apache.org/jira/browse/KAFKA-17322 Project: Kafka

Re: kafka-native image and -javaagent

2024-08-09 Thread Ivan Yurchenko
Hi Neil, I haven't personally tried. But since kafka-native contains not really a Java application, but a GraalVM-compiled native application, it doesn't support Java agents. Best, Ivan On Thu, Aug 8, 2024, at 22:36, Neil Buesing wrote: > I am attempt to set javaagents for the kafka-native im

Re: [DISCUSS] Apache Kafka 3.8.0 release

2024-06-12 Thread Ivan Yurchenko
Hi, I'll try to do all the fixes and changes for KIP-899 [1] sooner today, but please proceed with the release if I don't manage. Ivan [1] https://github.com/apache/kafka/pull/13277 On Wed, Jun 12, 2024, at 12:54, Josep Prat wrote: > Hi Luke, > I think Jose, also mentioned that it won't be rea

Re: [VOTE] KIP-899: Allow producer and consumer clients to rebootstrap

2024-05-22 Thread Ivan Yurchenko
igs, with > or without prefixes. So, we don't need to add the new config to > StreamsConfig explicitly. > > For KIP-932, it says for each new consumer config, we need to determine > whether it should be added to ShareConsumer config too. > > Thanks, > > Jun >

Re: [VOTE] KIP-899: Allow producer and consumer clients to rebootstrap

2024-05-15 Thread Ivan Yurchenko
> > >> On 15 Apr 2024, at 14:16, Chris Egerton > > wrote: > > > >> > > > >> Hi Ivan, > > > >> > > > >> Thanks for the KIP. After the recent changes, this LGTM. +1 (binding) > > > >> > > > >> Cheers

Re: [VOTE] KIP-899: Allow producer and consumer clients to rebootstrap

2024-05-15 Thread Ivan Yurchenko
; > > > +1 (non-binding) > > > > > > > > > > Andrew > > > > > > > > > >> On 15 Apr 2024, at 14:16, Chris Egerton > > > wrote: > > > > >> > > > > >> Hi Ivan, > > > > >&g

[jira] [Resolved] (KAFKA-16445) PATCH method for connector configuration

2024-05-10 Thread Ivan Yurchenko (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16445?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ivan Yurchenko resolved KAFKA-16445. Resolution: Fixed > PATCH method for connector configurat

Re: [VOTE] KIP-477: Add PATCH method for connector config in Connect REST API

2024-05-02 Thread Ivan Yurchenko
gt; > On Mon, Apr 8, 2024, 3:30 PM Chris Egerton > > > wrote: > > > > > > > Thanks Ivan! +1 (binding) from me. > > > > > > > > On Mon, Apr 8, 2024, 06:59 Ivan Yurchenko wrote: > > > > > > > > > Hello! > > >

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

2024-04-14 Thread Ivan Yurchenko
Congrats Greg! On Sun, Apr 14, 2024, at 22:51, Sophie Blee-Goldman wrote: > Congrats Greg! Happy to have you > > On Sun, Apr 14, 2024 at 9:26 AM Jorge Esteban Quilcate Otoya < > quilcate.jo...@gmail.com> wrote: > > > Congrats, Greg!! > > > > On Sun 14. Apr 2024 at 15.05, Josep Prat > > wrote: >

Re: [DISCUSS] KIP-899: Allow clients to rebootstrap

2024-04-12 Thread Ivan Yurchenko
cluster ID, so I > > think removing that from this KIP might > > be the best. I think there’s another KIP waiting to be written for > > ensuring consistency of clusters, but > > I think that wouldn’t conflict at all with this one. > > > > Thanks, > > Andrew &

Re: [DISCUSS] KIP-899: Allow clients to rebootstrap

2024-04-10 Thread Ivan Yurchenko
cluster ID, so I think > removing that from this KIP might > be the best. I think there’s another KIP waiting to be written for ensuring > consistency of clusters, but > I think that wouldn’t conflict at all with this one. > > Thanks, > Andrew > > > On 9

Re: [DISCUSS] KIP-899: Allow clients to rebootstrap

2024-04-09 Thread Ivan Yurchenko
ater on. > > Thanks, > Andrew > > > On 8 Apr 2024, at 18:24, Ivan Yurchenko wrote: > > > > Hello! > > > > I changed the KIP a bit, specifying that the certain benefit goes to > > consumers not participating in a group, but that other clients can b

RE: Re: [DISCUSS] KIP-899: Allow clients to rebootstrap

2024-04-08 Thread Ivan Yurchenko
/diffpagesbyversion.action?pageId=240881396&originalVersion=10&revisedVersion=11 On 2023/07/15 16:37:52 Ivan Yurchenko wrote: > Hello! > > I've made several changes to the KIP based on the comments: > > 1. Reduced the scope to producer and consumer clients only. &g

[VOTE] KIP-477: Add PATCH method for connector config in Connect REST API

2024-04-08 Thread Ivan Yurchenko
Hello! I'd like to put the subj KIP[1] to a vote. Thank you. Best regards, Ivan [1] https://cwiki.apache.org/confluence/display/KAFKA/KIP-477%3A+Add+PATCH+method+for+connector+config+in+Connect+REST+API

Re: [DISCUSS] KIP-477: Add PATCH method for connector config in Connect REST API

2024-03-28 Thread Ivan Yurchenko
blocking the KIP on fixing this. > > One final nit: Can you update the Jira ticket link in the KIP? > > Cheers, > > Chris > > On Wed, Mar 27, 2024 at 2:56 PM Ivan Yurchenko wrote: > > > Hi, > > > > I updated the KIP with the two following change

[jira] [Created] (KAFKA-16445) PATCH method for connecto configuration

2024-03-28 Thread Ivan Yurchenko (Jira)
Ivan Yurchenko created KAFKA-16445: -- Summary: PATCH method for connecto configuration Key: KAFKA-16445 URL: https://issues.apache.org/jira/browse/KAFKA-16445 Project: Kafka Issue Type

Re: [DISCUSS] KIP-477: Add PATCH method for connector config in Connect REST API

2024-03-27 Thread Ivan Yurchenko
good idea. I can update the KIP. > > Ivan > > > On Tue, Mar 26, 2024, at 14:19, Ivan Yurchenko wrote: > > Hi all, > > > > This KIP is a bit old now :) but I think its context hasn't changed much > > since then and the KIP is still valid. I would like

Re: [ANNOUNCE] New committer: Christo Lolov

2024-03-26 Thread Ivan Yurchenko
Congrats! On Tue, Mar 26, 2024, at 14:48, Lucas Brutschy wrote: > Congrats! > > On Tue, Mar 26, 2024 at 2:44 PM Federico Valeri wrote: > > > > Congrats! > > > > On Tue, Mar 26, 2024 at 2:27 PM Mickael Maison > > wrote: > > > > > > Congratulations Christo! > > > > > > On Tue, Mar 26, 2024 at 2:

Re: [DISCUSS] KIP-477: Add PATCH method for connector config in Connect REST API

2024-03-26 Thread Ivan Yurchenko
rs "null") as the value for to-be-removed > properties. This actually makes sense. AFAIU, `null` cannot be in the connector config since https://github.com/apache/kafka/pull/11333, so using it as a tombstone value is a good idea. I can update the KIP. Ivan On Tue, Mar 26, 2024,

RE: Re: [DISCUSS] KIP-477: Add PATCH method for connector config in Connect REST API

2024-03-26 Thread Ivan Yurchenko
27;re still interested in driving it, Ivan. > Hopefully we can give it the attention it deserves in the upcoming months! > > Cheers, > > Chris > > On Fri, Jun 28, 2019 at 4:56 AM Ivan Yurchenko > wrote: > > > Thank you for your feedback Ryanne! > > These a

[jira] [Created] (KAFKA-15931) Cached transaction index gets closed if tiered storage read is interrupted

2023-11-29 Thread Ivan Yurchenko (Jira)
Ivan Yurchenko created KAFKA-15931: -- Summary: Cached transaction index gets closed if tiered storage read is interrupted Key: KAFKA-15931 URL: https://issues.apache.org/jira/browse/KAFKA-15931

RemoteStorageManager.fetchLogSegment - how to deal with InterruptedException?

2023-11-17 Thread Ivan Yurchenko
Hello! `RemoteStorageManager.fetchLogSegment` is called in a background thread by the broker [1]. When a fetch request times out, the associated Future is cancelled [2] and the thread is interrupted. If the InterruptedException is propagated from the `RemoteStorageManager`, it pollutes the brok

Re: [ANNOUNCE] New Kafka PMC Member: Satish Duggana

2023-10-27 Thread Ivan Yurchenko
Congrats Satish! Ivan On Fri, Oct 27, 2023, at 19:02, Kamal Chandraprakash wrote: > Congratulations Satish! > > On Fri, Oct 27, 2023, 21:10 Jorge Esteban Quilcate Otoya < > quilcate.jo...@gmail.com> wrote: > > > Congratulations Satish!! > > > > On Fri, 27 Oct 2023 at 18:38, Mickael Maison > >

RE: [DISCUSS] KIP-905: Broker interceptors

2023-10-20 Thread Ivan Yurchenko
Hi David, I wonder if you considered interceptors that work not only on produce request, but on any request type the broker is receiving (apart from maybe internal request types). For example, being able to rewrite topic names in all the request types is essential to create a virtual clusters s

RE: Re: [DISCUSS] KIP-905: Broker interceptors

2023-10-20 Thread Ivan Yurchenko
Hi David and Ahmed, First, thank you David for the KIP. It would be very valuable for multiple use cases. Products like Conduktor Gateway [1] validate the demand and offer many potential use cases [2]. Now, I understand Ahmed's concerns about possible in-band interruptions, the are valid. Howe

Re: Apache Kafka 3.6.0 release

2023-09-08 Thread Ivan Yurchenko
Hi Satish and all, I wonder if https://issues.apache.org/jira/browse/KAFKA-14993 should be included in the 3.6 release plan. I'm thinking that when implemented, it would be a small, but still a change in the RSM contract: throw an exception instead of returning an empty InputStream. Maybe it sh

[VOTE] KIP-899: Allow producer and consumer clients to rebootstrap

2023-08-01 Thread Ivan Yurchenko
Hello, The discussion [1] for KIP-899 [2] has been open for quite some time. I'd like to put the KIP up for a vote. Best, Ivan [1] https://lists.apache.org/thread/m0ncbmfxs5m87sszby2jbmtjx2bdpcdl [2] https://cwiki.apache.org/confluence/display/KAFKA/KIP-899%3A+Allow+producer+and+consumer+clients

Re: [DISCUSS] KIP-899: Allow clients to rebootstrap

2023-07-15 Thread Ivan Yurchenko
ctedPageVersions=5 On Tue, 30 May 2023 at 08:23, Ivan Yurchenko wrote: > Hi Chris and all, > > > I believe the logic you've linked is only applicable for the producer and > > consumer clients; the admin client does something different (see [1]). > > I see, thank yo

Re: [DISCUSS] KIP-917: Additional custom metadata for remote log segment

2023-07-10 Thread Ivan Yurchenko
Hi! For those who are interested, I submitted the PR with the implementation: https://github.com/apache/kafka/pull/13984 Thanks! Ivan On Tue, 13 Jun 2023 at 15:05, Ivan Yurchenko wrote: > Hi Satish, > > I understand your point and I agree with it. TBH, I didn't take into &g

Re: [VOTE] KIP-917: Additional custom metadata for remote log segment

2023-07-10 Thread Ivan Yurchenko
eral KIP overview page and > move your KIP from "under discussion" to "approved" and 3.6.0 as the > version. > > Best, > > On Mon, Jun 19, 2023 at 12:20 PM Ivan Yurchenko > wrote: > > > Hi all, > > > > Thank you for your votes. > > >

Re: Apache Kafka 3.6.0 release

2023-06-19 Thread Ivan Yurchenko
hanks, > Satish. > > On Mon, 19 Jun 2023 at 16:03, Ivan Yurchenko > wrote: > > > > Hi, > > > > I would like to propose to include the newly accepted "KIP-917: > Additional > > custom metadata for remote log segment" [1] in the release plan. W

[jira] [Created] (KAFKA-15107) Additional custom metadata for remote log segment

2023-06-19 Thread Ivan Yurchenko (Jira)
Ivan Yurchenko created KAFKA-15107: -- Summary: Additional custom metadata for remote log segment Key: KAFKA-15107 URL: https://issues.apache.org/jira/browse/KAFKA-15107 Project: Kafka Issue

Re: Apache Kafka 3.6.0 release

2023-06-19 Thread Ivan Yurchenko
Hi, I would like to propose to include the newly accepted "KIP-917: Additional custom metadata for remote log segment" [1] in the release plan. Would it be possible? Thanks! Best, Ivan [1] https://cwiki.apache.org/confluence/display/KAFKA/KIP-917%3A+Additional+custom+metadata+for+remote+log+segm

Re: [VOTE] KIP-917: Additional custom metadata for remote log segment

2023-06-19 Thread Ivan Yurchenko
: > Thanks Ivan for addressing the comments in the KIP, LGTM. > > +1 > > On Tue, 13 Jun 2023 at 18:31, Luke Chen wrote: > > > > Looks good. Thanks for the update. > > > > On Tue, Jun 13, 2023 at 8:08 PM Ivan Yurchenko > > > wrote: > > > > &

Re: [VOTE] KIP-917: Additional custom metadata for remote log segment

2023-06-13 Thread Ivan Yurchenko
t it would be beneficial > to > > > > > implement. > > > > > > > > > > Vote +1 (non-binding) > > > > > > > > > > > > > > > -- > > > > > Divij Vaidya > > > > > > > > >

Re: [DISCUSS] KIP-917: Additional custom metadata for remote log segment

2023-06-13 Thread Ivan Yurchenko
metadata with > each segment. > > ~Satish. > > > On Wed, 7 Jun 2023 at 23:38, Ivan Yurchenko > wrote: > > > > Hi Satish, > > > > Thank you for your feedback. > > > > I've nothing against going from Map to byte[]. > > Serialization sh

Re: [DISCUSS] KIP-917: Additional custom metadata for remote log segment

2023-06-12 Thread Ivan Yurchenko
Hi all FYI, I opened this for a vote: https://lists.apache.org/thread/pnrkgomomwb03h8pfgs0k7dzwgpbtloz On Tue, 13 Jun 2023 at 08:01, Ivan Yurchenko wrote: > Hi Luke, > > > I saw you add the `RemoteLogSegmentMetadataRecord` and > > `RemoteLogSegmentMetadataSnapshotRecord`, I

Re: [DISCUSS] KIP-917: Additional custom metadata for remote log segment

2023-06-12 Thread Ivan Yurchenko
o > avoid confusion. > > Usually the "record" being added in the KIP will be the one affecting > the public interface, ex: API request/response. > So I'd like to confirm it. > > > Thanks. > Luke > > On Thu, Jun 8, 2023 at 2:09 AM Ivan Yurchenko > wrote:

[VOTE] KIP-917: Additional custom metadata for remote log segment

2023-06-12 Thread Ivan Yurchenko
Hello, Some interest in KIP-917 was expressed in the discussion thread [1]. After addressing all the comments there, I'm putting it up to a vote. Thanks! Best, Ivan [1] https://lists.apache.org/thread/qpccqd3jy5rzvbt5ngtzo3dg9pzp722y

Re: [DISCUSS] KIP-917: Additional custom metadata for remote log segment

2023-06-07 Thread Ivan Yurchenko
y opaque and it is the RSM implementation provider's > responsibility in serializing and deserializing the bytes. We can > introduce a property to guard the size with a configurable property > with a default value to avoid any unwanted large size values. > > Thanks, > Satish. > >

Re: [DISCUSS] KIP-917: Additional custom metadata for remote log segment

2023-05-29 Thread Ivan Yurchenko
Hi all, I want to bring this to a conclusion (positive or negative), so if there are no more questions in a couple of days, I'll put the KIP to the vote. Best, Ivan On Fri, 5 May 2023 at 18:42, Ivan Yurchenko wrote: > Hi Alexandre, > > > combining custom > > me

Re: [DISCUSS] KIP-899: Allow clients to rebootstrap

2023-05-29 Thread Ivan Yurchenko
rap only after " > metadata.max.age.ms" has elapsed since the last metadata update, and when > at least one request has been made to contact each known server and been > met with failure? > > [1] - > > https://github.com/apache/kafka/blob/c9a42c85e2c903329b3550181d2305

Re: [DISCUSS] KIP-917: Additional custom metadata for remote log segment

2023-05-05 Thread Ivan Yurchenko
on > more difficult. What do you think? > > I have a vague memory of this being discussed back when the tiered > storage KIP was started. Maybe Satish has more background on this. > > Thanks, > Alexandre > > Le lun. 17 avr. 2023 à 16:50, Ivan Yurchenko > a écrit : > &g

Re: [DISCUSS] KIP-917: Additional custom metadata for remote log segment

2023-04-17 Thread Ivan Yurchenko
gt; > Thanks, > Alexandre > > Le jeu. 6 avr. 2023 à 16:03, hzh0425 a écrit : > > > > I think it's a good idea as we may want to store remote segments in > different buckets > > > > > > > > | | > > hzhka...@163.com > > | > > |

[DISCUSS] KIP-917: Additional custom metadata for remote log segment

2023-04-06 Thread Ivan Yurchenko
Hello! I would like to start the discussion thread on KIP-917: Additional custom metadata for remote log segment [1] This KIP is fairly small and proposes to add a new field to the remote segment metadata. Thank you! Best, Ivan [1] https://cwiki.apache.org/confluence/display/KAFKA/KIP-917%3A+Ad

[jira] [Created] (KAFKA-14795) Provide message formatter for RemoteLogMetadata

2023-03-08 Thread Ivan Yurchenko (Jira)
Ivan Yurchenko created KAFKA-14795: -- Summary: Provide message formatter for RemoteLogMetadata Key: KAFKA-14795 URL: https://issues.apache.org/jira/browse/KAFKA-14795 Project: Kafka Issue

Re: [DISCUSS] KIP-899: Allow clients to rebootstrap

2023-02-19 Thread Ivan Yurchenko
ribe the cluster a second time. With that second attempt, would we > immediately resort to the bootstrap servers for any initial metadata > updates, or would we still try to go through the last-known set of brokers > first? > > Cheers, > > Chris > > On Mon, Feb 6, 2023 a

Re: [DISCUSS] KIP-899: Allow clients to rebootstrap

2023-02-06 Thread Ivan Yurchenko
Hi! There seems to be not much more discussion going, so I'm planning to start the vote in a couple of days. Thanks, Ivan On Wed, 18 Jan 2023 at 12:06, Ivan Yurchenko wrote: > Hello! > I would like to start the discussion thread on KIP-899: Allow clients to > rebootstrap. >

Re: [DISCUSS] KIP-899: Allow clients to rebootstrap

2023-01-26 Thread Ivan Yurchenko
time, why can't you just close the client and re-instantiate a new one > when needed? I'm not familiar with the stream thread, so I don't know if > that's possible. Another idea here is, would it make sense to expose a > maybeUpdateMetadata() API to serve such a purpos

Re: [DISCUSS] KIP-899: Allow clients to rebootstrap

2023-01-25 Thread Ivan Yurchenko
behaviour either by a reproducible manual test or by a branch with a > failing unit/integration test? > > Best, > Christo > > > On 18 Jan 2023, at 12:07, Ivan Yurchenko > wrote: > > > > Hello! > > I would like to start the discussion thread on KIP-899: Allow

[DISCUSS] KIP-899: Allow clients to rebootstrap

2023-01-18 Thread Ivan Yurchenko
Hello! I would like to start the discussion thread on KIP-899: Allow clients to rebootstrap. This KIP proposes to allow Kafka clients to repeat the bootstrap process when fetching metadata if none of the known nodes are available. https://cwiki.apache.org/confluence/display/KAFKA/KIP-899%3A+Allow+c

Re: [VOTE] KIP-797: Accept duplicate listener on port for IPv4/IPv6

2021-11-22 Thread Ivan Yurchenko
Hi, Thank you for the KIP. +1 (non-binding) Ivan On Tue, 23 Nov 2021 at 04:18, Luke Chen wrote: > Hi Matthew, > Thanks for the KIP. > It makes sense to allow IPv4 and IPv6 listening on the same port for the > listener config. > > +1 (non-binding) > > Thank you. > Luke > > On Mon, Nov 22, 202

[jira] [Created] (KAFKA-13376) Allow MirrorMaker producer and consumer customization per replication flow

2021-10-14 Thread Ivan Yurchenko (Jira)
Ivan Yurchenko created KAFKA-13376: -- Summary: Allow MirrorMaker producer and consumer customization per replication flow Key: KAFKA-13376 URL: https://issues.apache.org/jira/browse/KAFKA-13376

Re: CVE-2021-38153: Timing Attack Vulnerability for Apache Kafka Connect and Clients

2021-09-21 Thread Ivan Yurchenko
Hi Randall, Could you please share the JIRA ticket or the fixing commit? It might help to evaluate the impact better. Thank you! Ivan On Tue, 21 Sept 2021 at 19:37, Randall Hauch wrote: > Severity: moderate > > Description: > > Some components in Apache Kafka use `Arrays.equals` to validate a

Re: [idea] Kafka topic metadata

2021-06-15 Thread Ivan Yurchenko
uggestion earlier. I think this will be a great idea > > for a KIP. > > > > > https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals > > > > You were referring to metadata for the actual topic and not its contents. > > > > Sorry abo

Re: [idea] Kafka topic metadata

2021-06-14 Thread Ivan Yurchenko
Hi, Having metadata for topics seems pretty useful. Currently, one has to use external storage for this (e.g. a database) and the question of keeping topic and metadata in sync exists: A topic is deleted, how to delete its metadata? How to deal with delete-then-recreate scenarios (well, we have to

[jira] [Created] (KAFKA-12835) Topic IDs can mismatch on brokers (after interbroker protocol version update)

2021-05-21 Thread Ivan Yurchenko (Jira)
Ivan Yurchenko created KAFKA-12835: -- Summary: Topic IDs can mismatch on brokers (after interbroker protocol version update) Key: KAFKA-12835 URL: https://issues.apache.org/jira/browse/KAFKA-12835

[jira] [Created] (KAFKA-12430) emit.heartbeats.enabled = false should disable heartbeats topic creation

2021-03-05 Thread Ivan Yurchenko (Jira)
Ivan Yurchenko created KAFKA-12430: -- Summary: emit.heartbeats.enabled = false should disable heartbeats topic creation Key: KAFKA-12430 URL: https://issues.apache.org/jira/browse/KAFKA-12430 Project

Re: [VOTE] KIP - 405: Kafka Tiered Storage.

2021-02-15 Thread Ivan Yurchenko
Hi, Great! +1 (non-binding) Best, Ivan On Mon, 15 Feb 2021 at 22:47, Kowshik Prakasam wrote: > +1 (non-binding). Thanks for the excellent KIP! > > > Cheers, > Kowshik > > > > > > On Mon, Feb 15, 2021 at 2:50 AM Manikumar > wrote: > > > Hi Satish, > > > > Thanks for driving this KIP. I’m sure

[jira] [Created] (KAFKA-12235) ZkAdminManager.describeConfigs returns no config when 2+ configuration keys are specified

2021-01-25 Thread Ivan Yurchenko (Jira)
Ivan Yurchenko created KAFKA-12235: -- Summary: ZkAdminManager.describeConfigs returns no config when 2+ configuration keys are specified Key: KAFKA-12235 URL: https://issues.apache.org/jira/browse/KAFKA-12235

[jira] [Created] (KAFKA-9672) Dead broker in ISR cause isr-expiration to fail with exception

2020-03-06 Thread Ivan Yurchenko (Jira)
Ivan Yurchenko created KAFKA-9672: - Summary: Dead broker in ISR cause isr-expiration to fail with exception Key: KAFKA-9672 URL: https://issues.apache.org/jira/browse/KAFKA-9672 Project: Kafka

[jira] [Created] (KAFKA-9478) Controller may stop react on partition reassignment command in ZooKeeper

2020-01-28 Thread Ivan Yurchenko (Jira)
Ivan Yurchenko created KAFKA-9478: - Summary: Controller may stop react on partition reassignment command in ZooKeeper Key: KAFKA-9478 URL: https://issues.apache.org/jira/browse/KAFKA-9478 Project

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-12-27 Thread Ivan Yurchenko
Hi all, Jun: > (a) Cost: S3 list object requests cost $0.005 per 1000 requests. If you > have 100,000 partitions and want to pull the metadata for each partition at > the rate of 1/sec. It can cost $0.5/sec, which is roughly $40K per day. I want to note here, that no reasonably durable storage w

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-12-05 Thread Ivan Yurchenko
Hi Jun and others. Jun, All these are really valid concerns. Probably we should think about backing implementations like S3 with a metadata storage whose consistency model and pricing is better that pure S3, maybe even a Kafka topic (I guess this might be something you refer to as push-based appro

Re: [VOTE] KIP-531: Drop support for Scala 2.11 in Kafka 2.5

2019-11-18 Thread Ivan Yurchenko
Do I understand correctly, that non-commiters can also vote, despite their votes don't decide? If so, then +1 from me. Ivan On Mon, 18 Nov 2019 at 15:19, Ismael Juma wrote: > Hi all, > > People seemed supportive in general, so I'd like to start a vote on > KIP-531: > > > https://cwiki.apache.

[jira] [Created] (KAFKA-9143) DistributedHerder misleadingly log error on connector task reconfiguration

2019-11-05 Thread Ivan Yurchenko (Jira)
Ivan Yurchenko created KAFKA-9143: - Summary: DistributedHerder misleadingly log error on connector task reconfiguration Key: KAFKA-9143 URL: https://issues.apache.org/jira/browse/KAFKA-9143 Project

[jira] [Created] (KAFKA-9035) Improve

2019-10-13 Thread Ivan Yurchenko (Jira)
Ivan Yurchenko created KAFKA-9035: - Summary: Improve Key: KAFKA-9035 URL: https://issues.apache.org/jira/browse/KAFKA-9035 Project: Kafka Issue Type: Improvement Components

Re: Need some clarification of Kafka and MQTT

2019-10-09 Thread Ivan Yurchenko
where you run your Kafka Connect. An alternative would be to just write your own application that reads from Kafka and writes to MQTT or vice versa using Kafka's producers and consumers. I hope this helps! Best, Ivan Yurchenko [1] https://kafka.apache.org/documentation/#connect [2] https://d

Re: Are segment files identical in different replicas?

2019-09-02 Thread Ivan Yurchenko
s in > same order. > > about question 2, i'm not following what did you mean about "byte level", > could you plz to make more explanations? > > Best, > Lisheng > > > Ivan Yurchenko 于2019年8月30日周五 下午8:39写道: > > > Hi, > > > > Let's

Are segment files identical in different replicas?

2019-08-30 Thread Ivan Yurchenko
Hi, Let's say I have a topic-partition replicated to several replicas. On each replica there is a segment of this topic-partition containing records with offsets N..M. I'm trying to figure out: 1. Will the content of these segment files be identical on the logical level? I.e., will they contain th

Re: [DISCUSS] KIP-477: Add PATCH method for connector config in Connect REST API

2019-06-28 Thread Ivan Yurchenko
27;ve done so in an > atomic, non-racy way. > > So I don't think PATCH is strictly necessary nor sufficient for atomic > partial updates. That said, it doesn't hurt and I'm happy to support the > KIP. > > Ryanne > > On Tue, Jun 25, 2019 at 12:15 PM Ivan Yurche

Re: [DISCUSS] KIP-477: Add PATCH method for connector config in Connect REST API

2019-06-25 Thread Ivan Yurchenko
Hi, Since Kafka 2.3 has just been release and more people may have time to look at this now, I'd like to bump this discussion. Thanks. Ivan On Thu, 13 Jun 2019 at 17:20, Ivan Yurchenko wrote: > Hello, > > I'd like to start the discussion of KIP-477: Add PATCH method for &g

[DISCUSS] KIP-477: Add PATCH method for connector config in Connect REST API

2019-06-13 Thread Ivan Yurchenko
Hello, I'd like to start the discussion of KIP-477: Add PATCH method for connector config in Connect REST API. https://cwiki.apache.org/confluence/display/KAFKA/KIP-477%3A+Add+PATCH+method+for+connector+config+in+Connect+REST+API There is also a draft PR: https://github.com/apache/kafka/pull/693

Permission to create KIP

2019-05-30 Thread Ivan Yurchenko
Hi, I'd like to be able to create KIPs. Could someone please set up the permissions for me? The username is ivanyu . Thanks! Ivan

Add to contributors list

2019-03-22 Thread Ivan Yurchenko
Hello, Could somebody add me to the Kafka contributor list so I can assign myself tickets in JIRA? My ID/username is ivanyu. Thank you. Best, Ivan

Add to contributors list

2019-03-22 Thread Ivan Yurchenko
Hello, Could somebody add me to the Kafka contributor list so I can assign myself tickets in JIRA? My ID/username is ivanyu. Thank you. Best, Ivan