Re: [VOTE] KIP-869: Improve Streams State Restoration Visibility

2022-12-01 Thread Matthias J. Sax
Seems I am late to the party... Great KIP. Couple of questions from my side: (1) What is the purpose of `standby-updating-tasks`? It seems to be the same as the number of assigned standby task? Not sure how useful it would be? (2) `active-paused-tasks` / `standby-paused-tasks` -- what does "

Re: [DISCUSS] KIP-878: Autoscaling for Statically Partitioned Streams

2022-12-01 Thread Matthias J. Sax
Thanks for updating the KIP Sophie. I have the same question as Bruno. How can the user use the failure metric and what actions can be taken to react if the metric increases? Plus a few more: (1) Do we assume that user can reason about `subtopology-parallelism` metric to figure out if auto

Re: [DISCUSS] KIP-889 Versioned State Stores

2022-12-01 Thread Matthias J. Sax
Thanks Victoria! (1) About `ReadOnlyVersionedKeyValueStore` -- I am not sure about IQv1 vs IQv2. But you might be right that adding the interface later might not be an issue -- so it does not matter. Just wanted to double check. (2) About `delete(key, ts)` -- as already discussed, I agree t

[VOTE] KIP-878: Internal Topic Autoscaling for Kafka Streams

2022-12-01 Thread Sophie Blee-Goldman
Thanks to all who participated for a great discussion on this KIP. Seems we're ready to kick off the voting on this, but please don't hesitate to call out anything of concern or raise questions over on the voting thread. Otherwise, please give it a final look over and cast your vote! KIP-878: Int

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

2022-12-01 Thread Apache Jenkins Server
See

[jira] [Resolved] (KAFKA-14434) Why is this project not maintained anymore?

2022-12-01 Thread jianbin.chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14434?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] jianbin.chen resolved KAFKA-14434. -- Resolution: Invalid invalid > Why is this project not maintained anymore? > -

[jira] [Created] (KAFKA-14434) Why is this project not maintained anymore?

2022-12-01 Thread jianbin.chen (Jira)
jianbin.chen created KAFKA-14434: Summary: Why is this project not maintained anymore? Key: KAFKA-14434 URL: https://issues.apache.org/jira/browse/KAFKA-14434 Project: Kafka Issue Type: Impro

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

2022-12-01 Thread Apache Jenkins Server
See Changes: -- [...truncated 431514 lines...] [2022-12-01T23:35:43.096Z] [2022-12-01T23:35:43.096Z] Gradle Test Run :streams:integrationTest > Gradle Test Executor 170 >

Re: [DISCUSS] KIP-890 Server Side Defense

2022-12-01 Thread Matthias J. Sax
Thanks for the details Justine! 20) The client side change for 2 is removing the addPartitions to transaction call. We don't need to make this from the producer to the txn coordinator, only server side. I think I did not express myself clearly. I understand that we can (and should) change t

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

2022-12-01 Thread Apache Jenkins Server
See Changes: -- [...truncated 5375 lines...] [2022-12-01T19:23:30.657Z] [2022-12-01T19:23:30.657Z] > Task :core:compileTestScala [2022-12-01T19:23:30.657Z] [Warn] /home/j

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

2022-12-01 Thread Apache Jenkins Server
See Changes: -- [...truncated 4746 lines...] [2022-12-01T18:49:36.535Z] [Warn] /home/jenkins/workspace/Kafka_kafka_trunk/streams/streams-scala/src/main/scala/org/apache/ka

Re: [DISCUSS] KIP-858: Handle JBOD broker disk failure in KRaft

2022-12-01 Thread Igor Soarez
Hi Jun, Thank you for reviewing the KIP. Please find my replies to your comments below. 10. Thanks for pointing out this typo; it has been corrected. 11. I agree that the additional delay in switching to the future replica is undesirable, however I see a couple of issues if we forward the requ

Re: [DISCUSS] KIP-889 Versioned State Stores

2022-12-01 Thread Sagar
Thanks Victoria, I guess an advantage of exposing a method like delete(key, timestamp) could be that from a user's standpoint, it is a single operation and not 2. The equivalent of this method i.e put followed by get is not atomic so exposing it certainly sounds like a good idea. Thanks! Sagar.

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

2022-12-01 Thread Apache Jenkins Server
See Changes: -- [...truncated 514474 lines...] [2022-12-01T16:23:57.353Z] [2022-12-01T16:23:57.353Z] Gradle Test Run :streams:integrationTest > Gradle Test Executor 169 >

[jira] [Created] (KAFKA-14433) Clear all yammer metrics when test harnesses clean up

2022-12-01 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-14433: Summary: Clear all yammer metrics when test harnesses clean up Key: KAFKA-14433 URL: https://issues.apache.org/jira/browse/KAFKA-14433 Project: Kafka Issue T

[DISCUSS] KIP-893: The Kafka protocol should support nullable structs

2022-12-01 Thread David Jacot
Hi all, I have drafted a very small KIP which proposes to support nullable struct in the Kafka protocol. This is something that we plan to use for KIP-848. The KIP is here: https://cwiki.apache.org/confluence/x/YJIODg Please let me know what you think. Best, David

[jira] [Created] (KAFKA-14432) RocksDBStore relies on finalizers to not leak memory

2022-12-01 Thread Lucas Brutschy (Jira)
Lucas Brutschy created KAFKA-14432: -- Summary: RocksDBStore relies on finalizers to not leak memory Key: KAFKA-14432 URL: https://issues.apache.org/jira/browse/KAFKA-14432 Project: Kafka Issu

Re: Faster connector config dev cycles - A vision and prototype

2022-12-01 Thread Chris Egerton
Hi Greg, I really like this idea. It has the potential to both soften the learning curve for users coming up to speed with Kafka Connect, and address a pain point for both new and seasoned users experimenting with connector configs. Given that standalone mode is useful for docs and quickstarts in

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

2022-12-01 Thread Apache Jenkins Server
See

[jira] [Resolved] (KAFKA-14431) TopologyTestDriver instantiation raised NPE in JUnit5

2022-12-01 Thread Mehrdad Karami (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14431?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mehrdad Karami resolved KAFKA-14431. Resolution: Fixed Saw already fixed in truck branch > TopologyTestDriver instantiation ra

Re: [DISCUSS] KIP-881: Rack-aware Partition Assignment for Kafka Consumers

2022-12-01 Thread Rajini Sivaram
Hi Artem, Yes, clients can reset their `client.rack` config if the broker doesn't use a rack-aware selector. This config is only useful for improving locality with a rack-aware selector. On Wed, Nov 30, 2022 at 10:05 PM Artem Livshits wrote: > I think it's reasonable for practical scenarios. I

[jira] [Created] (KAFKA-14431) TopologyTestDriver instantiation raised NPE in JUnit5

2022-12-01 Thread Mehrdad Karami (Jira)
Mehrdad Karami created KAFKA-14431: -- Summary: TopologyTestDriver instantiation raised NPE in JUnit5 Key: KAFKA-14431 URL: https://issues.apache.org/jira/browse/KAFKA-14431 Project: Kafka Iss