|
@@ -62,6 +62,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_EDIT_LOG_AUTOROL
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_EDIT_LOG_AUTOROLL_MULTIPLIER_THRESHOLD_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ENABLE_RETRY_CACHE_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ENABLE_RETRY_CACHE_KEY;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_OBJECTS_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_OBJECTS_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY;
|
|
@@ -441,6 +443,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
Daemon nnrmthread = null; // NamenodeResourceMonitor thread
|
|
|
|
|
|
Daemon nnEditLogRoller = null; // NameNodeEditLogRoller thread
|
|
|
+
|
|
|
+ // A daemon to periodically clean up corrupt lazyPersist files
|
|
|
+ // from the name space.
|
|
|
+ Daemon lazyPersistFileScrubber = null;
|
|
|
/**
|
|
|
* When an active namenode will roll its own edit log, in # edits
|
|
|
*/
|
|
@@ -450,6 +456,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
*/
|
|
|
private final int editLogRollerInterval;
|
|
|
|
|
|
+ /**
|
|
|
+ * How frequently we scan and unlink corrupt lazyPersist files.
|
|
|
+ * (In seconds)
|
|
|
+ */
|
|
|
+ private final int lazyPersistFileScrubIntervalSec;
|
|
|
+
|
|
|
private volatile boolean hasResourcesAvailable = false;
|
|
|
private volatile boolean fsRunning = true;
|
|
|
|
|
@@ -857,6 +869,15 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
DFS_NAMENODE_EDIT_LOG_AUTOROLL_CHECK_INTERVAL_MS_DEFAULT);
|
|
|
this.inodeId = new INodeId();
|
|
|
|
|
|
+ this.lazyPersistFileScrubIntervalSec = conf.getInt(
|
|
|
+ DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC,
|
|
|
+ DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC_DEFAULT);
|
|
|
+
|
|
|
+ if (this.lazyPersistFileScrubIntervalSec == 0) {
|
|
|
+ throw new IllegalArgumentException(
|
|
|
+ DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC + " must be non-zero.");
|
|
|
+ }
|
|
|
+
|
|
|
// For testing purposes, allow the DT secret manager to be started regardless
|
|
|
// of whether security is enabled.
|
|
|
alwaysUseDelegationTokensForTests = conf.getBoolean(
|
|
@@ -930,7 +951,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
@VisibleForTesting
|
|
|
static RetryCache initRetryCache(Configuration conf) {
|
|
|
boolean enable = conf.getBoolean(DFS_NAMENODE_ENABLE_RETRY_CACHE_KEY,
|
|
|
- DFS_NAMENODE_ENABLE_RETRY_CACHE_DEFAULT);
|
|
|
+ DFS_NAMENODE_ENABLE_RETRY_CACHE_DEFAULT);
|
|
|
LOG.info("Retry cache on namenode is " + (enable ? "enabled" : "disabled"));
|
|
|
if (enable) {
|
|
|
float heapPercent = conf.getFloat(
|
|
@@ -1159,6 +1180,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
editLogRollerThreshold, editLogRollerInterval));
|
|
|
nnEditLogRoller.start();
|
|
|
|
|
|
+ if (lazyPersistFileScrubIntervalSec > 0) {
|
|
|
+ lazyPersistFileScrubber = new Daemon(new LazyPersistFileScrubber(
|
|
|
+ lazyPersistFileScrubIntervalSec));
|
|
|
+ lazyPersistFileScrubber.start();
|
|
|
+ }
|
|
|
+
|
|
|
cacheManager.startMonitorThread();
|
|
|
blockManager.getDatanodeManager().setShouldSendCachingCommands(true);
|
|
|
} finally {
|
|
@@ -1211,6 +1238,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
((NameNodeEditLogRoller)nnEditLogRoller.getRunnable()).stop();
|
|
|
nnEditLogRoller.interrupt();
|
|
|
}
|
|
|
+ if (lazyPersistFileScrubber != null) {
|
|
|
+ ((LazyPersistFileScrubber) lazyPersistFileScrubber.getRunnable()).stop();
|
|
|
+ lazyPersistFileScrubber.interrupt();
|
|
|
+ }
|
|
|
if (dir != null && getFSImage() != null) {
|
|
|
if (getFSImage().editLog != null) {
|
|
|
getFSImage().editLog.close();
|
|
@@ -2683,6 +2714,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
+ src + " for client " + clientMachine);
|
|
|
}
|
|
|
INodeFile myFile = INodeFile.valueOf(inode, src, true);
|
|
|
+
|
|
|
+ if (myFile.getLazyPersistFlag()) {
|
|
|
+ throw new UnsupportedOperationException(
|
|
|
+ "Cannot append to lazy persist file " + src);
|
|
|
+ }
|
|
|
// Opening an existing file for write - may need to recover lease.
|
|
|
recoverLeaseInternal(myFile, src, holder, clientMachine, false);
|
|
|
|
|
@@ -5001,6 +5037,71 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Daemon to periodically scan the namespace for lazyPersist files
|
|
|
+ * with missing blocks and unlink them.
|
|
|
+ */
|
|
|
+ class LazyPersistFileScrubber implements Runnable {
|
|
|
+ private volatile boolean shouldRun = true;
|
|
|
+ final int scrubIntervalSec;
|
|
|
+ public LazyPersistFileScrubber(final int scrubIntervalSec) {
|
|
|
+ this.scrubIntervalSec = scrubIntervalSec;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Periodically go over the list of lazyPersist files with missing
|
|
|
+ * blocks and unlink them from the namespace.
|
|
|
+ */
|
|
|
+ private void clearCorruptLazyPersistFiles()
|
|
|
+ throws SafeModeException, AccessControlException,
|
|
|
+ UnresolvedLinkException, IOException {
|
|
|
+
|
|
|
+ List<BlockCollection> filesToDelete = new ArrayList<BlockCollection>();
|
|
|
+
|
|
|
+ writeLock();
|
|
|
+
|
|
|
+ try {
|
|
|
+ final Iterator<Block> it = blockManager.getCorruptReplicaBlockIterator();
|
|
|
+
|
|
|
+ while (it.hasNext()) {
|
|
|
+ Block b = it.next();
|
|
|
+ BlockInfo blockInfo = blockManager.getStoredBlock(b);
|
|
|
+ if (blockInfo.getBlockCollection().getLazyPersistFlag()) {
|
|
|
+ filesToDelete.add(blockInfo.getBlockCollection());
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ for (BlockCollection bc : filesToDelete) {
|
|
|
+ LOG.warn("Removing lazyPersist file " + bc.getName() + " with no replicas.");
|
|
|
+ deleteInternal(bc.getName(), false, false, false);
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ writeUnlock();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void run() {
|
|
|
+ while (fsRunning && shouldRun) {
|
|
|
+ try {
|
|
|
+ clearCorruptLazyPersistFiles();
|
|
|
+ Thread.sleep(scrubIntervalSec * 1000);
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ FSNamesystem.LOG.info(
|
|
|
+ "LazyPersistFileScrubber was interrupted, exiting");
|
|
|
+ break;
|
|
|
+ } catch (Exception e) {
|
|
|
+ FSNamesystem.LOG.error(
|
|
|
+ "Ignoring exception in LazyPersistFileScrubber:", e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ public void stop() {
|
|
|
+ shouldRun = false;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
public FSImage getFSImage() {
|
|
|
return fsImage;
|
|
|
}
|