Re: [VOTE] KIP-390: Allow fine-grained configuration for compression

2019-01-22 Thread Becket Qin
Hi Dongjin, Thanks for the KIP. Sorry for being a little late on this, but I have some comments and have left them in the discussion thread to avoid polluting the voting thread. Could you take a look? Thanks, Jiangjie (Becket) Qin On Mon, Jan 21, 2019 at 3:17 PM Dongjin Lee wrote: > Hi dev,

Re: [DISCUSS] KIP-390: Add producer option to adjust compression level

2019-01-22 Thread Becket Qin
Hi Dongjin, Thanks for the KIP and sorry for being a bit late on the discussion. It makes sense to expose the configuration for compression types. But I am wondering if there is a better way to do that than what proposed in the KIP. What I feel confusing is that we are effectively sharing the con

Re: [DISCUSS] KIP-391: Allow Producing with Offsets for Cluster Replication

2019-01-22 Thread radai
the kip-320 conflict can be resolved by saying that the leader broker on the destination "stamps" is own local leader epoch on the incoming msgs - meaning the offsets "transfer" but leader epochs do not. On Mon, Jan 7, 2019 at 1:38 PM Edoardo Comar wrote: > > Hi, > I delayed starting the voting t

Re: [DISCUSS] KIP-415: Incremental Cooperative Rebalancing in Kafka Connect

2019-01-22 Thread Konstantine Karantasis
Hi Randall, thanks for you comments! Replying inline below: On Sat, Jan 19, 2019 at 4:26 PM Randall Hauch wrote: > Thanks for all this work, Konstantine. > > I have a question about when a member leaves. Here's the partial scenario, > repeated from the KIP: > > > Initial group and assignment: W

[jira] [Created] (KAFKA-7859) Replace LeaveGroup request/response with automated protocol

2019-01-22 Thread Boyang Chen (JIRA)
Boyang Chen created KAFKA-7859: -- Summary: Replace LeaveGroup request/response with automated protocol Key: KAFKA-7859 URL: https://issues.apache.org/jira/browse/KAFKA-7859 Project: Kafka Issue

[jira] [Created] (KAFKA-7858) Replace JoinGroup request/response with automated protocol

2019-01-22 Thread Boyang Chen (JIRA)
Boyang Chen created KAFKA-7858: -- Summary: Replace JoinGroup request/response with automated protocol Key: KAFKA-7858 URL: https://issues.apache.org/jira/browse/KAFKA-7858 Project: Kafka Issue Ty

Re: [DISCUSS] KIP-415: Incremental Cooperative Rebalancing in Kafka Connect

2019-01-22 Thread Konstantine Karantasis
Thank you Boyang for your follow up comments. Please find my replies inline, below: On Fri, Jan 18, 2019 at 12:00 PM Boyang Chen wrote: > Thanks a lot for the detailed explanation here Konstantine! I strongly > agree that a rolling start of > Kafka broker is not the optimal solution when we hav

Re: [DISCUSS] 2.1.1 bug-fix release

2019-01-22 Thread Colin McCabe
Hi all, If there are no objections, I'll create a release candidate for 2.1.1 tomorrow (Wednesday). thanks, all. Colin On Tue, Jan 15, 2019, at 11:56, Ismael Juma wrote: > Thanks for volunteering Colin. There are some important fixes since 2.1.0 > so I think this would be great. > > Ismael >

Build failed in Jenkins: kafka-2.1-jdk8 #112

2019-01-22 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Cleanup handling of mixed transactional/idempotent records -- [...truncated 2.81 MB...] org.apache.kafka.connect.runtime.ConnectMetricsTest > test

Build failed in Jenkins: kafka-0.11.0-jdk7 #411

2019-01-22 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Cleanup handling of mixed transactional/idempotent records -- [...truncated 1.56 MB...] org.apache.kafka.streams.integration.QueryableStateInt

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

2019-01-22 Thread Apache Jenkins Server
See Changes: [github] MINOR: Cleanup handling of mixed transactional/idempotent records -- [...truncated 2.27 MB...] org.apache.kafka.connect.transforms.RegexRouterTest >

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

2019-01-22 Thread Apache Jenkins Server
See Changes: [github] MINOR: Cleanup handling of mixed transactional/idempotent records -- [...truncated 2.27 MB...] org.apache.kafka.trogdor.agent.AgentTest > testCreate

[jira] [Created] (KAFKA-7855) Kafka Streams Maven Archetype quickstart fails to compile out of the box

2019-01-22 Thread Michael Drogalis (JIRA)
Michael Drogalis created KAFKA-7855: --- Summary: Kafka Streams Maven Archetype quickstart fails to compile out of the box Key: KAFKA-7855 URL: https://issues.apache.org/jira/browse/KAFKA-7855 Project:

[jira] [Created] (KAFKA-7857) Add AbstractCoordinatorConfig class to consolidate consumer coordinator configs between Consumer and Connect

2019-01-22 Thread Boyang Chen (JIRA)
Boyang Chen created KAFKA-7857: -- Summary: Add AbstractCoordinatorConfig class to consolidate consumer coordinator configs between Consumer and Connect Key: KAFKA-7857 URL: https://issues.apache.org/jira/browse/KAFKA-

[jira] [Created] (KAFKA-7856) Cryptographic Issues by Insufficient Entropy

2019-01-22 Thread Victor Sartori (JIRA)
Victor Sartori created KAFKA-7856: - Summary: Cryptographic Issues by Insufficient Entropy Key: KAFKA-7856 URL: https://issues.apache.org/jira/browse/KAFKA-7856 Project: Kafka Issue Type: Bug

Re: [VOTE] KIP-421: Support resolving externalized secrets in AbstractConfig

2019-01-22 Thread Ryanne Dolan
+1 non-binding, thanks! Ryanne On Tue, Jan 22, 2019 at 11:38 AM te...@confluent.io wrote: > > Hi all, > > We would like to start vote on KIP-421 to to enhance the AbstractConfig base > class to support replacing variables in configurations just prior to parsing > and validation. > > Link for t

Re: [VOTE] KIP-382 MirrorMaker 2.0

2019-01-22 Thread Ryanne Dolan
Thanks all, this is a large KIP and has sparked a lot of great discussion, both on and off the dev list. I'm closing the vote with the following results: +12 non-binding +3 binding The KIP is accepted! Thanks, Ryanne On Fri, Jan 11, 2019 at 9:48 PM Becket Qin wrote: > > Hi Ryanne, > > Thanks f

[VOTE] KIP-421: Support resolving externalized secrets in AbstractConfig

2019-01-22 Thread tejal
Hi all, We would like to start vote on KIP-421 to to enhance the AbstractConfig base class to support replacing variables in configurations just prior to parsing and validation. Link for the KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-421%3A+Support+resolving+externalized+secre

[jira] [Created] (KAFKA-7854) Behavior change in controller picking up partition reassignment tasks since 1.1.0

2019-01-22 Thread Zhanxiang (Patrick) Huang (JIRA)
Zhanxiang (Patrick) Huang created KAFKA-7854: Summary: Behavior change in controller picking up partition reassignment tasks since 1.1.0 Key: KAFKA-7854 URL: https://issues.apache.org/jira/browse/KAFKA

Re: [DISCUSS] KIP-158: Kafka Connect should allow source connectors to set topic-specific settings for new topics

2019-01-22 Thread Ryanne Dolan
Thanks Randall, makes sense to me. I suggest we change the topic.creation.enabled property name though. Sounds like it means topics are not created at all when disabled. Ryanne On Sat, Jan 19, 2019, 1:13 PM Randall Hauch Hi, > > Thanks again for all of the feedback. Based upon this feedback, I

[jira] [Created] (KAFKA-7853) Refactor ConsumerCoordinator/AbstractCoordinator to reduce constructor parameter list

2019-01-22 Thread Boyang Chen (JIRA)
Boyang Chen created KAFKA-7853: -- Summary: Refactor ConsumerCoordinator/AbstractCoordinator to reduce constructor parameter list Key: KAFKA-7853 URL: https://issues.apache.org/jira/browse/KAFKA-7853 Proje

[jira] [Created] (KAFKA-7852) Add public version of EmbeddedKafkaCluster

2019-01-22 Thread John Roesler (JIRA)
John Roesler created KAFKA-7852: --- Summary: Add public version of EmbeddedKafkaCluster Key: KAFKA-7852 URL: https://issues.apache.org/jira/browse/KAFKA-7852 Project: Kafka Issue Type: Improvemen

Re: [DISCUSS] KIP-390: Add producer option to adjust compression level

2019-01-22 Thread Dongjin Lee
Hello. I just fixed the draft implementation, with rebasing onto the latest trunk. The KIP was also restored. Please have a look, and if there is no major problem, please vote to the voting thread. You know, KIP freeze for 2.2.0 is almost imminent. Thanks, Dongjin On Tue, Jan 22, 2019 at 1:04 AM

[jira] [Created] (KAFKA-7851) Continue data streaming while rebalancing

2019-01-22 Thread Christian Richter (JIRA)
Christian Richter created KAFKA-7851: Summary: Continue data streaming while rebalancing Key: KAFKA-7851 URL: https://issues.apache.org/jira/browse/KAFKA-7851 Project: Kafka Issue Type: I

Re: [VOTE] KIP-396: Add Commit/List Offsets Operations to AdminClient

2019-01-22 Thread Mickael Maison
Thanks Ismael for the feedback. I think your point has 2 parts: - Having the reset functionality in the AdminClient: The fact we have a command line tool illustrate that this operation is relatively common. I seems valuable to be able to perform this operation directly via a proper API in addition

[jira] [Resolved] (KAFKA-6777) Wrong reaction on Out Of Memory situation

2019-01-22 Thread Seweryn Habdank-Wojewodzki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6777?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Seweryn Habdank-Wojewodzki resolved KAFKA-6777. --- Resolution: Won't Fix Last comment is accepted. We have to prepare ot