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

2022-09-01 Thread Jorge Esteban Quilcate Otoya
Hi Chris, Thanks for your feedback! 1. Yes, it will be context-dependent. I have added rules and scenarios to the nested notation to cover the happy path and edge cases. In short, backticks will be not be considered as part of the field name when they are wrapping a field name: first backtick at

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

2022-08-30 Thread Jorge Esteban Quilcate Otoya
Hi all, I'd like to start a discussion thread on KIP-864: Add End-To-End Latency Metrics to Connectors. This KIP aims to improve the metrics available on Source and Sink Connectors to measure end-to-end latency, including source and sink record conversion time, and sink record e2e latency

[jira] [Created] (KAFKA-14191) Add end-to-end latency metrics to Connectors

2022-08-30 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-14191: Summary: Add end-to-end latency metrics to Connectors Key: KAFKA-14191 URL: https://issues.apache.org/jira/browse/KAFKA-14191 Project

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

2022-08-12 Thread Jorge Esteban Quilcate Otoya
have worked nicely for them. > > Regards, > Robert > > On Fri, Aug 12, 2022 at 10:01 AM Jorge Esteban Quilcate Otoya < > quilcate.jo...@gmail.com> wrote: > > > Thanks Robert! And sorry for the late reply. > > > > That's a great catch and will require the current pro

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

2022-08-12 Thread Jorge Esteban Quilcate Otoya
such as root.`child.with.dot`. See Jsonata syntax for example: > http://docs.jsonata.org/simple > > Robert > > On Wed, Jun 29, 2022 at 3:34 AM Jorge Esteban Quilcate Otoya < > quilcate.jo...@gmail.com> wrote: > > > Thanks Chris! I have updated the KIP to include thi

Re: [ANNOUNCE] New Kafka PMC Member: A. Sophie Blee-Goldman

2022-08-02 Thread Jorge Esteban Quilcate Otoya
Congratulations, Sophie! On Tue, 2 Aug 2022, 23:26 Colin McCabe, wrote: > Congratulations! > > best, > Colin > > > On Tue, Aug 2, 2022, at 12:09, Matthew Benedict de Detrich wrote: > > Congratulations Sophie! > > > > -- > > Matthew de Detrich > > Aiven Deutschland GmbH > > Immanuelkirchstraße

Re: [ANNOUNCE] New Committer: Chris Egerton

2022-07-25 Thread Jorge Esteban Quilcate Otoya
Congratulations Chris! On Mon, 25 Jul 2022 at 20:27, Robin Moffatt wrote: > Congrats Chris! > > > -- > > Robin Moffatt | Principal Developer Advocate | ro...@confluent.io | @rmoff > > > On Mon, 25 Jul 2022 at 17:26, Mickael Maison wrote: > > > Hi all, > > > > The PMC for Apache Kafka has

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

2022-06-29 Thread Jorge Esteban Quilcate Otoya
> > > I'll bump the vote thread as well to see if there's agreement on adding > > this feature to Connect. > > > > Cheers, > > Jorge. > > > > On Wed, 15 Jun 2022 at 23:02, Jorge Esteban Quilcate Otoya < > > quilcate.jo...@gmail.com> wrote: &g

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

2022-06-28 Thread Jorge Esteban Quilcate Otoya
and it looks good to me. > > > > I’m +1 (binding) > > > > Thanks, > > John > > > > On Thu, Apr 21, 2022, at 09:10, Chris Egerton wrote: > > > This is a worthwhile addition to the SMTs that ship out of the box with > > > Kafka Connect. +1 non-binding

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

2022-06-28 Thread Jorge Esteban Quilcate Otoya
get delayed as a result. >> >> I'd be really curious to hear from Joshua and Tom on this front, though. >> Is >> it acceptable to move more incrementally here and settle on the syntax >> version property as our means of introducing new features, or is it >> prefe

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

2022-06-15 Thread Jorge Esteban Quilcate Otoya
le to implement things monolithically and try to get everything (or > at least, as much as possible) right the first time? > > Thanks again for your continued effort on this KIP! > > Cheers, > > Chris > > On Wed, Jun 8, 2022 at 5:41 PM Jorge Esteban Quilcate Otoya < >

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

2022-06-08 Thread Jorge Esteban Quilcate Otoya
an important part > of Joshua's KIP and his feedback on this KIP; I think it's worth pursuing > if we can. > My understanding from Joshua's feedback is that by including support for deep-scan, we are already covering the recursive functionality. Though, I may be missing something. >

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

2022-05-24 Thread Jorge Esteban Quilcate Otoya
y-handed, is to > change it to "field.syntax.version" with permitted values of "V1" (default, > equivalent to "field.style = plain") and "V2" (equivalent to "field.style = > nested"). This would leave us room in the future to make further chan

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

2022-05-15 Thread Jorge Esteban Quilcate Otoya
ath and have different escaping rules. > An > > > > alternative would be to come up with a parallel set of config names > > (e.g. > > > > as well as "field" an SMT might support "path") which were defined to > > > > always take paths, thus av

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

2022-04-21 Thread Jorge Esteban Quilcate Otoya
Hi all, I'd like to start a vote on KIP-821: https://cwiki.apache.org/confluence/display/KAFKA/KIP-821%3A+Connect+Transforms+support+for+nested+structures Thanks, Jorge

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

2022-04-20 Thread Jorge Esteban Quilcate Otoya
ine when it's in separate lines in the editor. Hopefully, it's fixed now. > > Thanks again for working through this, and congratulations on a > well-written KIP! > > Cheers, > > Chris > > On Tue, Apr 19, 2022 at 2:06 PM Jorge Esteban Quilcate Otoya < > quilcate.

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

2022-04-19 Thread Jorge Esteban Quilcate Otoya
"on.missing.parent" and "on.existing.field" > as the new property names for InsertField. > - Why is the "on_existing_field" (or "on.existing.field") property only > applied when the field style is nested? Couldn't it be useful for > non-nested field

Re: [VOTE] KIP-622 Add currentSystemTimeMs and currentStreamTimeMs to ProcessorContext

2022-04-14 Thread Jorge Esteban Quilcate Otoya
Yet another quick FYI. While implementing KIP-820, we found that `api.MockProcessorContext` was missing these new methods as well. We added the new method to the new `api.MockProcessorContext` via https://issues.apache.org/jira/browse/KAFKA-13654. Please let us know if there are any concerns. I

[jira] [Created] (KAFKA-13822) Update Kafka Streams Adjust Thread Count tests to new Processor API

2022-04-12 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-13822: Summary: Update Kafka Streams Adjust Thread Count tests to new Processor API Key: KAFKA-13822 URL: https://issues.apache.org/jira/browse/KAFKA

[jira] [Created] (KAFKA-13821) Update Kafka Streams demo to new Processor API

2022-04-12 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-13821: Summary: Update Kafka Streams demo to new Processor API Key: KAFKA-13821 URL: https://issues.apache.org/jira/browse/KAFKA-13821

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

2022-04-09 Thread Jorge Esteban Quilcate Otoya
be if any of the field elements > specified with InsertField already exist in the record value? Will we just > overwrite them? What's the behavior of InsertField today under similar > circumstances? > The current behavior is to overwrite the value. > > Cheers, > > Chris &

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

2022-03-31 Thread Jorge Esteban Quilcate Otoya
h maps as well. > Can we update it to reflect this? > > References: > > [1] - https://stackoverflow.com/a/17808731 > [2] - > > https://github.com/apache/kafka/blob/7243facb8d69a7252e6b9556b5eaee13e41bab7f/connect/api/src/main/java/org/apache/kafka/connect/transforms/Transfor

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

2022-03-28 Thread Jorge Esteban Quilcate Otoya
s that involve, for > example, a > > > combination of the Flatten and Cast SMTs) is pretty high. I think this > > > should be an opt-in feature, at least until the next major release. One > > way > > > we could accomplish this is by introducing a new "field.st

Re: [DISCUSS] KIP-634: Complementary support for headers in Kafka Streams DSL

2022-03-21 Thread Jorge Esteban Quilcate Otoya
ecord` as > value. > >> - Extend `Record` to: > >>- Implement `RecordMetadata` to expose `topic`, `partition`, and > `offset` > >> - Use `Headers` abstraction introduce on this KIP instead of core one > >> > >> KIP: > >> > https://c

Re: [VOTE] KIP-820: Extend KStream process with new Processor API

2022-03-21 Thread Jorge Esteban Quilcate Otoya
lso include `final String... > > stateStoreNames`. > > >> And in javadocs explains that if users want to connect state stores to > > this > > >> processor, they could use the `connectState` API instead. > > >> > > >> Otherwise, I'm +1. > >

[jira] [Resolved] (KAFKA-13742) Quota byte-rate/request metrics are loaded only when at least one quota is register

2022-03-17 Thread Jorge Esteban Quilcate Otoya (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13742?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jorge Esteban Quilcate Otoya resolved KAFKA-13742. -- Resolution: Not A Problem Given that I got a better

[jira] [Resolved] (KAFKA-13744) Quota metric tags are inconsistent

2022-03-17 Thread Jorge Esteban Quilcate Otoya (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13744?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jorge Esteban Quilcate Otoya resolved KAFKA-13744. -- Resolution: Not A Problem > Quota metric t

Re: [DISCUSS] KIP-820: Extend KStream process with new Processor API

2022-03-17 Thread Jorge Esteban Quilcate Otoya
FixedKeyProcessorContext > - FixedKeyRecord > > > -Matthias > > > On 3/10/22 3:15 PM, Jorge Esteban Quilcate Otoya wrote: > > Thanks all! > > > > I agree with Matthias and Jon on going forward with the new > > `FixedKeyRecord` approach. > > The KI

[jira] [Created] (KAFKA-13744) Quota metric tags are inconsistent

2022-03-15 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-13744: Summary: Quota metric tags are inconsistent Key: KAFKA-13744 URL: https://issues.apache.org/jira/browse/KAFKA-13744 Project: Kafka

[jira] [Created] (KAFKA-13742) Quota byte-rate/request metrics are loaded only when at least one quota is register

2022-03-15 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-13742: Summary: Quota byte-rate/request metrics are loaded only when at least one quota is register Key: KAFKA-13742 URL: https://issues.apache.org/jira

Re: [DISCUSS] KIP-820: Extend KStream process with new Processor API

2022-03-10 Thread Jorge Esteban Quilcate Otoya
emind" (instead of say "enforce", since we cannot > > really do it) users the semantics of "processValue". Personally I felt > that > > adding the new set of APIs for that purpose only is a bit overkill, and > > hence was leaning towards just the runtime validati

Re: [DISCUSS] KIP-820: Extend KStream process with new Processor API

2022-03-07 Thread Jorge Esteban Quilcate Otoya
nstraint. > > As I said, with this proposal, the devil is in the details, > so if anyone thinks the API can be simplified, I suggest you > check out the branch and try out your proposal. I'd be very > happy to have a simplier solution, but I'm also pretty sure > this complexity is

Re: [DISCUSS] KIP-820: Extend KStream process with new Processor API

2022-03-06 Thread Jorge Esteban Quilcate Otoya
ngly prefer compile > time checks and I am happy to extend the API surface area to achieve it > -- however, I won't be surprised if others don't like this idea... > > > > -Matthias > > On 2/27/22 6:20 AM, Jorge Esteban Quilcate Otoya wrote: > > Thanks, Guozhang. > &g

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

2022-02-28 Thread Jorge Esteban Quilcate Otoya
like the `field.style` configuration flag approach. Thanks for pointing out the `recursive` approach. Will add `nested` at the moment, let's check the demand for `recursive` to consider it as part of this or another KIP. I have added the following on the KIP: ``` Future KIPs could extend this su

Re: [DISCUSS] KIP-820: Extend KStream process with new Processor API

2022-02-27 Thread Jorge Esteban Quilcate Otoya
for those who > mistakenly change the key, I think that enforcing everyone to `setValue` > may incur more costs.. > > Guozhang > > On Fri, Feb 25, 2022 at 12:54 PM Jorge Esteban Quilcate Otoya < > quilcate.jo...@gmail.com> wrote: > > > Hi all, > > > > Appreciate

Re: [DISCUSS] KIP-820: Extend KStream process with new Processor API

2022-02-25 Thread Jorge Esteban Quilcate Otoya
reciate > Matthias > > > > > extending the offer, but if Jorge doesn’t want to redesign the dsl > right > > > > > now, we’re better off just accepting the work he’s willing to do. > > > > > > > > > > Specifically, this KIP is quite a

Re: [DISCUSS] KIP-820: Extend KStream process with new Processor API

2022-02-17 Thread Jorge Esteban Quilcate Otoya
eturn to have a single consolidated > > ProcessorSupplier programming interface (i.e. we would eventually > > deprecate ValueTransformerWithKeySupplier). > > > > On Wed, Feb 16, 2022 at 10:57 AM Jorge Esteban Quilcate Otoya < > > quilcate.jo...@gmail.com> wrote: >

Re: [DISCUSS] KIP-820: Extend KStream process with new Processor API

2022-02-16 Thread Jorge Esteban Quilcate Otoya
fic comments, but I just wanted to mention I like the > direction of > >>> the KIP. My team is a big user of "transform" methods because of the > >>> ability to chain them, and I have always found the terminology > challenging > >>> to

[VOTE] KIP-634: Complementary support for headers and record metadata in Kafka Streams DSL

2022-02-15 Thread Jorge Esteban Quilcate Otoya
Hi all, I'd like to start a vote on KIP-634 which proposes extending support for Headers and Record Metadata in Kafka Streams https://cwiki.apache.org/confluence/display/KAFKA/KIP-634%3A+Complementary+support+for+headers+and+record+metadata+in+Kafka+Streams+DSL Thanks, Jorge

[VOTE] KIP-820: Extend KStream process with new Processor API

2022-02-15 Thread Jorge Esteban Quilcate Otoya
Hi all, I'd like to start a vote on KIP-820 which proposes extending KStream to use the new Processor API https://cwiki.apache.org/confluence/display/KAFKA/KIP-820%3A+Extend+KStream+process+with+new+Processor+API Thanks, Jorge

Re: [DISCUSS] KIP-820: Extend KStream process with new Processor API

2022-02-15 Thread Jorge Esteban Quilcate Otoya
about just deprecating the KStream methods and not the > > > Transform* interfaces is that you can keep your proposal just scoped to > > > KStream and not have any consequences for the rest of the DSL. > > > > > > Thanks again, > > > John > >

Re: [DISCUSS] KIP-820: Extend KStream process with new Processor API

2022-02-14 Thread Jorge Esteban Quilcate Otoya
ting the KStream methods and not the > Transform* interfaces is that you can keep your proposal just scoped to > KStream and not have any consequences for the rest of the DSL. > > Thanks again, > John > > On Fri, Feb 11, 2022, at 06:43, Jorge Esteban Quilcate Otoya wrote: > > Thanks, Jo

Re: [DISCUSS] KIP-634: Complementary support for headers in Kafka Streams DSL

2022-02-11 Thread Jorge Esteban Quilcate Otoya
! On Thu, 10 Feb 2022 at 13:15, Jorge Esteban Quilcate Otoya < quilcate.jo...@gmail.com> wrote: > > What do you think about instead adding topic and > partition to Record? > > This is a very interesting idea. Forgot to consider this addition from > KIP-478. > > `Reco

Re: [DISCUSS] KIP-820: Extend KStream process with new Processor API

2022-02-11 Thread Jorge Esteban Quilcate Otoya
o the existing > > > > transformers, by the way, that are also forbidden to forward > > > > anything during punctuation. > > > > > > > > For what it's worth, I think this is the best tradeoff. > > > > > > > > The only alternative

Re: [DISCUSS] KIP-820: Extend KStream process with new Processor API

2022-02-10 Thread Jorge Esteban Quilcate Otoya
t` have to be a public API? It seems to > me > > > > that this can be completely abstracted away from user interfaces as > an > > > > internal class, and we call the `setKey` before calling > > user-instantiated > > > > `process` function, and then in

Re: [DISCUSS] KIP-634: Complementary support for headers in Kafka Streams DSL

2022-02-10 Thread Jorge Esteban Quilcate Otoya
> (or if we're using a default serde from the config), will > Streams automatically wrap it downstream of the record- > mapping operator? > > Otherwise, your proposal looks good to me! > > Thanks, > -John > > On Tue, 2022-02-08 at 18:06 +, Jorge Esteban Quilcate > Ot

Re: [ANNOUNCE] New committer: Luke Chen

2022-02-10 Thread Jorge Esteban Quilcate Otoya
Congratulations Luke! On Thu, 10 Feb 2022 at 09:20, Bruno Cadonna wrote: > Congrats, Luke! Very well deserved! > > Best, > Bruno > > On 10.02.22 09:20, Manikumar wrote: > > Congrats Luke! > > > > On Thu, Feb 10, 2022 at 1:36 PM Mickael Maison > > > wrote: > > > >> Congratulations Luke! > >> >

[jira] [Created] (KAFKA-13662) Migrate DeserializationExceptionHandler to latest ProcessorContext API

2022-02-09 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-13662: Summary: Migrate DeserializationExceptionHandler to latest ProcessorContext API Key: KAFKA-13662 URL: https://issues.apache.org/jira/browse/KAFKA

Re: [DISCUSS] KIP-820: Extend KStream process with new Processor API

2022-02-09 Thread Jorge Esteban Quilcate Otoya
more efficient. > I'm not sure how much overhead it may incur to check if the key did not > change: if it is just a reference equality check maybe it's okay. What's > your take on this? > > > Guozhang > > On Tue, Feb 8, 2022 at 5:17 AM Jorge Esteban Quilcate Otoya < > quilc

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

2022-02-08 Thread Jorge Esteban Quilcate Otoya
Hi Dev team, I'd like to start a new discussion thread on Kafka Connect KIP-821: https://cwiki.apache.org/confluence/display/KAFKA/KIP-821%3A+Connect+Transforms+support+for+nested+structures This KIP is aimed to include support for nested structures on the existing SMTs — where this make sense.

[jira] [Created] (KAFKA-13656) Connect Transforms support for nested structures

2022-02-08 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-13656: Summary: Connect Transforms support for nested structures Key: KAFKA-13656 URL: https://issues.apache.org/jira/browse/KAFKA-13656

Re: [DISCUSS] KIP-634: Complementary support for headers in Kafka Streams DSL

2022-02-08 Thread Jorge Esteban Quilcate Otoya
/pull/10265/files for both Processors backing changes on the KStream API. - It is proposing to still extend `To` class for backwards compatibility. Looking forward to your feedback. Regards, Jorge. On Thu, 4 Mar 2021 at 18:38, Jorge Esteban Quilcate Otoya < quilcate.jo...@gmail.com> wrote:

[DISCUSS] KIP-820: Extend KStream process with new Processor API

2022-02-08 Thread Jorge Esteban Quilcate Otoya
Hi Dev team, I'd like to start a new discussion thread on Kafka Streams KIP-820: https://cwiki.apache.org/confluence/display/KAFKA/KIP-820%3A+Extend+KStream+process+with+new+Processor+API This KIP is aimed to extend the current `KStream#process` API to return output values that could be chained

[jira] [Created] (KAFKA-13654) Extend KStream process with new Processor API

2022-02-08 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-13654: Summary: Extend KStream process with new Processor API Key: KAFKA-13654 URL: https://issues.apache.org/jira/browse/KAFKA-13654 Project

[jira] [Resolved] (KAFKA-13117) After processors, migrate TupleForwarder and CacheFlushListener

2021-11-23 Thread Jorge Esteban Quilcate Otoya (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13117?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jorge Esteban Quilcate Otoya resolved KAFKA-13117. -- Resolution: Fixed [https://github.com/apache/kafka/pull/11481

[jira] [Resolved] (KAFKA-10543) Convert KTable joins to new PAPI

2021-11-09 Thread Jorge Esteban Quilcate Otoya (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10543?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jorge Esteban Quilcate Otoya resolved KAFKA-10543. -- Resolution: Fixed https://github.com/apache/kafka/pull/11412

[jira] [Created] (KAFKA-13429) Update gitignore to include new modules

2021-11-02 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-13429: Summary: Update gitignore to include new modules Key: KAFKA-13429 URL: https://issues.apache.org/jira/browse/KAFKA-13429 Project

[jira] [Resolved] (KAFKA-10540) Convert KStream aggregations to new PAPI

2021-10-19 Thread Jorge Esteban Quilcate Otoya (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10540?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jorge Esteban Quilcate Otoya resolved KAFKA-10540. -- Resolution: Fixed https://github.com/apache/kafka/pull/11315

[jira] [Resolved] (KAFKA-10539) Convert KStreamImpl joins to new PAPI

2021-10-19 Thread Jorge Esteban Quilcate Otoya (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10539?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jorge Esteban Quilcate Otoya resolved KAFKA-10539. -- Resolution: Fixed https://github.com/apache/kafka/pull/11356

[jira] [Resolved] (KAFKA-10544) Convert KTable aggregations to new PAPI

2021-09-23 Thread Jorge Esteban Quilcate Otoya (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10544?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jorge Esteban Quilcate Otoya resolved KAFKA-10544. -- Resolution: Fixed https://github.com/apache/kafka/pull/11316

[jira] [Resolved] (KAFKA-10542) Convert KTable maps to new PAPI

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

[jira] [Resolved] (KAFKA-13201) Convert KTable suppress to new PAPI

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

[jira] [Created] (KAFKA-13201) Convert KTable suppress to new PAPI

2021-08-13 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-13201: Summary: Convert KTable suppress to new PAPI Key: KAFKA-13201 URL: https://issues.apache.org/jira/browse/KAFKA-13201 Project: Kafka

[jira] [Resolved] (KAFKA-10541) Convert KTable filters to new PAPI

2021-07-20 Thread Jorge Esteban Quilcate Otoya (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10541?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jorge Esteban Quilcate Otoya resolved KAFKA-10541. -- Fix Version/s: 3.0.0 Assignee: John Roesler

[jira] [Resolved] (KAFKA-10537) Convert KStreamImpl filters to new PAPI

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

[jira] [Resolved] (KAFKA-10538) Convert KStreamImpl maps to new PAPI

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

[DISCUSS] KIP-756: Move StreamsResetter tool outside of core

2021-06-08 Thread Jorge Esteban Quilcate Otoya
Hi everyone, Hope you are all well and safe. I'd like to propose the following KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-756%3A+Move+StreamsResetter+tool+outside+of+core The goal is to move `StreamsResetter` tool outside of `core` module, and allow Kafka Streams users to access

[jira] [Resolved] (KAFKA-12536) Add Instant-based methods to ReadOnlySessionStore

2021-05-07 Thread Jorge Esteban Quilcate Otoya (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12536?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jorge Esteban Quilcate Otoya resolved KAFKA-12536. -- Fix Version/s: 3.0.0 Resolution: Fixed > Add Inst

[jira] [Resolved] (KAFKA-10434) Remove deprecated methods on WindowStore

2021-05-07 Thread Jorge Esteban Quilcate Otoya (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10434?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jorge Esteban Quilcate Otoya resolved KAFKA-10434. -- Resolution: Fixed > Remove deprecated methods on WindowSt

[jira] [Resolved] (KAFKA-12451) Remove deprecation annotation on long-based read operations in WindowStore

2021-05-07 Thread Jorge Esteban Quilcate Otoya (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12451?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jorge Esteban Quilcate Otoya resolved KAFKA-12451. -- Fix Version/s: 3.0.0 Resolution: Fixed https

[jira] [Resolved] (KAFKA-12450) Remove deprecated methods from ReadOnlyWindowStore

2021-05-07 Thread Jorge Esteban Quilcate Otoya (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12450?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jorge Esteban Quilcate Otoya resolved KAFKA-12450. -- Fix Version/s: 3.0.0 Resolution: Fixed > Rem

Re: [VOTE} KIP-733: change Kafka Streams default replication factor config

2021-04-15 Thread Jorge Esteban Quilcate Otoya
+1 Thanks Matthias! On Thu, 15 Apr 2021, 20:48 Israel Ekpo, wrote: > Makes perfect sense to me > > +1 as well. > > Thanks Matthias. > > > On Thu, Apr 15, 2021 at 2:41 PM Guozhang Wang wrote: > > > +1 as well. Thanks! > > > > On Wed, Apr 14, 2021 at 4:30 PM Bill Bejeck wrote: > > > > > Thanks

[jira] [Resolved] (KAFKA-12533) Migrate KStream stateless operators to new Processor API

2021-03-31 Thread Jorge Esteban Quilcate Otoya (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12533?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jorge Esteban Quilcate Otoya resolved KAFKA-12533. -- Resolution: Duplicate > Migrate KStream stateless operat

[jira] [Resolved] (KAFKA-12532) Migrate Stream operators to new Processor API

2021-03-31 Thread Jorge Esteban Quilcate Otoya (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12532?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jorge Esteban Quilcate Otoya resolved KAFKA-12532. -- Resolution: Duplicate > Migrate Stream operators to

[jira] [Created] (KAFKA-12536) Add Instant-based methods to ReadOnlySessionStore

2021-03-23 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-12536: Summary: Add Instant-based methods to ReadOnlySessionStore Key: KAFKA-12536 URL: https://issues.apache.org/jira/browse/KAFKA-12536

[jira] [Created] (KAFKA-12533) Migrate KStream stateless operators to new Processor API

2021-03-23 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-12533: Summary: Migrate KStream stateless operators to new Processor API Key: KAFKA-12533 URL: https://issues.apache.org/jira/browse/KAFKA-12533

[jira] [Created] (KAFKA-12532) Migrate Stream operators to new Processor API

2021-03-23 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-12532: Summary: Migrate Stream operators to new Processor API Key: KAFKA-12532 URL: https://issues.apache.org/jira/browse/KAFKA-12532 Project

Re: [VOTE] KIP-666: Add Instant-based methods to ReadOnlySessionStore

2021-03-12 Thread Jorge Esteban Quilcate Otoya
gt; > > > I’m +1 (binding) > > > > > > -John > > > > > > On Mon, Sep 21, 2020, at 12:35, Sophie Blee-Goldman wrote: > > >> Thanks for pointing out the vote in the discussion thread, this email > > >> somehow skipped my inbox ¯\_(ツ

[jira] [Created] (KAFKA-12450) Remove deprecated methods from ReadOnlyWindowStore

2021-03-10 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-12450: Summary: Remove deprecated methods from ReadOnlyWindowStore Key: KAFKA-12450 URL: https://issues.apache.org/jira/browse/KAFKA-12450

[jira] [Created] (KAFKA-12451) Remove deprecation annotation on long-based read operations in WindowStore

2021-03-10 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-12451: Summary: Remove deprecation annotation on long-based read operations in WindowStore Key: KAFKA-12451 URL: https://issues.apache.org/jira/browse

[jira] [Created] (KAFKA-12449) Remove deprecated WindowStore#put

2021-03-10 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-12449: Summary: Remove deprecated WindowStore#put Key: KAFKA-12449 URL: https://issues.apache.org/jira/browse/KAFKA-12449 Project: Kafka

Re: [DISCUSS] KIP-634: Complementary support for headers in Kafka Streams DSL

2021-03-04 Thread Jorge Esteban Quilcate Otoya
+Streams+DSL Looking forward to your feedback, Cheers and stay safe, Jorge. On Thu, Oct 29, 2020 at 12:33 AM Jorge Esteban Quilcate Otoya < quilcate.jo...@gmail.com> wrote: > Thanks Sophie! Haven't followed KIP-478 but sounds great. > I'll be happy to help on that migration to

[jira] [Created] (KAFKA-12287) Add WARN logging on consumer-groups when reset-offsets by timestamp or duration can't find an offset and defaults to latest.

2021-02-03 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-12287: Summary: Add WARN logging on consumer-groups when reset-offsets by timestamp or duration can't find an offset and defaults to latest. Key: KAFKA-12287

Re: [DISCUSS] KIP-706: Add method "Producer#produce" to return CompletionStage instead of Future

2021-01-30 Thread Jorge Esteban Quilcate Otoya
Hi all, My 2c: Pros of "Produce": - Same as Kafka Protocol "ProduceRequest" and simple enough "producer#produce()"; though consumer also differ between "FetchRequest" and "consumer#poll". - As reference, other libraries also use "produce", e.g.: librdkafka-derived libraries:

Re: [DISCUSS] KIP-634: Complementary support for headers in Kafka Streams DSL

2020-10-28 Thread Jorge Esteban Quilcate Otoya
all > DSL operators > to the new PAPI before adding header support for those operators. But that > definitely > sounds achievable here > > On Wed, Oct 28, 2020 at 11:10 AM Jorge Esteban Quilcate Otoya < > quilcate.jo...@gmail.com> wrote: > > > Hi Matt

Re: [DISCUSS] KIP-634: Complementary support for headers in Kafka Streams DSL

2020-10-28 Thread Jorge Esteban Quilcate Otoya
an `Iterable` similar to `ProducerRecord`)? We could either > add it to your KIP or do a new KIP just for the PAPI. > > Thoughts? > > > -Matthias > > On 7/16/20 4:05 PM, Jorge Esteban Quilcate Otoya wrote: > > Hi everyone, > > > > Bumping this thread to c

Re: [ANNOUNCE] New committer: A. Sophie Blee-Goldman

2020-10-19 Thread Jorge Esteban Quilcate Otoya
Congratulations Sophie!! So well deserved. On Mon, Oct 19, 2020 at 6:31 PM Bill Bejeck wrote: > Congratulations Sophie! > > -Bill > > On Mon, Oct 19, 2020 at 12:49 PM Leah Thomas wrote: > > > Congrats Sophie! > > > > On Mon, Oct 19, 2020 at 11:41 AM Matthias J. Sax > wrote: > > > > > Hi all,

Re: [DISCUSS] KIP-666: Add Instant-based methods to ReadOnlySessionStore

2020-09-21 Thread Jorge Esteban Quilcate Otoya
reviewed the KIP, and it looks good to > me. > > Thanks, > John > > On Tue, Sep 1, 2020, at 12:35, Jorge Esteban Quilcate Otoya wrote: > > Thanks Sophie! > > > > > one nit: you missed updating the startTime long to Instant in both > > appearances of t

Re: [VOTE] KIP-667: Remove deprecated methods from ReadOnlyWindowStore

2020-09-08 Thread Jorge Esteban Quilcate Otoya
m +1 (binding) for doing this in > 3.0. > > Thanks, > -John > > On Mon, 2020-09-07 at 12:18 +0100, Jorge Esteban Quilcate > Otoya wrote: > > Hi everyone, > > > > I'd like to start a thread to vote for KIP-667 and fix deprecation notes > on > > ReadOnlyWindowSt

[VOTE] KIP-667: Remove deprecated methods from ReadOnlyWindowStore

2020-09-07 Thread Jorge Esteban Quilcate Otoya
Hi everyone, I'd like to start a thread to vote for KIP-667 and fix deprecation notes on ReadOnlyWindowStore: https://cwiki.apache.org/confluence/display/KAFKA/KIP-667%3A+Remove+deprecated+methods+from+ReadOnlyWindowStore Discussion thread:

[VOTE] KIP-666: Add Instant-based methods to ReadOnlySessionStore

2020-09-07 Thread Jorge Esteban Quilcate Otoya
Hi everyone, I'd like to start a thread to vote for KIP-666 and align instant-based operations on Interactive Query APIs between Window and Session stores: https://cwiki.apache.org/confluence/display/KAFKA/KIP-666%3A+Add+Instant-based+methods+to+ReadOnlySessionStore Discussion thread:

[jira] [Created] (KAFKA-10459) Document IQ APIs where order does not hold between stores

2020-09-03 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-10459: Summary: Document IQ APIs where order does not hold between stores Key: KAFKA-10459 URL: https://issues.apache.org/jira/browse/KAFKA-10459

Re: [DISCUSS] KIP-666: Add Instant-based methods to ReadOnlySessionStore

2020-09-01 Thread Jorge Esteban Quilcate Otoya
WindowStore methods, and provide a default > implementation > on the SessionStore interface which then just calls the corresponding > long-based method. > WDYT? > > -Sophie > > On Fri, Aug 28, 2020 at 11:31 AM Jorge Esteban Quilcate Otoya < > quilcate.jo...@gmai

[DISCUSS] KIP-666: Add Instant-based methods to ReadOnlySessionStore

2020-08-28 Thread Jorge Esteban Quilcate Otoya
Hi everyone, I'd like to discuss the following proposal to align IQ Session Store API with the Window Store one. https://cwiki.apache.org/confluence/display/KAFKA/KIP-666%3A+Add+Instant-based+methods+to+ReadOnlySessionStore Looking forward to your feedback. Cheers, Jorge.

[DISCUSS] KIP-667: Remove deprecated methods from ReadOnlyWindowStore

2020-08-28 Thread Jorge Esteban Quilcate Otoya
Hi everyone, I'd like to propose these changes to the Window Store API. https://cwiki.apache.org/confluence/display/KAFKA/KIP-667%3A+Remove+deprecated+methods+from+ReadOnlyWindowStore As these changes involve removing deprecated methods, this KIP is targeting the next major release v3.0.

[jira] [Created] (KAFKA-10445) Align IQ SessionStore API with Instant-based methods as ReadOnlyWindowStore

2020-08-28 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-10445: Summary: Align IQ SessionStore API with Instant-based methods as ReadOnlyWindowStore Key: KAFKA-10445 URL: https://issues.apache.org/jira/browse

[jira] [Created] (KAFKA-10434) Remove deprecated methods on WindowStore

2020-08-25 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-10434: Summary: Remove deprecated methods on WindowStore Key: KAFKA-10434 URL: https://issues.apache.org/jira/browse/KAFKA-10434 Project

[jira] [Created] (KAFKA-10409) Refactor Kafka Streams RocksDb iterators

2020-08-17 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-10409: Summary: Refactor Kafka Streams RocksDb iterators Key: KAFKA-10409 URL: https://issues.apache.org/jira/browse/KAFKA-10409 Project

Re: [VOTE] KIP-648: Renaming getter method for Interactive Queries

2020-07-29 Thread Jorge Esteban Quilcate Otoya
+1 (non-binding). Thanks John! On Wed, Jul 29, 2020 at 3:00 PM Navinder Brar wrote: > +1 (non-binding). Thanks John, looks good to me. > > ~NavinderOn Wednesday, 29 July, 2020, 04:32:25 am IST, John Thomas < > johnthote...@live.com> wrote: > > Hello everyone, > > I'd like to kick-off a

Re: [VOTE] KIP-450: Sliding Window Aggregations in the DSL

2020-07-29 Thread Jorge Esteban Quilcate Otoya
Thanks Leah! This will be a great addition. +1 (non-binding) Very happy that KIP-617 is being used already :D Cheers, Jorge. On Wed, Jul 29, 2020 at 2:28 PM John Roesler wrote: > Thanks for the awesome KIP, Leah, > > I’m +1 (binding) > > Thanks, > John > > On Tue, Jul 28, 2020, at 19:10,

<    1   2   3   >