Re: [DISCUSS] How hard is it to separate the logic layer and the storage layer of Kafka broker?

2018-11-02 Thread Yuanjin Lin
Colin, Thanks for the meaningful reply! We are 100% sure those HDDs are the bottleneck. Almost 90% alerts are about HDDs. I am the guy who have to deal with it. The common scenario would be 100-400 partitions per HDD(2TB size). Due to some historical reasons, developers in my company tend to put

Re: [VOTE] - KIP-213 Support non-key joining in KTable

2018-11-02 Thread Jan Filipiak
reminder On 30.10.2018 15:47, Adam Bellemare wrote: > Hi All > > I would like to call a vote on > https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable. > This allows a Kafka Streams DSL user to perform KTable to KTable > foreign-key joins on their data. I

[DISCUSS] How hard is it to separate the logic layer and the storage layer of Kafka broker?

2018-11-02 Thread Yuanjin Lin
Hi all, I am a software engineer from Zhihu.com. Kafka is so great and used heavily in Zhihu. There are probably over 2K Kafka brokers in total. However, we are suffering from the problem that the performance degrades rapidly when the number of topics increases(sadly, we are using HDD). We are

Re: [VOTE] - KIP-213 Support non-key joining in KTable

2018-11-02 Thread Adam Bellemare
As expected :) But still, thanks none-the-less! On Fri, Nov 2, 2018 at 3:36 AM Jan Filipiak wrote: > reminder > > On 30.10.2018 15:47, Adam Bellemare wrote: > > Hi All > > > > I would like to call a vote on > > >

Re: KSQL with changelog topic

2018-11-02 Thread Matthias J. Sax
I see. Makes sense. AFAIK, this is not supported atm, because KSQL cannot handle the `Windowed` datatype yet. -Matthias On 11/1/18 9:28 PM, Boyang Chen wrote: > For example, I have a stream windowed aggregation changlog topic > KStream--XXX--changelog-26, does KSQL support a simple query

Re: [DISCUSS] How hard is it to separate the logic layer and the storage layer of Kafka broker?

2018-11-02 Thread Colin McCabe
On Fri, Nov 2, 2018, at 03:14, Yuanjin Lin wrote: > Hi all, > > I am a software engineer from Zhihu.com. Kafka is so great and used heavily > in Zhihu. There are probably over 2K Kafka brokers in total. > > However, we are suffering from the problem that the performance degrades > rapidly when

[jira] [Created] (KAFKA-7586) Connector status endpoint HTTP response codes

2018-11-02 Thread Rod Cordova (JIRA)
Rod Cordova created KAFKA-7586: -- Summary: Connector status endpoint HTTP response codes Key: KAFKA-7586 URL: https://issues.apache.org/jira/browse/KAFKA-7586 Project: Kafka Issue Type:

Re: Apache Kafka blog on more partitions support

2018-11-02 Thread Mayuresh Gharat
Thanks Jun for sharing this. Looks nice ! Do we intend to shed light on how much time is required, on an average, for new Leader election. Also would it be good to add "if the controller waits for the LeaderAndIsrResponses before sending shutDown_OK to the shutting down broker". Thanks,

Re: [VOTE] 2.0.1 RC0

2018-11-02 Thread Ewen Cheslack-Postava
+1 -Ewen On Thu, Nov 1, 2018 at 10:10 AM Manikumar wrote: > We were waiting for the system test results. There were few failures: > KAFKA-7579, KAFKA-7559, KAFKA-7561 > they are not blockers for 2.0.1 release. We need more votes from > PMC/committers :) > > Thanks Stanislav! for the system

Re: [VOTE] - KIP-213 Support non-key joining in KTable

2018-11-02 Thread Jan Filipiak
Hi Adam, congrats for pulling it of! As I mentioned its not something I can use in can / would use in production. So I am throwing a non binding minus one in here. I don't expect it todo any harm for the vote. Thanks for the credits :) Best Jan On 30.10.2018 15:47, Adam Bellemare wrote: >

Re: Apache Kafka blog on more partitions support

2018-11-02 Thread harshach
Thanks Jun for sharing the post. Minor Nit: Date says  December 16, 2019. Did this test measured the replication affects on the overall cluster health and performance? It looks like we are suggesting with 200k partitions and 4k per broker max size of a cluster should be around 50 brokers?

Re: KSQL with changelog topic

2018-11-02 Thread Boyang Chen
Thanks for confirming! From: Matthias J. Sax Sent: Saturday, November 3, 2018 1:35 AM To: dev@kafka.apache.org Subject: Re: KSQL with changelog topic I see. Makes sense. AFAIK, this is not supported atm, because KSQL cannot handle the `Windowed` datatype yet.

Re: Apache Kafka blog on more partitions support

2018-11-02 Thread Jun Rao
Hi, Harsha, The recommendation is the upper bound for partitions. One can certainly have more than 50 brokers in a cluster if there are fewer partitions per broker. I will clarify this in the blog. Thanks, Jun On Fri, Nov 2, 2018 at 12:07 PM, wrote: > Thanks Jun for sharing the post. > Minor

Re: Apache Kafka blog on more partitions support

2018-11-02 Thread Jun Rao
Hi, Mayuresh, Most of the controlled shutdown time is in leader election. The controller currently doesn't wait for LeaderAndIsrRequest to be sent out before responding to the controlled shutdown request. Thanks, Jun On Fri, Nov 2, 2018 at 1:58 PM, Mayuresh Gharat wrote: > Thanks Jun for

[jira] [Created] (KAFKA-7584) StreamsConfig throws ClassCastException if max.in.flight.request.per.connect is specified as String

2018-11-02 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-7584: -- Summary: StreamsConfig throws ClassCastException if max.in.flight.request.per.connect is specified as String Key: KAFKA-7584 URL:

Apache Kafka blog on more partitions support

2018-11-02 Thread Jun Rao
Hi, Everyone, The follow is the preview of a blog on Kafka supporting more partitions. https://drive.google.com/file/d/122TK0oCoforc2cBWfW_yaEBjTMoX6yMt Please let me know if you have any comments by Tuesday. Thanks, Jun

[jira] [Created] (KAFKA-7585) Avoid classloader when passing serializers to KafkaProducer constructors

2018-11-02 Thread Sherwin Chiu (JIRA)
Sherwin Chiu created KAFKA-7585: --- Summary: Avoid classloader when passing serializers to KafkaProducer constructors Key: KAFKA-7585 URL: https://issues.apache.org/jira/browse/KAFKA-7585 Project: Kafka