On Tue, Jan 30, 2018 at 1:38 PM, David Espinosa <espi...@gmail.com> wrote:
> Hi Andrey,
> My topics are replicated with a replicated factor equals to the number of
> nodes, 3 in this test.
> Didn't know about the kip-227.
> The problems I see at 70k topics coming from ZK are related to any
> operation where ZK has to retrieve topics metadata. Just listing topics at
> 50K or 60k you will experience a big delay in the response. I have no more
> details about these problems, but is easy to reproduce the latency in the
> topics list request.

AFAIK kafka doesn't do a full list as part of normal operations from
ZK. If you have requirements in your consumer/producer code on doing
--describe, then that would be a problem. I think that can be worked
around. Based on my profiling data so far, while things are working in
non-failure mode, none of the ZK functions pop up as "hot methods".

> Thanks me for pointing me to this parameter,  vm.max_map_count, it wasn't
> on my radar. Could you tell me what value you use?

I set it to the max allowable on Amzn Linux: vm.max_map_count=1215752192

> The other way around about topic naming, I think the longer the topic names
> are the sooner jute.maxbuffer overflows.

I see; what value(s) have you tried with and how much gain did you you see?

> David
>
>
> 2018-01-30 4:40 GMT+01:00 Andrey Falko <afa...@salesforce.com>:
>
>> On Sun, Jan 28, 2018 at 8:45 AM, David Espinosa <espi...@gmail.com> wrote:
>> > Hi Monty,
>> >
>> > I'm also planning to use a big amount of topics in Kafka, so recently I
>> > made a test within a 3 nodes kafka cluster where I created 100k topics
>> with
>> > one partition. Sent 1M messages in total.
>>
>> Are your topic partitions replicated?
>>
>> > These are my conclusions:
>> >
>> >    - There is not any limitation on kafka regarding the number of topics
>> >    but on Zookeeper and in the system where Kafka nodes is allocated.
>>
>> There are also the problems being addressed in KIP-227:
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> 227%3A+Introduce+Incremental+FetchRequests+to+Increase+
>> Partition+Scalability
>>
>> >    - Zookeeper will start having problems from 70k topics, which can be
>> >    solved modifying a buffer parameter on the JVM (-Djute.maxbuffer).
>> >    Performance is reduced.
>>
>> What kind of problems do you see at 70k topics? If performance is
>> reduced w/ modifying jute.maxbuffer, won't that effect the performance
>> of kafka interms of how long it takes to recover from broker failure,
>> creating/deleting topics, producing and consuming?
>>
>> >    - Open file descriptors of the system are equivalent to [number of
>> >    topics]X[number of partitions per topic]. Set to 128k in my test to
>> avoid
>> >    problems.
>> >    - System needs a big amount of memory for page caching.
>>
>> I also had to tune vm.max_map_count much higher.
>>
>> >
>> > So, after creating 100k with the required setup (system+JVM) but seeing
>> > problems at 70k, I feel safe by not creating more than 50k, and always
>> will
>> > have Zookeeper as my first suspect if a problem comes. I think with
>> proper
>> > resources (memory) and system setup (open file descriptors), you don't
>> have
>> > any real limitation regarding partitions.
>>
>> I can confirm the 50k number. After about 40k-45k topics, I start
>> seeing slow down in consume offset commit latencies that eclipse 50ms.
>> Hopefully KIP-227 will alleviate that problem and leave ZK as the last
>> remaining hurdle. I'm testing with 3x replication per partition and 10
>> brokers.
>>
>> > By the way, I used long topic names (about 30 characters), which can be
>> > important for ZK.
>>
>> I'd like to learn more about this, are you saying that long topic
>> names would improve ZK performance because that relates to bumping up
>> jute.maxbuffer?
>>
>> > Hope this information is of your help.
>> >
>> > David
>> >
>> > 2018-01-28 2:22 GMT+01:00 Monty Hindman <montyhind...@gmail.com>:
>> >
>> >> I'm designing a system and need some more clarity regarding Kafka's
>> >> recommended limits on the number of topics and/or partitions. At a high
>> >> level, our system would work like this:
>> >>
>> >> - A user creates a job X (X is a UUID).
>> >> - The user uploads data for X to an input topic: X.in.
>> >> - Workers process the data, writing results to an output topic: X.out.
>> >> - The user downloads the data from X.out.
>> >>
>> >> It's important for the system that data for different jobs be kept
>> >> separate, and that input and output data be kept separate. By
>> "separate" I
>> >> mean that there needs to be a reasonable way for users and the system's
>> >> workers to query for the data they need (by job-id and by
>> input-vs-output)
>> >> and not get the data they don't need.
>> >>
>> >> Based on expected usage and our data retention policy, we would not
>> expect
>> >> to need more than 12,000 active jobs at any one time -- in other words,
>> >> 24,000 topics. If we were to have 5 partitions per topic (our cluster
>> has 5
>> >> brokers), that would imply 120,000 partitions. [These number refer only
>> to
>> >> main/primary partitions, not any replicas that might exist.]
>> >>
>> >> Those numbers seem to be far larger than the suggested limits I see
>> online.
>> >> For example, the Kafka FAQ on these matters seems to imply that the most
>> >> relevant limit is the number of partitions (rather than topics) and
>> sort of
>> >> implies that 10,000 partitions might be a suggested guideline (
>> >> https://goo.gl/fQs2md). Also implied is that systems should use fewer
>> >> topics and instead partition the data within topics if further
>> separation
>> >> is needed (the FAQ entry uses the example of partitioning by user ID,
>> which
>> >> is roughly analogous to job ID in my use case).
>> >>
>> >> The guidance in the FAQ is unclear to me:
>> >>
>> >> - Does the suggested limit of 10,000 refer to the total number of
>> >> partitions (ie, main partitions plus any replicas) or just the main
>> >> partitions?
>> >>
>> >> - If the most important limitation is number of partitions (rather than
>> >> number of topics), how does the suggested strategy of using fewer topics
>> >> and then partitioning by some other attribute (ie job ID) help at all?
>> >>
>> >> - Is my use case just a bad fit for Kafka? Or, is there a way for us to
>> use
>> >> Kafka while still supporting the kinds of query patterns that we need
>> (ie,
>> >> by job ID and by input-vs-output)?
>> >>
>> >> Thanks in advance for any guidance.
>> >>
>> >> Monty
>> >>
>>

Reply via email to