|
@@ -177,7 +177,7 @@ public class DFSClient implements java.io.Closeable {
|
|
|
final ReplaceDatanodeOnFailure dtpReplaceDatanodeOnFailure;
|
|
|
final FileSystem.Statistics stats;
|
|
|
final int hdfsTimeout; // timeout value for a DFS operation.
|
|
|
- final LeaseRenewer leaserenewer;
|
|
|
+ private final String authority;
|
|
|
final SocketCache socketCache;
|
|
|
final Conf dfsClientConf;
|
|
|
private Random r = new Random();
|
|
@@ -347,9 +347,9 @@ public class DFSClient implements java.io.Closeable {
|
|
|
this.hdfsTimeout = Client.getTimeout(conf);
|
|
|
this.ugi = UserGroupInformation.getCurrentUser();
|
|
|
|
|
|
- final String authority = nameNodeUri == null? "null": nameNodeUri.getAuthority();
|
|
|
- this.leaserenewer = LeaseRenewer.getInstance(authority, ugi, this);
|
|
|
- this.clientName = leaserenewer.getClientName(dfsClientConf.taskId);
|
|
|
+ this.authority = nameNodeUri == null? "null": nameNodeUri.getAuthority();
|
|
|
+ this.clientName = "DFSClient_" + dfsClientConf.taskId + "_" +
|
|
|
+ DFSUtil.getRandom().nextInt() + "_" + Thread.currentThread().getId();
|
|
|
|
|
|
this.socketCache = new SocketCache(dfsClientConf.socketCacheCapacity);
|
|
|
|
|
@@ -477,7 +477,30 @@ public class DFSClient implements java.io.Closeable {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /** Put a file. */
|
|
|
+ /** Return the lease renewer instance. The renewer thread won't start
|
|
|
+ * until the first output stream is created. The same instance will
|
|
|
+ * be returned until all output streams are closed.
|
|
|
+ */
|
|
|
+ public synchronized LeaseRenewer getLeaseRenewer() throws IOException {
|
|
|
+ return LeaseRenewer.getInstance(authority, ugi, this);
|
|
|
+ }
|
|
|
+
|
|
|
+ /** Get a lease and start automatic renewal */
|
|
|
+ private void beginFileLease(final String src, final DFSOutputStream out)
|
|
|
+ throws IOException {
|
|
|
+ getLeaseRenewer().put(src, out, this);
|
|
|
+ }
|
|
|
+
|
|
|
+ /** Stop renewal of lease for the file. */
|
|
|
+ void endFileLease(final String src) throws IOException {
|
|
|
+ getLeaseRenewer().closeFile(src, this);
|
|
|
+ }
|
|
|
+
|
|
|
+
|
|
|
+ /** Put a file. Only called from LeaseRenewer, where proper locking is
|
|
|
+ * enforced to consistently update its local dfsclients array and
|
|
|
+ * client's filesBeingWritten map.
|
|
|
+ */
|
|
|
void putFileBeingWritten(final String src, final DFSOutputStream out) {
|
|
|
synchronized(filesBeingWritten) {
|
|
|
filesBeingWritten.put(src, out);
|
|
@@ -490,7 +513,7 @@ public class DFSClient implements java.io.Closeable {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /** Remove a file. */
|
|
|
+ /** Remove a file. Only called from LeaseRenewer. */
|
|
|
void removeFileBeingWritten(final String src) {
|
|
|
synchronized(filesBeingWritten) {
|
|
|
filesBeingWritten.remove(src);
|
|
@@ -566,6 +589,14 @@ public class DFSClient implements java.io.Closeable {
|
|
|
clientRunning = false;
|
|
|
closeAllFilesBeingWritten(true);
|
|
|
socketCache.clear();
|
|
|
+
|
|
|
+ try {
|
|
|
+ // remove reference to this client and stop the renewer,
|
|
|
+ // if there is no more clients under the renewer.
|
|
|
+ getLeaseRenewer().closeClient(this);
|
|
|
+ } catch (IOException ioe) {
|
|
|
+ LOG.info("Exception occurred while aborting the client. " + ioe);
|
|
|
+ }
|
|
|
closeConnectionToNamenode();
|
|
|
}
|
|
|
|
|
@@ -606,7 +637,7 @@ public class DFSClient implements java.io.Closeable {
|
|
|
closeAllFilesBeingWritten(false);
|
|
|
socketCache.clear();
|
|
|
clientRunning = false;
|
|
|
- leaserenewer.closeClient(this);
|
|
|
+ getLeaseRenewer().closeClient(this);
|
|
|
// close connections to the namenode
|
|
|
closeConnectionToNamenode();
|
|
|
}
|
|
@@ -1103,7 +1134,7 @@ public class DFSClient implements java.io.Closeable {
|
|
|
final DFSOutputStream result = DFSOutputStream.newStreamForCreate(this,
|
|
|
src, masked, flag, createParent, replication, blockSize, progress,
|
|
|
buffersize, dfsClientConf.createChecksum());
|
|
|
- leaserenewer.put(src, result, this);
|
|
|
+ beginFileLease(src, result);
|
|
|
return result;
|
|
|
}
|
|
|
|
|
@@ -1153,7 +1184,7 @@ public class DFSClient implements java.io.Closeable {
|
|
|
flag, createParent, replication, blockSize, progress, buffersize,
|
|
|
checksum);
|
|
|
}
|
|
|
- leaserenewer.put(src, result, this);
|
|
|
+ beginFileLease(src, result);
|
|
|
return result;
|
|
|
}
|
|
|
|
|
@@ -1239,7 +1270,7 @@ public class DFSClient implements java.io.Closeable {
|
|
|
+ src + " on client " + clientName);
|
|
|
}
|
|
|
final DFSOutputStream result = callAppend(stat, src, buffersize, progress);
|
|
|
- leaserenewer.put(src, result, this);
|
|
|
+ beginFileLease(src, result);
|
|
|
return result;
|
|
|
}
|
|
|
|