|
@@ -20,7 +20,8 @@ package org.apache.hadoop.fs;
|
|
import java.io.Closeable;
|
|
import java.io.Closeable;
|
|
import java.io.FileNotFoundException;
|
|
import java.io.FileNotFoundException;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
-import java.lang.ref.WeakReference;
|
|
|
|
|
|
+import java.lang.ref.PhantomReference;
|
|
|
|
+import java.lang.ref.ReferenceQueue;
|
|
import java.net.URI;
|
|
import java.net.URI;
|
|
import java.net.URISyntaxException;
|
|
import java.net.URISyntaxException;
|
|
import java.security.PrivilegedExceptionAction;
|
|
import java.security.PrivilegedExceptionAction;
|
|
@@ -32,7 +33,6 @@ import java.util.HashMap;
|
|
import java.util.HashSet;
|
|
import java.util.HashSet;
|
|
import java.util.IdentityHashMap;
|
|
import java.util.IdentityHashMap;
|
|
import java.util.Iterator;
|
|
import java.util.Iterator;
|
|
-import java.util.LinkedList;
|
|
|
|
import java.util.List;
|
|
import java.util.List;
|
|
import java.util.Map;
|
|
import java.util.Map;
|
|
import java.util.NoSuchElementException;
|
|
import java.util.NoSuchElementException;
|
|
@@ -2920,16 +2920,6 @@ public abstract class FileSystem extends Configured implements Closeable {
|
|
volatile int readOps;
|
|
volatile int readOps;
|
|
volatile int largeReadOps;
|
|
volatile int largeReadOps;
|
|
volatile int writeOps;
|
|
volatile int writeOps;
|
|
- /**
|
|
|
|
- * Stores a weak reference to the thread owning this StatisticsData.
|
|
|
|
- * This allows us to remove StatisticsData objects that pertain to
|
|
|
|
- * threads that no longer exist.
|
|
|
|
- */
|
|
|
|
- final WeakReference<Thread> owner;
|
|
|
|
-
|
|
|
|
- StatisticsData(WeakReference<Thread> owner) {
|
|
|
|
- this.owner = owner;
|
|
|
|
- }
|
|
|
|
|
|
|
|
/**
|
|
/**
|
|
* Add another StatisticsData object to this one.
|
|
* Add another StatisticsData object to this one.
|
|
@@ -3000,17 +2990,37 @@ public abstract class FileSystem extends Configured implements Closeable {
|
|
* Thread-local data.
|
|
* Thread-local data.
|
|
*/
|
|
*/
|
|
private final ThreadLocal<StatisticsData> threadData;
|
|
private final ThreadLocal<StatisticsData> threadData;
|
|
-
|
|
|
|
|
|
+
|
|
/**
|
|
/**
|
|
- * List of all thread-local data areas. Protected by the Statistics lock.
|
|
|
|
|
|
+ * Set of all thread-local data areas. Protected by the Statistics lock.
|
|
|
|
+ * The references to the statistics data are kept using phantom references
|
|
|
|
+ * to the associated threads. Proper clean-up is performed by the cleaner
|
|
|
|
+ * thread when the threads are garbage collected.
|
|
*/
|
|
*/
|
|
- private LinkedList<StatisticsData> allData;
|
|
|
|
|
|
+ private final Set<StatisticsDataReference> allData;
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Global reference queue and a cleaner thread that manage statistics data
|
|
|
|
+ * references from all filesystem instances.
|
|
|
|
+ */
|
|
|
|
+ private static final ReferenceQueue<Thread> STATS_DATA_REF_QUEUE;
|
|
|
|
+ private static final Thread STATS_DATA_CLEANER;
|
|
|
|
+
|
|
|
|
+ static {
|
|
|
|
+ STATS_DATA_REF_QUEUE = new ReferenceQueue<Thread>();
|
|
|
|
+ // start a single daemon cleaner thread
|
|
|
|
+ STATS_DATA_CLEANER = new Thread(new StatisticsDataReferenceCleaner());
|
|
|
|
+ STATS_DATA_CLEANER.
|
|
|
|
+ setName(StatisticsDataReferenceCleaner.class.getName());
|
|
|
|
+ STATS_DATA_CLEANER.setDaemon(true);
|
|
|
|
+ STATS_DATA_CLEANER.start();
|
|
|
|
+ }
|
|
|
|
|
|
public Statistics(String scheme) {
|
|
public Statistics(String scheme) {
|
|
this.scheme = scheme;
|
|
this.scheme = scheme;
|
|
- this.rootData = new StatisticsData(null);
|
|
|
|
|
|
+ this.rootData = new StatisticsData();
|
|
this.threadData = new ThreadLocal<StatisticsData>();
|
|
this.threadData = new ThreadLocal<StatisticsData>();
|
|
- this.allData = null;
|
|
|
|
|
|
+ this.allData = new HashSet<StatisticsDataReference>();
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -3020,7 +3030,7 @@ public abstract class FileSystem extends Configured implements Closeable {
|
|
*/
|
|
*/
|
|
public Statistics(Statistics other) {
|
|
public Statistics(Statistics other) {
|
|
this.scheme = other.scheme;
|
|
this.scheme = other.scheme;
|
|
- this.rootData = new StatisticsData(null);
|
|
|
|
|
|
+ this.rootData = new StatisticsData();
|
|
other.visitAll(new StatisticsAggregator<Void>() {
|
|
other.visitAll(new StatisticsAggregator<Void>() {
|
|
@Override
|
|
@Override
|
|
public void accept(StatisticsData data) {
|
|
public void accept(StatisticsData data) {
|
|
@@ -3032,6 +3042,63 @@ public abstract class FileSystem extends Configured implements Closeable {
|
|
}
|
|
}
|
|
});
|
|
});
|
|
this.threadData = new ThreadLocal<StatisticsData>();
|
|
this.threadData = new ThreadLocal<StatisticsData>();
|
|
|
|
+ this.allData = new HashSet<StatisticsDataReference>();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * A phantom reference to a thread that also includes the data associated
|
|
|
|
+ * with that thread. On the thread being garbage collected, it is enqueued
|
|
|
|
+ * to the reference queue for clean-up.
|
|
|
|
+ */
|
|
|
|
+ private class StatisticsDataReference extends PhantomReference<Thread> {
|
|
|
|
+ private final StatisticsData data;
|
|
|
|
+
|
|
|
|
+ public StatisticsDataReference(StatisticsData data, Thread thread) {
|
|
|
|
+ super(thread, STATS_DATA_REF_QUEUE);
|
|
|
|
+ this.data = data;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public StatisticsData getData() {
|
|
|
|
+ return data;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Performs clean-up action when the associated thread is garbage
|
|
|
|
+ * collected.
|
|
|
|
+ */
|
|
|
|
+ public void cleanUp() {
|
|
|
|
+ // use the statistics lock for safety
|
|
|
|
+ synchronized (Statistics.this) {
|
|
|
|
+ /*
|
|
|
|
+ * If the thread that created this thread-local data no longer exists,
|
|
|
|
+ * remove the StatisticsData from our list and fold the values into
|
|
|
|
+ * rootData.
|
|
|
|
+ */
|
|
|
|
+ rootData.add(data);
|
|
|
|
+ allData.remove(this);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Background action to act on references being removed.
|
|
|
|
+ */
|
|
|
|
+ private static class StatisticsDataReferenceCleaner implements Runnable {
|
|
|
|
+ @Override
|
|
|
|
+ public void run() {
|
|
|
|
+ while (true) {
|
|
|
|
+ try {
|
|
|
|
+ StatisticsDataReference ref =
|
|
|
|
+ (StatisticsDataReference)STATS_DATA_REF_QUEUE.remove();
|
|
|
|
+ ref.cleanUp();
|
|
|
|
+ } catch (Throwable th) {
|
|
|
|
+ // the cleaner thread should continue to run even if there are
|
|
|
|
+ // exceptions, including InterruptedException
|
|
|
|
+ LOG.warn("exception in the cleaner thread but it will continue to "
|
|
|
|
+ + "run", th);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -3040,14 +3107,12 @@ public abstract class FileSystem extends Configured implements Closeable {
|
|
public StatisticsData getThreadStatistics() {
|
|
public StatisticsData getThreadStatistics() {
|
|
StatisticsData data = threadData.get();
|
|
StatisticsData data = threadData.get();
|
|
if (data == null) {
|
|
if (data == null) {
|
|
- data = new StatisticsData(
|
|
|
|
- new WeakReference<Thread>(Thread.currentThread()));
|
|
|
|
|
|
+ data = new StatisticsData();
|
|
threadData.set(data);
|
|
threadData.set(data);
|
|
|
|
+ StatisticsDataReference ref =
|
|
|
|
+ new StatisticsDataReference(data, Thread.currentThread());
|
|
synchronized(this) {
|
|
synchronized(this) {
|
|
- if (allData == null) {
|
|
|
|
- allData = new LinkedList<StatisticsData>();
|
|
|
|
- }
|
|
|
|
- allData.add(data);
|
|
|
|
|
|
+ allData.add(ref);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return data;
|
|
return data;
|
|
@@ -3105,21 +3170,9 @@ public abstract class FileSystem extends Configured implements Closeable {
|
|
*/
|
|
*/
|
|
private synchronized <T> T visitAll(StatisticsAggregator<T> visitor) {
|
|
private synchronized <T> T visitAll(StatisticsAggregator<T> visitor) {
|
|
visitor.accept(rootData);
|
|
visitor.accept(rootData);
|
|
- if (allData != null) {
|
|
|
|
- for (Iterator<StatisticsData> iter = allData.iterator();
|
|
|
|
- iter.hasNext(); ) {
|
|
|
|
- StatisticsData data = iter.next();
|
|
|
|
- visitor.accept(data);
|
|
|
|
- if (data.owner.get() == null) {
|
|
|
|
- /*
|
|
|
|
- * If the thread that created this thread-local data no
|
|
|
|
- * longer exists, remove the StatisticsData from our list
|
|
|
|
- * and fold the values into rootData.
|
|
|
|
- */
|
|
|
|
- rootData.add(data);
|
|
|
|
- iter.remove();
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
|
|
+ for (StatisticsDataReference ref: allData) {
|
|
|
|
+ StatisticsData data = ref.getData();
|
|
|
|
+ visitor.accept(data);
|
|
}
|
|
}
|
|
return visitor.aggregate();
|
|
return visitor.aggregate();
|
|
}
|
|
}
|
|
@@ -3226,7 +3279,7 @@ public abstract class FileSystem extends Configured implements Closeable {
|
|
@Override
|
|
@Override
|
|
public String toString() {
|
|
public String toString() {
|
|
return visitAll(new StatisticsAggregator<String>() {
|
|
return visitAll(new StatisticsAggregator<String>() {
|
|
- private StatisticsData total = new StatisticsData(null);
|
|
|
|
|
|
+ private StatisticsData total = new StatisticsData();
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public void accept(StatisticsData data) {
|
|
public void accept(StatisticsData data) {
|
|
@@ -3259,7 +3312,7 @@ public abstract class FileSystem extends Configured implements Closeable {
|
|
*/
|
|
*/
|
|
public void reset() {
|
|
public void reset() {
|
|
visitAll(new StatisticsAggregator<Void>() {
|
|
visitAll(new StatisticsAggregator<Void>() {
|
|
- private StatisticsData total = new StatisticsData(null);
|
|
|
|
|
|
+ private StatisticsData total = new StatisticsData();
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public void accept(StatisticsData data) {
|
|
public void accept(StatisticsData data) {
|
|
@@ -3281,6 +3334,11 @@ public abstract class FileSystem extends Configured implements Closeable {
|
|
public String getScheme() {
|
|
public String getScheme() {
|
|
return scheme;
|
|
return scheme;
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ @VisibleForTesting
|
|
|
|
+ synchronized int getAllThreadLocalDataSize() {
|
|
|
|
+ return allData.size();
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|