|
@@ -40,10 +40,10 @@ import org.apache.hadoop.net.NetUtils;
|
|
|
import org.apache.hadoop.net.NetworkTopology;
|
|
|
import org.apache.hadoop.net.NodeBase;
|
|
|
import org.apache.hadoop.net.ScriptBasedMapping;
|
|
|
-import org.apache.hadoop.util.Daemon;
|
|
|
import org.apache.hadoop.util.ReflectionUtils;
|
|
|
|
|
|
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
|
|
|
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
|
|
|
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
@@ -119,8 +119,6 @@ public class ClientContext {
|
|
|
private NodeBase clientNode;
|
|
|
private boolean topologyResolutionEnabled;
|
|
|
|
|
|
- private Daemon deadNodeDetectorThr = null;
|
|
|
-
|
|
|
/**
|
|
|
* The switch to DeadNodeDetector.
|
|
|
*/
|
|
@@ -130,12 +128,18 @@ public class ClientContext {
|
|
|
* Detect the dead datanodes in advance, and share this information among all
|
|
|
* the DFSInputStreams in the same client.
|
|
|
*/
|
|
|
- private DeadNodeDetector deadNodeDetector = null;
|
|
|
+ private volatile DeadNodeDetector deadNodeDetector = null;
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Count the reference of ClientContext.
|
|
|
+ */
|
|
|
+ private int counter = 0;
|
|
|
|
|
|
/**
|
|
|
* ShortCircuitCache array size.
|
|
|
*/
|
|
|
private final int clientShortCircuitNum;
|
|
|
+ private Configuration configuration;
|
|
|
|
|
|
private ClientContext(String name, DfsClientConf conf,
|
|
|
Configuration config) {
|
|
@@ -149,6 +153,7 @@ public class ClientContext {
|
|
|
this.shortCircuitCache[i] = ShortCircuitCache.fromConf(scConf);
|
|
|
}
|
|
|
|
|
|
+ this.configuration = config;
|
|
|
this.peerCache = new PeerCache(scConf.getSocketCacheCapacity(),
|
|
|
scConf.getSocketCacheExpiry());
|
|
|
this.keyProviderCache = new KeyProviderCache(
|
|
@@ -159,11 +164,6 @@ public class ClientContext {
|
|
|
this.byteArrayManager = ByteArrayManager.newInstance(
|
|
|
conf.getWriteByteArrayManagerConf());
|
|
|
this.deadNodeDetectionEnabled = conf.isDeadNodeDetectionEnabled();
|
|
|
- if (deadNodeDetectionEnabled && deadNodeDetector == null) {
|
|
|
- deadNodeDetector = new DeadNodeDetector(name, config);
|
|
|
- deadNodeDetectorThr = new Daemon(deadNodeDetector);
|
|
|
- deadNodeDetectorThr.start();
|
|
|
- }
|
|
|
initTopologyResolution(config);
|
|
|
}
|
|
|
|
|
@@ -201,6 +201,7 @@ public class ClientContext {
|
|
|
context.printConfWarningIfNeeded(conf);
|
|
|
}
|
|
|
}
|
|
|
+ context.reference();
|
|
|
return context;
|
|
|
}
|
|
|
|
|
@@ -301,17 +302,33 @@ public class ClientContext {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Close dead node detector thread.
|
|
|
+ * Increment the counter. Start the dead node detector thread if there is no
|
|
|
+ * reference.
|
|
|
+ */
|
|
|
+ synchronized void reference() {
|
|
|
+ counter++;
|
|
|
+ if (deadNodeDetectionEnabled && deadNodeDetector == null) {
|
|
|
+ deadNodeDetector = new DeadNodeDetector(name, configuration);
|
|
|
+ deadNodeDetector.start();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Decrement the counter. Close the dead node detector thread if there is no
|
|
|
+ * reference.
|
|
|
*/
|
|
|
- public void stopDeadNodeDetectorThread() {
|
|
|
- if (deadNodeDetectorThr != null) {
|
|
|
- deadNodeDetectorThr.interrupt();
|
|
|
+ synchronized void unreference() {
|
|
|
+ Preconditions.checkState(counter > 0);
|
|
|
+ counter--;
|
|
|
+ if (counter == 0 && deadNodeDetectionEnabled && deadNodeDetector != null) {
|
|
|
+ deadNodeDetector.interrupt();
|
|
|
try {
|
|
|
- deadNodeDetectorThr.join();
|
|
|
+ deadNodeDetector.join();
|
|
|
} catch (InterruptedException e) {
|
|
|
LOG.warn("Encountered exception while waiting to join on dead " +
|
|
|
"node detector thread.", e);
|
|
|
}
|
|
|
+ deadNodeDetector = null;
|
|
|
}
|
|
|
}
|
|
|
}
|