Re: [DISCUSS] KIP-361: Add Consumer Configuration to Disable Auto Topic Creation

2018-08-22 Thread Dhruvil Shah
*"suppress" is the opposite of "allow", so setting suppress.auto.create.topics=false would mean that we do _not_ allow auto topic creation; when set to true, the server configuration will determine whether we allow automatic creation or not.* Sorry, I meant suppress.auto.create.topics=true above

Re: [DISCUSS] KIP-361: Add Consumer Configuration to Disable Auto Topic Creation

2018-08-22 Thread Dhruvil Shah
To be clear, we will allow auto topic creation only when server config auto.create.topics.enable=true and consumer config allow.auto.create.topics=true; when either is false, we would not create the topic if it does not exist. "suppress" is the opposite of "allow", so setting

Re: [DISCUSS] KIP-361: Add Consumer Configuration to Disable Auto Topic Creation

2018-08-22 Thread Matthias J. Sax
@Dhruvil: Yes, I had "suppress.auto.topic.creation" or "allow.auto.topic.creation" in mind. @Brandon: I think it would be "allow=false" or "suppress=true" to enable this feature. Ie, default values would be "allow=true" or "suppress=false". Not sure if this changes any preferences for you.

[jira] [Created] (KAFKA-7329) Continuous warning message of LEADER_NOT_AVAILABLE

2018-08-22 Thread Vasudevan Seshadri (JIRA)
Vasudevan Seshadri created KAFKA-7329: - Summary: Continuous warning message of LEADER_NOT_AVAILABLE Key: KAFKA-7329 URL: https://issues.apache.org/jira/browse/KAFKA-7329 Project: Kafka

Re: [DISCUSS] KIP-361: Add Consumer Configuration to Disable Auto Topic Creation

2018-08-22 Thread Ismael Juma
1 seems reasonable to me, would be good to hear other opinions. With regards to 2, yes, we discussed deprecating and eventually removing the server config. But we'd like to make it easier to get auto create behaviour on the producer (and maybe the consumer although that's less clear) before we do

[jira] [Created] (KAFKA-7328) Java version for Kafka

2018-08-22 Thread Vasudevan Seshadri (JIRA)
Vasudevan Seshadri created KAFKA-7328: - Summary: Java version for Kafka Key: KAFKA-7328 URL: https://issues.apache.org/jira/browse/KAFKA-7328 Project: Kafka Issue Type: Improvement

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

2018-08-22 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-7327) kafak主节点cpu 内存持续飙高,不回收,最后服务挂掉问题?

2018-08-22 Thread ruiliang (JIRA)
ruiliang created KAFKA-7327: --- Summary: kafak主节点cpu 内存持续飙高,不回收,最后服务挂掉问题? Key: KAFKA-7327 URL: https://issues.apache.org/jira/browse/KAFKA-7327 Project: Kafka Issue Type: Bug Components:

Re: [DISCUSS] KIP-361: Add Consumer Configuration to Disable Auto Topic Creation

2018-08-22 Thread Dhruvil Shah
Hi Ismael, Thanks for the comments. Replies below. 1. We could throw an InvalidConfigurationException at run-time either when building the MetadataRequest or when an ApiVersions response is received. Because this is a configuration problem, I think users would likely see the exception on the

Re: [DISCUSS] KIP-361: Add Consumer Configuration to Disable Auto Topic Creation

2018-08-22 Thread Brandon Kirchner
“allow=false” seems a bit more intuitive to me than “suppress=false” Brandon > On Aug 22, 2018, at 8:48 PM, Ted Yu wrote: > > We may also consider : > > "suppress.auto.topic.creation" > > or > > "allow.auto.topic.creation" > > w.r.t. suppress or allow, I don't have strong opinion either.

Re: [DISCUSS] KIP-361: Add Consumer Configuration to Disable Auto Topic Creation

2018-08-22 Thread Ted Yu
We may also consider : "suppress.auto.topic.creation" or "allow.auto.topic.creation" w.r.t. suppress or allow, I don't have strong opinion either. It's just a matter of choosing the proper default value. Cheers On Wed, Aug 22, 2018 at 6:00 PM Dhruvil Shah wrote: > Hi Matthias, > > Do you

Re: [DISCUSS] KIP-361: Add Consumer Configuration to Disable Auto Topic Creation

2018-08-22 Thread Dhruvil Shah
Hi Matthias, Do you mean something like "suppress.auto.create.topic"? I am leaning a bit towards "allow.auto.create.topics" but I don't have a strong preference either. Let's wait to hear if anyone else has an opinion on this. Thanks, Dhruvil On Tue, Aug 21, 2018 at 5:28 PM Matthias J. Sax

Re: [DISCUSS] KIP-358: Migrate Streams API to Duration instead of long ms times

2018-08-22 Thread Matthias J. Sax
Thanks a lot for the KIP. From my understanding, the idea of the KIP is to improve the public API at DSL level. However, not all public methods listed are part of DSL level API, but part of runtime API. Those methods are called during processing and are on the hot code path. I am not sure, if we

Re: [DISCUSS] KIP-110: Add Codec for ZStandard Compression (Updated)

2018-08-22 Thread Dongjin Lee
Jason, Great. +1 for UNSUPPORTED_COMPRESSION_TYPE. Best, Dongjin On Thu, Aug 23, 2018 at 8:19 AM Jason Gustafson wrote: > Hey Dongjin, > > Yeah that's right. For what it's worth, librdkafka also appears to handle > unexpected error codes. I expect that most client implementations would >

Re: Current Kafka Steams and KSQL Performance Metrics / Benchmarks?

2018-08-22 Thread Guozhang Wang
Hello Adam, Thanks for your interests in working on Kafka Streams / KSQL potential performance improvements (I thought the non-key joining will take most of your time :P ) Currently there is no published performance numbers for latest versions of Streams AFAIK. Personally I ran the Streams

Re: [DISCUSS] KIP-110: Add Codec for ZStandard Compression (Updated)

2018-08-22 Thread Jason Gustafson
Hey Dongjin, Yeah that's right. For what it's worth, librdkafka also appears to handle unexpected error codes. I expect that most client implementations would either pass through the raw type or convert to an enum using something like what the java client does. Since we're expecting the client to

[jira] [Created] (KAFKA-7326) Let KStream.print() to flush on each printed line

2018-08-22 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-7326: Summary: Let KStream.print() to flush on each printed line Key: KAFKA-7326 URL: https://issues.apache.org/jira/browse/KAFKA-7326 Project: Kafka Issue Type:

[DISCUSS] KIP-363: Allow performance tools to print final results to output file

2018-08-22 Thread Attila Sasvári
Hi all, I have created a minor KIP to allow consumer and producer performance tools to print final results to output file in CSV format. https://cwiki.apache.org/confluence/display/KAFKA/KIP-363%3A+Allow+performance+tools+to+print+final+results+to+output+file Please take a look and share your

Re: [jira] [Created] (KAFKA-7325) Support dynamic gap session window

2018-08-22 Thread Daniel Roy
unsuscribe On Wed, Aug 22, 2018 at 7:22 PM, Lei Chen (JIRA) wrote: > Lei Chen created KAFKA-7325: > --- > > Summary: Support dynamic gap session window > Key: KAFKA-7325 > URL:

[jira] [Created] (KAFKA-7325) Support dynamic gap session window

2018-08-22 Thread Lei Chen (JIRA)
Lei Chen created KAFKA-7325: --- Summary: Support dynamic gap session window Key: KAFKA-7325 URL: https://issues.apache.org/jira/browse/KAFKA-7325 Project: Kafka Issue Type: Improvement

[jira] [Resolved] (KAFKA-6343) OOM as the result of creation of 5k topics

2018-08-22 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6343?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-6343. Resolution: Fixed Assignee: Alex Dunayevsky Fix Version/s: 2.1.0 Merged the PR to trunk.

[DISCUSS] KIP-362: Dynamic Session Window Support

2018-08-22 Thread Lei Chen
Hi All, I created a KIP to add dynamic gap session window support to Kafka Streams DSL. https://cwiki.apache.org/confluence/display/KAFKA/KIP-362%3A+Support+dynamic+gap+session+window Please take a look, Thanks, Lei

Re: [DISCUSS] KIP-357: Add support to list ACLs per principal

2018-08-22 Thread Manikumar
Hi Viktor, We already have a method in Authorizer interface to get acls for a given principal. We will use this method to fetch acls and filter the results for requested Resources. Authorizer { def getAcls(principal: KafkaPrincipal): Map[Resource, Set[Acl]] } Currently AdminClient API doesn't

Re: [VOTE] KIP-336: Consolidate ExtendedSerializer/Serializer and ExtendedDeserializer/Deserializer

2018-08-22 Thread Jason Gustafson
Hey Viktor, This is a nice cleanup. Just a couple quick questions: 1. Rather than returning null for the default `deserialize(topic, data)`, would it be better to throw UnsupportedOperationException? I assume that internally we'll always invoke the api which takes headers. Similarly for

Re: [DISCUSS] KIP-356: Add KafkaConsumer fetch-error-rate and fetch-error-total metrics

2018-08-22 Thread Kevin Lu
Bumping this as it has been a week without any responses. Regards, Kevin On Wed, Aug 15, 2018 at 11:40 PM Kevin Lu wrote: > Hi friends! :) > > I believe we currently have a gap in KafkaConsumer metrics for errors > since the KafkaConsumer is complex and are many places where things can go >

Re: Current Kafka Steams and KSQL Performance Metrics / Benchmarks?

2018-08-22 Thread Adam Bellemare
Blog post in question: https://www.confluent.io/blog/ksql-february-release-streaming-sql-for-apache-kafka/ On Wed, Aug 22, 2018 at 10:01 AM, Adam Bellemare wrote: > Hi All > > I am looking for performance metrics related to Kafka Streams and KSQL. I > have been scouring various blogs, including

Current Kafka Steams and KSQL Performance Metrics / Benchmarks?

2018-08-22 Thread Adam Bellemare
Hi All I am looking for performance metrics related to Kafka Streams and KSQL. I have been scouring various blogs, including the confluent one, looking for any current performance metrics or benchmarks, official or otherwise, on both Kafka Streams and KSQL for Kafka 2.x +. Unfortunately, almost

Re: [VOTE] KIP-336: Consolidate ExtendedSerializer/Serializer and ExtendedDeserializer/Deserializer

2018-08-22 Thread Satish Duggana
+1 On Wed, Aug 22, 2018 at 4:45 PM, Ted Yu wrote: > +1 > Original message From: Kamal Chandraprakash < > kamal.chandraprak...@gmail.com> Date: 8/22/18 3:19 AM (GMT-08:00) To: > dev@kafka.apache.org Subject: Re: [VOTE] KIP-336: Consolidate > ExtendedSerializer/Serializer and

Re: [ANNOUNCE] New Kafka PMC member: Dong Lin

2018-08-22 Thread Satish Duggana
Congrats Dong Lin! On Wed, Aug 22, 2018 at 10:08 AM, Abhimanyu Nagrath < abhimanyunagr...@gmail.com> wrote: > Congratulations, Dong! > > On Wed, Aug 22, 2018 at 6:20 AM Dhruvil Shah wrote: > > > Congratulations, Dong! > > > > On Tue, Aug 21, 2018 at 4:38 PM Jason Gustafson > > wrote: > > > > >

Re: [DISCUSS] KIP-357: Add support to list ACLs per principal

2018-08-22 Thread Viktor Somogyi-Vass
Hi Manikumar, Implementation-wise is it just a filter over the returned ACL listing or do you plan to add new methods to the Authorizer as well? Thanks, Viktor On Fri, Aug 17, 2018 at 9:18 PM Priyank Shah wrote: > +1(non-binding) > > Thanks. > Priyank > > On 8/16/18, 6:01 AM, "Manikumar"

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

2018-08-22 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: Split at first occurrence of '=' in kafka.py props parsing -- [...truncated 2.48 MB...]

Re: [VOTE] KIP-346 - Improve LogCleaner behavior on error

2018-08-22 Thread Stanislav Kozlovski
Hi everybody, @Jason - I've updated the section. Thanks for the reminder I'm glad to say that the vote *has passed* with 3 binding votes (Jason, Gwen, Harsha) and 6 non-binding votes (Dhruvil, Colin, Mickael, Manikumar, Ray, Ted, Thomas). The PR is ready for review at

Re: [VOTE] KIP-336: Consolidate ExtendedSerializer/Serializer and ExtendedDeserializer/Deserializer

2018-08-22 Thread Ted Yu
+1 Original message From: Kamal Chandraprakash Date: 8/22/18 3:19 AM (GMT-08:00) To: dev@kafka.apache.org Subject: Re: [VOTE] KIP-336: Consolidate ExtendedSerializer/Serializer and ExtendedDeserializer/Deserializer +1 Thanks for the KIP! On Wed, Aug 22, 2018 at 2:48 PM

Re: [VOTE] KIP-336: Consolidate ExtendedSerializer/Serializer and ExtendedDeserializer/Deserializer

2018-08-22 Thread Kamal Chandraprakash
+1 Thanks for the KIP! On Wed, Aug 22, 2018 at 2:48 PM Viktor Somogyi-Vass wrote: > Hi All, > > I'd like to start a vote on this KIP ( > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=87298242) > which aims to refactor ExtendedSerializer/Serializer and >

[VOTE] KIP-336: Consolidate ExtendedSerializer/Serializer and ExtendedDeserializer/Deserializer

2018-08-22 Thread Viktor Somogyi-Vass
Hi All, I'd like to start a vote on this KIP ( https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=87298242) which aims to refactor ExtendedSerializer/Serializer and ExtendedDeserializer/Deserializer. To summarize what's the motivation: When headers were introduced by KIP-82 the

[jira] [Resolved] (KAFKA-7279) partitionsFor implicitly creates topic for the existent topic

2018-08-22 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7279?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-7279. -- Resolution: Duplicate Resolving as duplicate of KAFKA-7320/KIP-361 > partitionsFor implicitly creates

Re: [DISCUSS] KIP-110: Add Codec for ZStandard Compression (Updated)

2018-08-22 Thread Dongjin Lee
Jason and Ismael, It seems like the only thing we need to regard if we define a new error code (i.e., UNSUPPORTED_COMPRESSION_TYPE) would be the implementation of the other language clients, right? At least, this strategy causes any problem for Java client. Do I understand correctly? Thanks,

Re: [DISCUSS] KIP-110: Add Codec for ZStandard Compression (Updated)

2018-08-22 Thread Dongjin Lee
Jason, > I think we would only use this error code when we /know/ that zstd was in use and the client doesn't support it? This is true if either 1) the message needs down-conversion and we encounter a zstd compressed message, or 2) if the topic is explicitly configured to use zstd. Yes, it is

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-08-22 Thread Eno Thereska
Ok thanks, if you guys are seeing this at LinkedIn then the motivation makes more sense. Eno On Tue, Aug 21, 2018 at 5:39 PM, Becket Qin wrote: > Hi Eno, > > Thanks for the comments. This KIP is not really about improving the > performance in general. It is about ensuring the cluster state can

Re: [DISCUSS] KIP-358: Migrate Streams API to Duration instead of long ms times

2018-08-22 Thread Nikolay Izhikov
Dear, commiters. Please, pay attention to this KIP and share your opinion. В Вт, 21/08/2018 в 11:14 -0500, John Roesler пишет: > I'll solicit more reviews. Let's get at least one committer to chime in > before we start a vote (since we need their approval anyway). > -John > > On Mon, Aug 20,