|
@@ -18,10 +18,6 @@
|
|
|
|
|
|
package org.apache.hadoop.hdfs;
|
|
|
|
|
|
-import static org.junit.Assert.assertEquals;
|
|
|
-import static org.junit.Assert.assertTrue;
|
|
|
-import static org.junit.Assert.fail;
|
|
|
-
|
|
|
import java.io.BufferedReader;
|
|
|
import java.io.File;
|
|
|
import java.io.FileOutputStream;
|
|
@@ -40,10 +36,13 @@ import org.apache.hadoop.fs.FSInputStream;
|
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
|
import org.apache.hadoop.fs.FileUtil;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
+import org.apache.hadoop.hdfs.inotify.Event;
|
|
|
+import org.apache.hadoop.hdfs.inotify.EventBatch;
|
|
|
import org.apache.hadoop.hdfs.protocol.DirectoryListing;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSImage;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FSImageFormat;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
@@ -51,6 +50,9 @@ import org.apache.hadoop.util.StringUtils;
|
|
|
import org.apache.log4j.Logger;
|
|
|
import org.junit.Test;
|
|
|
|
|
|
+import static org.apache.hadoop.hdfs.inotify.Event.CreateEvent;
|
|
|
+import static org.junit.Assert.*;
|
|
|
+
|
|
|
/**
|
|
|
* This tests data transfer protocol handling in the Datanode. It sends
|
|
|
* various forms of wrong data and verifies that Datanode handles it well.
|
|
@@ -74,6 +76,7 @@ public class TestDFSUpgradeFromImage {
|
|
|
private static final String HADOOP023_RESERVED_IMAGE =
|
|
|
"hadoop-0.23-reserved.tgz";
|
|
|
private static final String HADOOP2_RESERVED_IMAGE = "hadoop-2-reserved.tgz";
|
|
|
+ private static final String HADOOP252_IMAGE = "hadoop-252-dfs-dir.tgz";
|
|
|
|
|
|
private static class ReferenceFileInfo {
|
|
|
String path;
|
|
@@ -620,4 +623,100 @@ public class TestDFSUpgradeFromImage {
|
|
|
numDataNodes(1).enableManagedDfsDirsRedundancy(false).
|
|
|
manageDataDfsDirs(false), null);
|
|
|
}
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testPreserveEditLogs() throws Exception {
|
|
|
+ unpackStorage(HADOOP252_IMAGE, HADOOP_DFS_DIR_TXT);
|
|
|
+ /**
|
|
|
+ * The pre-created image has the following edits:
|
|
|
+ * mkdir /input; mkdir /input/dir1~5
|
|
|
+ * copyFromLocal randome_file_1 /input/dir1
|
|
|
+ * copyFromLocal randome_file_2 /input/dir2
|
|
|
+ * mv /input/dir1/randome_file_1 /input/dir3/randome_file_3
|
|
|
+ * rmdir /input/dir1
|
|
|
+ */
|
|
|
+ Configuration conf = new HdfsConfiguration();
|
|
|
+ conf = UpgradeUtilities.initializeStorageStateConf(1, conf);
|
|
|
+ MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
|
|
|
+ .format(false)
|
|
|
+ .manageDataDfsDirs(false)
|
|
|
+ .manageNameDfsDirs(false)
|
|
|
+ .startupOption(StartupOption.UPGRADE)
|
|
|
+ .build();
|
|
|
+ DFSInotifyEventInputStream ieis =
|
|
|
+ cluster.getFileSystem().getInotifyEventStream(0);
|
|
|
+
|
|
|
+ EventBatch batch;
|
|
|
+ Event.CreateEvent ce;
|
|
|
+ Event.RenameEvent re;
|
|
|
+
|
|
|
+ // mkdir /input
|
|
|
+ batch = TestDFSInotifyEventInputStream.waitForNextEvents(ieis);
|
|
|
+ assertEquals(1, batch.getEvents().length);
|
|
|
+ assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.CREATE);
|
|
|
+ ce = (Event.CreateEvent) batch.getEvents()[0];
|
|
|
+ assertEquals(ce.getPath(), "/input");
|
|
|
+
|
|
|
+ // mkdir /input/dir1~5
|
|
|
+ for (int i = 1; i <= 5; i++) {
|
|
|
+ batch = TestDFSInotifyEventInputStream.waitForNextEvents(ieis);
|
|
|
+ assertEquals(1, batch.getEvents().length);
|
|
|
+ assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.CREATE);
|
|
|
+ ce = (Event.CreateEvent) batch.getEvents()[0];
|
|
|
+ assertEquals(ce.getPath(), "/input/dir" + i);
|
|
|
+ }
|
|
|
+ // copyFromLocal randome_file_1~2 /input/dir1~2
|
|
|
+ for (int i = 1; i <= 2; i++) {
|
|
|
+ batch = TestDFSInotifyEventInputStream.waitForNextEvents(ieis);
|
|
|
+ assertEquals(1, batch.getEvents().length);
|
|
|
+ if (batch.getEvents()[0].getEventType() != Event.EventType.CREATE) {
|
|
|
+ FSImage.LOG.debug("");
|
|
|
+ }
|
|
|
+ assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.CREATE);
|
|
|
+
|
|
|
+ // copyFromLocal randome_file_1 /input/dir1, CLOSE
|
|
|
+ batch = TestDFSInotifyEventInputStream.waitForNextEvents(ieis);
|
|
|
+ assertEquals(1, batch.getEvents().length);
|
|
|
+ assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.CLOSE);
|
|
|
+
|
|
|
+ // copyFromLocal randome_file_1 /input/dir1, CLOSE
|
|
|
+ batch = TestDFSInotifyEventInputStream.waitForNextEvents(ieis);
|
|
|
+ assertEquals(1, batch.getEvents().length);
|
|
|
+ assertTrue(batch.getEvents()[0].getEventType() ==
|
|
|
+ Event.EventType.RENAME);
|
|
|
+ re = (Event.RenameEvent) batch.getEvents()[0];
|
|
|
+ assertEquals(re.getDstPath(), "/input/dir" + i + "/randome_file_" + i);
|
|
|
+ }
|
|
|
+
|
|
|
+ // mv /input/dir1/randome_file_1 /input/dir3/randome_file_3
|
|
|
+ long txIDBeforeRename = batch.getTxid();
|
|
|
+ batch = TestDFSInotifyEventInputStream.waitForNextEvents(ieis);
|
|
|
+ assertEquals(1, batch.getEvents().length);
|
|
|
+ assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.RENAME);
|
|
|
+ re = (Event.RenameEvent) batch.getEvents()[0];
|
|
|
+ assertEquals(re.getDstPath(), "/input/dir3/randome_file_3");
|
|
|
+
|
|
|
+
|
|
|
+ // rmdir /input/dir1
|
|
|
+ batch = TestDFSInotifyEventInputStream.waitForNextEvents(ieis);
|
|
|
+ assertEquals(1, batch.getEvents().length);
|
|
|
+ assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.UNLINK);
|
|
|
+ assertEquals(((Event.UnlinkEvent) batch.getEvents()[0]).getPath(),
|
|
|
+ "/input/dir1");
|
|
|
+ long lastTxID = batch.getTxid();
|
|
|
+
|
|
|
+ // Start inotify from the tx before rename /input/dir1/randome_file_1
|
|
|
+ ieis = cluster.getFileSystem().getInotifyEventStream(txIDBeforeRename);
|
|
|
+ batch = TestDFSInotifyEventInputStream.waitForNextEvents(ieis);
|
|
|
+ assertEquals(1, batch.getEvents().length);
|
|
|
+ assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.RENAME);
|
|
|
+ re = (Event.RenameEvent) batch.getEvents()[0];
|
|
|
+ assertEquals(re.getDstPath(), "/input/dir3/randome_file_3");
|
|
|
+
|
|
|
+ // Try to read beyond available edits
|
|
|
+ ieis = cluster.getFileSystem().getInotifyEventStream(lastTxID + 1);
|
|
|
+ assertNull(ieis.poll());
|
|
|
+
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
}
|