|
@@ -16,6 +16,7 @@
|
|
|
* limitations under the License.
|
|
|
*/
|
|
|
package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
|
|
|
+import com.google.common.collect.Iterators;
|
|
|
import com.google.common.util.concurrent.Uninterruptibles;
|
|
|
import org.apache.hadoop.fs.CreateFlag;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
@@ -145,6 +146,36 @@ public class TestLazyPersistFiles extends LazyPersistTestCase {
|
|
|
Assert.assertTrue(fs.exists(path1));
|
|
|
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * If NN restarted then lazyPersist files should not deleted
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testFileShouldNotDiscardedIfNNRestarted() throws IOException,
|
|
|
+ InterruptedException {
|
|
|
+ getClusterBuilder().setRamDiskReplicaCapacity(2).build();
|
|
|
+ final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
+ Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
|
|
|
+ makeTestFile(path1, BLOCK_SIZE, true);
|
|
|
+ ensureFileReplicasOnStorageType(path1, RAM_DISK);
|
|
|
+
|
|
|
+ cluster.shutdownDataNodes();
|
|
|
+
|
|
|
+ cluster.restartNameNodes();
|
|
|
+
|
|
|
+ // wait for the replication monitor to mark the file as corrupt
|
|
|
+ Thread.sleep(2 * DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT * 1000);
|
|
|
+
|
|
|
+ Long corruptBlkCount = (long) Iterators.size(cluster.getNameNode()
|
|
|
+ .getNamesystem().getBlockManager().getCorruptReplicaBlockIterator());
|
|
|
+
|
|
|
+ // Check block detected as corrupted
|
|
|
+ assertThat(corruptBlkCount, is(1L));
|
|
|
+
|
|
|
+ // Ensure path1 exist.
|
|
|
+ Assert.assertTrue(fs.exists(path1));
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Concurrent read from the same node and verify the contents.
|
|
|
*/
|