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 everything to Kafka, cuz it makes them
feel safe. Although, we have over 2k servers for Kafka. I still can receive
alerts everyday.

If the modification I proposed is not too hard, I will begin to do it next
month.


On Sat, Nov 3, 2018 at 1:36 AM Colin McCabe  wrote:

> 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 the number of topics increases(sadly, we are using HDD).
>
> Hi Yuanjin,
>
> How many partitions are you trying to create?
>
> Do you have benchmarks confirming that disk I/O is your bottleneck?  There
> are a few cases where large numbers of partitions may impose CPU and
> garbage collection burdens.  The patch on
> https://github.com/apache/kafka/pull/5206 illustrates one of them.
>
> > We are considering separating the logic layer and the storage layer of
> Kafka
> > broker like Apache Pulsar.
> >
> > After the modification, a server may have several Kafka brokers and more
> > topics. Those brokers all connect to a sole storage engine via RP The
> > sole storage can do the load balancing work easily, and avoid creating
> too
> > many files which hurts HDD.
> >
> > Is it hard? I think replacing the stuff in `Kafka.Log` would be enough,
> > right?
>
> It would help to know what the problem is here.  If the problem is a large
> number of files, then maybe the simplest approach would be creating fewer
> files.  You don't need to introduce a new layer of servers in order to do
> that.  You could use something like RocksDB to store messages and indices,
> or create your own file format which combined together things which were
> previously separate.  For example, we could combine the timeindex and index
> files.
>
> As I understand it, Pulsar made the decision to combine together data from
> multiple partitions in a single file.  Sometimes a very large number of
> partitions.  This is great for writing, but not so good if you want to read
> historical data from a single topic.
>
> regards,
> Colin
>


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 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,
>
> Mayuresh
>
> On Fri, Nov 2, 2018 at 12:07 PM  wrote:
>
> > 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?
> >
> > Thanks,
> > Harsha
> > On Nov 2, 2018, 11:50 AM -0700, Jun Rao , wrote:
> > > 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
> >
>
>
> --
> -Regards,
> Mayuresh R. Gharat
> (862) 250-7125
>


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 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?
>
> Thanks,
> Harsha
> On Nov 2, 2018, 11:50 AM -0700, Jun Rao , wrote:
> > 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
>


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 test results.
>
> Thanks,
> Manikumar
>
> On Thu, Nov 1, 2018 at 10:20 PM Eno Thereska 
> wrote:
>
> > Anything else holding this up?
> >
> > Thanks
> > Eno
> >
> > On Thu, Nov 1, 2018 at 10:27 AM Jakub Scholz  wrote:
> >
> > > +1 (non-binding) ... I used the staged binaries and run tests with
> > > different clients.
> > >
> > > On Fri, Oct 26, 2018 at 4:29 AM Manikumar 
> > > wrote:
> > >
> > > > Hello Kafka users, developers and client-developers,
> > > >
> > > > This is the first candidate for release of Apache Kafka 2.0.1.
> > > >
> > > > This is a bug fix release closing 49 tickets:
> > > > https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+2.0.1
> > > >
> > > > Release notes for the 2.0.1 release:
> > > > http://home.apache.org/~manikumar/kafka-2.0.1-rc0/RELEASE_NOTES.html
> > > >
> > > > *** Please download, test and vote by  Tuesday, October 30, end of
> day
> > > >
> > > > Kafka's KEYS file containing PGP keys we use to sign the release:
> > > > http://kafka.apache.org/KEYS
> > > >
> > > > * Release artifacts to be voted upon (source and binary):
> > > > http://home.apache.org/~manikumar/kafka-2.0.1-rc0/
> > > >
> > > > * Maven artifacts to be voted upon:
> > > > https://repository.apache.org/content/groups/staging/
> > > >
> > > > * Javadoc:
> > > > http://home.apache.org/~manikumar/kafka-2.0.1-rc0/javadoc/
> > > >
> > > > * Tag to be voted upon (off 2.0 branch) is the 2.0.1 tag:
> > > > https://github.com/apache/kafka/releases/tag/2.0.1-rc0
> > > >
> > > > * Documentation:
> > > > http://kafka.apache.org/20/documentation.html
> > > >
> > > > * Protocol:
> > > > http://kafka.apache.org/20/protocol.html
> > > >
> > > > * Successful Jenkins builds for the 2.0 branch:
> > > > Unit/integration tests:
> > > https://builds.apache.org/job/kafka-2.0-jdk8/177/
> > > >
> > > > /**
> > > >
> > > > Thanks,
> > > > Manikumar
> > > >
> > >
> >
>


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,

Mayuresh

On Fri, Nov 2, 2018 at 12:07 PM  wrote:

> 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?
>
> Thanks,
> Harsha
> On Nov 2, 2018, 11:50 AM -0700, Jun Rao , wrote:
> > 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
>


-- 
-Regards,
Mayuresh R. Gharat
(862) 250-7125


[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: Improvement
  Components: KafkaConnect
Affects Versions: 1.1.1
Reporter: Rod Cordova


It appears that the HTTP response code for the status endpoint is always 
200/OK. It would be extremely helpful if the response code was NOT 200/OK if 
there are FAILED connector states or task states. Having to parse the response 
body JSON in order to determine actual health adds unnecessary burden on the 
upstream client doing the health check.

Based on the existing documentation you can see that even though there is a 
FAILED worker task the HTTP response code is 200/OK 
([https://docs.confluent.io/current/connect/references/restapi.html#get--connectors-(string-name)-status])

We run these connectors in Docker containers and have resorted to Docker health 
checks (calling a simple script to parse the JSON status response body) but 
would much prefer to leverage the HTTP response code as that is the lowest 
common denominator supported by most proxies/load balancers etc.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


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.



-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 like
>
> ```
>
> select * from
>
> changelog_topic
>
> limit 10
>
> ```
>
> I'm not sure because the serde for window store changelog is compound 
> Windowed.
>
>
> Hope this makes sense. Thank you!
>
>
> 
> From: Matthias J. Sax 
> Sent: Friday, November 2, 2018 5:06 AM
> To: dev@kafka.apache.org
> Subject: Re: KSQL with changelog topic
>
> I am not sure if I understand the question. Can you elaborate?
>
> -Matthias
>
>
> On 11/1/18 10:38 AM, Boyang Chen wrote:
>> Hey there,
>>
>>
>> a dumb question that I haven't found the answer anywhere: does KSQL support 
>> query towards Stream applications'
>>
>> changelog topic? Especially window store changelog, since it has a more 
>> complex key schema.
>>
>>
>> Thanks!
>>
>
>



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?

Thanks,
Harsha
On Nov 2, 2018, 11:50 AM -0700, Jun Rao , wrote:
> 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
  Issue Type: Improvement
  Components: producer 
Affects Versions: 2.0.0
Reporter: Sherwin Chiu


When using 
{quote}{{public KafkaProducer(Properties properties, Serializer 
keySerializer, Serializer valueSerializer)}}
{quote}
I was surprised to run into the following error.
{quote}{{Class org.apache.kafka.common.serialization.StringSerializer could not 
be found.}}
{quote}
Is there a reason why KafkaProducer needs to reload the class I already passed 
in?  I was expecting this constructor to function similarly to 
{quote}{{public KafkaProducer(Map configs, Serializer 
keySerializer, Serializer valueSerializer)}}
{quote}
Why not convert/cast the Properties to a Map and add the 
serializers just like the Map config constructor?  I see that it's trying to 
keep the Properties construct.  Is there a reason for this?



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


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-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: https://issues.apache.org/jira/browse/KAFKA-7584
 Project: Kafka
  Issue Type: Improvement
  Components: streams
Affects Versions: 2.0.0
Reporter: Matthias J. Sax
Assignee: Matthias J. Sax


Setting 
{quote}{{props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE);}}
{{props.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, "3");}}
{quote}
results in
{quote}{{java.lang.ClassCastException: java.lang.String cannot be cast to 
java.lang.Integer}}{{at 
org.apache.kafka.streams.StreamsConfig.checkIfUnexpectedUserSpecifiedConsumerConfig(StreamsConfig.java:875)}}
{{ at 
org.apache.kafka.streams.StreamsConfig.getProducerConfigs(StreamsConfig.java:1071)}}
{quote}
 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


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 the number of topics increases(sadly, we are using HDD).

Hi Yuanjin,

How many partitions are you trying to create?

Do you have benchmarks confirming that disk I/O is your bottleneck?  There are 
a few cases where large numbers of partitions may impose CPU and garbage 
collection burdens.  The patch on https://github.com/apache/kafka/pull/5206 
illustrates one of them.

> We are considering separating the logic layer and the storage layer of Kafka
> broker like Apache Pulsar.
> 
> After the modification, a server may have several Kafka brokers and more
> topics. Those brokers all connect to a sole storage engine via RP The
> sole storage can do the load balancing work easily, and avoid creating too
> many files which hurts HDD.
> 
> Is it hard? I think replacing the stuff in `Kafka.Log` would be enough,
> right?

It would help to know what the problem is here.  If the problem is a large 
number of files, then maybe the simplest approach would be creating fewer 
files.  You don't need to introduce a new layer of servers in order to do that. 
 You could use something like RocksDB to store messages and indices, or create 
your own file format which combined together things which were previously 
separate.  For example, we could combine the timeindex and index files.

As I understand it, Pulsar made the decision to combine together data from 
multiple partitions in a single file.  Sometimes a very large number of 
partitions.  This is great for writing, but not so good if you want to read 
historical data from a single topic.

regards,
Colin


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 like
> 
> ```
> 
> select * from
> 
> changelog_topic
> 
> limit 10
> 
> ```
> 
> I'm not sure because the serde for window store changelog is compound 
> Windowed.
> 
> 
> Hope this makes sense. Thank you!
> 
> 
> 
> From: Matthias J. Sax 
> Sent: Friday, November 2, 2018 5:06 AM
> To: dev@kafka.apache.org
> Subject: Re: KSQL with changelog topic
> 
> I am not sure if I understand the question. Can you elaborate?
> 
> -Matthias
> 
> 
> On 11/1/18 10:38 AM, Boyang Chen wrote:
>> Hey there,
>>
>>
>> a dumb question that I haven't found the answer anywhere: does KSQL support 
>> query towards Stream applications'
>>
>> changelog topic? Especially window store changelog, since it has a more 
>> complex key schema.
>>
>>
>> Thanks!
>>
> 
> 



signature.asc
Description: OpenPGP digital signature


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
> >
> 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 have been using this in production for
> > some time and I have composed a PR that enables this. It is a fairly
> > extensive PR, but I believe it will add considerable value to the Kafka
> > Streams DSL.
> >
> > The PR can be found here:
> > https://github.com/apache/kafka/pull/5527
> >
> > See
> http://mail-archives.apache.org/mod_mbox/kafka-dev/201810.mbox/browser
> > for previous discussion thread.
> >
> > I would also like to give a shout-out to Jan Filipiak who helped me out
> > greatly in this project, and who led the initial work into this problem.
> > Without Jan's help and insight I do not think this would have been
> possible
> > to get to this point.
> >
> > Adam
> >
>


[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 considering separating the logic layer and the storage layer of Kafka
broker like Apache Pulsar.

After the modification, a server may have several Kafka brokers and more
topics. Those brokers all connect to a sole storage engine via RPC. The
sole storage can do the load balancing work easily, and avoid creating too
many files which hurts HDD.

Is it hard? I think replacing the stuff in `Kafka.Log` would be enough,
right?

Regards,
Lin.


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 have been using this in production for
> some time and I have composed a PR that enables this. It is a fairly
> extensive PR, but I believe it will add considerable value to the Kafka
> Streams DSL.
>
> The PR can be found here:
> https://github.com/apache/kafka/pull/5527
>
> See http://mail-archives.apache.org/mod_mbox/kafka-dev/201810.mbox/browser
> for previous discussion thread.
>
> I would also like to give a shout-out to Jan Filipiak who helped me out
> greatly in this project, and who led the initial work into this problem.
> Without Jan's help and insight I do not think this would have been possible
> to get to this point.
>
> Adam
>


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:
> 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 have been using this in production for
> some time and I have composed a PR that enables this. It is a fairly
> extensive PR, but I believe it will add considerable value to the Kafka
> Streams DSL.
>
> The PR can be found here:
> https://github.com/apache/kafka/pull/5527
>
> See http://mail-archives.apache.org/mod_mbox/kafka-dev/201810.mbox/browser
> for previous discussion thread.
>
> I would also like to give a shout-out to Jan Filipiak who helped me out
> greatly in this project, and who led the initial work into this problem.
> Without Jan's help and insight I do not think this would have been possible
> to get to this point.
>
> Adam
>