[UPDATE] KIP-341: Update Sticky Assignor's User Data Protocol

2019-04-15 Thread Vahid Hashemian
Just a heads up to the community that the implementation of this KIP is almost complete. I'd like to just mention that there was a slight deviation in implementation from the approved KIP. I have updated the KIP to keep it consistent with the final implementation. To check what has changes please

[jira] [Created] (KAFKA-8238) Log how many bytes and messages were read from __consumer_offsets

2019-04-15 Thread Colin P. McCabe (JIRA)
Colin P. McCabe created KAFKA-8238: -- Summary: Log how many bytes and messages were read from __consumer_offsets Key: KAFKA-8238 URL: https://issues.apache.org/jira/browse/KAFKA-8238 Project: Kafka

Re: [DISCUSSION] KIP-421: Automatically resolve external configurations.

2019-04-15 Thread Tejal Adsul
Hi All, I have updated the KIP to address the comments in the discussion. I have added the flow as to how dynamic config values will be resolved. Please could you’ll review the updated changes and let me know your feedback. Thanks, Tejal On 2019/03/21 20:38:54, Tejal Adsul wrote: > I have

Request KIP Permissions

2019-04-15 Thread Jukka Karvanen
Hi, Could you please grant me write access to KIP proposals? I am planning to make KIP for KAFKA-8233: Helper class to make it simpler to write test logic with TopologyTestDriver Wiki ID: jkarvanen Best regards, Jukka Karvanen

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

2019-04-15 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-7778: Add KTable.suppress to Scala API (#6314) [vahid.hashemian] KAFKA-7471: Multiple Consumer Group Management Feature (#5726) --

[DISCUSS] KIP-421: Automatically resolve external configurations.

2019-04-15 Thread TEJAL ADSUL
Hi All, I have updated the KIP to address the comments in the discussion. I have added the flow as to how dynamic config values will be resolved. Please could you’ll review the updated changes and let me know your feedback. Thanks, Tejal On 2019/03/21 20:36:24, Tejal Adsul wrote: > @Colin

Re: [DISCUSS] KIP-455: Create an Administrative API for Replica Reassignment

2019-04-15 Thread Colin McCabe
link: https://cwiki.apache.org/confluence/display/KAFKA/KIP-455%3A+Create+an+Administrative+API+for+Replica+Reassignment C. On Mon, Apr 15, 2019, at 18:07, Colin McCabe wrote: > Hi all, > > We've been having discussions on a few different KIPs (KIP-236, > KIP-435, etc.) about what the Admin

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

2019-04-15 Thread Apache Jenkins Server
See -- [...truncated 73 B...] No credentials specified > git rev-parse --is-inside-work-tree # timeout=10 Fetching changes from the remote Git repository > git config remote.origin.url

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

2019-04-15 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-6635; Producer close awaits pending transactions (#5971) -- [...truncated 2.38 MB...] org.apache.kafka.streams.TopologyTestDriverTest >

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

2019-04-15 Thread Apache Jenkins Server
See -- [...truncated 73 B...] No credentials specified > git rev-parse --is-inside-work-tree # timeout=10 Fetching changes from the remote Git repository > git config remote.origin.url

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

2019-04-15 Thread Apache Jenkins Server
See -- [...truncated 570 B...] > git config remote.origin.url https://github.com/apache/kafka.git # timeout=10 Fetching upstream changes from https://github.com/apache/kafka.git > git

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

2019-04-15 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-6635; Producer close awaits pending transactions (#5971) -- [...truncated 2.36 MB...] org.apache.kafka.connect.json.JsonConverterTest >

[jira] [Resolved] (KAFKA-8228) Exactly once semantics break during server restart for kafka-streams application

2019-04-15 Thread Boquan Tang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8228?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Boquan Tang resolved KAFKA-8228. Resolution: Duplicate This might duplicate KAFKA-7866, close for now and watch that ticket. >

[DISCUSS] KIP-455: Create an Administrative API for Replica Reassignment

2019-04-15 Thread Colin McCabe
Hi all, We've been having discussions on a few different KIPs (KIP-236, KIP-435, etc.) about what the Admin Client replica reassignment API should look like. The current API is really hard to extend and maintain, which is a big source of problems. I think it makes sense to have a KIP that

Re: [DISCUSS] KIP-449: Add connector contexts to Connect worker logs

2019-04-15 Thread Randall Hauch
Thanks, Konstantine. I'm not sure it's within the scope of this PR to address the `worker.id` MDC parameter, especially because the behavior is not yet well defined as to what happens when the `client.id` is set to a non-unique value. I guess I'd prefer to limit the scope to just the additional

[jira] [Created] (KAFKA-8237) Untangle TopicDeletionManager and add test cases

2019-04-15 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-8237: -- Summary: Untangle TopicDeletionManager and add test cases Key: KAFKA-8237 URL: https://issues.apache.org/jira/browse/KAFKA-8237 Project: Kafka Issue

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

2019-04-15 Thread Sophie Blee-Goldman
Thanks for the feedback Matthias and Bill. After discussing offline we realized the type of windows I originally had in mind were quite different, and I agree now that the semantics outlined by Matthias are the direction to go in here. I will update the KIP accordingly with the new semantics (and

Re: [DISCUSSION] KIP-422: Add support for user/client configuration in the Kafka Admin Client

2019-04-15 Thread Colin McCabe
Hi all, In KIP-133: Describe and Alter Configs Admin APIs, there is "future work" section that explains: > Future Work > ... > > 2. Support for reading and updating client, user and replication quotas. We > initially included that in the KIP, but it subsequently became apparent > that a

Re: [DISCUSS] KIP-449: Add connector contexts to Connect worker logs

2019-04-15 Thread Konstantine Karantasis
Thanks for the quick updates on the KIP Randall! 3. Indeed. I thought I remembered some ambiguity in the numbering of tasks, but looking closer I think we are fine, especially now that you mention explicitly 0-based indexing for task ids. 4. Fine with the example as is. 5. Sounds good. 6.

Re: [DISCUSS] KIP-252: Extend ACLs to allow filtering based on ip ranges and subnets

2019-04-15 Thread Colin McCabe
On Mon, Apr 8, 2019, at 06:38, Sönke Liebau wrote: > Hi Colin, > > quick summary up front: I totally agree, and always have! I think we > misunderstood each other a little :) > I was never really opposed the idea of restricting which ACL features can > be used, I was just opposed to doing it

Re: [DISCUSS] KIP-449: Add connector contexts to Connect worker logs

2019-04-15 Thread Chris Egerton
Hi Randall, Thanks for the KIP. Debugging Connect workers definitely becomes harder as the number of connectors and tasks increases, and your proposal would simplify the process of sifting through logs and finding relevant information faster and more accurately. I have a couple small comments:

[jira] [Created] (KAFKA-8236) Incorporate version control for Kafka Streams Application Reset

2019-04-15 Thread Boyang Chen (JIRA)
Boyang Chen created KAFKA-8236: -- Summary: Incorporate version control for Kafka Streams Application Reset Key: KAFKA-8236 URL: https://issues.apache.org/jira/browse/KAFKA-8236 Project: Kafka

Jenkins build is back to normal : kafka-trunk-jdk11 #436

2019-04-15 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-435: Incremental Partition Reassignment

2019-04-15 Thread Viktor Somogyi-Vass
A follow up on the batching topic to clarify my points above. Generally I think that batching should be a core feature as Colin said the controller should possess all information that are related. Also Cruise Control (or really any 3rd party admin system) might build upon this to give more

Re: [VOTE] KIP-422: Use the default value of max.poll.interval in Streams

2019-04-15 Thread Jun Rao
The voting thread seems to be referring to the wrong KIP number. It should be KIP-442 instead of KIP-422. Thanks, Jun On Wed, Apr 3, 2019 at 7:03 PM John Roesler wrote: > Thanks all. The KIP-442 vote has passed with 3 binding votes (Guozhang, > Bill, and Damian) and one non-binding vote (me)

Jenkins build is back to normal : kafka-trunk-jdk8 #3551

2019-04-15 Thread Apache Jenkins Server
See

Re: Kafka Source Code Debugging

2019-04-15 Thread Kartik Kalaghatgi
Checkout : https://cwiki.apache.org/confluence/display/KAFKA/Developer+Setup Thanks, Kartik On Mon, 15 Apr 2019 at 22:03, Shubham Singh wrote: > Hi Team, > > I wanted to debug Kafka github (Git hub link > ) source code on my eclipse. > Please let me know how to set up and make kafka run in

Kafka Source Code Debugging

2019-04-15 Thread Shubham Singh
Hi Team, I wanted to debug Kafka github (Git hub link ) source code on my eclipse. Please let me know how to set up and make kafka run in debug mode. Thanks, Shubham

Re: [DISCUSS] KIP-435: Incremental Partition Reassignment

2019-04-15 Thread Viktor Somogyi-Vass
Hey Guys, I'll reply to you all in this email: @Jun: 1. yes, it'd be a good idea to add this feature, I'll write this into the KIP. I was actually thinking about introducing a dynamic config called reassignment.parallel.partition.count and reassignment.parallel.replica.count. The first property

Jenkins build is back to normal : kafka-2.2-jdk8 #80

2019-04-15 Thread Apache Jenkins Server
See

KAFKA-7471: Multiple Consumer Group Management PR merging with trunk

2019-04-15 Thread Alex D
Hello guys, Any comments/updates on Multiple Consumer Group Management PR: https://github.com/apache/kafka/pull/5726 ? Best Regards, Alex Dunayevsky

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

2019-04-15 Thread Apache Jenkins Server
See Changes: [github] KAFKA-8232; Test topic delete completion rather than intermediate state -- [...truncated 2.36 MB...] org.apache.kafka.connect.data.ValuesTest >

Re: [VOTE] KIP-419: Safely notify Kafka Connect SourceTask is stopped

2019-04-15 Thread Edoardo Comar
Thanks Andrew. +1 (non-binding) -- Edoardo Comar IBM Event Streams IBM UK Ltd, Hursley Park, SO21 2JN From: Mickael Maison To: dev Date: 10/04/2019 10:14 Subject:Re: [VOTE] KIP-419: Safely notify Kafka Connect SourceTask is

[Jira][Created](KAFKA-8100)kafka consumer not refresh metadata for dynamic topic deletion

2019-04-15 Thread Shengnan YU
Hi everyone: I found that KafkaConsumer will not refresh its topic metadata when some unused topics are deleted, which leads to continuous UNKNOWN_TOPIC_PARTITION warning. In source code KafkaProducer will remove unused topic after a expire time but KafkaConsumer not. I know it may be a design

[jira] [Created] (KAFKA-8235) NoSuchElementException when restoring state after a clean shutdown of a Kafka Streams application

2019-04-15 Thread Andrew Klopper (JIRA)
Andrew Klopper created KAFKA-8235: - Summary: NoSuchElementException when restoring state after a clean shutdown of a Kafka Streams application Key: KAFKA-8235 URL: https://issues.apache.org/jira/browse/KAFKA-8235

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

2019-04-15 Thread Apache Jenkins Server
See -- [...truncated 593 B...] Fetching upstream changes from https://github.com/apache/kafka.git > git --version # timeout=10 > git fetch --tags --progress

[jira] [Created] (KAFKA-8234) Multi-module support for JAAS config property

2019-04-15 Thread Gabor Somogyi (JIRA)
Gabor Somogyi created KAFKA-8234: Summary: Multi-module support for JAAS config property Key: KAFKA-8234 URL: https://issues.apache.org/jira/browse/KAFKA-8234 Project: Kafka Issue Type:

[jira] [Resolved] (KAFKA-8232) Flaky test kafka.admin.TopicCommandWithAdminClientTest.testTopicDeletion

2019-04-15 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8232?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-8232. --- Resolution: Fixed Reviewer: Manikumar Fix Version/s: 2.2.1 > Flaky test

[jira] [Created] (KAFKA-8233) Helper class to make it simpler to write test logic TopologyTestDriver

2019-04-15 Thread Jukka Karvanen (JIRA)
Jukka Karvanen created KAFKA-8233: - Summary: Helper class to make it simpler to write test logic TopologyTestDriver Key: KAFKA-8233 URL: https://issues.apache.org/jira/browse/KAFKA-8233 Project:

[VOTE] KIP-437: Custom replacement for MaskField SMT

2019-04-15 Thread Valeria Vasylieva
Hi all, Since there are no more objections/proposals I would like to start the vote on KIP-437. See: KIP-437 and related PR I will be grateful to