Re: [DISCUSS] KIP-834: Pause / Resume KafkaStreams Topologies

2022-05-06 Thread Guozhang Wang
I'm in favor of the "just pausing the instance itself“ option as well. As for EOS, the point is that when the processing is paused, we would not trigger any `producer.send` during the time, and the transaction timeout is sort of relying on that behavior, so my point was that it's probably better

[jira] [Created] (KAFKA-13885) Add new metrics for partitioner logic introduced in KIP-794

2022-05-06 Thread Artem Livshits (Jira)
Artem Livshits created KAFKA-13885: -- Summary: Add new metrics for partitioner logic introduced in KIP-794 Key: KAFKA-13885 URL: https://issues.apache.org/jira/browse/KAFKA-13885 Project: Kafka

Re: [DISCUSS] KIP-834: Pause / Resume KafkaStreams Topologies

2022-05-06 Thread Jim Hughes
Hi Matthias, Since the only thing which will be paused is processing the topology, I think we can let commits happen naturally. Good point about getting the paused state to new members; it is seeming like the "building block" approach is a good one to keep things simple at first. Cheers, Jim

Re: [DISCUSS] KIP-834: Pause / Resume KafkaStreams Topologies

2022-05-06 Thread Matthias J. Sax
I think it's tricky to propagate a pauseAll() via the rebalance protocol. New members joining the group would need to get paused, too? Could there be weird race conditions with overlapping pauseAll() and resumeAll() calls on different instanced while there could be a errors / network

Jenkins build is still unstable: Kafka » Kafka Branch Builder » trunk #913

2022-05-06 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-834: Pause / Resume KafkaStreams Topologies

2022-05-06 Thread Jim Hughes
Hi Bill, Great questions; I'll do my best to reply inline: On Fri, May 6, 2022 at 3:21 PM Bill Bejeck wrote: > Hi Jim, > > Thanks for the KIP. I have a couple of meta-questions as well: > > 1) Regarding pausing only a subset of running instances, I'm thinking there > may be a use case for

Re: [DISCUSS] KIP-834: Pause / Resume KafkaStreams Topologies

2022-05-06 Thread Bill Bejeck
Hi Jim, Thanks for the KIP. I have a couple of meta-questions as well: 1) Regarding pausing only a subset of running instances, I'm thinking there may be a use case for pausing all of them. Would it make sense to also allow for pausing all instances by adding a method `pauseAll()` or

[jira] [Created] (KAFKA-13884) KRaft Obsever are not required to flush on every append

2022-05-06 Thread Jose Armando Garcia Sancio (Jira)
Jose Armando Garcia Sancio created KAFKA-13884: -- Summary: KRaft Obsever are not required to flush on every append Key: KAFKA-13884 URL: https://issues.apache.org/jira/browse/KAFKA-13884

Jenkins build is still unstable: Kafka » Kafka Branch Builder » trunk #912

2022-05-06 Thread Apache Jenkins Server
See

[DISCUSS] KIP-835: Monitor KRaft Controller Quorum Health

2022-05-06 Thread José Armando García Sancio
Hi all, I created a KIP for adding a mechanism to monitor the health of the KRaft Controller quorum through metrics. See KIP-835: https://cwiki.apache.org/confluence/x/0xShD Thanks for your feedback, -José

Re: [DISCUSS] KIP-832 Allow creating a producer/consumer using a producer/consumer config

2022-05-06 Thread François Rosière
The KIP has been updated to reflect the last discussion https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882578#KIP832:Allowcreatingaproducer/consumerusingaproducer/consumerconfig-ProposedChanges Le ven. 6 mai 2022 à 20:44, François Rosière a écrit : > Hello, > > No problem

Re: [DISCUSS] KIP-832 Allow creating a producer/consumer using a producer/consumer config

2022-05-06 Thread François Rosière
Hello, No problem to also add a constructor taking the StreamsConfig in the TopologyTestDriver. Summary about the changes to apply: - Create 2 new constructors in KafkaProducer - Create a new constructor in KafkaConsumer and increase de visibility of an existing one - Create a new

Re: [DISCUSS] KIP-834: Pause / Resume KafkaStreams Topologies

2022-05-06 Thread Jim Hughes
Hi Guozhang, Thanks for the feedback; responses inline below: On Fri, May 6, 2022 at 1:09 PM Guozhang Wang wrote: > Hello Jim, > > Thanks for the proposed KIP. I have some meta questions about it: > > 1) Would an instance always pause/resume all of its current owned > topologies (i.e. the

[GitHub] [kafka-site] cadonna merged pull request #407: Fix link to old version

2022-05-06 Thread GitBox
cadonna merged PR #407: URL: https://github.com/apache/kafka-site/pull/407 -- 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] [Resolved] (KAFKA-10888) Sticky partition leads to uneven product msg, resulting in abnormal delays in some partitions

2022-05-06 Thread Jun Rao (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10888?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-10888. - Fix Version/s: 3.3.0 Resolution: Fixed merged the PR to trunk. Thanks [~alivshits] for the

[jira] [Created] (KAFKA-13883) KIP-835: Monitor Quorum

2022-05-06 Thread Jose Armando Garcia Sancio (Jira)
Jose Armando Garcia Sancio created KAFKA-13883: -- Summary: KIP-835: Monitor Quorum Key: KAFKA-13883 URL: https://issues.apache.org/jira/browse/KAFKA-13883 Project: Kafka Issue

Re: [DISCUSS] KIP-834: Pause / Resume KafkaStreams Topologies

2022-05-06 Thread Guozhang Wang
Hello Jim, Thanks for the proposed KIP. I have some meta questions about it: 1) Would an instance always pause/resume all of its current owned topologies (i.e. the named topologies), or are there any scenarios where we only want to pause/resume a subset of them? 2) From a user's perspective, do

Jenkins build is still unstable: Kafka » Kafka Branch Builder » 3.1 #114

2022-05-06 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-832 Allow creating a producer/consumer using a producer/consumer config

2022-05-06 Thread John Roesler
Thanks for the KIP, François! I'm generally in favor of your KIP, since you're proposing to follow the existing pattern of the constructors for both Producer and Consumer, but with the config object instead of Properties or Map configs. Also, because we already have this pattern in Streams, and

Jenkins build is still unstable: Kafka » Kafka Branch Builder » trunk #911

2022-05-06 Thread Apache Jenkins Server
See

Re: [VOTE] 3.1.1 RC1

2022-05-06 Thread Michal Tóth
Hello I have executed some produce/consume system tests which all passed. Also everything passed from https://github.com/tombentley/kafka-verify-rc - checking signatures, checksums, with gradle unit & integration tests, etc. Good from me (non-binding). pi 6. 5. 2022 o 14:30 David Jacot

Re: [VOTE] 3.2.0 RC1

2022-05-06 Thread David Jacot
Thanks for running the release, Bruno. I performed the following validations: * Verified all checksums and signatures. * Built from source and ran unit tests. * Ran the first quickstart steps for both ZK and KRaft. * Spotchecked the doc and the Javadocs. +1 (binding) Best, David On Thu, May 5,

Re: [VOTE] 3.1.1 RC1

2022-05-06 Thread David Jacot
Thanks for running the release, Tom. I performed the following validations: * Verified all checksums and signatures. * Built from source and ran unit tests. * Ran the first quickstart steps for both ZK and KRaft. * Spotchecked the Javadocs. I noticed the same issues as others on the website. I

Build failed in Jenkins: Kafka » Kafka Branch Builder » 3.2 #46

2022-05-06 Thread Apache Jenkins Server
See Changes: -- [...truncated 525071 lines...] [2022-05-06T12:07:26.432Z] > Task :connect:api:copyDependantLibs UP-TO-DATE [2022-05-06T12:07:26.432Z] > Task :connect:api:jar

Jenkins build is still unstable: Kafka » Kafka Branch Builder » trunk #910

2022-05-06 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-832 Allow creating a producer/consumer using a producer/consumer config

2022-05-06 Thread François Rosière
To stay consistent with existing code, we should simply add 2 constructors. One with ser/deser and one without. So that, users have the choice to use one or the other. I updated the KIP accordingly. Le ven. 6 mai 2022 à 12:55, François Rosière a écrit : > On the other hand, the KafkaConsumer

Jenkins build is still unstable: Kafka » Kafka Branch Builder » 3.1 #113

2022-05-06 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-832 Allow creating a producer/consumer using a producer/consumer config

2022-05-06 Thread François Rosière
On the other hand, the KafkaConsumer constructor with a config + serializer and deserializer already exists but is not public. It would also complexify a bit the caller to not have the serializer/deserializer exposed at constructor level. Once the KIP would have been implemented, for streams,

[jira] [Created] (KAFKA-13882) Dockerfile for previewing website

2022-05-06 Thread Tom Bentley (Jira)
Tom Bentley created KAFKA-13882: --- Summary: Dockerfile for previewing website Key: KAFKA-13882 URL: https://issues.apache.org/jira/browse/KAFKA-13882 Project: Kafka Issue Type: Task

[jira] [Created] (KAFKA-13881) Add package.java for public package javadoc

2022-05-06 Thread Tom Bentley (Jira)
Tom Bentley created KAFKA-13881: --- Summary: Add package.java for public package javadoc Key: KAFKA-13881 URL: https://issues.apache.org/jira/browse/KAFKA-13881 Project: Kafka Issue Type: Task

Re: [DISCUSS] KIP-832 Allow creating a producer/consumer using a producer/consumer config

2022-05-06 Thread François Rosière
Hello, We may create a constructor with a single parameter which is the config but then, I would need to give the serializer/deserializer by also overriding the config. Like I would do for the interceptors. So, no real opinion on that, both solutions are ok for me. Maybe easier to take the

Re: [DISCUSS] KIP-831: Add metric for log recovery progress

2022-05-06 Thread Luke Chen
Hi James, Thanks for your input. For the `RemainingBytesToRecovery` metric proposal, I think there's one thing I didn't make it clear. Currently, when log manager start up, we'll try to load all logs (segments), and during the log loading, we'll try to recover logs if necessary. And the logs

Re: [DISCUSS] KIP-832 Allow creating a producer/consumer using a producer/consumer config

2022-05-06 Thread Bruno Cadonna
Hi Francois, Thank you for updating the KIP! Now the motivation of the KIP is much clearer. I would still be interested in: >> 2. Why do you only want to change/add the constructors that take the >> properties objects and de/serializers and you do not also want to >> add/change the

[GitHub] [kafka-site] cadonna commented on pull request #407: Fix link to old version

2022-05-06 Thread GitBox
cadonna commented on PR #407: URL: https://github.com/apache/kafka-site/pull/407#issuecomment-1119359385 Call for review: @bbejeck -- 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