Re: [Discuss] KIP-389: Enforce group.max.size to cap member metadata growth

2019-01-09 Thread Stanislav Kozlovski
Sounds good! I've updated the KIP with another small section under Motivation. If there aren't any objections or further recommendations, I plan on starting a VOTE thread in the following days. Best, Stanislav On Wed, Jan 9, 2019 at 8:54 PM Gwen Shapira wrote: > Thanks for the data driven appro

Re: [DISCUSS] KIP-362: Dynamic Session Window Support

2019-01-09 Thread Guozhang Wang
Hello Lei, Just checking what's the current status of this KIP. We have a KIP deadline for 2.2 on 24th and wondering if this one may be able to make it. Guozhang On Sat, Dec 15, 2018 at 1:01 PM Lei Chen wrote: > Sorry for the late reply Matthias. Have been busy with other work recently. > I'l

[jira] [Created] (KAFKA-7809) Getting uncaught exception in kafka-producer-network-thread | producer-7686: java.lang.OutOfMemoryError: Java heap space

2019-01-09 Thread Suman Kalyan Ghosh (JIRA)
Suman Kalyan Ghosh created KAFKA-7809: - Summary: Getting uncaught exception in kafka-producer-network-thread | producer-7686: java.lang.OutOfMemoryError: Java heap space Key: KAFKA-7809 URL: https://issues.ap

Re: [DISCUSSION] KIP-412: Extend Admin API to support dynamic application log levels

2019-01-09 Thread Stanislav Kozlovski
Sorry about cutting the last message short. I was meaning to say that in the future we would be able to introduce finer-grained logging configuration (e.g enable debug logs for operations pertaining to this topic) and that would be easier to do if we are to know what the target resource of an Incre

[jira] [Resolved] (KAFKA-6431) Lock contention in Purgatory

2019-01-09 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6431?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sriharsha Chintalapani resolved KAFKA-6431. --- Resolution: Fixed > Lock contention in Purgatory > --

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

2019-01-09 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Add 2.1 version metadata upgrade (#6111) -- [...truncated 2.25 MB...] org.apache.kafka.streams.test.OutputVerifierTest > shouldFailIfValueI

Re: [VOTE] KIP-382 MirrorMaker 2.0

2019-01-09 Thread Ryanne Dolan
Thanks Jun. > 103. My point was that the MirrorMakerConnector can die while the Heartbeat connector is still alive. So, one can't solely rely on Heartbeat for monitoring? Each cluster will have a heartbeat topic produced by MirrorHeartbeatConnector, which doesn't have an associated "source" other

Re: [VOTE] - KIP-213 (new vote) - Simplified and revised.

2019-01-09 Thread Guozhang Wang
Hello Adam, I'm +1 on the current proposal, thanks! Guozhang On Mon, Jan 7, 2019 at 6:13 AM Adam Bellemare wrote: > Hi All > > I would like to call a new vote on KIP-213. The design has changed > substantially. Perhaps more importantly, the KIP and associated > documentation has been greatly

Re: [DISCUSS] KIP-307: Allow to define custom processor names with KStreams DSL

2019-01-09 Thread Guozhang Wang
Hello Florian, Just checking if have read about my previous email and if you feel happy about it. We have the 2.2 KIP freeze deadline at 24th this month, while the PR itself is getting quite close. So it'll be great if we can get the agreement on it and get it into 2.2.0 release. Guozhang On M

Re: [VOTE] KIP-382 MirrorMaker 2.0

2019-01-09 Thread Jun Rao
Hi, Ryanne, 103. My point was that the MirrorMakerConnector can die while the Heartbeat connector is still alive. So, one can't solely rely on Heartbeat for monitoring? 105. Hmm, maybe I don't understand how this is done. Let's say we replica topic1 from cluster A to cluster B. My understanding i

Re: [DISCUSS] KIP-378: Enable Dependency Injection for Kafka Streams handlers

2019-01-09 Thread Guozhang Wang
Hello Wladimir, Just checking if you are still working on this KIP. We have the 2.2 KIP freeze deadline by 24th this month, and it'll be great to complete this KIP by then so 2.2.0 release could have this feature. Guozhang On Mon, Dec 3, 2018 at 11:26 PM Guozhang Wang wrote: > Hello Wladimir,

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

2019-01-09 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-7786; Ignore OffsetsForLeaderEpoch response if epoch changed while -- [...truncated 917.14 KB...] kafka.zk.ReassignPartitionsZNodeTest > testDecod

[jira] [Created] (KAFKA-7808) AdminClient#describeTopic should not throw InvalidTopic if topic name is not found

2019-01-09 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-7808: Summary: AdminClient#describeTopic should not throw InvalidTopic if topic name is not found Key: KAFKA-7808 URL: https://issues.apache.org/jira/browse/KAFKA-7808 Proj

[jira] [Resolved] (KAFKA-7786) Fast update of leader epoch may stall partition fetching due to FENCED_LEADER_EPOCH

2019-01-09 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7786?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-7786. Resolution: Fixed Fix Version/s: 2.1.1 2.2.0 > Fast update of lea

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

2019-01-09 Thread Apache Jenkins Server
See

Re: [Discuss] KIP-389: Enforce group.max.size to cap member metadata growth

2019-01-09 Thread Gwen Shapira
Thanks for the data driven approach, Stanislav. I love it :) And thank you for sharing your formula, Boyang. I totally agree that rebalance latency will not grow linearly with the consumer group size. My recommendation, considering what we know today: 1. Add the limit config, and set it to MAX_INT

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

2019-01-09 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-7799; Fix flaky test RestServerTest.testCORSEnabled (#6106) -- [...truncated 427.31 KB...] kafka.server.IsrExpirationTest > testIsrExpirationIfNoFe

[jira] [Created] (KAFKA-7807) Special Characters in Topic Name Appear to Break Replication

2019-01-09 Thread Joseph Niemiec (JIRA)
Joseph Niemiec created KAFKA-7807: - Summary: Special Characters in Topic Name Appear to Break Replication Key: KAFKA-7807 URL: https://issues.apache.org/jira/browse/KAFKA-7807 Project: Kafka

[jira] [Created] (KAFKA-7806) Windowed Aggregations should wrap default key serde if none is specified

2019-01-09 Thread John Roesler (JIRA)
John Roesler created KAFKA-7806: --- Summary: Windowed Aggregations should wrap default key serde if none is specified Key: KAFKA-7806 URL: https://issues.apache.org/jira/browse/KAFKA-7806 Project: Kafka

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

2019-01-09 Thread Apache Jenkins Server
See

Re: [VOTE] [REMINDER] KIP-383 Pluggable interface for SSL Factory

2019-01-09 Thread Harsha
HI All, We are looking forward to this KIP. Appreciate if others can take a look at the kip and vote on this thread. Thanks Harsha On Fri, Dec 21, 2018, at 4:41 AM, Damian Guy wrote: > must be my gmail playing up. This appears to be the DISCUSS thread to me... > e > On Thu, 20 Dec 2018 a

Delayed messages with skewed data when subscribing to many partitions

2019-01-09 Thread Sönke Liebau
Hi all, we've just had a case where we suspect that messages get delayed from being consumed under certain circumstances. I don't necessarily think this is a bug, hence have not opened a jira yet but wanted to discuss here - there's probably a best practice that I just don't know about. The scena

[jira] [Created] (KAFKA-7805) Use --bootstrap-server option in ducktape tests where applicable

2019-01-09 Thread Viktor Somogyi (JIRA)
Viktor Somogyi created KAFKA-7805: - Summary: Use --bootstrap-server option in ducktape tests where applicable Key: KAFKA-7805 URL: https://issues.apache.org/jira/browse/KAFKA-7805 Project: Kafka

[jira] [Created] (KAFKA-7804) Update the docs for KIP-377

2019-01-09 Thread Viktor Somogyi (JIRA)
Viktor Somogyi created KAFKA-7804: - Summary: Update the docs for KIP-377 Key: KAFKA-7804 URL: https://issues.apache.org/jira/browse/KAFKA-7804 Project: Kafka Issue Type: Improvement

Re: [DISCUSSION] KIP-412: Extend Admin API to support dynamic application log levels

2019-01-09 Thread Stanislav Kozlovski
Hey Ryanne, thanks for taking a look at the KIP! I think that it is useful to specify the distinction between a standard Kafka config and the log level configs. The log level can be looked at as a separate resource as it does not change the behavior of the Kafka broker in any way. In terms of prac

Re: [Discuss] KIP-389: Enforce group.max.size to cap member metadata growth

2019-01-09 Thread Stanislav Kozlovski
Hey everybody, I ran a quick benchmark and took some heap dumps to gauge how much memory each consumer in a group is using, all done locally. The setup was the following: 10 topics with 10 partitions each (100 partitions total) and one consumer group with 10 members, then expanded to 20 members. H

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

2019-01-09 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Simplify handling of KafkaProducer serializer overrides [jason] KAFKA-6833; Producer should await metadata for unknown partitions -- [...truncated

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

2019-01-09 Thread Apache Jenkins Server
See Changes: [github] KAFKA-7799; Fix flaky test RestServerTest.testCORSEnabled (#6106) -- [...truncated 4.50 MB...] org.apache.kafka.streams.test.OutputVerifierTest > sh

Jenkins build is back to normal : kafka-2.0-jdk8 #211

2019-01-09 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-201: Rationalising Policy interfaces

2019-01-09 Thread Tom Bentley
Hi Anna and Mickael, Anna, did you have any comments about the points I made? Mickael, we really need the vote to be passed before there's even any work to do. With the exception of Ismael, the KIP didn't seem to get the attention of any of the other committers. Kind regards, Tom On Thu, 13 De

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

2019-01-09 Thread Apache Jenkins Server
See Changes: [github] KAFKA-7799; Fix flaky test RestServerTest.testCORSEnabled (#6106) -- [...truncated 2.25 MB...] org.apache.kafka.streams.test.OutputVerifierTest > s

[jira] [Created] (KAFKA-7803) Streams internal topics config is not updated when the code is changed

2019-01-09 Thread Tim Van Laer (JIRA)
Tim Van Laer created KAFKA-7803: --- Summary: Streams internal topics config is not updated when the code is changed Key: KAFKA-7803 URL: https://issues.apache.org/jira/browse/KAFKA-7803 Project: Kafka

[jira] [Resolved] (KAFKA-6833) KafkaProducer throws "Invalid partition given with record" exception

2019-01-09 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6833?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-6833. Resolution: Fixed Fix Version/s: 2.1.1 2.2.0 > KafkaProducer thro

[jira] [Resolved] (KAFKA-7799) Fix flaky test RestServerTest.testCORSEnabled

2019-01-09 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7799?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-7799. Resolution: Fixed > Fix flaky test RestServerTest.testCORSEnabled > ---