|
@@ -169,7 +169,7 @@ import org.apache.hadoop.hdfs.server.namenode.ha.StandbyCheckpointer;
|
|
import org.apache.hadoop.hdfs.server.namenode.ha.StandbyState;
|
|
import org.apache.hadoop.hdfs.server.namenode.ha.StandbyState;
|
|
import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean;
|
|
import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean;
|
|
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
|
|
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
|
|
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
|
|
|
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotManager;
|
|
import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
|
|
import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
|
|
@@ -307,6 +307,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
/** The namespace tree. */
|
|
/** The namespace tree. */
|
|
FSDirectory dir;
|
|
FSDirectory dir;
|
|
private final BlockManager blockManager;
|
|
private final BlockManager blockManager;
|
|
|
|
+ private final SnapshotManager snapshotManager;
|
|
private final DatanodeStatistics datanodeStatistics;
|
|
private final DatanodeStatistics datanodeStatistics;
|
|
|
|
|
|
// Block pool ID used by this namenode
|
|
// Block pool ID used by this namenode
|
|
@@ -464,6 +465,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_DEFAULT);
|
|
DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_DEFAULT);
|
|
|
|
|
|
this.blockManager = new BlockManager(this, this, conf);
|
|
this.blockManager = new BlockManager(this, this, conf);
|
|
|
|
+ this.snapshotManager = new SnapshotManager(this);
|
|
this.datanodeStatistics = blockManager.getDatanodeManager().getDatanodeStatistics();
|
|
this.datanodeStatistics = blockManager.getDatanodeManager().getDatanodeStatistics();
|
|
|
|
|
|
this.fsOwner = UserGroupInformation.getCurrentUser();
|
|
this.fsOwner = UserGroupInformation.getCurrentUser();
|
|
@@ -2963,29 +2965,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
getEditLog().logSync();
|
|
getEditLog().logSync();
|
|
}
|
|
}
|
|
|
|
|
|
- /**
|
|
|
|
- * Set the given directory as a snapshottable directory.
|
|
|
|
- * If the path is already a snapshottable directory, this is a no-op.
|
|
|
|
- * Otherwise, the {@link INodeDirectory} of the path is replaced by an
|
|
|
|
- * {@link INodeDirectorySnapshottable}.
|
|
|
|
- */
|
|
|
|
- void setSnapshottable(final String path) throws IOException {
|
|
|
|
- writeLock();
|
|
|
|
- try {
|
|
|
|
- final INodeDirectory d = INodeDirectory.valueOf(dir.getINode(path), path);
|
|
|
|
- if (d.isSnapshottable()) {
|
|
|
|
- //The directory is already a snapshottable directory.
|
|
|
|
- return;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- final INodeDirectorySnapshottable s
|
|
|
|
- = INodeDirectorySnapshottable.newInstance(d);
|
|
|
|
- dir.replaceINodeDirectory(path, d, s);
|
|
|
|
- } finally {
|
|
|
|
- writeUnlock();
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
/** Persist all metadata about this file.
|
|
/** Persist all metadata about this file.
|
|
* @param src The string representation of the path
|
|
* @param src The string representation of the path
|
|
* @param clientName The string representation of the client
|
|
* @param clientName The string representation of the client
|