|
@@ -19,20 +19,28 @@ package org.apache.hadoop.hdfs;
|
|
import static org.junit.Assert.assertEquals;
|
|
import static org.junit.Assert.assertEquals;
|
|
import static org.junit.Assert.assertTrue;
|
|
import static org.junit.Assert.assertTrue;
|
|
|
|
|
|
|
|
+import java.io.File;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
|
|
+import java.io.RandomAccessFile;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
|
+import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
|
+import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
|
|
+import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.TestInterDatanodeProtocol;
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.TestInterDatanodeProtocol;
|
|
import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
|
|
import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
|
|
|
|
+import org.apache.hadoop.security.UserGroupInformation;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
|
|
|
|
public class TestLeaseRecovery {
|
|
public class TestLeaseRecovery {
|
|
@@ -148,4 +156,55 @@ public class TestLeaseRecovery {
|
|
if (cluster != null) {cluster.shutdown();}
|
|
if (cluster != null) {cluster.shutdown();}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Block Recovery when the meta file not having crcs for all chunks in block
|
|
|
|
+ * file
|
|
|
|
+ */
|
|
|
|
+ @Test
|
|
|
|
+ public void testBlockRecoveryWithLessMetafile() throws Exception {
|
|
|
|
+ Configuration conf = new Configuration();
|
|
|
|
+ conf.set(DFSConfigKeys.DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY,
|
|
|
|
+ UserGroupInformation.getCurrentUser().getShortUserName());
|
|
|
|
+ MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1)
|
|
|
|
+ .build();
|
|
|
|
+ Path file = new Path("/testRecoveryFile");
|
|
|
|
+ DistributedFileSystem dfs = cluster.getFileSystem();
|
|
|
|
+ FSDataOutputStream out = dfs.create(file);
|
|
|
|
+ int count = 0;
|
|
|
|
+ while (count < 2 * 1024 * 1024) {
|
|
|
|
+ out.writeBytes("Data");
|
|
|
|
+ count += 4;
|
|
|
|
+ }
|
|
|
|
+ out.hsync();
|
|
|
|
+ // abort the original stream
|
|
|
|
+ ((DFSOutputStream) out.getWrappedStream()).abort();
|
|
|
|
+
|
|
|
|
+ LocatedBlocks locations = cluster.getNameNodeRpc().getBlockLocations(
|
|
|
|
+ file.toString(), 0, count);
|
|
|
|
+ ExtendedBlock block = locations.get(0).getBlock();
|
|
|
|
+ DataNode dn = cluster.getDataNodes().get(0);
|
|
|
|
+ BlockLocalPathInfo localPathInfo = dn.getBlockLocalPathInfo(block, null);
|
|
|
|
+ File metafile = new File(localPathInfo.getMetaPath());
|
|
|
|
+ assertTrue(metafile.exists());
|
|
|
|
+
|
|
|
|
+ // reduce the block meta file size
|
|
|
|
+ RandomAccessFile raf = new RandomAccessFile(metafile, "rw");
|
|
|
|
+ raf.setLength(metafile.length() - 20);
|
|
|
|
+ raf.close();
|
|
|
|
+
|
|
|
|
+ // restart DN to make replica to RWR
|
|
|
|
+ DataNodeProperties dnProp = cluster.stopDataNode(0);
|
|
|
|
+ cluster.restartDataNode(dnProp, true);
|
|
|
|
+
|
|
|
|
+ // try to recover the lease
|
|
|
|
+ DistributedFileSystem newdfs = (DistributedFileSystem) FileSystem
|
|
|
|
+ .newInstance(cluster.getConfiguration(0));
|
|
|
|
+ count = 0;
|
|
|
|
+ while (++count < 10 && !newdfs.recoverLease(file)) {
|
|
|
|
+ Thread.sleep(1000);
|
|
|
|
+ }
|
|
|
|
+ assertTrue("File should be closed", newdfs.recoverLease(file));
|
|
|
|
+
|
|
|
|
+ }
|
|
}
|
|
}
|