|
@@ -76,6 +76,10 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RETRY_CACHE_EXPI
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RETRY_CACHE_HEAP_PERCENT_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RETRY_CACHE_HEAP_PERCENT_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LEASE_RECHECK_INTERVAL_MS_KEY;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LEASE_RECHECK_INTERVAL_MS_DEFAULT;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_LOCK_HOLD_TO_RELEASE_LEASE_MS_KEY;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_LOCK_HOLD_TO_RELEASE_LEASE_MS_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_ENABLED_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_DEFAULT;
|
|
@@ -372,7 +376,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
private final UserGroupInformation fsOwner;
|
|
|
private final String supergroup;
|
|
|
private final boolean standbyShouldCheckpoint;
|
|
|
-
|
|
|
+
|
|
|
+ /** Interval between each check of lease to release. */
|
|
|
+ private final long leaseRecheckIntervalMs;
|
|
|
+ /** Maximum time the lock is hold to release lease. */
|
|
|
+ private final long maxLockHoldToReleaseLeaseMs;
|
|
|
+
|
|
|
// Scan interval is not configurable.
|
|
|
private static final long DELEGATION_TOKEN_REMOVER_SCAN_INTERVAL =
|
|
|
TimeUnit.MILLISECONDS.convert(1, TimeUnit.HOURS);
|
|
@@ -791,6 +800,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
DFSConfigKeys.DFS_NAMENODE_EDEKCACHELOADER_INTERVAL_MS_KEY,
|
|
|
DFSConfigKeys.DFS_NAMENODE_EDEKCACHELOADER_INTERVAL_MS_DEFAULT);
|
|
|
|
|
|
+ this.leaseRecheckIntervalMs = conf.getLong(
|
|
|
+ DFS_NAMENODE_LEASE_RECHECK_INTERVAL_MS_KEY,
|
|
|
+ DFS_NAMENODE_LEASE_RECHECK_INTERVAL_MS_DEFAULT);
|
|
|
+ this.maxLockHoldToReleaseLeaseMs = conf.getLong(
|
|
|
+ DFS_NAMENODE_MAX_LOCK_HOLD_TO_RELEASE_LEASE_MS_KEY,
|
|
|
+ DFS_NAMENODE_MAX_LOCK_HOLD_TO_RELEASE_LEASE_MS_DEFAULT);
|
|
|
+
|
|
|
// For testing purposes, allow the DT secret manager to be started regardless
|
|
|
// of whether security is enabled.
|
|
|
alwaysUseDelegationTokensForTests = conf.getBoolean(
|
|
@@ -834,6 +850,16 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
return retryCache;
|
|
|
}
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
+ public long getLeaseRecheckIntervalMs() {
|
|
|
+ return leaseRecheckIntervalMs;
|
|
|
+ }
|
|
|
+
|
|
|
+ @VisibleForTesting
|
|
|
+ public long getMaxLockHoldToReleaseLeaseMs() {
|
|
|
+ return maxLockHoldToReleaseLeaseMs;
|
|
|
+ }
|
|
|
+
|
|
|
void lockRetryCache() {
|
|
|
if (retryCache != null) {
|
|
|
retryCache.lock();
|
|
@@ -3083,9 +3109,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
if(nrCompleteBlocks == nrBlocks) {
|
|
|
finalizeINodeFileUnderConstruction(src, pendingFile,
|
|
|
iip.getLatestSnapshotId(), false);
|
|
|
- NameNode.stateChangeLog.warn("BLOCK*"
|
|
|
- + " internalReleaseLease: All existing blocks are COMPLETE,"
|
|
|
- + " lease removed, file closed.");
|
|
|
+ NameNode.stateChangeLog.warn("BLOCK*" +
|
|
|
+ " internalReleaseLease: All existing blocks are COMPLETE," +
|
|
|
+ " lease removed, file " + src + " closed.");
|
|
|
return true; // closed!
|
|
|
}
|
|
|
|
|
@@ -3122,9 +3148,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
blockManager.checkMinReplication(lastBlock)) {
|
|
|
finalizeINodeFileUnderConstruction(src, pendingFile,
|
|
|
iip.getLatestSnapshotId(), false);
|
|
|
- NameNode.stateChangeLog.warn("BLOCK*"
|
|
|
- + " internalReleaseLease: Committed blocks are minimally replicated,"
|
|
|
- + " lease removed, file closed.");
|
|
|
+ NameNode.stateChangeLog.warn("BLOCK*" +
|
|
|
+ " internalReleaseLease: Committed blocks are minimally" +
|
|
|
+ " replicated, lease removed, file" + src + " closed.");
|
|
|
return true; // closed!
|
|
|
}
|
|
|
// Cannot close file right now, since some blocks
|
|
@@ -3167,7 +3193,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
finalizeINodeFileUnderConstruction(src, pendingFile,
|
|
|
iip.getLatestSnapshotId(), false);
|
|
|
NameNode.stateChangeLog.warn("BLOCK* internalReleaseLease: "
|
|
|
- + "Removed empty last block and closed file.");
|
|
|
+ + "Removed empty last block and closed file " + src);
|
|
|
return true;
|
|
|
}
|
|
|
// start recovery of the last block for this file
|