|
@@ -34,6 +34,7 @@ import org.apache.hadoop.hdfs.DFSOutputStream;
|
|
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
|
|
|
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
|
|
@@ -135,4 +136,51 @@ public class TestFSImage {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Ensure mtime and atime can be loaded from fsimage.
|
|
|
+ */
|
|
|
+ @Test(timeout=60000)
|
|
|
+ public void testLoadMtimeAtime() throws Exception {
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ MiniDFSCluster cluster = null;
|
|
|
+ try {
|
|
|
+ cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
|
|
+ cluster.waitActive();
|
|
|
+ DistributedFileSystem hdfs = cluster.getFileSystem();
|
|
|
+ String userDir = hdfs.getHomeDirectory().toUri().getPath().toString();
|
|
|
+ Path file = new Path(userDir, "file");
|
|
|
+ Path dir = new Path(userDir, "/dir");
|
|
|
+ Path link = new Path(userDir, "/link");
|
|
|
+ hdfs.createNewFile(file);
|
|
|
+ hdfs.mkdirs(dir);
|
|
|
+ hdfs.createSymlink(file, link, false);
|
|
|
+
|
|
|
+ long mtimeFile = hdfs.getFileStatus(file).getModificationTime();
|
|
|
+ long atimeFile = hdfs.getFileStatus(file).getAccessTime();
|
|
|
+ long mtimeDir = hdfs.getFileStatus(dir).getModificationTime();
|
|
|
+ long mtimeLink = hdfs.getFileLinkStatus(link).getModificationTime();
|
|
|
+ long atimeLink = hdfs.getFileLinkStatus(link).getAccessTime();
|
|
|
+
|
|
|
+ // save namespace and restart cluster
|
|
|
+ hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
|
|
|
+ hdfs.saveNamespace();
|
|
|
+ hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
|
|
|
+ cluster.shutdown();
|
|
|
+ cluster = new MiniDFSCluster.Builder(conf).format(false)
|
|
|
+ .numDataNodes(1).build();
|
|
|
+ cluster.waitActive();
|
|
|
+ hdfs = cluster.getFileSystem();
|
|
|
+
|
|
|
+ assertEquals(mtimeFile, hdfs.getFileStatus(file).getModificationTime());
|
|
|
+ assertEquals(atimeFile, hdfs.getFileStatus(file).getAccessTime());
|
|
|
+ assertEquals(mtimeDir, hdfs.getFileStatus(dir).getModificationTime());
|
|
|
+ assertEquals(mtimeLink, hdfs.getFileLinkStatus(link).getModificationTime());
|
|
|
+ assertEquals(atimeLink, hdfs.getFileLinkStatus(link).getAccessTime());
|
|
|
+ } finally {
|
|
|
+ if (cluster != null) {
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|