HADOOP-12829. StatisticsDataReferenceCleaner swallows interrupt exceptions (Gregory Chanan via cmccabe)
This commit is contained in:
parent
9ed17f181d
commit
d9c409a428
|
@ -719,6 +719,9 @@ Release 2.9.0 - UNRELEASED
|
||||||
HADOOP-12714. Fix hadoop-mapreduce-client-nativetask unit test which fails
|
HADOOP-12714. Fix hadoop-mapreduce-client-nativetask unit test which fails
|
||||||
because it is not able to open the "glibc bug spill" file. (cmccabe)
|
because it is not able to open the "glibc bug spill" file. (cmccabe)
|
||||||
|
|
||||||
|
HADOOP-12829. StatisticsDataReferenceCleaner swallows interrupt exceptions
|
||||||
|
(Gregory Chanan via cmccabe)
|
||||||
|
|
||||||
Release 2.8.0 - UNRELEASED
|
Release 2.8.0 - UNRELEASED
|
||||||
|
|
||||||
INCOMPATIBLE CHANGES
|
INCOMPATIBLE CHANGES
|
||||||
|
|
|
@ -3184,15 +3184,16 @@ public abstract class FileSystem extends Configured implements Closeable {
|
||||||
private static class StatisticsDataReferenceCleaner implements Runnable {
|
private static class StatisticsDataReferenceCleaner implements Runnable {
|
||||||
@Override
|
@Override
|
||||||
public void run() {
|
public void run() {
|
||||||
while (true) {
|
while (!Thread.interrupted()) {
|
||||||
try {
|
try {
|
||||||
StatisticsDataReference ref =
|
StatisticsDataReference ref =
|
||||||
(StatisticsDataReference)STATS_DATA_REF_QUEUE.remove();
|
(StatisticsDataReference)STATS_DATA_REF_QUEUE.remove();
|
||||||
ref.cleanUp();
|
ref.cleanUp();
|
||||||
|
} catch (InterruptedException ie) {
|
||||||
|
LOG.warn("Cleaner thread interrupted, will stop", ie);
|
||||||
|
Thread.currentThread().interrupt();
|
||||||
} catch (Throwable th) {
|
} catch (Throwable th) {
|
||||||
// the cleaner thread should continue to run even if there are
|
LOG.warn("Exception in the cleaner thread but it will continue to "
|
||||||
// exceptions, including InterruptedException
|
|
||||||
LOG.warn("exception in the cleaner thread but it will continue to "
|
|
||||||
+ "run", th);
|
+ "run", th);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue