|
@@ -29,6 +29,7 @@ import static org.mockito.Mockito.spy;
|
|
|
|
|
|
import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
+import java.util.EnumSet;
|
|
|
import java.util.List;
|
|
|
import java.util.Random;
|
|
|
|
|
@@ -40,9 +41,12 @@ import org.apache.hadoop.fs.FileStatus;
|
|
|
import org.apache.hadoop.fs.Options.Rename;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
+import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
+import org.apache.hadoop.hdfs.DFSOutputStream;
|
|
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
|
|
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.protocol.QuotaExceededException;
|
|
@@ -102,6 +106,7 @@ public class TestRenameWithSnapshots {
|
|
|
|
|
|
@Before
|
|
|
public void setUp() throws Exception {
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE);
|
|
|
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPL).format(true)
|
|
|
.build();
|
|
|
cluster.waitActive();
|
|
@@ -2289,4 +2294,49 @@ public class TestRenameWithSnapshots {
|
|
|
assertEquals(0, diff.getChildrenDiff().getList(ListType.DELETED).size());
|
|
|
assertEquals(0, diff.getChildrenDiff().getList(ListType.CREATED).size());
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Rename of the underconstruction file in snapshot should not fail NN restart
|
|
|
+ * after checkpoint. Unit test for HDFS-5425.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testRenameUCFileInSnapshot() throws Exception {
|
|
|
+ final Path test = new Path("/test");
|
|
|
+ final Path foo = new Path(test, "foo");
|
|
|
+ final Path bar = new Path(foo, "bar");
|
|
|
+ hdfs.mkdirs(foo);
|
|
|
+ // create a file and keep it as underconstruction.
|
|
|
+ hdfs.create(bar);
|
|
|
+ SnapshotTestHelper.createSnapshot(hdfs, test, "s0");
|
|
|
+ // rename bar --> bar2
|
|
|
+ final Path bar2 = new Path(foo, "bar2");
|
|
|
+ hdfs.rename(bar, bar2);
|
|
|
+
|
|
|
+ // save namespace and restart
|
|
|
+ restartClusterAndCheckImage(true);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Similar with testRenameUCFileInSnapshot, but do renaming first and then
|
|
|
+ * append file without closing it. Unit test for HDFS-5425.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testAppendFileAfterRenameInSnapshot() throws Exception {
|
|
|
+ final Path test = new Path("/test");
|
|
|
+ final Path foo = new Path(test, "foo");
|
|
|
+ final Path bar = new Path(foo, "bar");
|
|
|
+ DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
|
|
|
+ SnapshotTestHelper.createSnapshot(hdfs, test, "s0");
|
|
|
+ // rename bar --> bar2
|
|
|
+ final Path bar2 = new Path(foo, "bar2");
|
|
|
+ hdfs.rename(bar, bar2);
|
|
|
+ // append file and keep it as underconstruction.
|
|
|
+ FSDataOutputStream out = hdfs.append(bar2);
|
|
|
+ out.writeByte(0);
|
|
|
+ ((DFSOutputStream) out.getWrappedStream()).hsync(
|
|
|
+ EnumSet.of(SyncFlag.UPDATE_LENGTH));
|
|
|
+
|
|
|
+ // save namespace and restart
|
|
|
+ restartClusterAndCheckImage(true);
|
|
|
+ }
|
|
|
}
|