Re: Broker side partition round robin

2020-06-02 Thread Vinicius Scheidegger
Hi Liam,

(+adding imgur links to the images)
First of all, thanks for checking my doubt.
I understand that the reason I notice this behavior is because our case
differs from yours in one point: I'm not writing terabytes in a single day.
I'm writing MB, but distribution matters in these MBs, this
because processing in the consumer side is CPU intensive (complex machine
learning algo), so a real equally distributed load is a requirement,
otherwise messages start queuing (which is what I don't want).

Our system is also in production and this bad distribution is generating
queuing in some consumers while others are idle.
I tried both approaches:
Round Robbin - actually gives me a distributed load in a big time window,
but as I grow the number of producers I get several messages in a single
partition while others don't get anything.
Hash of the key - here we used a random UUID as the key - still a bad
distribution


[image: image.png]
(imgur link: https://i.imgur.com/ZhQq9Uh.png )


When I start, let's say 10 producers, we can get 5 messages going to one
partition while others have none - i understand that this is because the
round robin is internal to the producer.
[image: image.png]
  (imgur link: https://i.imgur.com/Hv8TUDL.png 
)
(The picture above is what I believe is happening)


it would surprise me that this hasn't come up before, that's why I'm pretty
sure I'm missing something here...
We're currently analyzing some solutions, one of them is building our own
partitioner with shared memory (yes, that's how far we got on this),
although I believe a better solution would be to have this on Kafka broker
side and not depend on custom code.

[image: image.png]
  (imgur link: https://i.imgur.com/JR8QvZH.png 
)

Above is the draft of one of our current ideas of a possible design. Based
on the shared memory we could decide the partition and send the messages
directly there (the number of producers, partitions and consumers here are
simplified).
This if we don't find a solution within Kafka to mitigate this distribution
issue - even though the thought design imposes limits and add layers we
didn't had in our initial design.

My question is, do we really need to develop this??
Is equal distribution in a scenario with multiple producers something that
can be achieved in Kafka without custom development?
Having never checked out the broker code when receiving a message - is
partition definition even possible on the broker side?
If this really does not exist, would a feature like that benefit other
people, is it worth checking out instead of the above solution?
Should I move this message to the dev forum? - no one gave me much
attention there too (but maybe my messages are too big/boring - hahaha)

Thanks


On Tue, Jun 2, 2020 at 1:08 PM Vinicius Scheidegger <
vinicius.scheideg...@gmail.com> wrote:

> Hi Liam,
>
> First of all, thanks for checking my doubt.
> I understand that the reason I notice this behavior is because our case
> differs from yours in one point: I'm not writing terabytes in a single day.
> I'm writing MB, but distribution matters in these MBs, this
> because processing in the consumer side is CPU intensive (complex machine
> learning algo), so a real equally distributed load is a requirement,
> otherwise messages start queuing (which is what I don't want).
>
> Our system is also in production and this bad distribution is generating
> queuing in some consumers while others are idle.
> I tried both approaches:
> Round Robbin - actually gives me a distributed load in a big time window,
> but as I grow the number of producers I get several messages in a single
> partition while others don't get anything.
> Hash of the key - here we used a random UUID as the key - still a bad
> distribution
>
>
> [image: image.png]
>
>
>
> When I start, let's say 10 producers, we can get 5 messages going to one
> partition while others have none - i understand that this is because the
> round robin is internal to the producer.
> [image: image.png]
>
> (The picture above is what I believe is happening)
>
>
> it would surprise me that this hasn't come up before, that's why I'm
> pretty sure I'm missing something here...
> We're currently analyzing some solutions, one of them is building our own
> partitioner with shared memory (yes, that's how far we got on this),
> although I believe a better solution would be to have this on Kafka broker
> side and not depend on custom code.
>
> [image: image.png]
> Above is the draft of one of our current ideas of a possible design. Based
> on the shared memory we could decide the partition and send the messages
> directly there (the number of producers, partitions and consumers here are
> simplified).
> This if we don't find a solution within Kafka to mitigate this
> distribution issue - even though the thought design imposes limits and add
> layers we didn't had in our initial design.
>
> My 

Re: How to manually start ingesting in kafka source connector ?

2020-06-02 Thread Yu Watanabe
Thank you for the link.

In my environment I  use strimzi (0.17.0).
I realized that connector offset is created when kafka connect start
up . So before creating connector , I am able to set offset for Kafka
Connect.

In addition to you link ,scenario in below link also helped my idea.
https://www.confluent.jp/blog/kafka-connect-deep-dive-jdbc-source-connector/#starting-table-capture

Thanks,
Yu

On Thu, May 28, 2020 at 9:40 PM Robin Moffatt  wrote:
>
> You could look at
> https://rmoff.net/2019/08/15/reset-kafka-connect-source-connector-offsets/ and
> experiment with creating the connector elsewhere to see if you can pre-empt
> the key value that Kafka Connect will use when writing the offsets, and so
> do your list 2 - 1 - 3 instead
>
>
> --
>
> Robin Moffatt | Senior Developer Advocate | ro...@confluent.io | @rmoff
>
>
> On Thu, 28 May 2020 at 10:12, Yu Watanabe  wrote:
>
> > Robin
> >
> > Thank you for the reply.
> >
> > Any way to not automatically start after creating connector ?
> >
> > I am trying to find a way to change connector offset  as described in
> > below link before starting connector ..
> >
> >
> > https://www.confluent.jp/blog/kafka-connect-deep-dive-jdbc-source-connector/#starting-table-capture
> >
> > Steps I want to do will be
> >
> > 1. Create jdbc connector
> > 2. Change connector offset
> > 3. Start connector
> >
> > Thanks,
> > Yu
> >
> > On Thu, May 28, 2020 at 6:01 PM Robin Moffatt  wrote:
> > >
> > > When you create the connector, it will start.
> > >
> > >
> > > --
> > >
> > > Robin Moffatt | Senior Developer Advocate | ro...@confluent.io | @rmoff
> > >
> > >
> > > On Thu, 28 May 2020 at 04:12, Yu Watanabe  wrote:
> > >
> > > > Dear community .
> > > >
> > > > I would like to ask question related to source connector in kafka
> > > > connect (2.4.0) .
> > > >
> > > > Is there a way to manually start source connector after registering to
> > > > kafka connect ?
> > > >
> > > > Looking at the document , I found PAUSE API ,
> > > >
> > > >
> > > >
> > https://docs.confluent.io/current/connect/references/restapi.html#put--connectors-(string-name)-pause
> > > >
> > > > however, could not find set initial state for individual tasks in
> > > > connector properties ..
> > > >
> > > > https://docs.confluent.io/current/connect/managing/configuring.html
> > > >
> > > > I appreciate if I could get some help.
> > > >
> > > > Best Regards,
> > > > Yu Watanabe
> > > >
> > > > --
> > > > Yu Watanabe
> > > >
> > > > linkedin: www.linkedin.com/in/yuwatanabe1/
> > > > twitter:   twitter.com/yuwtennis
> > > >
> >
> >
> >
> > --
> > Yu Watanabe
> >
> > linkedin: www.linkedin.com/in/yuwatanabe1/
> > twitter:   twitter.com/yuwtennis
> >



-- 
Yu Watanabe

linkedin: www.linkedin.com/in/yuwatanabe1/
twitter:   twitter.com/yuwtennis


Re: Disk space - sharp increase in usage

2020-06-02 Thread Andrew Otto
WMF recently had an issue
 where Kafka broker
disks were filling up with log segment data.  It turned out that Kafka was
not deleting old log segments because the oldest log segment had a message
with a Kafka timestamp a year in the future.  Since the oldest log segment
had a message newer than any others, Kafka could not respect the
retention.ms setting and delete any old log segments.  We mitigated this by
setting retention.bytes, which overrode retention.ms and allowed Kafka to
prune old logs.  For us, this could be prevented from happening again by
setting message.timestamp.difference.max.ms.

Not sure if this is your problem, but it is at least something to check!
 :)

On Tue, Jun 2, 2020 at 6:26 AM Liam Clarke-Hutchinson <
liam.cla...@adscale.co.nz> wrote:

> Hi Victoria,
>
> There are no metrics of when a config was changed. However, if you've been
> capturing the JMX metrics from the brokers, the metric
> kafka.cluster:name=ReplicasCount,partition=*,topic=*,type=Partition
> would show if replication factor was increased.
>
> As for retention time, if you're sure that there's not been an increase in
> data ingestion, best metric to look into for that is
> kafka.log:name=LogSegments... as an increase in that would either be caused
> by a large influx of data or an increase in retention time.
>
> Lastly, check the logs and metrics for the log cleaner, in case there's any
> issues occurring preventing logs from being cleaned.
> kafka.log:name=max-clean-time-secs,type=LogCleaner
> and kafka.log:name=time-since-last-run-ms,type=LogCleanerManager would be
> most useful here.
>
> The ZK logs won't be much use (ZK being where the config is stored) unless
> you had audit logging enabled, which is disabled by default.
>
> Good luck,
>
> Liam Clarke-Hutchinson
>
>
> On Tue, 2 Jun. 2020, 8:50 pm Victoria Zuberman, <
> victoria.zuber...@imperva.com> wrote:
>
> > Regards kafka-logs directory, it was an interesting lead, we checked and
> > it is the same.
> >
> > Regards replication factor and retention, I am not looking for current
> > information, I am look for metrics that can give me information about a
> > change.
> >
> > Still looking for more ideas
> >
> > On 02/06/2020, 11:31, "Peter Bukowinski"  wrote:
> >
> > CAUTION: This message was sent from outside the company. Do not click
> > links or open attachments unless you recognize the sender and know the
> > content is safe.
> >
> >
> > > On Jun 2, 2020, at 12:56 AM, Victoria Zuberman <
> > victoria.zuber...@imperva.com> wrote:
> > >
> > > Hi,
> > >
> > > Background:
> > > Kafka cluster
> > > 7 brokers, with 4T disk each
> > > version 2.3 (recently upgraded from 0.1.0 via 1.0.1)
> > >
> > > Problem:
> > > Used disk space went from 40% to 80%.
> > > Looking for root cause.
> > >
> > > Suspects:
> > >
> > >  1.  Incoming traffic
> > >
> > > Ruled out, according to metrics no significant change in “bytes in”
> > for topics in cluster
> > >
> > >  1.  Upgrade
> > >
> > > The raise started on the day of upgrade to 2.3
> > >
> > > But we upgraded another cluster in the same way and we don’t see
> > similar issue there
> > >
> > > Is there a known change or issue at 2.3 related to disk space
> usage?
> > >
> > >  1.  Replication factor
> > >
> > > Is there a way to see whether replication factor of any topic was
> > changed recently? Didn’t find in metrics...
> >
> > You can use the kafka-topics.sh script to check the replica count for
> > all your topics. Upgrading would not have affected the replica count,
> > though.
> >
> > >  1.  Retention
> > >
> > > Is there a way to see whether retention was changed recently?
> Didn’t
> > find in metrics...
> >
> > You can use  kafka-topics.sh —-zookeeper host:2181 --describe
> > --topics-with-overrides
> > to list any topics with non-default retention, but I’m guessing
> that’s
> > not it.
> >
> > If your disk usage went from 40 to 80% on all brokers — effectively
> > doubled — it could be that your kafka data log directory path(s) changed
> > during the upgrade. As you upgraded each broker and (re)started the
> kafka,
> > it would have left the existing data under the old one path and created
> new
> > topic partition directories and logs under the new path as it rejoined
> the
> > cluster. Have you verified that your data log directory locations are the
> > same as they used to be?
> >
> > > Would appreciate any other ideas or investigation leads
> > >
> > > Thanks,
> > > Victoria
> > >
> > > ---
> > > NOTICE:
> > > This email and all attachments are confidential, may be
> proprietary,
> > and may be privileged or otherwise protected from disclosure. They are
> > intended solely for the individual or entity to whom the email is
> > addressed. 

Re: Broker side partition round robin

2020-06-02 Thread Liam Clarke-Hutchinson
Hi, I can't see your images, they're not making it through the mailing
list, sorry.

On Tue, 2 Jun. 2020, 11:09 pm Vinicius Scheidegger, <
vinicius.scheideg...@gmail.com> wrote:

> Hi Liam,
>
> First of all, thanks for checking my doubt.
> I understand that the reason I notice this behavior is because our case
> differs from yours in one point: I'm not writing terabytes in a single day.
> I'm writing MB, but distribution matters in these MBs, this
> because processing in the consumer side is CPU intensive (complex machine
> learning algo), so a real equally distributed load is a requirement,
> otherwise messages start queuing (which is what I don't want).
>
> Our system is also in production and this bad distribution is generating
> queuing in some consumers while others are idle.
> I tried both approaches:
> Round Robbin - actually gives me a distributed load in a big time window,
> but as I grow the number of producers I get several messages in a single
> partition while others don't get anything.
> Hash of the key - here we used a random UUID as the key - still a bad
> distribution
>
>
> [image: image.png]
>
>
>
> When I start, let's say 10 producers, we can get 5 messages going to one
> partition while others have none - i understand that this is because the
> round robin is internal to the producer.
> [image: image.png]
>
> (The picture above is what I believe is happening)
>
>
> it would surprise me that this hasn't come up before, that's why I'm
> pretty sure I'm missing something here...
> We're currently analyzing some solutions, one of them is building our own
> partitioner with shared memory (yes, that's how far we got on this),
> although I believe a better solution would be to have this on Kafka broker
> side and not depend on custom code.
>
> [image: image.png]
> Above is the draft of one of our current ideas of a possible design. Based
> on the shared memory we could decide the partition and send the messages
> directly there (the number of producers, partitions and consumers here are
> simplified).
> This if we don't find a solution within Kafka to mitigate this
> distribution issue - even though the thought design imposes limits and add
> layers we didn't had in our initial design.
>
> My question is, do we really need to develop this??
> Is equal distribution in a scenario with multiple producers something that
> can be achieved in Kafka without custom development?
> Having never checked out the broker code when receiving a message - is
> partition definition even possible on the broker side?
> If this really does not exist, would a feature like that benefit other
> people, is it worth checking out instead of the above solution?
> Should I move this message to the dev forum? - no one gave me much
> attention there too (but maybe my messages are too big/boring - hahaha)
>
> Thanks
>
>
> On Tue, Jun 2, 2020 at 10:47 AM Liam Clarke-Hutchinson <
> liam.cla...@adscale.co.nz> wrote:
>
>> Hi Vinicius,
>>
>> As you note, the cluster doesn't load balance producers, it relies on them
>> using a partition strategy to do so.
>>
>> In production, I've never had actual broker load skew develop from
>> multiple
>> independent producers using round robining - and we're talking say 20 - 50
>> producers (depending on scaling) writing terabytes over a day.
>>
>> And load skew / hot brokers is something I monitor closely.
>>
>> The only time I've seen load skew is when a key based partition strategy
>> was used, and keys weren't evenly distributed.
>>
>> So in other words, in theory there's no guarantee, but in my experience,
>> round robining multiple producers works fine.
>>
>> Cheers,
>>
>> Liam Clarke
>>
>> On Mon, 1 Jun. 2020, 11:55 pm Vinicius Scheidegger, <
>> vinicius.scheideg...@gmail.com> wrote:
>>
>> > Hey guys, I need some help here...
>> >
>> > Is this a flaw in the design (maybe a discussion point for a KIP?), is
>> > Kafka not supposed to perform equal load balancing with multiple
>> producers
>> > or am I missing something (which is what I believe is happening)?
>> >
>> > On Wed, May 27, 2020 at 2:40 PM Vinicius Scheidegger <
>> > vinicius.scheideg...@gmail.com> wrote:
>> >
>> >> Does anyone know whether we could really have an "out of the box"
>> >> solution to do round robin over the partitions when we have multiple
>> >> producers?
>> >> By that I mean, a round robin on the broker side (or maybe some way to
>> >> synchronize all producers).
>> >>
>> >> Thank you,
>> >>
>> >> On Tue, May 26, 2020 at 1:41 PM Vinicius Scheidegger <
>> >> vinicius.scheideg...@gmail.com> wrote:
>> >>
>> >>> Yes, I checked it. The issue is that RoundRobbinPartitioner is bound
>> to
>> >>> the producer. In a scenario with multiple producers it doesn't
>> guarantee
>> >>> equal distribution - from what I understood and from my tests, the
>> >>> following situation happens with it:
>> >>>
>> >>> [image: image.png]
>> >>>
>> >>> Of course, the first partition is not always 1 and each producer may
>> >>> start in a 

Re: Broker side partition round robin

2020-06-02 Thread Vinicius Scheidegger
Hi Liam,

First of all, thanks for checking my doubt.
I understand that the reason I notice this behavior is because our case
differs from yours in one point: I'm not writing terabytes in a single day.
I'm writing MB, but distribution matters in these MBs, this
because processing in the consumer side is CPU intensive (complex machine
learning algo), so a real equally distributed load is a requirement,
otherwise messages start queuing (which is what I don't want).

Our system is also in production and this bad distribution is generating
queuing in some consumers while others are idle.
I tried both approaches:
Round Robbin - actually gives me a distributed load in a big time window,
but as I grow the number of producers I get several messages in a single
partition while others don't get anything.
Hash of the key - here we used a random UUID as the key - still a bad
distribution


[image: image.png]



When I start, let's say 10 producers, we can get 5 messages going to one
partition while others have none - i understand that this is because the
round robin is internal to the producer.
[image: image.png]

(The picture above is what I believe is happening)


it would surprise me that this hasn't come up before, that's why I'm pretty
sure I'm missing something here...
We're currently analyzing some solutions, one of them is building our own
partitioner with shared memory (yes, that's how far we got on this),
although I believe a better solution would be to have this on Kafka broker
side and not depend on custom code.

[image: image.png]
Above is the draft of one of our current ideas of a possible design. Based
on the shared memory we could decide the partition and send the messages
directly there (the number of producers, partitions and consumers here are
simplified).
This if we don't find a solution within Kafka to mitigate this distribution
issue - even though the thought design imposes limits and add layers we
didn't had in our initial design.

My question is, do we really need to develop this??
Is equal distribution in a scenario with multiple producers something that
can be achieved in Kafka without custom development?
Having never checked out the broker code when receiving a message - is
partition definition even possible on the broker side?
If this really does not exist, would a feature like that benefit other
people, is it worth checking out instead of the above solution?
Should I move this message to the dev forum? - no one gave me much
attention there too (but maybe my messages are too big/boring - hahaha)

Thanks


On Tue, Jun 2, 2020 at 10:47 AM Liam Clarke-Hutchinson <
liam.cla...@adscale.co.nz> wrote:

> Hi Vinicius,
>
> As you note, the cluster doesn't load balance producers, it relies on them
> using a partition strategy to do so.
>
> In production, I've never had actual broker load skew develop from multiple
> independent producers using round robining - and we're talking say 20 - 50
> producers (depending on scaling) writing terabytes over a day.
>
> And load skew / hot brokers is something I monitor closely.
>
> The only time I've seen load skew is when a key based partition strategy
> was used, and keys weren't evenly distributed.
>
> So in other words, in theory there's no guarantee, but in my experience,
> round robining multiple producers works fine.
>
> Cheers,
>
> Liam Clarke
>
> On Mon, 1 Jun. 2020, 11:55 pm Vinicius Scheidegger, <
> vinicius.scheideg...@gmail.com> wrote:
>
> > Hey guys, I need some help here...
> >
> > Is this a flaw in the design (maybe a discussion point for a KIP?), is
> > Kafka not supposed to perform equal load balancing with multiple
> producers
> > or am I missing something (which is what I believe is happening)?
> >
> > On Wed, May 27, 2020 at 2:40 PM Vinicius Scheidegger <
> > vinicius.scheideg...@gmail.com> wrote:
> >
> >> Does anyone know whether we could really have an "out of the box"
> >> solution to do round robin over the partitions when we have multiple
> >> producers?
> >> By that I mean, a round robin on the broker side (or maybe some way to
> >> synchronize all producers).
> >>
> >> Thank you,
> >>
> >> On Tue, May 26, 2020 at 1:41 PM Vinicius Scheidegger <
> >> vinicius.scheideg...@gmail.com> wrote:
> >>
> >>> Yes, I checked it. The issue is that RoundRobbinPartitioner is bound to
> >>> the producer. In a scenario with multiple producers it doesn't
> guarantee
> >>> equal distribution - from what I understood and from my tests, the
> >>> following situation happens with it:
> >>>
> >>> [image: image.png]
> >>>
> >>> Of course, the first partition is not always 1 and each producer may
> >>> start in a different point in time, anyway my point is that it does not
> >>> guarantee equal distribution.
> >>>
> >>> The other option pointed out is to select the partition myself - either
> >>> a shared memory on the producers (assuming that this is possible - I
> mean I
> >>> would need to guarantee that producers CAN share a synchronized
> memory) or
> 

Re: Disk space - sharp increase in usage

2020-06-02 Thread Liam Clarke-Hutchinson
Hi Victoria,

There are no metrics of when a config was changed. However, if you've been
capturing the JMX metrics from the brokers, the metric
kafka.cluster:name=ReplicasCount,partition=*,topic=*,type=Partition
would show if replication factor was increased.

As for retention time, if you're sure that there's not been an increase in
data ingestion, best metric to look into for that is
kafka.log:name=LogSegments... as an increase in that would either be caused
by a large influx of data or an increase in retention time.

Lastly, check the logs and metrics for the log cleaner, in case there's any
issues occurring preventing logs from being cleaned.
kafka.log:name=max-clean-time-secs,type=LogCleaner
and kafka.log:name=time-since-last-run-ms,type=LogCleanerManager would be
most useful here.

The ZK logs won't be much use (ZK being where the config is stored) unless
you had audit logging enabled, which is disabled by default.

Good luck,

Liam Clarke-Hutchinson


On Tue, 2 Jun. 2020, 8:50 pm Victoria Zuberman, <
victoria.zuber...@imperva.com> wrote:

> Regards kafka-logs directory, it was an interesting lead, we checked and
> it is the same.
>
> Regards replication factor and retention, I am not looking for current
> information, I am look for metrics that can give me information about a
> change.
>
> Still looking for more ideas
>
> On 02/06/2020, 11:31, "Peter Bukowinski"  wrote:
>
> CAUTION: This message was sent from outside the company. Do not click
> links or open attachments unless you recognize the sender and know the
> content is safe.
>
>
> > On Jun 2, 2020, at 12:56 AM, Victoria Zuberman <
> victoria.zuber...@imperva.com> wrote:
> >
> > Hi,
> >
> > Background:
> > Kafka cluster
> > 7 brokers, with 4T disk each
> > version 2.3 (recently upgraded from 0.1.0 via 1.0.1)
> >
> > Problem:
> > Used disk space went from 40% to 80%.
> > Looking for root cause.
> >
> > Suspects:
> >
> >  1.  Incoming traffic
> >
> > Ruled out, according to metrics no significant change in “bytes in”
> for topics in cluster
> >
> >  1.  Upgrade
> >
> > The raise started on the day of upgrade to 2.3
> >
> > But we upgraded another cluster in the same way and we don’t see
> similar issue there
> >
> > Is there a known change or issue at 2.3 related to disk space usage?
> >
> >  1.  Replication factor
> >
> > Is there a way to see whether replication factor of any topic was
> changed recently? Didn’t find in metrics...
>
> You can use the kafka-topics.sh script to check the replica count for
> all your topics. Upgrading would not have affected the replica count,
> though.
>
> >  1.  Retention
> >
> > Is there a way to see whether retention was changed recently? Didn’t
> find in metrics...
>
> You can use  kafka-topics.sh —-zookeeper host:2181 --describe
> --topics-with-overrides
> to list any topics with non-default retention, but I’m guessing that’s
> not it.
>
> If your disk usage went from 40 to 80% on all brokers — effectively
> doubled — it could be that your kafka data log directory path(s) changed
> during the upgrade. As you upgraded each broker and (re)started the kafka,
> it would have left the existing data under the old one path and created new
> topic partition directories and logs under the new path as it rejoined the
> cluster. Have you verified that your data log directory locations are the
> same as they used to be?
>
> > Would appreciate any other ideas or investigation leads
> >
> > Thanks,
> > Victoria
> >
> > ---
> > NOTICE:
> > This email and all attachments are confidential, may be proprietary,
> and may be privileged or otherwise protected from disclosure. They are
> intended solely for the individual or entity to whom the email is
> addressed. However, mistakes sometimes happen in addressing emails. If you
> believe that you are not an intended recipient, please stop reading
> immediately. Do not copy, forward, or rely on the contents in any way.
> Notify the sender and/or Imperva, Inc. by telephone at +1 (650) 832-6006
> and then delete or destroy any copy of this email and its attachments. The
> sender reserves and asserts all rights to confidentiality, as well as any
> privileges that may apply. Any disclosure, copying, distribution or action
> taken or omitted to be taken by an unintended recipient in reliance on this
> message is prohibited and may be unlawful.
> > Please consider the environment before printing this email.
>
>
>


Re: Disk space - sharp increase in usage

2020-06-02 Thread Victoria Zuberman
Regards kafka-logs directory, it was an interesting lead, we checked and it is 
the same.

Regards replication factor and retention, I am not looking for current 
information, I am look for metrics that can give me information about a change.

Still looking for more ideas

On 02/06/2020, 11:31, "Peter Bukowinski"  wrote:

CAUTION: This message was sent from outside the company. Do not click links 
or open attachments unless you recognize the sender and know the content is 
safe.


> On Jun 2, 2020, at 12:56 AM, Victoria Zuberman 
 wrote:
>
> Hi,
>
> Background:
> Kafka cluster
> 7 brokers, with 4T disk each
> version 2.3 (recently upgraded from 0.1.0 via 1.0.1)
>
> Problem:
> Used disk space went from 40% to 80%.
> Looking for root cause.
>
> Suspects:
>
>  1.  Incoming traffic
>
> Ruled out, according to metrics no significant change in “bytes in” for 
topics in cluster
>
>  1.  Upgrade
>
> The raise started on the day of upgrade to 2.3
>
> But we upgraded another cluster in the same way and we don’t see similar 
issue there
>
> Is there a known change or issue at 2.3 related to disk space usage?
>
>  1.  Replication factor
>
> Is there a way to see whether replication factor of any topic was changed 
recently? Didn’t find in metrics...

You can use the kafka-topics.sh script to check the replica count for all 
your topics. Upgrading would not have affected the replica count, though.

>  1.  Retention
>
> Is there a way to see whether retention was changed recently? Didn’t find 
in metrics...

You can use  kafka-topics.sh —-zookeeper host:2181 --describe 
--topics-with-overrides
to list any topics with non-default retention, but I’m guessing that’s not 
it.

If your disk usage went from 40 to 80% on all brokers — effectively doubled 
— it could be that your kafka data log directory path(s) changed during the 
upgrade. As you upgraded each broker and (re)started the kafka, it would have 
left the existing data under the old one path and created new topic partition 
directories and logs under the new path as it rejoined the cluster. Have you 
verified that your data log directory locations are the same as they used to be?

> Would appreciate any other ideas or investigation leads
>
> Thanks,
> Victoria
>
> ---
> NOTICE:
> This email and all attachments are confidential, may be proprietary, and 
may be privileged or otherwise protected from disclosure. They are intended 
solely for the individual or entity to whom the email is addressed. However, 
mistakes sometimes happen in addressing emails. If you believe that you are not 
an intended recipient, please stop reading immediately. Do not copy, forward, 
or rely on the contents in any way. Notify the sender and/or Imperva, Inc. by 
telephone at +1 (650) 832-6006 and then delete or destroy any copy of this 
email and its attachments. The sender reserves and asserts all rights to 
confidentiality, as well as any privileges that may apply. Any disclosure, 
copying, distribution or action taken or omitted to be taken by an unintended 
recipient in reliance on this message is prohibited and may be unlawful.
> Please consider the environment before printing this email.




Re: Disk space - sharp increase in usage

2020-06-02 Thread Peter Bukowinski


> On Jun 2, 2020, at 12:56 AM, Victoria Zuberman 
>  wrote:
> 
> Hi,
> 
> Background:
> Kafka cluster
> 7 brokers, with 4T disk each
> version 2.3 (recently upgraded from 0.1.0 via 1.0.1)
> 
> Problem:
> Used disk space went from 40% to 80%.
> Looking for root cause.
> 
> Suspects:
> 
>  1.  Incoming traffic
> 
> Ruled out, according to metrics no significant change in “bytes in” for 
> topics in cluster
> 
>  1.  Upgrade
> 
> The raise started on the day of upgrade to 2.3
> 
> But we upgraded another cluster in the same way and we don’t see similar 
> issue there
> 
> Is there a known change or issue at 2.3 related to disk space usage?
> 
>  1.  Replication factor
> 
> Is there a way to see whether replication factor of any topic was changed 
> recently? Didn’t find in metrics...

You can use the kafka-topics.sh script to check the replica count for all your 
topics. Upgrading would not have affected the replica count, though.

>  1.  Retention
> 
> Is there a way to see whether retention was changed recently? Didn’t find in 
> metrics...

You can use  kafka-topics.sh —-zookeeper host:2181 --describe 
--topics-with-overrides
to list any topics with non-default retention, but I’m guessing that’s not it.

If your disk usage went from 40 to 80% on all brokers — effectively doubled — 
it could be that your kafka data log directory path(s) changed during the 
upgrade. As you upgraded each broker and (re)started the kafka, it would have 
left the existing data under the old one path and created new topic partition 
directories and logs under the new path as it rejoined the cluster. Have you 
verified that your data log directory locations are the same as they used to be?

> Would appreciate any other ideas or investigation leads
> 
> Thanks,
> Victoria
> 
> ---
> NOTICE:
> This email and all attachments are confidential, may be proprietary, and may 
> be privileged or otherwise protected from disclosure. They are intended 
> solely for the individual or entity to whom the email is addressed. However, 
> mistakes sometimes happen in addressing emails. If you believe that you are 
> not an intended recipient, please stop reading immediately. Do not copy, 
> forward, or rely on the contents in any way. Notify the sender and/or 
> Imperva, Inc. by telephone at +1 (650) 832-6006 and then delete or destroy 
> any copy of this email and its attachments. The sender reserves and asserts 
> all rights to confidentiality, as well as any privileges that may apply. Any 
> disclosure, copying, distribution or action taken or omitted to be taken by 
> an unintended recipient in reliance on this message is prohibited and may be 
> unlawful.
> Please consider the environment before printing this email.


Disk space - sharp increase in usage

2020-06-02 Thread Victoria Zuberman
Hi,

Background:
Kafka cluster
7 brokers, with 4T disk each
version 2.3 (recently upgraded from 0.1.0 via 1.0.1)

Problem:
Used disk space went from 40% to 80%.
Looking for root cause.

Suspects:

  1.  Incoming traffic

Ruled out, according to metrics no significant change in “bytes in” for topics 
in cluster

  1.  Upgrade

The raise started on the day of upgrade to 2.3

But we upgraded another cluster in the same way and we don’t see similar issue 
there

Is there a known change or issue at 2.3 related to disk space usage?

  1.  Replication factor

Is there a way to see whether replication factor of any topic was changed 
recently? Didn’t find in metrics...

  1.  Retention

Is there a way to see whether retention was changed recently? Didn’t find in 
metrics...

Would appreciate any other ideas or investigation leads

Thanks,
Victoria

---
NOTICE:
This email and all attachments are confidential, may be proprietary, and may be 
privileged or otherwise protected from disclosure. They are intended solely for 
the individual or entity to whom the email is addressed. However, mistakes 
sometimes happen in addressing emails. If you believe that you are not an 
intended recipient, please stop reading immediately. Do not copy, forward, or 
rely on the contents in any way. Notify the sender and/or Imperva, Inc. by 
telephone at +1 (650) 832-6006 and then delete or destroy any copy of this 
email and its attachments. The sender reserves and asserts all rights to 
confidentiality, as well as any privileges that may apply. Any disclosure, 
copying, distribution or action taken or omitted to be taken by an unintended 
recipient in reliance on this message is prohibited and may be unlawful.
Please consider the environment before printing this email.