|
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertNull;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
|
|
|
|
import java.io.FileNotFoundException;
|
|
|
+import java.util.List;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
@@ -104,19 +105,18 @@ public class TestSnapshotPathINodes {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- static Snapshot getSnapshot(INodesInPath inodesInPath, String name) {
|
|
|
+ static Snapshot getSnapshot(INodesInPath inodesInPath, String name,
|
|
|
+ int index) {
|
|
|
if (name == null) {
|
|
|
return null;
|
|
|
}
|
|
|
- final int i = inodesInPath.getSnapshotRootIndex() - 1;
|
|
|
- final INode inode = inodesInPath.getINodes()[i];
|
|
|
+ final INode inode = inodesInPath.getINode(index - 1);
|
|
|
return inode.asDirectory().getSnapshot(DFSUtil.string2Bytes(name));
|
|
|
}
|
|
|
|
|
|
static void assertSnapshot(INodesInPath inodesInPath, boolean isSnapshot,
|
|
|
final Snapshot snapshot, int index) {
|
|
|
assertEquals(isSnapshot, inodesInPath.isSnapshot());
|
|
|
- assertEquals(index, inodesInPath.getSnapshotRootIndex());
|
|
|
assertEquals(Snapshot.getSnapshotId(isSnapshot ? snapshot : null),
|
|
|
inodesInPath.getPathSnapshotId());
|
|
|
if (!isSnapshot) {
|
|
@@ -124,7 +124,7 @@ public class TestSnapshotPathINodes {
|
|
|
inodesInPath.getLatestSnapshotId());
|
|
|
}
|
|
|
if (isSnapshot && index >= 0) {
|
|
|
- assertEquals(Snapshot.Root.class, inodesInPath.getINodes()[index].getClass());
|
|
|
+ assertEquals(Snapshot.Root.class, inodesInPath.getINode(index).getClass());
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -142,38 +142,35 @@ public class TestSnapshotPathINodes {
|
|
|
String[] names = INode.getPathNames(file1.toString());
|
|
|
byte[][] components = INode.getPathComponents(names);
|
|
|
INodesInPath nodesInPath = INodesInPath.resolve(fsdir.rootDir, components);
|
|
|
- INode[] inodes = nodesInPath.getINodes();
|
|
|
// The number of inodes should be equal to components.length
|
|
|
- assertEquals(inodes.length, components.length);
|
|
|
+ assertEquals(nodesInPath.length(), components.length);
|
|
|
// The returned nodesInPath should be non-snapshot
|
|
|
assertSnapshot(nodesInPath, false, null, -1);
|
|
|
|
|
|
// The last INode should be associated with file1
|
|
|
assertTrue("file1=" + file1 + ", nodesInPath=" + nodesInPath,
|
|
|
- inodes[components.length - 1] != null);
|
|
|
- assertEquals(inodes[components.length - 1].getFullPathName(),
|
|
|
+ nodesInPath.getINode(components.length - 1) != null);
|
|
|
+ assertEquals(nodesInPath.getINode(components.length - 1).getFullPathName(),
|
|
|
file1.toString());
|
|
|
- assertEquals(inodes[components.length - 2].getFullPathName(),
|
|
|
+ assertEquals(nodesInPath.getINode(components.length - 2).getFullPathName(),
|
|
|
sub1.toString());
|
|
|
- assertEquals(inodes[components.length - 3].getFullPathName(),
|
|
|
+ assertEquals(nodesInPath.getINode(components.length - 3).getFullPathName(),
|
|
|
dir.toString());
|
|
|
|
|
|
// Call getExistingPathINodes and request only one INode. This is used
|
|
|
// when identifying the INode for a given path.
|
|
|
nodesInPath = INodesInPath.resolve(fsdir.rootDir, components, 1, false);
|
|
|
- inodes = nodesInPath.getINodes();
|
|
|
- assertEquals(inodes.length, 1);
|
|
|
+ assertEquals(nodesInPath.length(), 1);
|
|
|
assertSnapshot(nodesInPath, false, null, -1);
|
|
|
- assertEquals(inodes[0].getFullPathName(), file1.toString());
|
|
|
+ assertEquals(nodesInPath.getINode(0).getFullPathName(), file1.toString());
|
|
|
|
|
|
// Call getExistingPathINodes and request 2 INodes. This is usually used
|
|
|
// when identifying the parent INode of a given path.
|
|
|
nodesInPath = INodesInPath.resolve(fsdir.rootDir, components, 2, false);
|
|
|
- inodes = nodesInPath.getINodes();
|
|
|
- assertEquals(inodes.length, 2);
|
|
|
+ assertEquals(nodesInPath.length(), 2);
|
|
|
assertSnapshot(nodesInPath, false, null, -1);
|
|
|
- assertEquals(inodes[1].getFullPathName(), file1.toString());
|
|
|
- assertEquals(inodes[0].getFullPathName(), sub1.toString());
|
|
|
+ assertEquals(nodesInPath.getINode(1).getFullPathName(), file1.toString());
|
|
|
+ assertEquals(nodesInPath.getINode(0).getFullPathName(), sub1.toString());
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -191,53 +188,49 @@ public class TestSnapshotPathINodes {
|
|
|
String[] names = INode.getPathNames(snapshotPath);
|
|
|
byte[][] components = INode.getPathComponents(names);
|
|
|
INodesInPath nodesInPath = INodesInPath.resolve(fsdir.rootDir, components);
|
|
|
- INode[] inodes = nodesInPath.getINodes();
|
|
|
// Length of inodes should be (components.length - 1), since we will ignore
|
|
|
// ".snapshot"
|
|
|
- assertEquals(inodes.length, components.length - 1);
|
|
|
+ assertEquals(nodesInPath.length(), components.length - 1);
|
|
|
// SnapshotRootIndex should be 3: {root, Testsnapshot, sub1, s1, file1}
|
|
|
- final Snapshot snapshot = getSnapshot(nodesInPath, "s1");
|
|
|
+ final Snapshot snapshot = getSnapshot(nodesInPath, "s1", 3);
|
|
|
assertSnapshot(nodesInPath, true, snapshot, 3);
|
|
|
// Check the INode for file1 (snapshot file)
|
|
|
- INode snapshotFileNode = inodes[inodes.length - 1];
|
|
|
+ INode snapshotFileNode = nodesInPath.getLastINode();
|
|
|
assertINodeFile(snapshotFileNode, file1);
|
|
|
assertTrue(snapshotFileNode.getParent().isWithSnapshot());
|
|
|
|
|
|
// Call getExistingPathINodes and request only one INode.
|
|
|
nodesInPath = INodesInPath.resolve(fsdir.rootDir, components, 1, false);
|
|
|
- inodes = nodesInPath.getINodes();
|
|
|
- assertEquals(inodes.length, 1);
|
|
|
+ assertEquals(nodesInPath.length(), 1);
|
|
|
// The snapshotroot (s1) is not included in inodes. Thus the
|
|
|
// snapshotRootIndex should be -1.
|
|
|
assertSnapshot(nodesInPath, true, snapshot, -1);
|
|
|
// Check the INode for file1 (snapshot file)
|
|
|
- assertINodeFile(inodes[inodes.length - 1], file1);
|
|
|
+ assertINodeFile(nodesInPath.getLastINode(), file1);
|
|
|
|
|
|
// Call getExistingPathINodes and request 2 INodes.
|
|
|
nodesInPath = INodesInPath.resolve(fsdir.rootDir, components, 2, false);
|
|
|
- inodes = nodesInPath.getINodes();
|
|
|
- assertEquals(inodes.length, 2);
|
|
|
+ assertEquals(nodesInPath.length(), 2);
|
|
|
// There should be two INodes in inodes: s1 and snapshot of file1. Thus the
|
|
|
// SnapshotRootIndex should be 0.
|
|
|
assertSnapshot(nodesInPath, true, snapshot, 0);
|
|
|
- assertINodeFile(inodes[inodes.length - 1], file1);
|
|
|
+ assertINodeFile(nodesInPath.getLastINode(), file1);
|
|
|
|
|
|
// Resolve the path "/TestSnapshot/sub1/.snapshot"
|
|
|
String dotSnapshotPath = sub1.toString() + "/.snapshot";
|
|
|
names = INode.getPathNames(dotSnapshotPath);
|
|
|
components = INode.getPathComponents(names);
|
|
|
nodesInPath = INodesInPath.resolve(fsdir.rootDir, components);
|
|
|
- inodes = nodesInPath.getINodes();
|
|
|
- // The number of INodes returned should be components.length - 1 since we
|
|
|
- // will ignore ".snapshot"
|
|
|
- assertEquals(inodes.length, components.length - 1);
|
|
|
+ // The number of INodes returned should still be components.length
|
|
|
+ // since we put a null in the inode array for ".snapshot"
|
|
|
+ assertEquals(nodesInPath.length(), components.length);
|
|
|
|
|
|
// No SnapshotRoot dir is included in the resolved inodes
|
|
|
assertSnapshot(nodesInPath, true, snapshot, -1);
|
|
|
- // The last INode should be the INode for sub1
|
|
|
- final INode last = inodes[inodes.length - 1];
|
|
|
- assertEquals(last.getFullPathName(), sub1.toString());
|
|
|
- assertFalse(last instanceof INodeFile);
|
|
|
+ // The last INode should be null, the last but 1 should be sub1
|
|
|
+ assertNull(nodesInPath.getLastINode());
|
|
|
+ assertEquals(nodesInPath.getINode(-2).getFullPathName(), sub1.toString());
|
|
|
+ assertTrue(nodesInPath.getINode(-2).isDirectory());
|
|
|
|
|
|
String[] invalidPathComponent = {"invalidDir", "foo", ".snapshot", "bar"};
|
|
|
Path invalidPath = new Path(invalidPathComponent[0]);
|
|
@@ -275,16 +268,15 @@ public class TestSnapshotPathINodes {
|
|
|
String[] names = INode.getPathNames(snapshotPath);
|
|
|
byte[][] components = INode.getPathComponents(names);
|
|
|
INodesInPath nodesInPath = INodesInPath.resolve(fsdir.rootDir, components);
|
|
|
- INode[] inodes = nodesInPath.getINodes();
|
|
|
// Length of inodes should be (components.length - 1), since we will ignore
|
|
|
// ".snapshot"
|
|
|
- assertEquals(inodes.length, components.length - 1);
|
|
|
+ assertEquals(nodesInPath.length(), components.length - 1);
|
|
|
// SnapshotRootIndex should be 3: {root, Testsnapshot, sub1, s2, file1}
|
|
|
- snapshot = getSnapshot(nodesInPath, "s2");
|
|
|
+ snapshot = getSnapshot(nodesInPath, "s2", 3);
|
|
|
assertSnapshot(nodesInPath, true, snapshot, 3);
|
|
|
|
|
|
// Check the INode for file1 (snapshot file)
|
|
|
- final INode inode = inodes[inodes.length - 1];
|
|
|
+ final INode inode = nodesInPath.getLastINode();
|
|
|
assertEquals(file1.getName(), inode.getLocalName());
|
|
|
assertTrue(inode.asFile().isWithSnapshot());
|
|
|
}
|
|
@@ -293,25 +285,34 @@ public class TestSnapshotPathINodes {
|
|
|
String[] names = INode.getPathNames(file1.toString());
|
|
|
byte[][] components = INode.getPathComponents(names);
|
|
|
INodesInPath nodesInPath = INodesInPath.resolve(fsdir.rootDir, components);
|
|
|
- INode[] inodes = nodesInPath.getINodes();
|
|
|
// The length of inodes should be equal to components.length
|
|
|
- assertEquals(inodes.length, components.length);
|
|
|
+ assertEquals(nodesInPath.length(), components.length);
|
|
|
// The number of non-null elements should be components.length - 1 since
|
|
|
// file1 has been deleted
|
|
|
- assertEquals(nodesInPath.getNumNonNull(), components.length - 1);
|
|
|
+ assertEquals(getNumNonNull(nodesInPath), components.length - 1);
|
|
|
// The returned nodesInPath should be non-snapshot
|
|
|
assertSnapshot(nodesInPath, false, snapshot, -1);
|
|
|
// The last INode should be null, and the one before should be associated
|
|
|
// with sub1
|
|
|
- assertNull(inodes[components.length - 1]);
|
|
|
- assertEquals(inodes[components.length - 2].getFullPathName(),
|
|
|
+ assertNull(nodesInPath.getINode(components.length - 1));
|
|
|
+ assertEquals(nodesInPath.getINode(components.length - 2).getFullPathName(),
|
|
|
sub1.toString());
|
|
|
- assertEquals(inodes[components.length - 3].getFullPathName(),
|
|
|
+ assertEquals(nodesInPath.getINode(components.length - 3).getFullPathName(),
|
|
|
dir.toString());
|
|
|
hdfs.deleteSnapshot(sub1, "s2");
|
|
|
hdfs.disallowSnapshot(sub1);
|
|
|
}
|
|
|
|
|
|
+ private int getNumNonNull(INodesInPath iip) {
|
|
|
+ List<INode> inodes = iip.getReadOnlyINodes();
|
|
|
+ for (int i = inodes.size() - 1; i >= 0; i--) {
|
|
|
+ if (inodes.get(i) != null) {
|
|
|
+ return i+1;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return 0;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* for snapshot file while adding a new file after snapshot.
|
|
|
*/
|
|
@@ -333,39 +334,37 @@ public class TestSnapshotPathINodes {
|
|
|
String[] names = INode.getPathNames(snapshotPath);
|
|
|
byte[][] components = INode.getPathComponents(names);
|
|
|
INodesInPath nodesInPath = INodesInPath.resolve(fsdir.rootDir, components);
|
|
|
- INode[] inodes = nodesInPath.getINodes();
|
|
|
// Length of inodes should be (components.length - 1), since we will ignore
|
|
|
// ".snapshot"
|
|
|
- assertEquals(inodes.length, components.length - 1);
|
|
|
+ assertEquals(nodesInPath.length(), components.length - 1);
|
|
|
// The number of non-null inodes should be components.length - 2, since
|
|
|
// snapshot of file3 does not exist
|
|
|
- assertEquals(nodesInPath.getNumNonNull(), components.length - 2);
|
|
|
- s4 = getSnapshot(nodesInPath, "s4");
|
|
|
+ assertEquals(getNumNonNull(nodesInPath), components.length - 2);
|
|
|
+ s4 = getSnapshot(nodesInPath, "s4", 3);
|
|
|
|
|
|
// SnapshotRootIndex should still be 3: {root, Testsnapshot, sub1, s4, null}
|
|
|
assertSnapshot(nodesInPath, true, s4, 3);
|
|
|
|
|
|
// Check the last INode in inodes, which should be null
|
|
|
- assertNull(inodes[inodes.length - 1]);
|
|
|
+ assertNull(nodesInPath.getINode(nodesInPath.length() - 1));
|
|
|
}
|
|
|
|
|
|
// Check the inodes for /TestSnapshot/sub1/file3
|
|
|
String[] names = INode.getPathNames(file3.toString());
|
|
|
byte[][] components = INode.getPathComponents(names);
|
|
|
INodesInPath nodesInPath = INodesInPath.resolve(fsdir.rootDir, components);
|
|
|
- INode[] inodes = nodesInPath.getINodes();
|
|
|
// The number of inodes should be equal to components.length
|
|
|
- assertEquals(inodes.length, components.length);
|
|
|
+ assertEquals(nodesInPath.length(), components.length);
|
|
|
|
|
|
// The returned nodesInPath should be non-snapshot
|
|
|
assertSnapshot(nodesInPath, false, s4, -1);
|
|
|
|
|
|
// The last INode should be associated with file3
|
|
|
- assertEquals(inodes[components.length - 1].getFullPathName(),
|
|
|
+ assertEquals(nodesInPath.getINode(components.length - 1).getFullPathName(),
|
|
|
file3.toString());
|
|
|
- assertEquals(inodes[components.length - 2].getFullPathName(),
|
|
|
+ assertEquals(nodesInPath.getINode(components.length - 2).getFullPathName(),
|
|
|
sub1.toString());
|
|
|
- assertEquals(inodes[components.length - 3].getFullPathName(),
|
|
|
+ assertEquals(nodesInPath.getINode(components.length - 3).getFullPathName(),
|
|
|
dir.toString());
|
|
|
hdfs.deleteSnapshot(sub1, "s4");
|
|
|
hdfs.disallowSnapshot(sub1);
|
|
@@ -380,15 +379,15 @@ public class TestSnapshotPathINodes {
|
|
|
String[] names = INode.getPathNames(file1.toString());
|
|
|
byte[][] components = INode.getPathComponents(names);
|
|
|
INodesInPath nodesInPath = INodesInPath.resolve(fsdir.rootDir, components);
|
|
|
- INode[] inodes = nodesInPath.getINodes();
|
|
|
// The number of inodes should be equal to components.length
|
|
|
- assertEquals(inodes.length, components.length);
|
|
|
+ assertEquals(nodesInPath.length(), components.length);
|
|
|
|
|
|
// The last INode should be associated with file1
|
|
|
- assertEquals(inodes[components.length - 1].getFullPathName(),
|
|
|
+ assertEquals(nodesInPath.getINode(components.length - 1).getFullPathName(),
|
|
|
file1.toString());
|
|
|
// record the modification time of the inode
|
|
|
- final long modTime = inodes[inodes.length - 1].getModificationTime();
|
|
|
+ final long modTime = nodesInPath.getINode(nodesInPath.length() - 1)
|
|
|
+ .getModificationTime();
|
|
|
|
|
|
// Create a snapshot for the dir, and check the inodes for the path
|
|
|
// pointing to a snapshot file
|
|
@@ -403,14 +402,13 @@ public class TestSnapshotPathINodes {
|
|
|
names = INode.getPathNames(snapshotPath);
|
|
|
components = INode.getPathComponents(names);
|
|
|
INodesInPath ssNodesInPath = INodesInPath.resolve(fsdir.rootDir, components);
|
|
|
- INode[] ssInodes = ssNodesInPath.getINodes();
|
|
|
// Length of ssInodes should be (components.length - 1), since we will
|
|
|
// ignore ".snapshot"
|
|
|
- assertEquals(ssInodes.length, components.length - 1);
|
|
|
- final Snapshot s3 = getSnapshot(ssNodesInPath, "s3");
|
|
|
+ assertEquals(ssNodesInPath.length(), components.length - 1);
|
|
|
+ final Snapshot s3 = getSnapshot(ssNodesInPath, "s3", 3);
|
|
|
assertSnapshot(ssNodesInPath, true, s3, 3);
|
|
|
// Check the INode for snapshot of file1
|
|
|
- INode snapshotFileNode = ssInodes[ssInodes.length - 1];
|
|
|
+ INode snapshotFileNode = ssNodesInPath.getLastINode();
|
|
|
assertEquals(snapshotFileNode.getLocalName(), file1.getName());
|
|
|
assertTrue(snapshotFileNode.asFile().isWithSnapshot());
|
|
|
// The modification time of the snapshot INode should be the same with the
|
|
@@ -423,14 +421,14 @@ public class TestSnapshotPathINodes {
|
|
|
components = INode.getPathComponents(names);
|
|
|
INodesInPath newNodesInPath = INodesInPath.resolve(fsdir.rootDir, components);
|
|
|
assertSnapshot(newNodesInPath, false, s3, -1);
|
|
|
- INode[] newInodes = newNodesInPath.getINodes();
|
|
|
// The number of inodes should be equal to components.length
|
|
|
- assertEquals(newInodes.length, components.length);
|
|
|
+ assertEquals(newNodesInPath.length(), components.length);
|
|
|
// The last INode should be associated with file1
|
|
|
final int last = components.length - 1;
|
|
|
- assertEquals(newInodes[last].getFullPathName(), file1.toString());
|
|
|
+ assertEquals(newNodesInPath.getINode(last).getFullPathName(),
|
|
|
+ file1.toString());
|
|
|
// The modification time of the INode for file3 should have been changed
|
|
|
- Assert.assertFalse(modTime == newInodes[last].getModificationTime());
|
|
|
+ Assert.assertFalse(modTime == newNodesInPath.getINode(last).getModificationTime());
|
|
|
hdfs.deleteSnapshot(sub1, "s3");
|
|
|
hdfs.disallowSnapshot(sub1);
|
|
|
}
|