[DISCUSS] KIP-508: Make Suppression State Queriable - rebooted.

2020-02-14 Thread Dongjin Lee
Hi devs, I'd like to reboot the discussion on KIP-508, which aims to support a Materialized variant of KTable#suppress. It was initially submitted several months ago but closed by the inactivity. - KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-508%3A+Make+Suppression+State+Queriable

Re: [DISCUSS] Apache Kafka 2.5.0 release

2020-02-14 Thread Randall Hauch
Hi, David. I just filed https://issues.apache.org/jira/browse/KAFKA-9556 that identifies two pretty minor issues with the new KIP-558 that adds new Connect REST API endpoints to get the list of topics used by a connector. The impact is high: the feature cannot be fully disabled, and Connect does

Re: [DISCUSS] KIP-553: Enable TLSv1.3 by default and disable all protocols except [TLSV1.2, TLSV1.3]

2020-02-14 Thread Nikolay Izhikov
Hello, Kafka team. I ran system tests that use SSL for the TLSv1.3. You can find the results of the tests in the Jira ticket [1], [2], [3], [4]. I also, need a changes [5] in `security_config.py` to execute system tests with TLSv1.3(more info in PR description). Please, take a look. Test

[jira] [Resolved] (KAFKA-9554) Define the SPI for Tiered Storage framework

2020-02-14 Thread Satish Duggana (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9554?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Satish Duggana resolved KAFKA-9554. --- Resolution: Duplicate Duplicate of https://issues.apache.org/jira/browse/KAFKA-9548 >

[jira] [Created] (KAFKA-9556) KIP-558 cannot be fully disabled and when enabled topic reset not working on connector deletion

2020-02-14 Thread Randall Hauch (Jira)
Randall Hauch created KAFKA-9556: Summary: KIP-558 cannot be fully disabled and when enabled topic reset not working on connector deletion Key: KAFKA-9556 URL: https://issues.apache.org/jira/browse/KAFKA-9556

Re: [DISCUSS] KIP-570: Add leader epoch in StopReplicaRequest

2020-02-14 Thread Jason Gustafson
Hey David, Thanks, it makes sense to prevent reordering, especially for the case of reassignment. When a topic is deleted, however, I am not sure we will have a bumped epoch to send. I guess for that case, we could send a sentinel which would take the existing semantics of overriding any existing

[jira] [Created] (KAFKA-9557) Thread-level "process" metrics are computed incorrectly

2020-02-14 Thread John Roesler (Jira)
John Roesler created KAFKA-9557: --- Summary: Thread-level "process" metrics are computed incorrectly Key: KAFKA-9557 URL: https://issues.apache.org/jira/browse/KAFKA-9557 Project: Kafka Issue

[jira] [Resolved] (KAFKA-9536) Integration tests for KIP-558

2020-02-14 Thread Konstantine Karantasis (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9536?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Konstantine Karantasis resolved KAFKA-9536. --- Resolution: Fixed   Fixed with [https://github.com/apache/kafka/pull/8085]

[jira] [Created] (KAFKA-9559) Change the default "default serde" from ByteArraySerde to null

2020-02-14 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-9559: -- Summary: Change the default "default serde" from ByteArraySerde to null Key: KAFKA-9559 URL: https://issues.apache.org/jira/browse/KAFKA-9559 Project:

[jira] [Created] (KAFKA-9560) Connector::validate is utilized concurrently by the framework, but not documented as thread-safe

2020-02-14 Thread Chris Egerton (Jira)
Chris Egerton created KAFKA-9560: Summary: Connector::validate is utilized concurrently by the framework, but not documented as thread-safe Key: KAFKA-9560 URL: https://issues.apache.org/jira/browse/KAFKA-9560

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

2020-02-14 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9512: Flaky Test -- [...truncated 2.89 MB...] org.apache.kafka.streams.test.OutputVerifierTest >

Re: [DISCUSS] Apache Kafka 2.5.0 release

2020-02-14 Thread Konstantine Karantasis
Hi David, I want to confirm what Randall mentions above. The code fixes for KAFKA-9556 were in place before code freeze on Wed, but we spent a bit more time hardening the conditions of the integration tests and fixing some jenkins branch builders to run the test on repeat. Best, Konstantine On

[jira] [Resolved] (KAFKA-9557) Thread-level "process" metrics are computed incorrectly

2020-02-14 Thread John Roesler (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9557?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] John Roesler resolved KAFKA-9557. - Fix Version/s: 2.6.0 Resolution: Fixed > Thread-level "process" metrics are computed

Re: [VOTE] KIP-568: Explicit rebalance triggering on the Consumer

2020-02-14 Thread Sophie Blee-Goldman
Thanks all! This KIP passes with 3 binding votes (John, Bill, and Guozhang) and 3 non-binding votes (Navinder, Konstantine, Boyang, and Bruno). I'll call for review on a PR shortly. Sophie On Fri, Feb 14, 2020 at 12:25 AM Bruno Cadonna wrote: > Thanks! > > +1 (non-binding) > > Best, >

[jira] [Created] (KAFKA-9558) getListOffsetsCalls doesn't update node in case of leader change

2020-02-14 Thread Sanjana Kaundinya (Jira)
Sanjana Kaundinya created KAFKA-9558: Summary: getListOffsetsCalls doesn't update node in case of leader change Key: KAFKA-9558 URL: https://issues.apache.org/jira/browse/KAFKA-9558 Project:

Re: [DISCUSS] KIP-508: Make Suppression State Queriable - rebooted.

2020-02-14 Thread John Roesler
Hi Dongjin, Thanks for the KIP! Can you explain more about why the internal data structures of suppression should be queriable? The motivation just says that users might want to do it, which seems like it could justify literally anything :) One design point of Suppression is that if you

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

2020-02-14 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9512: Flaky Test -- [...truncated 2.87 MB...] org.apache.kafka.streams.TopologyTestDriverTest >

回复:[Discuss] KIP-571: Add option to force remove members in StreamsResetter

2020-02-14 Thread feyman2009
Hi, Boyang You can call me Feyman :) Thanks for your quick reply with great advices! I have updated the KIP-571 , would you mind to see if it looks good ? Thanks ! -- 发件人:Boyang Chen 发送时间:2020年2月14日(星期五) 08:35

[jira] [Created] (KAFKA-9561) Update task input partitions when topic metadata changes

2020-02-14 Thread Boyang Chen (Jira)
Boyang Chen created KAFKA-9561: -- Summary: Update task input partitions when topic metadata changes Key: KAFKA-9561 URL: https://issues.apache.org/jira/browse/KAFKA-9561 Project: Kafka Issue

Re: Possible to create Scrum board under Kafka project in JIRA?

2020-02-14 Thread Harsha Chintalapani
All the dependency JIRAs are created and linked to the EPIC here https://issues.apache.org/jira/browse/KAFKA-7739 . Lets drive it through that. -Harsha On Fri, Feb 14, 2020 at 2:28 AM, Alexandre Dupriez < alexandre.dupr...@gmail.com> wrote: > Good morning, > > Would it be possible to allow the

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

2020-02-14 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9557: correct thread process-rate sensor to measure throughput [github] KAFKA-9556; Fix two issues with KIP-558 and expand testing coverage

[jira] [Created] (KAFKA-9562) Streams not making progress under heavy failures with EOS enabled on 2.5 branch

2020-02-14 Thread John Roesler (Jira)
John Roesler created KAFKA-9562: --- Summary: Streams not making progress under heavy failures with EOS enabled on 2.5 branch Key: KAFKA-9562 URL: https://issues.apache.org/jira/browse/KAFKA-9562 Project:

Re: [VOTE] KIP-568: Explicit rebalance triggering on the Consumer

2020-02-14 Thread Bruno Cadonna
Thanks! +1 (non-binding) Best, Bruno On Fri, Feb 14, 2020 at 1:57 AM Boyang Chen wrote: > > +1 (non-binding) > > On Thu, Feb 13, 2020 at 4:45 PM Guozhang Wang wrote: > > > +1 (binding). > > > > > > Guozhang > > > > On Tue, Feb 11, 2020 at 5:29 PM Guozhang Wang wrote: > > > > > Hi Sophie, > >

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

2020-02-14 Thread Apache Jenkins Server
See Changes: [github] HOTFIX: Fix breakage in `ConsumerPerformanceTest` (#8113) -- [...truncated 2.89 MB...] org.apache.kafka.streams.test.ConsumerRecordFactoryTest >

Jenkins build is back to normal : kafka-2.5-jdk8 #22

2020-02-14 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-9554) Define the SPI for Tiered Storage framework

2020-02-14 Thread Alexandre Dupriez (Jira)
Alexandre Dupriez created KAFKA-9554: Summary: Define the SPI for Tiered Storage framework Key: KAFKA-9554 URL: https://issues.apache.org/jira/browse/KAFKA-9554 Project: Kafka Issue

[jira] [Created] (KAFKA-9555) Topic-based implementation for the RLMM

2020-02-14 Thread Alexandre Dupriez (Jira)
Alexandre Dupriez created KAFKA-9555: Summary: Topic-based implementation for the RLMM Key: KAFKA-9555 URL: https://issues.apache.org/jira/browse/KAFKA-9555 Project: Kafka Issue Type:

Possible to create Scrum board under Kafka project in JIRA?

2020-02-14 Thread Alexandre Dupriez
Good morning, Would it be possible to allow the the Apache Kafka project in JIRA to be included in a new Scrum board? I can see there is already a Kanban board for Cloudera and tried to create a Scrum board for Tiered-Storage but don't have the permissions to include Apache Kafka. Thank you,