Re: [DISCUSS] KIP-825: introduce a new API to control when aggregated results are produced

2022-03-21 Thread Hao Li
Hi John, Yes. For naming, `trigger` is similar to Flink's trigger, but it has a different meaning in our case. `emit` sounds like an action to emit? How about `emitTrigger`? I'm open to suggestions for the naming. For deprecating `Suppressed.untilWindowClose`, I agree with Guozhang we can

Re: [DISCUSS] KIP-825: introduce a new API to control when aggregated results are produced

2022-03-21 Thread Guozhang Wang
I think the following case is only doable via `suppress`: stream .groupBy(..) .windowBy(..) .aggregate(..) //result in a KTable> .mapValues(..) .suppress(Suppressed.untilWindowClose) // since we can trace back to parent node, to find a window definition Guozhang On Mon, Mar 21, 2022

Re: [VOTE} KIP-796: Interactive Query v2

2022-03-21 Thread John Roesler
Hello, all, During the PR reviews for this KIP, there were several late concerns raised about the IQv2 APIs. I filed tickets under KAFKA-13479 and promised to revisit them before the API was released. Unfortunately, I have not had time to circle back on those concerns. Now that the 3.2 branch

Re: [DISCUSS] KIP-825: introduce a new API to control when aggregated results are produced

2022-03-21 Thread John Roesler
Thanks, Guozhang! To clarify, I was asking specifically about deprecating just the method ‘untilWindowClose’. I might not be thinking clearly about it, though. What does untilWindowClose do that this KIP doesn’t cover? Thanks, John On Mon, Mar 21, 2022, at 20:31, Guozhang Wang wrote: > Just

Re: [DISCUSS] KIP-825: introduce a new API to control when aggregated results are produced

2022-03-21 Thread Guozhang Wang
Just my 2c: Suppressed is in `suppress` whose application scope is much larger and hence more flexible. I.e. it can be used anywhere for a `KTable` (but internally we would check whether certain emit policies like `untilWindowClose` is valid or not), whereas `trigger` as for now is only applicable

Re: [DISCUSS] KIP-825: introduce a new API to control when aggregated results are produced

2022-03-21 Thread John Roesler
Hi all, Thanks for the Kip, Hao! For what it’s worth, I’m also in favor of your latest framing of the API, I think the name is fine. I assume it’s inspired by Flink? It’s not identical to the concept of a trigger in Flink, which specifies when to evaluate the window, which might be confusing

Re: [DISCUSS] KIP-825: introduce a new API to control when aggregated results are produced

2022-03-21 Thread Guozhang Wang
Hi Hao, For 2), I think it's a good idea in general to use a separate function on the Time/SessionWindowedKStream itself, to achieve the same effect that, for now, the emitting control is only for windowed aggregations as in this KIP, than overloading existing functions. We can discuss further

Re: [DISCUSS] KIP-825: introduce a new API to control when aggregated results are produced

2022-03-21 Thread Hao Li
Hi Guozhang, Thanks for the feedback. 1. I agree to have an `Emitted` control class and two static constructors named `onWindowClose` and `onEachUpdate`. 2. For the API function changes, I'm thinking of adding a new function called `trigger` to `TimeWindowedKStream` and

[VOTE] KIP-794: Strictly Uniform Sticky Partitioner

2022-03-21 Thread Artem Livshits
Hi all, I'd like to start a vote on https://cwiki.apache.org/confluence/display/KAFKA/KIP-794%3A+Strictly+Uniform+Sticky+Partitioner . -Artem

Re: [DISCUSS] Apache Kafka 3.2.0 release

2022-03-21 Thread Bruno Cadonna
Hi Kafkateers, Last week we reached feature freeze for the next major release of Apache Kafka. I cut the release branch for 3.2 and bumped trunk to 3.3.0-SNAPSHOT. From this point, commits that should go into 3.2.0 should be merged into trunk and cherry-picked to 3.2. I removed the KIPs

[jira] [Resolved] (KAFKA-13682) Implement auto preferred leader election in KRaft Controller

2022-03-21 Thread Jose Armando Garcia Sancio (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13682?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jose Armando Garcia Sancio resolved KAFKA-13682. Resolution: Fixed > Implement auto preferred leader election in

[jira] [Resolved] (KAFKA-13587) Implement unclean leader election in KIP-704

2022-03-21 Thread Jose Armando Garcia Sancio (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13587?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jose Armando Garcia Sancio resolved KAFKA-13587. Resolution: Fixed > Implement unclean leader election in KIP-704

[jira] [Resolved] (KAFKA-6718) Rack Aware Stand-by Task Assignment for Kafka Streams

2022-03-21 Thread Bruno Cadonna (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-6718?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bruno Cadonna resolved KAFKA-6718. -- Resolution: Fixed > Rack Aware Stand-by Task Assignment for Kafka Streams >

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

2022-03-21 Thread Jorge Esteban Quilcate Otoya
Hi all, With the acceptance of KIP-820 which will enable easier access to the Record's metadata and headers and the potential design of a new version of the DSL, I will set this KIP as inactive/dormant for the time being. Thanks, everyone for the great discussions! Jorge. On Wed, 16 Feb 2022 at

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

2022-03-21 Thread Jorge Esteban Quilcate Otoya
Hi Dev team, The vote for KIP-820 has passed with: 3 +1 (binding) votes from John, Matthias, and Guozhang. This KIP is now accepted. Thanks for your feedback and votes! Jorge. On Thu, 17 Mar 2022 at 17:30, Guozhang Wang wrote: > Reviewed the updated docs, and recasting my +1 vote again,

Re: [DISCUSS] KIP-714: Client metrics and observability

2022-03-21 Thread Jun Rao
Hi, Kirk, Sarat, A few more comments. 40. GetTelemetrySubscriptionsResponseV0 : RequestedMetrics Array[string] uses "Array[0] empty string" to represent all metrics subscribed. We had a similar issue with the topics field in MetadataRequest and used the following convention. In version 1 and

Jenkins build is unstable: Kafka » Kafka Branch Builder » 3.2 #1

2022-03-21 Thread Apache Jenkins Server
See

[jira] [Resolved] (KAFKA-13728) PushHttpMetricsReporter no longer pushes metrics when network failure is recovered.

2022-03-21 Thread Guozhang Wang (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13728?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-13728. --- Fix Version/s: 3.2.0 Resolution: Fixed > PushHttpMetricsReporter no longer pushes

Re: [DISCUSS] KIP-651 - Support PEM format for SSL certificates and

2022-03-21 Thread Rajini Sivaram
For the background on the current implementation: We use Java's keystore loading for JKS/PKCS12 keystore files and these files require passwords. We retained the same requirement for PEM files as well for consistency, even though it is not a language restriction anymore. When PEM keys are provided

Article on kafka.apache.org

2022-03-21 Thread Leah Atkins
Hi! Sorry for interrupting you. Our expert can write an article and publish it on kafka.apache.org. Is it possible? Thanks. Kind regards, Leah Atkins

Re: [VOTE] KIP-653: Upgrade log4j to log4j2

2022-03-21 Thread Ismael Juma
Hi Edoardo, Thanks for the information. That's definitely useful. A couple of questions for you and the rest of the group: 1. Did you test the branch using log4j 1.x configs? 2. Given the release of https://github.com/qos-ch/reload4j, does it really make sense to force breakage on users in a

RE: [VOTE] KIP-653: Upgrade log4j to log4j2

2022-03-21 Thread Edoardo Comar
Hi Ismael and Luke, we've tested Dongjin code - porting her preview releases and PR to different Kafka code levels (2.8.1+, 3.1.0+, trunk). We're happy with it and would love it if her PR was merged in 3.2.0. To chime in on the issue of compatibility, as we have experienced it, the main

RE: [DISCUSS] KIP-651 - Support PEM format for SSL certificates and

2022-03-21 Thread Dejan Maric
Regarding this KIP, we've created an MR that removes the need for specifying *keyPassword *when PEM certificates and private key are provided as files: https://github.com/apache/kafka/pull/11916 We think that Kafka should not enforce the use of passwords on private keys. It would be sufficient

Build failed in Jenkins: Kafka » Kafka Branch Builder » trunk #788

2022-03-21 Thread Apache Jenkins Server
See Changes: -- [...truncated 598536 lines...] [2022-03-21T13:24:00.505Z] [2022-03-21T13:24:00.505Z]

Jenkins build is unstable: Kafka » Kafka Branch Builder » trunk #787

2022-03-21 Thread Apache Jenkins Server
See

[GitHub] [kafka-site] tombentley merged pull request #401: Add Tom's public key to KEYS

2022-03-21 Thread GitBox
tombentley merged pull request #401: URL: https://github.com/apache/kafka-site/pull/401 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail:

[jira] [Created] (KAFKA-13756) Connect validate endpoint should return proper response on name and connector class error

2022-03-21 Thread Daniel Urban (Jira)
Daniel Urban created KAFKA-13756: Summary: Connect validate endpoint should return proper response on name and connector class error Key: KAFKA-13756 URL: https://issues.apache.org/jira/browse/KAFKA-13756