Re: [VOTE] KIP-441: Smooth Scaling Out for Kafka Streams

2019-09-11 Thread Vinoth Chandar
+1 (non-binding). On Fri, Sep 6, 2019 at 12:46 AM Bruno Cadonna wrote: > +1 (non-binding) > > On Fri, Sep 6, 2019 at 12:32 AM Guozhang Wang wrote: > > > > +1 (binding). > > > > On Thu, Sep 5, 2019 at 2:47 PM John Roesler wrote: > > > > > Hello, all, > > > > > > After a great discussion, I'd

Re: _consumer_offsets is becoming rather big. How to purge?

2019-09-11 Thread Colin McCabe
Hi Ash, At first guess, you probably had a problem with your log cleaner thread, which resulted in the offsets log not being cleaned. Check if that thread is running. best, Colin On Wed, Sep 11, 2019, at 09:52, Ash G wrote: > Bump, no reply, > > It seems this condition was missed by devs

KIP-521: Enable redirection of Connect's log4j messages to a file by default

2019-09-11 Thread Konstantine Karantasis
Hi all. While we are in the midst of some very interesting KIP discussions, I'd like to bring a brief and useful KIP on the table as well. It's about enabling redirection of log4j logging to a file for Kafka Connect by default, in a way similar to how this is done for Kafka brokers today. You

[jira] [Resolved] (KAFKA-8875) CreateTopic API should check topic existence before replication factor

2019-09-11 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8875?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-8875. Fix Version/s: 2.4.0 Resolution: Fixed > CreateTopic API should check topic

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

2019-09-11 Thread Apache Jenkins Server
See Changes: [github] KAFKA-8817: Remove timeout for the whole test (#7313) [github] MINOR: Add api version to uncaught exception message (#7311) [cmccabe] KAFKA-8345 (KIP-455): Controller and KafkaApi changes

[jira] [Resolved] (KAFKA-8886) Make Authorizer create/delete methods asynchronous

2019-09-11 Thread Rajini Sivaram (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8886?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-8886. --- Reviewer: Manikumar Resolution: Fixed > Make Authorizer create/delete methods

Re: [DISCUSS] KIP-519: Make SSL context/engine configuration extensible

2019-09-11 Thread Rajini Sivaram
Kafka already has the notion of custom configs. And we support reconfigurable custom configs for some interfaces e.g. MetricsReporter. We also recently added custom reconfigurable configs for Authorizer under KIP-504. The issue with custom configs for SSL is described in

RE: [DISCUSS] KIP-519: Make SSL context/engine configuration extensible

2019-09-11 Thread Pellerin, Clement
Indeed, this is a general problem requiring a more general solution than KIP-519. I'm glad there was work done on this already. So config.originals() still contains unknown configs but nothing has been validated and cast to the proper type. How does validation work for an extension point that

[jira] [Resolved] (KAFKA-8856) Add Streams Config for Backward-compatible Metrics

2019-09-11 Thread Bruno Cadonna (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8856?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bruno Cadonna resolved KAFKA-8856. -- Resolution: Fixed > Add Streams Config for Backward-compatible Metrics >

[jira] [Created] (KAFKA-8899) Optimize Partition.maybeIncrementLeaderHW

2019-09-11 Thread Lucas Bradstreet (Jira)
Lucas Bradstreet created KAFKA-8899: --- Summary: Optimize Partition.maybeIncrementLeaderHW Key: KAFKA-8899 URL: https://issues.apache.org/jira/browse/KAFKA-8899 Project: Kafka Issue Type:

Re: [DISCUSS] KIP-521: Enable redirection of Connect's log4j messages to a file by default

2019-09-11 Thread Gwen Shapira
Great idea. It will greatly improve the ops experience. Can't believe we didn't do it before. On Wed, Sep 11, 2019 at 2:07 PM Konstantine Karantasis wrote: > > *** Missed the [DISCUSS] tag in the previous email. Reposting here, please > reply in this thread instead *** > > Hi all. > > While we

[DISCUSS] Streams-Broker compatibility regression in 2.2.1 release

2019-09-11 Thread Matthias J. Sax
Hi, recently a user reported an issue upgrading a Kafka Streams application from 2.2.0 to 2.2.1 (cf https://mail-archives.apache.org/mod_mbox/kafka-users/201908.mbox/) After some investigation, we identified https://issues.apache.org/jira/browse/KAFKA-7895 to be the root cause of the problem.

[DISCUSS] KIP-521: Enable redirection of Connect's log4j messages to a file by default

2019-09-11 Thread Konstantine Karantasis
*** Missed the [DISCUSS] tag in the previous email. Reposting here, please reply in this thread instead *** Hi all. While we are in the midst of some very interesting KIP discussions, I'd like to bring a brief and useful KIP on the table as well. It's about enabling redirection of log4j logging

Re: [DISCUSS] KIP-521: Enable redirection of Connect's log4j messages to a file by default

2019-09-11 Thread Konstantine Karantasis
Thanks Gwen! Indeed, it's a common setup and it's been missing for some time. I agree, it'll be nice to have this in place by default. I'm guessing previous attempts missed that such a change needs a KIP. Cheers, Konstantine On Wed, Sep 11, 2019 at 2:16 PM Gwen Shapira wrote: > Great idea.

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

2019-09-11 Thread Apache Jenkins Server
See Changes: [github] KAFKA-8886; Make Authorizer create/delete API asynchronous (#7316) [wangguoz] KAFKA-8856: Add Streams config for backward-compatible metrics (#7279)

[jira] [Created] (KAFKA-8900) Stalled partitions

2019-09-11 Thread Luke Stephenson (Jira)
Luke Stephenson created KAFKA-8900: -- Summary: Stalled partitions Key: KAFKA-8900 URL: https://issues.apache.org/jira/browse/KAFKA-8900 Project: Kafka Issue Type: Bug Components:

Jenkins build is back to normal : kafka-2.3-jdk8 #103

2019-09-11 Thread Apache Jenkins Server
See

Re: [DISCUSS] Streams-Broker compatibility regression in 2.2.1 release

2019-09-11 Thread Alisson Sales
Thanks for letting the community/users know. The proposal seems sensible. I'm wondering if is it worth to add a note about this on the release notes here: https://www.apache.org/dist/kafka/2.2.1/RELEASE_NOTES.html. On Thu, Sep 12, 2019 at 5:23 AM Matthias J. Sax wrote: > Hi, > > recently a

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

2019-09-11 Thread Apache Jenkins Server
See Changes: [github] KAFKA-8886; Make Authorizer create/delete API asynchronous (#7316) -- [...truncated 1.90 MB...] at

Re: [DISCUSS] KIP-507: Securing Internal Connect REST Endpoints

2019-09-11 Thread Chris Egerton
Hi all, I've updated KIP-507 to reflect the changes inspired by Randall's recent feedback. In addition, after some further research, I've decided to remove the proposed default value for the internal.request.key.size and instead, should no value be provided, rely on the default key size for the

Re: [VOTE] KIP-520: Augment Consumer.committed(partition) to allow multiple partitions

2019-09-11 Thread Kamal Chandraprakash
Thanks for the KIP! LGTM, +1 (non-binding). On Wed, Sep 11, 2019 at 3:23 AM Matthias J. Sax wrote: > I don't have a strong preference. So I am also fine to deprecate the > existing methods. Let's see what Jason thinks. > > Can you update the KIP to reflect the semantics of the return `Map`

[jira] [Created] (KAFKA-8897) Increase Version of RocksDB

2019-09-11 Thread Bruno Cadonna (Jira)
Bruno Cadonna created KAFKA-8897: Summary: Increase Version of RocksDB Key: KAFKA-8897 URL: https://issues.apache.org/jira/browse/KAFKA-8897 Project: Kafka Issue Type: Improvement

Re: [DISCUSS] KIP-519: Make SSL context/engine configuration extensible

2019-09-11 Thread Maulin Vasavada
Hi all, Since the "custom config" seems the main topic of interest let us talk about it. 1. I want to confirm that I interpret the definition of 'custom config of SslEngineFactory' the same way Clement is suggesting - "a config that does not exist in Kafka but is specified by a custom

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

2019-09-11 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-8747; Add atomic counter to fix flaky testEventQueueTime test -- [...truncated 2.03 MB...] kafka.server.KafkaMetricReporterClusterIdTest >

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

2019-09-11 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-8898) if there is no message for poll, kafka consumer apply memory

2019-09-11 Thread linking12 (Jira)
linking12 created KAFKA-8898: Summary: if there is no message for poll, kafka consumer apply memory Key: KAFKA-8898 URL: https://issues.apache.org/jira/browse/KAFKA-8898 Project: Kafka Issue

Topology with loops (intermediate topics) and potential bug in TopologyTestDriver

2019-09-11 Thread Adam Domanski
Hi, I probably found a bug in TopologyTestDriver for quite non-trivial Kafka Streams topology. The streaming logic is the following: There is a concept of children and parents. Children are aggregated under parent. Some children of master parent can send poison pills to other parents. Such

Re: [DISCUSS] KIP-511: Collect and Expose Client's Name and Version in the Brokers

2019-09-11 Thread David Jacot
Hi all, I have discussed with Magnus about the various options to get his view from a librdkafka perspective and he has suggested a good alternative. It seems we could continue with the idea to use the ApiVersionsRequest/Response but we a different failing back strategy. When a broker get an

RE: [DISCUSS] KIP-519: Make SSL context/engine configuration extensible

2019-09-11 Thread Pellerin, Clement
I'm sorry if I divert the discussion, but without this issue, it would have been pretty trivial to update KIP-383 to go as far as you did. I am also happy to get a discussion going, the KIP-383 thread was a desolate place. Kafka needs to know about custom configs because it validates the

Re: _consumer_offsets is becoming rather big. How to purge?

2019-09-11 Thread Ash G
Bump, no reply, It seems this condition was missed by devs when this feature was designed and implemented. On 2019/09/06 14:45:47, Ash G wrote: > > _consumer_offsets is becoming rather big > 1 TB. Is there a way to purge > dead/inactive consumer id rows from it? > I am assuming

Re: Topology with loops (intermediate topics) and potential bug in TopologyTestDriver

2019-09-11 Thread John Roesler
Hi Adam, I haven't looked into your project yet, but just wanted to mention this to see if it explains your observations. TopologyTestDriver processes every input record fully and synchronously. So, when you pipe an input record in, it traverses all internal processing, including intermediate

Re: [DISCUSS] KIP-470: TopologyTestDriver test input and output usability improvements

2019-09-11 Thread John Roesler
Thanks for the update, Jukka! I'd be in favor of the current proposal. Not sure how the others feel. If people generally feel positive, it might be time to start a vote. Thanks, -John On Sat, Sep 7, 2019 at 12:40 AM Jukka Karvanen wrote: > > Hi, > > Sorry; I need to rollback right away the one