Re: [DISCUSS] KIP-1046: Expose producer.id.expiration.check.interval.ms as dynamic broker configuration

2024-05-16 Thread Jorge Esteban Quilcate Otoya
ok that > we are exposing the value though. Can we just include some information > about the current default, the documentation etc that is already defined as > this will now become part of the public documentation? > > Thanks, > Justine > > On Thu, May 16, 2024 at 1

[DISCUSS] KIP-1046: Expose producer.id.expiration.check.interval.ms as dynamic broker configuration

2024-05-16 Thread Jorge Esteban Quilcate Otoya
Hi dev team, I'd like to start a discussion thread for KIP-1046: https://cwiki.apache.org/confluence/display/KAFKA/KIP-1046%3A+Expose+producer.id.expiration.check.interval.ms+as+dynamic+broker+configuration This KIP aims to align how tuning configurations for Producer ID expiration checks are

[jira] [Resolved] (KAFKA-16685) RLMTask warning logs do not include parent exception trace

2024-05-08 Thread Jorge Esteban Quilcate Otoya (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16685?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jorge Esteban Quilcate Otoya resolved KAFKA-16685. -- Resolution: Fixed Merged https://github.com/apache/kafka/pull

[jira] [Created] (KAFKA-16691) Support for nested structures: TimestampConverter

2024-05-07 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-16691: Summary: Support for nested structures: TimestampConverter Key: KAFKA-16691 URL: https://issues.apache.org/jira/browse/KAFKA-16691

[jira] [Created] (KAFKA-16690) Support for nested structures: HeaderFrom

2024-05-07 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-16690: Summary: Support for nested structures: HeaderFrom Key: KAFKA-16690 URL: https://issues.apache.org/jira/browse/KAFKA-16690 Project

[jira] [Resolved] (KAFKA-14226) Introduce support for nested structures

2024-05-07 Thread Jorge Esteban Quilcate Otoya (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14226?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jorge Esteban Quilcate Otoya resolved KAFKA-14226. -- Resolution: Fixed Merged: https://github.com/apache/kafka

[jira] [Created] (KAFKA-16685) RSM Task warn logs do not include parent exception trace

2024-05-07 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-16685: Summary: RSM Task warn logs do not include parent exception trace Key: KAFKA-16685 URL: https://issues.apache.org/jira/browse/KAFKA-16685

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

2024-04-14 Thread Jorge Esteban Quilcate Otoya
Congrats, Greg!! On Sun 14. Apr 2024 at 15.05, Josep Prat wrote: > Congrats Greg!!! > > > Best, > > Josep Prat > Open Source Engineering Director, aivenjosep.p...@aiven.io | > +491715557497 | aiven.io > Aiven Deutschland GmbH > Alexanderufer 3-7, 10117 Berlin > Geschäftsführer: Oskari

Re: [ANNOUNCE] New committer: Christo Lolov

2024-03-26 Thread Jorge Esteban Quilcate Otoya
Congrats Christo!! On Tue 26. Mar 2024 at 14.33, Apoorv Mittal wrote: > Congrats Christo! > > Regards, > Apoorv Mittal > +44 7721681581 > > > On Tue, Mar 26, 2024 at 12:05 PM Luke Chen wrote: > > > Hi, Everyone, > > > > The PMC of Apache Kafka is pleased to announce a new Kafka committer: > >

Re: [DISCUSS] Different retention semantics for active segment rotation

2024-03-21 Thread Jorge Esteban Quilcate Otoya
> > Thanks. > Luke > > On Thu, Mar 21, 2024 at 3:33 PM Jorge Esteban Quilcate Otoya < > quilcate.jo...@gmail.com> wrote: > > > Hi dev community, > > > > I'd like to share some findings on how rotation of active segment differ > > depending on whether

[DISCUSS] Different retention semantics for active segment rotation

2024-03-21 Thread Jorge Esteban Quilcate Otoya
Hi dev community, I'd like to share some findings on how rotation of active segment differ depending on whether topic retention is time- or size-based. I was (wrongly) under the assumption that active segments are only rotated when segment configs (segment.bytes (1GiB) or segment.ms (7d)) or

Re: [VOTE] KIP-956: Tiered Storage Quotas

2024-03-19 Thread Jorge Esteban Quilcate Otoya
Thanks Abhjeet! Looking forward for this one. +1 (non-binding). On Thu, 14 Mar 2024 at 06:08, Luke Chen wrote: > Thanks for the KIP! > +1 from me. > > Luke > > On Sun, Mar 10, 2024 at 8:44 AM Satish Duggana > wrote: > > > Thanks Abhijeet for the KIP, +1 from me. > > > > > > On Sat, 9 Mar 2024

Re: [DISCUSS] KIP-956: Tiered Storage Quotas

2024-03-19 Thread Jorge Esteban Quilcate Otoya
uota configs, such as > *replica.alter.log.dirs.io.max.bytes.per.second* as pointed out by Jun in > the thread. > > Also, we can revisit the names of the components during implementation, > since those are not exposed to the user. > > Please let me know if you have any further concerns. > > Regards, > Abhijeet. > > &

Re: [DISCUSS] KIP-956: Tiered Storage Quotas

2024-03-11 Thread Jorge Esteban Quilcate Otoya
Hi Abhijeet, Thanks for the KIP! Looks good to me. I just have a minor comments on naming: Would it be work to align the config names to existing quota names? e.g. `remote.log.manager.copy.byte.rate.quota` (or similar) instead of `remote.log.manager.copy.max.bytes.per.second`? Same for new

[jira] [Created] (KAFKA-16264) Expose `producer.id.expiration.check.interval.ms` as dynamic broker configuration

2024-02-16 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-16264: Summary: Expose `producer.id.expiration.check.interval.ms` as dynamic broker configuration Key: KAFKA-16264 URL: https://issues.apache.org/jira

[jira] [Created] (KAFKA-16229) Slow expiration of Producer IDs leading to high CPU usage

2024-02-06 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-16229: Summary: Slow expiration of Producer IDs leading to high CPU usage Key: KAFKA-16229 URL: https://issues.apache.org/jira/browse/KAFKA-16229

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

2024-01-31 Thread Jorge Esteban Quilcate Otoya
Hi Kamal, Thanks for this KIP! It should help to solve one of the main issues with tiered storage at the moment that is dealing with individual consumer configurations to avoid flooding logs with interrupted exceptions. One of the topics discussed in [1][2] was on the semantics of `

Re: [ANNOUNCE] New Kafka PMC Member: Divij Vaidya

2023-12-27 Thread Jorge Esteban Quilcate Otoya
Congratulations Divij!! On Wed 27. Dec 2023 at 14.56, Tom Bentley wrote: > Congratulations! > > On Thu, 28 Dec 2023 at 06:17, Philip Nee wrote: > > > congrats divij! > > > > On Wed, Dec 27, 2023 at 8:55 AM Justine Olshan > > > > wrote: > > > > > Congratulations Divij! > > > > > > On Wed, Dec

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

2023-11-23 Thread Jorge Esteban Quilcate Otoya
A bit late, but happy this KIP is being adopted. Thanks, Christo! On Thu 23. Nov 2023 at 15.15, Christo Lolov wrote: > Hello all, > > With 3 +1 binding and 1 +1 non-binding votes KIP-963 is adopted 拾! > I will get down to implementing it. > > Best, > Christo > > On Tue, 21 Nov 2023 at 07:22,

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

2023-11-20 Thread Jorge Esteban Quilcate Otoya
t; > data > > > > > > internally. > > > > > > > > > > > > Thanks, > > > > > > Jorge. > > > > > > > > > > > > On Thu, 9 Nov 2023 at 10:51, Luke Chen > wrote: > > > > > > >

Re: [VOTE] KIP-997: Partition-Level Throughput Metrics

2023-11-15 Thread Jorge Esteban Quilcate Otoya
Qichao, thanks again for leading this proposal! +1 (non-binding) Cheers, Jorge. On Wed, 15 Nov 2023 at 19:17, Divij Vaidya wrote: > +1 (binding) > > I was involved in the discussion thread for this KIP and support it in its > current form. > > -- > Divij Vaidya > > > > On Wed, Nov 15, 2023 at

Re: [DISCUSS] KIP-1002: Fetch remote segment indexes at once

2023-11-13 Thread Jorge Esteban Quilcate Otoya
her as proposed in the KIP. > > -- > Divij Vaidya > > > > On Fri, Nov 10, 2023 at 4:00 PM Jorge Esteban Quilcate Otoya < > quilcate.jo...@gmail.com> wrote: > > > Hello everyone, > > > > I would like to start the discussion on a KIP for Tiered S

[DISCUSS] KIP-1003: Signal next segment when remote fetching

2023-11-10 Thread Jorge Esteban Quilcate Otoya
Hi there, I would like to start the discussion on a KIP for Tiered Storage. It's about improving cross-segment latencies by enabling Remote Storage Manager implementation to pre-fetch across segments. Have a look:

[DISCUSS] KIP-1002: Fetch remote segment indexes at once

2023-11-10 Thread Jorge Esteban Quilcate Otoya
Hello everyone, I would like to start the discussion on a KIP for Tiered Storage. It's about improving cross-segment latencies by reducing calls to fetch indexes individually. Have a look: https://cwiki.apache.org/confluence/display/KAFKA/KIP-1002%3A+Fetch+remote+segment+indexes+at+once Cheers,

[jira] [Created] (KAFKA-15806) Signal next segment when remote fetching

2023-11-10 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-15806: Summary: Signal next segment when remote fetching Key: KAFKA-15806 URL: https://issues.apache.org/jira/browse/KAFKA-15806 Project

[jira] [Created] (KAFKA-15805) Fetch Remote Indexes at once

2023-11-10 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-15805: Summary: Fetch Remote Indexes at once Key: KAFKA-15805 URL: https://issues.apache.org/jira/browse/KAFKA-15805 Project: Kafka

Re: [DISCUSS] KIP-977: Partition-Level Throughput Metrics

2023-11-09 Thread Jorge Esteban Quilcate Otoya
are updated. > > Please take another look and let me know if you have any more concerns. > > Best, > Qichao Chu > Software Engineer | Data - Kafka > [image: Uber] <https://uber.com/> > > > On Wed, Nov 8, 2023 at 6:29 AM Jorge Esteban Quilcate Otoya &l

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

2023-11-09 Thread Jorge Esteban Quilcate Otoya
Remote deletion metrics, should we also emit a metric to > > expose the oldest segment time? > > Users can configure the topic retention either by size (or) time. If time > > is configured, then emitting > > the oldest segment time allows the user to configure an alert on top of >

Re: [DISCUSS] KIP-977: Partition-Level Throughput Metrics

2023-11-07 Thread Jorge Esteban Quilcate Otoya
Hi Qichao, Thanks for the KIP! This will be a valuable contribution and improve the tooling for troubleshooting. I have a couple of comments: 1. It's unclear from the `metrics.verbosity` description what the supported values are. In the description mentions "If the value is high ... In the low

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

2023-11-01 Thread Jorge Esteban Quilcate Otoya
t much data actually being written in remote > storage. Do these two scenarios demonstrate the usefulness I would have > from such a metric and do the benefits make sense to you? > > 7. I agree. I have changed TotalRemoteLogSizeComputationTime, > TotalRemoteLogSizeBytes, and TotalRem

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

2023-10-27 Thread Jorge Esteban Quilcate Otoya
Congratulations Satish!! On Fri, 27 Oct 2023 at 18:38, Mickael Maison wrote: > Congratulations Satish! > > On Fri, Oct 27, 2023 at 5:18 PM Lucas Brutschy > wrote: > > > > Congrats! > > > > On Fri, Oct 27, 2023 at 5:06 PM Manikumar > wrote: > > > > > > Congrats! > > > > > > On Fri, Oct 27,

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

2023-10-27 Thread Jorge Esteban Quilcate Otoya
Hi Christo, Thanks for proposing KIP, this metrics will certainly be useful to operate Kafka Tiered Storage as it becomes production-ready. 1. Given that the scope of the KIPs has grown to cover more metrics, what do you think about introducing latency metrics for RSM operations? Copy and delete

Re: [ANNOUNCE] New committer: Yash Mayya

2023-09-21 Thread Jorge Esteban Quilcate Otoya
Congratulations, Yash! On Thu 21. Sep 2023 at 21.57, Randall Hauch wrote: > Congratulations, Yash! > > On Thu, Sep 21, 2023 at 12:31 PM Satish Duggana > wrote: > > > Congratulations Yash!! > > > > On Thu, 21 Sept 2023 at 22:57, Viktor Somogyi-Vass > > wrote: > > > > > > Congrats Yash! > > > >

[jira] [Created] (KAFKA-15314) No Quota applied if client-id is null or empty

2023-08-08 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-15314: Summary: No Quota applied if client-id is null or empty Key: KAFKA-15314 URL: https://issues.apache.org/jira/browse/KAFKA-15314

Re: [DISCUSS] KIP-935: Extend AlterConfigPolicy with existing configurations

2023-07-25 Thread Jorge Esteban Quilcate Otoya
KIP is updated now: https://cwiki.apache.org/confluence/display/KAFKA/KIP-935%3A+Extend+AlterConfigPolicy+with+existing+configurations Looking forward to your feedback, Many thanks, Jorge. On Tue, 25 Jul 2023 at 16:59, Jorge Esteban Quilcate Otoya < quilcate.jo...@gmail.com> wrote: >

Re: [DISCUSS] KIP-935: Extend AlterConfigPolicy with existing configurations

2023-07-25 Thread Jorge Esteban Quilcate Otoya
ame`. Let me know if there's any issues with this; otherwise I will update the mail thread once the KIP is updated. Many thanks, Jorge. On Tue, 20 Jun 2023 at 11:56, Jorge Esteban Quilcate Otoya < quilcate.jo...@gmail.com> wrote: > Thanks Colin! You're right. I started this KIP o

[VOTE] KIP-934: Add DeleteTopicPolicy

2023-07-25 Thread Jorge Esteban Quilcate Otoya
Hi All, I'd like to start the vote for KIP-934: Add DeleteTopicPolicy: https://cwiki.apache.org/confluence/x/-xE0Dw Regards, Jorge.

Re: [VOTE] KIP-930: Tiered Storage Metrics

2023-07-25 Thread Jorge Esteban Quilcate Otoya
+1 (non-binding) Thanks, Abhijeet! On Tue, 25 Jul 2023 at 14:22, Abhijeet Kumar wrote: > Please find the updated link to the KIP: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-930%3A+Rename+ambiguous+Tiered+Storage+Metrics > > Updated the KIP as per our conversation on the

Re: [DISCUSS] KIP-930: Tiered Storage Metrics

2023-07-25 Thread Jorge Esteban Quilcate Otoya
Hi Abhijeet, Thanks for this KIP, I pretty much agree with the renaming and new names look good to me. Cheers, Jorge. On Tue, 25 Jul 2023 at 12:56, Satish Duggana wrote: > Hi Abhijeet, > Thanks for keeping this KIP only to renaming the existing metrics for > better clarity. These new names

[jira] [Created] (KAFKA-15231) Add ability to pause/resume Remote Log Manager tasks

2023-07-21 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-15231: Summary: Add ability to pause/resume Remote Log Manager tasks Key: KAFKA-15231 URL: https://issues.apache.org/jira/browse/KAFKA-15231

Re: [VOTE] KIP-852: Optimize calculation of size for log in remote tier

2023-07-13 Thread Jorge Esteban Quilcate Otoya
+1 (non-binding) Thanks for the KIP! Jorge. On Thu, 13 Jul 2023 at 12:26, Luke Chen wrote: > +1 (binding) from me. > > Thanks for the KIP! > > Luke > > On Sun, Jul 2, 2023 at 11:49 AM Kamal Chandraprakash < > kamal.chandraprak...@gmail.com> wrote: > > > +1 (non-binding). Thanks for the KIP! >

Re: [DISCUSS] KIP-852 Optimize calculation of size for log in remote tier

2023-07-13 Thread Jorge Esteban Quilcate Otoya
Thanks Divij. I was confusing with the metric tags used by clients that are based on topic and partition. Ideally partition label could be at a DEBUG recording level, but that's outside the scope of this KIP. Looks good to me, thanks again! Jorge. On Wed, 12 Jul 2023 at 15:55, Divij Vaidya

[jira] [Created] (KAFKA-15181) Race condition on partition assigned to TopicBasedRemoteLogMetadataManager

2023-07-12 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-15181: Summary: Race condition on partition assigned to TopicBasedRemoteLogMetadataManager Key: KAFKA-15181 URL: https://issues.apache.org/jira/browse

Re: [ANNOUNCE] New committer: Greg Harris

2023-07-11 Thread Jorge Esteban Quilcate Otoya
Congrats Greg!! On Tue 11. Jul 2023 at 15.20, Federico Valeri wrote: > Congrats Greg! > > On Tue, Jul 11, 2023 at 3:55 AM Luke Chen wrote: > > > > Congrats Greg! > > > > Luke > > > > On Tue, Jul 11, 2023 at 8:19 AM Matthew de Detrich > > wrote: > > > > > Congratulations, well deserved! > > >

[jira] [Created] (KAFKA-15147) Measure pending and outstanding Remote Segment operations

2023-07-05 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-15147: Summary: Measure pending and outstanding Remote Segment operations Key: KAFKA-15147 URL: https://issues.apache.org/jira/browse/KAFKA-15147

[jira] [Created] (KAFKA-15142) Add Client Metadata to RemoteStorageFetchInfo

2023-07-03 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-15142: Summary: Add Client Metadata to RemoteStorageFetchInfo Key: KAFKA-15142 URL: https://issues.apache.org/jira/browse/KAFKA-15142 Project

[jira] [Resolved] (KAFKA-15131) Improve RemoteStorageManager exception handling documentation

2023-07-03 Thread Jorge Esteban Quilcate Otoya (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15131?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jorge Esteban Quilcate Otoya resolved KAFKA-15131. -- Resolution: Fixed > Improve RemoteStorageManager except

Re: [DISCUSS] KIP-852 Optimize calculation of size for log in remote tier

2023-07-02 Thread Jorge Esteban Quilcate Otoya
Thanks Divij, this KIP is a super useful improvement to Tiered Storage. I have a couple of minor comments to the KIP, otherwise I'm +1 on this proposal: 1. APIs haven't used getter naming convention on TS as far as I can see (e.g `RLMM#remoteLogSegmentMetadata()`). We could rename the proposed

Re: [DISCUSS] KIP-940: Broker extension point for validating record contents at produce time

2023-06-30 Thread Jorge Esteban Quilcate Otoya
Thank you both for the replies! A couple more comments: On Tue, 27 Jun 2023 at 14:57, Edoardo Comar wrote: > Hi Jorge > thanks for the feedback. Comments inline below > > > 1. Similar to Kirk's first point, I'm also concerned on how would the > > plugin developers / operators be able to apply

[jira] [Created] (KAFKA-15135) RLM listener configurations passed but ignored by RLMM

2023-06-30 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-15135: Summary: RLM listener configurations passed but ignored by RLMM Key: KAFKA-15135 URL: https://issues.apache.org/jira/browse/KAFKA-15135

[jira] [Created] (KAFKA-15131) Improve RemoteStorageManager exception handling

2023-06-28 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-15131: Summary: Improve RemoteStorageManager exception handling Key: KAFKA-15131 URL: https://issues.apache.org/jira/browse/KAFKA-15131

Re: [DISCUSS] KIP-940: Broker extension point for validating record contents at produce time

2023-06-21 Thread Jorge Esteban Quilcate Otoya
Hi Eduardo, Adrian. Thanks for the KIP. I agree that allowing custom validations on the broker-side addresses a real gap as you clearly stated on the motivation. Some initial thoughts from my side: 1. Similar to Kirk's first point, I'm also concerned on how would the plugin developers /

Re: [DISCUSS] KIP-935: Extend AlterConfigPolicy with existing configurations

2023-06-20 Thread Jorge Esteban Quilcate Otoya
es are Optionals, and > express deletions as Optional.empty > > The old API should just stay the same, bugs and all, for compatibility > reasons. But for the new API we should choose one of the above, I think. > I'm not completely sure which... > > best, > Colin > > On

Re: [ANNOUNCE] New committer: Divij Vaidya

2023-06-13 Thread Jorge Esteban Quilcate Otoya
Awesome news! Congrats Divij! On Tue 13. Jun 2023 at 19.30, Tom Bentley wrote: > Congratulations Divij! > > On Tue, 13 Jun 2023 at 17:21, Mickael Maison > wrote: > > > Congratulations! > > > > On Tue, Jun 13, 2023 at 6:05 PM Yash Mayya wrote: > > > > > > Congratulations Divij! > > > > > > On

Re: [DISCUSS] KIP-935: Extend AlterConfigPolicy with existing configurations

2023-06-12 Thread Jorge Esteban Quilcate Otoya
See KAFKA-14195. Some deletions are not handled correctly. And this cannot > be fixed without a kip because of backwards compatibility. > > Colin > > On Wed, Jun 7, 2023, at 17:07, Jorge Esteban Quilcate Otoya wrote: > > Thank Colin. > > > > I've took a closer look on

Re: [DISCUSS] KIP-935: Extend AlterConfigPolicy with existing configurations

2023-06-07 Thread Jorge Esteban Quilcate Otoya
on the existing map and just including the resulting config instead, though this would break compatibility with existing implementations. Thanks, Jorge. On Wed, 7 Jun 2023 at 08:38, Colin McCabe wrote: > On Tue, Jun 6, 2023, at 06:57, Jorge Esteban Quilcate Otoya wrote: > > Tha

Re: [DISCUSS] KIP-934: Add DeleteTopicPolicy

2023-06-06 Thread Jorge Esteban Quilcate Otoya
solution than deletion policies. > > One last note: if we do this, we should pass the UUID of the topic as well > as the name. > > best, > Colin > > > On Mon, May 22, 2023, at 09:18, Jorge Esteban Quilcate Otoya wrote: > > Hi everyone, > > > > I'd like to st

Re: [DISCUSS] KIP-935: Extend AlterConfigPolicy with existing configurations

2023-06-06 Thread Jorge Esteban Quilcate Otoya
May 23, 2023, at 03:03, Christo Lolov wrote: > > Hello! > > > > This proposal will address problems with configuration dependencies > which I > > run into very frequently, so I am fully supporting the development of > this > > feature! > > > > Best,

[jira] [Created] (KAFKA-15051) docs: add missing connector plugin endpoint to documentation

2023-06-02 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-15051: Summary: docs: add missing connector plugin endpoint to documentation Key: KAFKA-15051 URL: https://issues.apache.org/jira/browse/KAFKA-15051

Re: [DISCUSS] KIP-934: Add DeleteTopicPolicy

2023-05-23 Thread Jorge Esteban Quilcate Otoya
5hpkfc1orm8r2tsmn [3] https://github.com/apache/kafka/pull/4281#issuecomment-1035154386 > > Best, > Christo > > On Mon, 22 May 2023 at 17:19, Jorge Esteban Quilcate Otoya < > quilcate.jo...@gmail.com> wrote: > > > Hi everyone, > > > > I'd like to start

[DISCUSS] KIP-934: Add DeleteTopicPolicy

2023-05-22 Thread Jorge Esteban Quilcate Otoya
Hi everyone, I'd like to start a discussion for KIP-934 < https://cwiki.apache.org/confluence/display/KAFKA/KIP-934%3A+Add+DeleteTopicPolicy> which proposes adding a new policy for when deleting topics. There have been related KIPs in the past that haven't been accepted and seem

[DISCUSS] KIP-935: Extend AlterConfigPolicy with existing configurations

2023-05-22 Thread Jorge Esteban Quilcate Otoya
Hi everyone, I'd like to start a discussion for KIP-935 < https://cwiki.apache.org/confluence/display/KAFKA/KIP-935%3A+Extend+AlterConfigPolicy+with+existing+configurations> which proposes extend AlterConfigPolicy with existing configuration to enable more complex policies. There have been

[jira] [Created] (KAFKA-15014) KIP-935: Extend AlterConfigPolicy with existing configurations

2023-05-22 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-15014: Summary: KIP-935: Extend AlterConfigPolicy with existing configurations Key: KAFKA-15014 URL: https://issues.apache.org/jira/browse/KAFKA-15014

[jira] [Created] (KAFKA-15013) KIP-934: Add DeleteTopicPolicy

2023-05-22 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-15013: Summary: KIP-934: Add DeleteTopicPolicy Key: KAFKA-15013 URL: https://issues.apache.org/jira/browse/KAFKA-15013 Project: Kafka

Re: [DISCUSS] KIP-877: Mechanism for plugins and connectors to register metrics

2023-05-15 Thread Jorge Esteban Quilcate Otoya
Hi Mickael, Just to check the status of this KIP as it looks very useful. I can see how new Tiered Storage interfaces and plugins may benefit from this. Cheers, Jorge. On Mon, 6 Feb 2023 at 23:00, Chris Egerton wrote: > Hi Mickael, > > I agree that adding a getter method for Monitorable isn't

Re: [VOTE] KIP-864: Add End-To-End Latency Metrics to Connectors

2023-05-10 Thread Jorge Esteban Quilcate Otoya
> > > wrote: > > > > > Thanks for the KIP > > > > > > +1 (binding) > > > > > > On Thu, Jan 26, 2023 at 4:36 PM Jorge Esteban Quilcate Otoya > > > wrote: > > > > > > > > Hi all, > > > > >

[jira] [Created] (KAFKA-14843) Connector plugins config endpoint does not include Common configs

2023-03-24 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-14843: Summary: Connector plugins config endpoint does not include Common configs Key: KAFKA-14843 URL: https://issues.apache.org/jira/browse/KAFKA-14843

[jira] [Created] (KAFKA-14815) Move Kafka documentation to Markdown/Hugo

2023-03-16 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-14815: Summary: Move Kafka documentation to Markdown/Hugo Key: KAFKA-14815 URL: https://issues.apache.org/jira/browse/KAFKA-14815 Project

Re: [ANNOUNCE] New Kafka PMC Member: Chris Egerton

2023-03-09 Thread Jorge Esteban Quilcate Otoya
So well deserved! Congratulations Chris!!! On Thu, 9 Mar 2023 at 22:09, Lucas Brutschy wrote: > Congratulations! > > On Thu, Mar 9, 2023 at 8:48 PM Roman Schmitz > wrote: > > > > Congratulations Chris! > > > > Am Do., 9. März 2023 um 20:33 Uhr schrieb Chia-Ping Tsai < > chia7...@gmail.com > >

[DISCUSS] Migrating documentation to Markdown/Hugo

2023-03-07 Thread Jorge Esteban Quilcate Otoya
Hi all, I wanted to share on the mailing list a follow up from https://issues.apache.org/jira/browse/KAFKA-2967 to gather feedback from a wider audience: I put together a static-site generation with Hugo, and migrated most of the documentation under `docs/` to Markdown using Pandoc:

Re: [VOTE] KIP-821: Connect Transforms support for nested structures

2023-01-27 Thread Jorge Esteban Quilcate Otoya
ks, > Bill > > On Fri, Jan 27, 2023 at 10:11 AM Jorge Esteban Quilcate Otoya < > quilcate.jo...@gmail.com> wrote: > > > Hi all, > > > > I'm pleased to announce that the KIP-821 has been accepted with 3 binding > > votes from Chris, John, Mickael, 1 non-bind

Re: [VOTE] KIP-821: Connect Transforms support for nested structures

2023-01-27 Thread Jorge Esteban Quilcate Otoya
at 16:21, Mickael Maison wrote: > +1 (binding) > Thanks for the KIP > > On Tue, Jun 28, 2022 at 10:42 PM Jorge Esteban Quilcate Otoya > wrote: > > > > Hi everyone, > > > > I'd like to bump this vote thread. Currently it's missing 1 +1 binding > vote &

Re: [DISCUSS] KIP-864: Add End-To-End Latency Metrics to Connectors

2023-01-26 Thread Jorge Esteban Quilcate Otoya
Chris > > On Mon, Dec 5, 2022 at 10:06 AM Jorge Esteban Quilcate Otoya < > quilcate.jo...@gmail.com> wrote: > > > Sure! I have a added the following to the proposed changes section: > > > > ``` > > The per-record metrics will definitely be added to

[VOTE] KIP-864: Add End-To-End Latency Metrics to Connectors

2023-01-26 Thread Jorge Esteban Quilcate Otoya
Hi all, I'd like to call for a vote on KIP-864, which proposes to add metrics to measure end-to-end latency in source and sink connectors. KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-864%3A+Add+End-To-End+Latency+Metrics+to+Connectors Discussion thread:

Re: Re: [DISCUSS] KIP-821: Connect Transforms support for nested structures

2023-01-25 Thread Jorge Esteban Quilcate Otoya
vote thread anyways; I'm sure this won't block us. > > Cheers, and thanks for all your hard work on this! > > Chris > > On Thu, Sep 1, 2022 at 1:33 PM Jorge Esteban Quilcate Otoya < > quilcate.jo...@gmail.com> wrote: > > > Hi Chris, > > > > T

Re: [VOTE] KIP-821: Connect Transforms support for nested structures

2023-01-25 Thread Jorge Esteban Quilcate Otoya
n wrote: > > +1 (binding). Thanks Jorge, great stuff! > > > > We should probably verify with the people that have already cast +1 votes > > that they're still on board, since the design has shifted a bit since the > > last vote was casted. > > > > On 2022/0

Re: [ANNOUNCE] New committer: Josep Prat

2022-12-20 Thread Jorge Esteban Quilcate Otoya
Congrats Josep!! On Tue, 20 Dec 2022, 17:31 Greg Harris, wrote: > Congratulations Josep! > > On Tue, Dec 20, 2022 at 9:29 AM Chris Egerton > wrote: > > > Congrats Josep! Well-earned. > > > > On Tue, Dec 20, 2022, 12:26 Jun Rao wrote: > > > > > Hi, Everyone, > > > > > > The PMC of Apache Kafka

Re: [DISCUSS] KIP-864: Add End-To-End Latency Metrics to Connectors

2022-12-05 Thread Jorge Esteban Quilcate Otoya
level? Otherwise, > it's possible that a vote for the KIP as it's written today would be a vote > in favor of unconditionally exposing these metrics at INFO level, even if > the performance testing reveals issues. > > Cheers, > > Chris > > On Sun, Dec 4, 2022 at 7:08 PM Jorge

Re: [DISCUSS] KIP-864: Add End-To-End Latency Metrics to Connectors

2022-12-04 Thread Jorge Esteban Quilcate Otoya
, Jorge. On Tue, 29 Nov 2022 at 19:40, Chris Egerton wrote: > Hi Jorge, > > Thanks! What were your thoughts on the possible benchmarking and/or > downgrading of per-record metrics to DEBUG? > > Cheers, > > Chris > > On Thu, Nov 24, 2022 at 8:20 AM Jorge Esteban

[jira] [Created] (KAFKA-14441) Benchmark performance impact of metrics library

2022-12-04 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-14441: Summary: Benchmark performance impact of metrics library Key: KAFKA-14441 URL: https://issues.apache.org/jira/browse/KAFKA-14441

Re: [DISCUSS] KIP-864: Add End-To-End Latency Metrics to Connectors

2022-11-24 Thread Jorge Esteban Quilcate Otoya
ding term where you can use, e.g., > "FileStreamSource" as the name of a connector class in a connector config > instead of "org.apache.kafka.connect.file.FileStreamSourceConnector". > > > Cheers, > > Chris > > On Fri, Nov 18, 2022 at 12:06 PM Jorge Este

[jira] [Created] (KAFKA-14409) Clean ProcessorParameters from casting

2022-11-19 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-14409: Summary: Clean ProcessorParameters from casting Key: KAFKA-14409 URL: https://issues.apache.org/jira/browse/KAFKA-14409 Project: Kafka

[jira] [Created] (KAFKA-14408) Consider enabling DEBUG log level on tests for Streams

2022-11-19 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-14408: Summary: Consider enabling DEBUG log level on tests for Streams Key: KAFKA-14408 URL: https://issues.apache.org/jira/browse/KAFKA-14408

Re: [DISCUSS] KIP-864: Add End-To-End Latency Metrics to Connectors

2022-11-18 Thread Jorge Esteban Quilcate Otoya
maximum and average values. > > Good question. I will remove it and change the record latency from DEBUG->INFO as it already cover the maximum metric. Hope it's clearer now, let me know if there any additional feedback. Thanks! > Thanks, > Mickael > > On Thu, Oct 20, 2022 at 9:58 P

Re: [ANNOUNCE] New Kafka PMC Member: Bruno Cadonna

2022-11-02 Thread Jorge Esteban Quilcate Otoya
Congratulations, Bruno!! On Wed, 2 Nov 2022, 09:06 Mickael Maison, wrote: > Congratulations Bruno! > > On Wed, Nov 2, 2022 at 8:33 AM Matthew Benedict de Detrich > wrote: > > > > Congratulations! > > > > On Wed, Nov 2, 2022 at 8:32 AM Josep Prat > > wrote: > > > > > Congrats Bruno! > > > > >

Re: [DISCUSS] KIP-864: Add End-To-End Latency Metrics to Connectors

2022-10-20 Thread Jorge Esteban Quilcate Otoya
ure wallclock - record timestamp on producer send callback - sink-batch-e2e-latency-before-put: measure time wallclock - record timestamp after consumer poll - sink-batch-e2e-latency-after-put: measure time wallclock - record timestamp after sink connector put. > Thanks again for the KIP! Looking forwa

Re: [DISCUSS] KIP-864: Add End-To-End Latency Metrics to Connectors

2022-09-15 Thread Jorge Esteban Quilcate Otoya
, a kindly reminder that the vote thread is open. Thanks! Jorge. On Thu, 8 Sept 2022 at 14:25, Jorge Esteban Quilcate Otoya < quilcate.jo...@gmail.com> wrote: > Great. I have updated the KIP to reflect this. > > Cheers, > Jorge. > > On Thu, 8 Sept 2022 at 12:26, Yash Mayya wro

[jira] [Created] (KAFKA-14232) Support for nested structures: InsertField

2022-09-13 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-14232: Summary: Support for nested structures: InsertField Key: KAFKA-14232 URL: https://issues.apache.org/jira/browse/KAFKA-14232 Project

[jira] [Created] (KAFKA-14231) Support for nested structures: ReplaceField

2022-09-13 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-14231: Summary: Support for nested structures: ReplaceField Key: KAFKA-14231 URL: https://issues.apache.org/jira/browse/KAFKA-14231 Project

[jira] [Created] (KAFKA-14230) Support for nested structures: Cast

2022-09-13 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-14230: Summary: Support for nested structures: Cast Key: KAFKA-14230 URL: https://issues.apache.org/jira/browse/KAFKA-14230 Project: Kafka

[jira] [Created] (KAFKA-14228) Support for nested structures: ValueToKey

2022-09-13 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-14228: Summary: Support for nested structures: ValueToKey Key: KAFKA-14228 URL: https://issues.apache.org/jira/browse/KAFKA-14228 Project

[jira] [Created] (KAFKA-14229) Support for nested structures: HoistValue

2022-09-13 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-14229: Summary: Support for nested structures: HoistValue Key: KAFKA-14229 URL: https://issues.apache.org/jira/browse/KAFKA-14229 Project

[jira] [Created] (KAFKA-14227) Support for nested structures: MaskField

2022-09-13 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-14227: Summary: Support for nested structures: MaskField Key: KAFKA-14227 URL: https://issues.apache.org/jira/browse/KAFKA-14227 Project

[jira] [Created] (KAFKA-14226) Introduce support for nested structures

2022-09-13 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-14226: Summary: Introduce support for nested structures Key: KAFKA-14226 URL: https://issues.apache.org/jira/browse/KAFKA-14226 Project

Re: [VOTE] KIP-864: Add End-To-End Latency Metrics to Connectors

2022-09-12 Thread Jorge Esteban Quilcate Otoya
Pressed send to soon. Updating subject. On Mon, 12 Sept 2022 at 11:45, Jorge Esteban Quilcate Otoya < quilcate.jo...@gmail.com> wrote: > Hi everyone, > > Thank you all for the positive discussion about KIP-864. > > I would like to start a voting thread, to introd

[VOTE]

2022-09-12 Thread Jorge Esteban Quilcate Otoya
Hi everyone, Thank you all for the positive discussion about KIP-864. I would like to start a voting thread, to introduce these new metrics for Connector tasks KIP: https://cwiki.apache.org/confluence/x/6I5rDQ Thanks, Jorge

Re: [DISCUSS] KIP-864: Add End-To-End Latency Metrics to Connectors

2022-09-08 Thread Jorge Esteban Quilcate Otoya
Great. I have updated the KIP to reflect this. Cheers, Jorge. On Thu, 8 Sept 2022 at 12:26, Yash Mayya wrote: > Thanks, I think it makes sense to define these metrics at a DEBUG recording > level. > > On Thu, Sep 8, 2022 at 2:51 PM Jorge Esteban Quilcate Otoya < > quilca

Re: [DISCUSS] KIP-864: Add End-To-End Latency Metrics to Connectors

2022-09-08 Thread Jorge Esteban Quilcate Otoya
that these are single-record metrics compared to existing batch-only could be explicitly defined by setting these metrics at a DEBUG or TRACE metric recording level, leaving the existing at INFO level. wdyt? > > Thanks, > Yash > > On Tue, Sep 6, 2022 at 4:42 PM Jorge Esteban Quilcate

Re: [DISCUSS] KIP-864: Add End-To-End Latency Metrics to Connectors

2022-09-06 Thread Jorge Esteban Quilcate Otoya
ughts as well. > > > > The other point Yash mentioned about the slightly flawed definition of > e2e > > is also true in a sense. But I have a feeling that's one the records are > > polled by the connector tasks, it would be difficult to track the final > leg > > via the framewor

Re: [DISCUSS] KIP-864: Add End-To-End Latency Metrics to Connectors

2022-09-01 Thread Jorge Esteban Quilcate Otoya
d the wall clock time, > it > > looks like a per record metric. However, the put-batch time measures the > > time to put a batch of records to external sink. So, I would assume the 2 > > can't be added as is to compute the e2e latency. Maybe I am missing > > something here.

  1   2   3   >