[DISCUSS] KIP-649: Dynamic Client Configuration

2020-07-29 Thread Ryan Dielhenn
Hi everyone, I would like to start a discussion on KIP 649: https://cwiki.apache.org/confluence/display/KAFKA/KIP-649%3A+Dynamic+Client+Configuration This proposal specifies the mechanisms that will enable dynamic configuration of producers and consumers. We have cherry picked a few producer

Build failed in Jenkins: kafka-trunk-jdk8 #4750

2020-07-29 Thread Apache Jenkins Server
See Changes: [github] KAFKA-10270: A broker to controller channel manager (#9012) -- [...truncated 6.37 MB...] org.apache.kafka.streams.internals.WindowStoreFacadeTest

[jira] [Created] (KAFKA-10325) Implement KIP-649: Dynamic Client Configuration

2020-07-29 Thread Ryan Dielhenn (Jira)
Ryan Dielhenn created KAFKA-10325: - Summary: Implement KIP-649: Dynamic Client Configuration Key: KAFKA-10325 URL: https://issues.apache.org/jira/browse/KAFKA-10325 Project: Kafka Issue

Build failed in Jenkins: kafka-trunk-jdk11 #1676

2020-07-29 Thread Apache Jenkins Server
See Changes: [github] KAFKA-10270: A broker to controller channel manager (#9012) -- [...truncated 6.41 MB...] org.apache.kafka.streams.TopologyTestDriverTest >

Re: [DISCUSS] KIP-630: Kafka Raft Snapshot

2020-07-29 Thread Jose Garcia Sancio
Thanks Ron for the additional comments and suggestions. Here are the changes to the KIP: https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=158864763=17=15 On Wed, Jul 29, 2020 at 8:44 AM Ron Dagostino wrote: > > Thanks, Jose. It's looking good. Here is one minor

Jenkins build is back to normal : kafka-trunk-jdk14 #325

2020-07-29 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

2020-07-29 Thread Jason Gustafson
Hey Jun, I added a section on "Cluster Bootstrapping" which discusses clusterId generation and the process through which brokers find the current leader. The quick summary is that the first controller will be responsible for generating the clusterId and persisting it in the metadata log. Before

[jira] [Resolved] (KAFKA-9210) kafka stream loss data

2020-07-29 Thread John Roesler (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9210?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] John Roesler resolved KAFKA-9210. - Resolution: Fixed > kafka stream loss data > -- > > Key:

Build failed in Jenkins: kafka-trunk-jdk14 #326

2020-07-29 Thread Apache Jenkins Server
See Changes: [github] KAFKA-10309: KafkaProducer's sendOffsetsToTransaction should not block -- [...truncated 3.20 MB...]

Build failed in Jenkins: kafka-trunk-jdk8 #4751

2020-07-29 Thread Apache Jenkins Server
See Changes: [github] KAFKA-10309: KafkaProducer's sendOffsetsToTransaction should not block -- [...truncated 3.18 MB...] org.apache.kafka.streams.TestTopicsTest >

Re: [VOTE] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-07-29 Thread Boyang Chen
Thanks David for the feedback! On Wed, Jul 29, 2020 at 7:53 AM David Jacot wrote: > Hi, Colin, Boyang, > > Colin, thanks for the clarification. Somehow, I thought that even if the > controller is ran independently, it > would still run the listeners of the broker and thus would be accessible by

[jira] [Resolved] (KAFKA-10270) Add a broker to controller channel manager to redirect AlterConfig

2020-07-29 Thread Boyang Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10270?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Boyang Chen resolved KAFKA-10270. - Resolution: Fixed > Add a broker to controller channel manager to redirect AlterConfig >

[jira] [Created] (KAFKA-10326) Both serializer and deserializer should be able to see the generated client id

2020-07-29 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-10326: -- Summary: Both serializer and deserializer should be able to see the generated client id Key: KAFKA-10326 URL: https://issues.apache.org/jira/browse/KAFKA-10326

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-07-29 Thread Colin McCabe
On Thu, Jul 23, 2020, at 23:02, Boyang Chen wrote: > Hey Colin, > > some more questions I have about the proposal: > > 1. We mentioned in the networking section that "The only time when clients > should contact a controller node directly is when they are debugging system > issues". But later we

[jira] [Created] (KAFKA-10321) shouldDieOnInvalidOffsetExceptionWhileRunning would block forever on JDK11

2020-07-29 Thread Boyang Chen (Jira)
Boyang Chen created KAFKA-10321: --- Summary: shouldDieOnInvalidOffsetExceptionWhileRunning would block forever on JDK11 Key: KAFKA-10321 URL: https://issues.apache.org/jira/browse/KAFKA-10321 Project:

[GitHub] [kafka-site] scott-confluent edited a comment on pull request #269: Kafka nav and hompeage redesign

2020-07-29 Thread GitBox
scott-confluent edited a comment on pull request #269: URL: https://github.com/apache/kafka-site/pull/269#issuecomment-663676137 This is an automated message from the Apache Git Service. To respond to the message, please log

[jira] [Created] (KAFKA-10322) InMemoryWindowStore restore keys format incompatibility (lack of sequenceNumber in keys on topic)

2020-07-29 Thread Jira
Tomasz Bradło created KAFKA-10322: - Summary: InMemoryWindowStore restore keys format incompatibility (lack of sequenceNumber in keys on topic) Key: KAFKA-10322 URL:

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

2020-07-29 Thread John Roesler
Thanks for the awesome KIP, Leah, I’m +1 (binding) Thanks, John On Tue, Jul 28, 2020, at 19:10, Guozhang Wang wrote: > +1 (binding) > > On Tue, Jul 28, 2020 at 4:44 PM Matthias J. Sax wrote: > > > +1 (binding) > > > > On 7/28/20 4:35 PM, Sophie Blee-Goldman wrote: > > > Thanks for the KIP!

Kafka zk1 password

2020-07-29 Thread Lukac, Dominik
Hello, I have a question regarding running kafka with the help of vagrant. I have tried to connect to the zk1 vm by using WinSCP, but it looks like it requires a password, which I cannot find. Is there some file, which documents the passwords for the different vms, or are they all the same? But

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,

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: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-07-29 Thread Unmesh Joshi
Hi, In the BrokerHeartbeat request and response, what is the reason to have LeaseStartTimeMs and LeaseEndTimeMs respectively? There are two points I was thinking of 1. The time used to track lease expiry will be monotonic clock on the active controller. So it won't be useful to use that value

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

2020-07-29 Thread Navinder Brar
+1 (non-binding). Thanks John, looks good to me. ~NavinderOn Wednesday, 29 July, 2020, 04:32:25 am IST, John Thomas wrote: Hello everyone, I'd like to kick-off a vote for KIP-648 : Renaming getter method for Interactive Queries

Re: [DISCUSS] KIP-431: Support of printing additional ConsumerRecord fields in DefaultMessageFormatter

2020-07-29 Thread Badai Aqrandista
Hi all I have created a PR for KIP-431 against the latest trunk: https://github.com/apache/kafka/pull/9099 Please review. Regards Badai On Tue, Jul 21, 2020 at 2:13 AM Matthias J. Sax wrote: > > Thanks Badai. LGTM. > > On 7/19/20 4:26 PM, Badai Aqrandista wrote: > > Hi all > > > > I have made

Re: [VOTE] KIP-617: Allow Kafka Streams State Stores to be iterated backwards

2020-07-29 Thread Jorge Esteban Quilcate Otoya
Thanks everyone for voting! With 3 binding votes (Matthias, Guozhang, and John) and 2 non-binding votes (Leah, and Sophie), will mark this KIP as accepted. Thanks, Jorge. On Tue, Jul 28, 2020 at 3:27 AM Matthias J. Sax wrote: > +1 (binding) > > On 7/27/20 4:55 PM, Guozhang Wang wrote: > >

Re: [DISCUSS] KIP-450: Sliding Windows

2020-07-29 Thread Leah Thomas
Thanks for the nits Matthias, I've updated the examples and language accordingly. Leah On Tue, Jul 28, 2020 at 6:43 PM Matthias J. Sax wrote: > Thanks Leah. Overall LGTM. > > A few nits: > > - the first figure shows window [9,19] but the window is not aligned > properly (it should be 1ms to

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

2020-07-29 Thread John Roesler
Hi John, Thanks for picking this up! I’m +1 (binding). -John On Wed, Jul 29, 2020, at 09:14, Jorge Esteban Quilcate Otoya wrote: > +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. > > > >

[jira] [Created] (KAFKA-10323) NullPointerException during rebalance

2020-07-29 Thread yazgoo (Jira)
yazgoo created KAFKA-10323: -- Summary: NullPointerException during rebalance Key: KAFKA-10323 URL: https://issues.apache.org/jira/browse/KAFKA-10323 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-10324) Pre-0.11 consumers can get stuck when messages are downconverted from V2 format

2020-07-29 Thread Tommy Becker (Jira)
Tommy Becker created KAFKA-10324: Summary: Pre-0.11 consumers can get stuck when messages are downconverted from V2 format Key: KAFKA-10324 URL: https://issues.apache.org/jira/browse/KAFKA-10324

Re: [VOTE] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-07-29 Thread David Jacot
Hi, Colin, Boyang, Colin, thanks for the clarification. Somehow, I thought that even if the controller is ran independently, it would still run the listeners of the broker and thus would be accessible by redirecting on the loopback interface. My mistake. Boyang, I have few questions/comments

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

2020-07-29 Thread Bruno Cadonna
Thanks John, +1 (non-binding) Best, Bruno On 29.07.20 01:02, John Thomas wrote: Hello everyone, I'd like to kick-off a vote for KIP-648 : Renaming getter method for Interactive Queries

Re: [DISCUSS] KIP-630: Kafka Raft Snapshot

2020-07-29 Thread Ron Dagostino
Thanks, Jose. It's looking good. Here is one minor correction: <<< If the Kafka topic partition leader receives a fetch request with an offset and epoch greater than or equal to the LBO (x + 1, a) >>> If the Kafka topic partition leader receives a fetch request with an offset and epoch greater