Re: Kafka topic partition distributing evenly on disks

2020-08-07 Thread William Reynolds
Hmm, that's odd, I am sure it was in the docs previously. Here is the
KIP on it 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-113%3A+Support+replicas+movement+between+log+directories
Basically the reassignment json that you get looks like this from the
initial generation and if you already have a realignment file you can
just add the log dirs section to each partition entry

{
  "version" : int,
  "partitions" : [
{
  "topic" : str,
  "partition" : int,
  "replicas" : [int],
  "log_dirs" : [str]<-- NEW. A log directory can be either
"any", or a valid absolute path that begins with '/'. This is an
optional filed. It is treated as an array of "any" if this field is
not explicitly specified in the json file.
},
...
  ]
}

Hope that helps
William

On 07/08/2020, Péter Nagykátai  wrote:
> Thank you William,
>
> I checked the doc and don't see any instructions regarding disks. Should I
> simply "move around" the topics and Kafka will assign the topics evenly on
> the two disks (per broker)? The current setup looks like this (for the
> topic in question, 15 primary, replica partitions):
>
> Broker 1 - disk 1: 8 partition
> Broker 1 - disk 2: 2 partition
>
> Broker 2 - disk 1: 8 partition
> Broker 2 - disk 2: 2 partition
>
> Broker 3 - disk 1: 8 partition
> Broker 3 - disk 2: 2 partition
>
> Thanks!
>
> On Fri, Aug 7, 2020 at 1:01 PM William Reynolds <
> william.reyno...@instaclustr.com> wrote:
>
>> Hi Péter,
>> Sounds like time to reassign the partitions you have across all the
>> brokers/data dirs using the instructions from here
>> https://kafka.apache.org/documentation/#basic_ops_automigrate. That
>> assumes that your partition strategy has somewhat evenly filled your
>> partitions and given it may move all the partitions it could be a bit
>> intensive so be sure to use the throttle option.
>> Cheers
>> William
>>
>> On 07/08/2020, Péter Nagykátai  wrote:
>> > Hello everybody,
>> >
>> > Thank you for the detailed answers. My issue is partly answered here:
>> >
>> >
>> >
>> >
>> > *This rule also applies to disk-level, which means that when a set
>> > ofpartitions assigned to a specific broker, each of the disks will get
>> > thesame number of partitions without considering the load of disks at
>> > thattime.*
>> >
>> >  I admit, I didn't provide enough info either.
>> >
>> > So my problem is that an existing topic got a huge surge of events for
>> this
>> > week. I knew that'll happen and I modified the partition count.
>> > Unfortunately, it occurred to me a bit later, that I'll likely need
>> > some
>> > extra disk space. So I added an extra disk to each broker. The thing I
>> > didn't know, that Kafka won't evenly distribute the partitions on the
>> > disks.
>> > So the question still remains:
>> >  Is there any way to have Kafka evenly distribute data on its disks?
>> > Also, what options do I have *after *I'm in the situation I described
>> > above? (preferably without deleting the topic)
>> >
>> > Thanks!
>> >
>> > On Fri, Aug 7, 2020 at 12:00 PM Yingshuan Song
>> > 
>> > wrote:
>> >
>> >> Hi Peter,
>> >> Agreed with Manoj and Vinicius, i think those rules led to this result
>> >> :
>> >>
>> >> 1)the partitions of a topic - N and replication number - R determine
>> >> the
>> >> real partition-replica count of this topic, which is N * R;
>> >> 2)   kafka can distribute partitions evenly among brokers, but it is
>> >> based
>> >> on the broker count when the topic was created, this is important.
>> >> If we create a topic (N - 4, R - 3) in a kafka cluster which contains
>> >> 3
>> >> kafka brokers, then 4 * 3 / 3 = 4 partitions will be assigned to each
>> >> broker.
>> >> But if a new broker was added into this cluster and another topic (N -
>> 4,
>> >> R
>> >> - 3) need to be created, then 4 * 3 / 4 = 3 partitions will be
>> >> assigned
>> >> to
>> >> each broker.
>> >> Kafka will not assign all those partitions to the new added broker
>> >> even
>> >> though it is idle and i think this is a shortcoming of kafka.
>> >> This rule also applies to disk-level, which means that when a set of
>> >> par

Re: Kafka topic partition distributing evenly on disks

2020-08-07 Thread William Reynolds
 with one data disk (mounted solely to hold Kafka
>> > data)
>> > > and recently added a new one.
>> > >
>> > > On Thu, Aug 6, 2020 at 10:13 PM  wrote:
>> > >
>> > > > What do you mean older disk ?
>> > > >
>> > > > On 8/6/20, 12:05 PM, "Péter Nagykátai" 
>> wrote:
>> > > >
>> > > > [External]
>> > > >
>> > > >
>> > > > Yeah, but it doesn't do that. My "older" disks have ~70
>> partitions,
>> > > the
>> > > > newer ones ~5 partitions. That's why I'm asking what went
>> > > > wrong.
>> > > >
>> > > > On Thu, Aug 6, 2020 at 8:35 PM 
>> > wrote:
>> > > >
>> > > > > Kafka  evenly distributed number of partition on each disk so
>> in
>> > > > your case
>> > > > > every disk should have 3/2 topic partitions .
>> > > > > It is producer job to evenly produce data by partition key
>> > > > to
>> > > topic
>> > > > > partition .
>> > > > > How it partition key , it is auto generated or producer
>> > > > sending
>> > key
>> > > > along
>> > > > > with message .
>> > > > >
>> > > > >
>> > > > > On 8/6/20, 7:29 AM, "Péter Nagykátai" 
>> > > wrote:
>> > > > >
>> > > > > [External]
>> > > > >
>> > > > >
>> > > > > Hello,
>> > > > >
>> > > > > I have a Kafka cluster with 3 brokers (v2.3.0) and each
>> > broker
>> > > > has 2
>> > > > > disks
>> > > > > attached. I added a new topic (heavyweight) and was
>> surprised
>> > > > that
>> > > > > even if
>> > > > > the topic has 15 partitions, those weren't distributed
>> evenly
>> > > on
>> > > > the
>> > > > > disks.
>> > > > > Thus I got one disk that's almost empty and the other
>> almost
>> > > > filled
>> > > > > up. Is
>> > > > > there any way to have Kafka evenly distribute data on its
>> > > disks?
>> > > > >
>> > > > > Thank you!
>> > > > >
>> > > > >
>> > > > > This e-mail and any files transmitted with it are for the
>> > > > sole
>> > use
>> > > > of the
>> > > > > intended recipient(s) and may contain confidential and
>> privileged
>> > > > > information. If you are not the intended recipient(s), please
>> > reply
>> > > > to the
>> > > > > sender and destroy all copies of the original message. Any
>> > > > unauthorized
>> > > > > review, use, disclosure, dissemination, forwarding, printing
>> > > > or
>> > > > copying of
>> > > > > this email, and/or any action taken in reliance on the
>> > > > contents
>> > of
>> > > > this
>> > > > > e-mail is strictly prohibited and may be unlawful. Where
>> > permitted
>> > > by
>> > > > > applicable law, this e-mail and other e-mail communications
>> sent
>> > to
>> > > > and
>> > > > > from Cognizant e-mail addresses may be monitored.
>> > > > > This e-mail and any files transmitted with it are for the
>> > > > sole
>> > use
>> > > > of the
>> > > > > intended recipient(s) and may contain confidential and
>> privileged
>> > > > > information. If you are not the intended recipient(s), please
>> > reply
>> > > > to the
>> > > > > sender and destroy all copies of the original message. Any
>> > > > unauthorized
>> > > > > review, use, disclosure, dissemination, forwarding, printing
>> > > > or
>> > > > copying of
>> > > > > this email, and/or any action taken in reliance on the
>> > > > conte

Re: kafka tuning(vm.max_map_count) and logs retention.

2020-07-23 Thread William Reynolds
Hi Vitali,
When I ran into it it was latest time being very large. Until we could get
the messages set right we set segment.ms to maxint so it only rolled based
on size.
Cheers
William

On Thu, 23 Jul 2020 at 4:46 pm, Vitalii Stoianov <
vitalii.stoianov...@gmail.com> wrote:

> Hi  William,
>
>
> ./kafka-console-consumer.sh --bootstrap-server localhost:9092 --property
> print.timestamp=true --topic test
> One of the messages TS output:
> CreateTime:1595485571406707 1595485026.850 1595485571.406 216301538579718
> {msg data}
>
> So which one of these is used to roll over a log segment?
> I was trying to find some explanation on the web but with no luck.
>
> Regards,
> Vitalii.
>
> On Thu, Jul 23, 2020 at 9:25 AM William Reynolds <
> william.reyno...@instaclustr.com> wrote:
>
> > Hi Vitali,
> > What are the timestamps in your message? I have seen this before where
> you
> > have timestamps well into the future so every few messages causes a log
> > roll and you end up with a very large amount of log files.
> >
> > *William*
> >
> > On Thu, 23 Jul 2020 at 16:22, Vitalii Stoianov <
> > vitalii.stoianov...@gmail.com> wrote:
> >
> > > Hi All,
> > >
> > > I also have noticed that the number of log/index files are too high and
> > log
> > > roll is happening more frequently than expected.
> > > The log.roll.hours is default (168) and log.segment.bytes is 1g and log
> > > files size in the topic partition folders are usually smaller than 1g.
> > >
> > > Regards,
> > > Vitalii.
> > >
> > > On Wed, Jul 22, 2020 at 8:15 PM Vitalii Stoianov <
> > > vitalii.stoianov...@gmail.com> wrote:
> > >
> > > > Hi All,
> > > >
> > > > According to this:
> > > https://docs.confluent.io/current/kafka/deployment.html
> > > > vm.max_map_count is depend on number of index file:
> > > > *find /tmp/kafka_logs -name '*index' | wc -l*
> > > >
> > > > In our test lab we have next setup:
> > > >
> > > > *Topic:test  PartitionCount:256  ReplicationFactor:2
> > > > Configs:segment.bytes=1073741824,retention.ms
> > > > <http://retention.ms
> > >
> >
> >=8640,message.format.version=2.3-IV1,max.message.bytes=4194304,unclean.leader.election.enable=true*
> > > >
> > > > No cleanup.policy set explicitly for topic or in server.properties
> so I
> > > > assume default: delete according to
> > > > https://kafka.apache.org/23/documentation.html#brokerconfigs
> > > >
> > > > I did a small script that counted the number of index files and for
> > this
> > > > topic it is:
> > > > ~638000.
> > > > Also if I check kafka log/data dir it contain some old log/index
> files
> > > > create date for which is older than 10 days.(retention for topic is
> one
> > > day)
> > > > Note: When i checked  log-cleaner.log it contains info only about
> > cleanup
> > > > for compacted logs.
> > > >
> > > > In order to set:  vm.max_map_count value correctly, I need to
> > > > understand the following:
> > > > Why do such old index/log files exist and not cleaned?
> > > > How properly set vm.max_map_count if index/logs is not freed on time
> ??
> > > >
> > > > Regards,
> > > > Vitalii.
> > > >
> > >
> >
>
-- 


*William Reynolds**Technical Operations Engineer*


<https://www.facebook.com/instaclustr>   <https://twitter.com/instaclustr>
<https://www.linkedin.com/company/instaclustr>

Read our latest technical blog posts here
<https://www.instaclustr.com/blog/>.

This email has been sent on behalf of Instaclustr Pty. Limited (Australia)
and Instaclustr Inc (USA).

This email and any attachments may contain confidential and legally
privileged information.  If you are not the intended recipient, do not copy
or disclose its content, but please reply to this email immediately and
highlight the error to the sender and then immediately delete the message.

Instaclustr values your privacy. Our privacy policy can be found at
https://www.instaclustr.com/company/policies/privacy-policy


Re: kafka tuning(vm.max_map_count) and logs retention.

2020-07-22 Thread William Reynolds
Hi Vitali,
What are the timestamps in your message? I have seen this before where you
have timestamps well into the future so every few messages causes a log
roll and you end up with a very large amount of log files.

*William*

On Thu, 23 Jul 2020 at 16:22, Vitalii Stoianov <
vitalii.stoianov...@gmail.com> wrote:

> Hi All,
>
> I also have noticed that the number of log/index files are too high and log
> roll is happening more frequently than expected.
> The log.roll.hours is default (168) and log.segment.bytes is 1g and log
> files size in the topic partition folders are usually smaller than 1g.
>
> Regards,
> Vitalii.
>
> On Wed, Jul 22, 2020 at 8:15 PM Vitalii Stoianov <
> vitalii.stoianov...@gmail.com> wrote:
>
> > Hi All,
> >
> > According to this:
> https://docs.confluent.io/current/kafka/deployment.html
> > vm.max_map_count is depend on number of index file:
> > *find /tmp/kafka_logs -name '*index' | wc -l*
> >
> > In our test lab we have next setup:
> >
> > *Topic:test  PartitionCount:256  ReplicationFactor:2
> > Configs:segment.bytes=1073741824,retention.ms
> >  >=8640,message.format.version=2.3-IV1,max.message.bytes=4194304,unclean.leader.election.enable=true*
> >
> > No cleanup.policy set explicitly for topic or in server.properties so I
> > assume default: delete according to
> > https://kafka.apache.org/23/documentation.html#brokerconfigs
> >
> > I did a small script that counted the number of index files and for this
> > topic it is:
> > ~638000.
> > Also if I check kafka log/data dir it contain some old log/index files
> > create date for which is older than 10 days.(retention for topic is one
> day)
> > Note: When i checked  log-cleaner.log it contains info only about cleanup
> > for compacted logs.
> >
> > In order to set:  vm.max_map_count value correctly, I need to
> > understand the following:
> > Why do such old index/log files exist and not cleaned?
> > How properly set vm.max_map_count if index/logs is not freed on time ??
> >
> > Regards,
> > Vitalii.
> >
>