Thanks for creating this KIP Stanislav.  My observations:

1) I agree with Colin that threads automatically re-launching threads generally isn't a great idea.  Metrics and/or monitoring threads are generally much safer.  And there's always the issue of what happens if the re-launcher dies?

2) I'm 100% with James in agreement with setting up the LogCleaner to skip over problematic partitions instead of dying.

3) There's a lot of "feature bloat" suggestions.  From how I see things, a message could get corrupted in one of several states:

3a) Message is corrupted by the leader partition saving to disk. Replicas have the same error. 3b) Message is corrupted by one of the replica partitions saving to disk.  Leader and other replica(s) unlikely to have the same error
3c) Disk corruption happens later (e.g. during partition move)

If we have the simplest solution, then all of the above will not cause the LogCleaner to crash and 3b/3c have a chance of manual recovery.

4) In most of the issues I'm seeing via work, most of the corruption seems persistent on the same log segment (i.e. a 3b/3c type of corruption).  The only improvement I can think of is that if such an error occurs, then have the option (configuration setting?) to create a <log_segment>.skip file (or something similar).  If the .skip file is there, don't re-scan the segment.  If you want a re-try or manage to fix the issue manually (e.g. copying from a replica), then the .skip file can be deleted after the segment is fixed and the LogCleaner will try again on the next iteration.

5) I'm in alignment with Colin's comment about hard drive failures. By the time you can reliably detect HDD hardware failures, it's less about improving the LogCleaner as much as that data needs to be moved to a new drive.

-Ray

On 7/25/18 11:55 AM, Dhruvil Shah wrote:
For the cleaner thread specifically, I do not think respawning will help at
all because we are more than likely to run into the same issue again which
would end up crashing the cleaner. Retrying makes sense for transient
errors or when you believe some part of the system could have healed
itself, both of which I think are not true for the log cleaner.

On Wed, Jul 25, 2018 at 11:08 AM Ron Dagostino <rndg...@gmail.com> wrote:

<<<respawning threads is likely to make things worse, by putting you in an
infinite loop which consumes resources and fires off continuous log
messages.
Hi Colin.  In case it could be relevant, one way to mitigate this effect is
to implement a backoff mechanism (if a second respawn is to occur then wait
for 1 minute before doing it; then if a third respawn is to occur wait for
2 minutes before doing it; then 4 minutes, 8 minutes, etc. up to some max
wait time).

I have no opinion on whether respawn is appropriate or not in this context,
but a mitigation like the increasing backoff described above may be
relevant in weighing the pros and cons.

Ron

On Wed, Jul 25, 2018 at 1:26 PM Colin McCabe <cmcc...@apache.org> wrote:

On Mon, Jul 23, 2018, at 23:20, James Cheng wrote:
Hi Stanislav! Thanks for this KIP!

I agree that it would be good if the LogCleaner were more tolerant of
errors. Currently, as you said, once it dies, it stays dead.

Things are better now than they used to be. We have the metric
       kafka.log:type=LogCleanerManager,name=time-since-last-run-ms
which we can use to tell us if the threads are dead. And as of 1.1.0,
we
have KIP-226, which allows you to restart the log cleaner thread,
without requiring a broker restart.

https://cwiki.apache.org/confluence/display/KAFKA/KIP-226+-+Dynamic+Broker+Configuration
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-226+-+Dynamic+Broker+Configuration

I've only read about this, I haven't personally tried it.
Thanks for pointing this out, James!  Stanislav, we should probably add a
sentence or two mentioning the KIP-226 changes somewhere in the KIP.
Maybe
in the intro section?

I think it's clear that requiring the users to manually restart the log
cleaner is not a very good solution.  But it's good to know that it's a
possibility on some older releases.

Some comments:
* I like the idea of having the log cleaner continue to clean as many
partitions as it can, skipping over the problematic ones if possible.

* If the log cleaner thread dies, I think it should automatically be
revived. Your KIP attempts to do that by catching exceptions during
execution, but I think we should go all the way and make sure that a
new
one gets created, if the thread ever dies.
This is inconsistent with the way the rest of Kafka works.  We don't
automatically re-create other threads in the broker if they terminate.
In
general, if there is a serious bug in the code, respawning threads is
likely to make things worse, by putting you in an infinite loop which
consumes resources and fires off continuous log messages.

* It might be worth trying to re-clean the uncleanable partitions. I've
seen cases where an uncleanable partition later became cleanable. I
unfortunately don't remember how that happened, but I remember being
surprised when I discovered it. It might have been something like a
follower was uncleanable but after a leader election happened, the log
truncated and it was then cleanable again. I'm not sure.
James, I disagree.  We had this behavior in the Hadoop Distributed File
System (HDFS) and it was a constant source of user problems.

What would happen is disks would just go bad over time.  The DataNode
would notice this and take them offline.  But then, due to some
"optimistic" code, the DataNode would periodically try to re-add them to
the system.  Then one of two things would happen: the disk would just
fail
immediately again, or it would appear to work and then fail after a short
amount of time.

The way the disk failed was normally having an I/O request take a really
long time and time out.  So a bunch of request handler threads would
basically slam into a brick wall when they tried to access the bad disk,
slowing the DataNode to a crawl.  It was even worse in the second
scenario,
if the disk appeared to work for a while, but then failed.  Any data that
had been written on that DataNode to that disk would be lost, and we
would
need to re-replicate it.

Disks aren't biological systems-- they don't heal over time.  Once
they're
bad, they stay bad.  The log cleaner needs to be robust against cases
where
the disk really is failing, and really is returning bad data or timing
out.
* For your metrics, can you spell out the full metric in JMX-style
format, such as:

  kafka.log:type=LogCleanerManager,name=uncleanable-partitions-count
               value=4

* For "uncleanable-partitions": topic-partition names can be very long.
I think the current max size is 210 characters (or maybe 240-ish?).
Having the "uncleanable-partitions" being a list could be very large
metric. Also, having the metric come out as a csv might be difficult to
work with for monitoring systems. If we *did* want the topic names to
be
accessible, what do you think of having the
       kafka.log:type=LogCleanerManager,topic=topic1,partition=2
I'm not sure if LogCleanerManager is the right type, but my example was
that the topic and partition can be tags in the metric. That will allow
monitoring systems to more easily slice and dice the metric. I'm not
sure what the attribute for that metric would be. Maybe something like
"uncleaned bytes" for that topic-partition? Or time-since-last-clean?
Or
maybe even just "Value=1".
I haven't though about this that hard, but do we really need the
uncleanable topic names to be accessible through a metric?  It seems like
the admin should notice that uncleanable partitions are present, and then
check the logs?

* About `max.uncleanable.partitions`, you said that this likely
indicates that the disk is having problems. I'm not sure that is the
case. For the 4 JIRAs that you mentioned about log cleaner problems,
all
of them are partition-level scenarios that happened during normal
operation. None of them were indicative of disk problems.
I don't think this is a meaningful comparison.  In general, we don't
accept JIRAs for hard disk problems that happen on a particular cluster.
If someone opened a JIRA that said "my hard disk is having problems" we
could close that as "not a Kafka bug."  This doesn't prove that disk
problems don't happen, but  just that JIRA isn't the right place for
them.
I do agree that the log cleaner has had a significant number of logic
bugs, and that we need to be careful to limit their impact.  That's one
reason why I think that a threshold of "number of uncleanable logs" is a
good idea, rather than just failing after one IOException.  In all the
cases I've seen where a user hit a logic bug in the log cleaner, it was
just one partition that had the issue.  We also should increase test
coverage for the log cleaner.

* About marking disks as offline when exceeding a certain threshold,
that actually increases the blast radius of log compaction failures.
Currently, the uncleaned partitions are still readable and writable.
Taking the disks offline would impact availability of the uncleanable
partitions, as well as impact all other partitions that are on the
disk.
In general, when we encounter I/O errors, we take the disk partition
offline.  This is spelled out in KIP-112 (

https://cwiki.apache.org/confluence/display/KAFKA/KIP-112%3A+Handle+disk+failure+for+JBOD
) :

- Broker assumes a log directory to be good after it starts, and mark
log directory as
bad once there is IOException when broker attempts to access (i.e. read
or write) the log directory.
- Broker will be offline if all log directories are bad.
- Broker will stop serving replicas in any bad log directory. New
replicas will only be created
on good log directory.
The behavior Stanislav is proposing for the log cleaner is actually more
optimistic than what we do for regular broker I/O, since we will tolerate
multiple IOExceptions, not just one.  But it's generally consistent.
Ignoring errors is not.  In any case, if you want to tolerate an
unlimited
number of I/O errors, you can just set the threshold to an infinite value
(although I think that would be a bad idea).

best,
Colin

-James


On Jul 23, 2018, at 5:46 PM, Stanislav Kozlovski <
stanis...@confluent.io> wrote:
I renamed the KIP and that changed the link. Sorry about that. Here
is
the
new link:

https://cwiki.apache.org/confluence/display/KAFKA/KIP-346+-+Improve+LogCleaner+behavior+on+error
On Mon, Jul 23, 2018 at 5:11 PM Stanislav Kozlovski <
stanis...@confluent.io>
wrote:

Hey group,

I created a new KIP about making log compaction more fault-tolerant.
Please give it a look here and please share what you think,
especially in
regards to the points in the "Needs Discussion" paragraph.

KIP: KIP-346
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-346+-+Limit+blast+radius+of+log+compaction+failure
--
Best,
Stanislav


--
Best,
Stanislav

Reply via email to