|
@@ -17,17 +17,22 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.dfs;
|
|
|
|
|
|
+import java.io.DataOutputStream;
|
|
|
import java.io.File;
|
|
|
+import java.net.InetSocketAddress;
|
|
|
+import java.net.Socket;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.dfs.DFSTestUtil;
|
|
|
+import org.apache.hadoop.dfs.FSConstants;
|
|
|
import org.apache.hadoop.dfs.MiniDFSCluster;
|
|
|
+import org.apache.hadoop.io.Text;
|
|
|
|
|
|
import junit.framework.TestCase;
|
|
|
|
|
|
-/** Test if a datanode can handle disk error correctly*/
|
|
|
+/** Test if a datanode can correctly handle errors during block read/write*/
|
|
|
public class TestDiskError extends TestCase {
|
|
|
public void testShutdown() throws Exception {
|
|
|
// bring up a cluster of 3
|
|
@@ -54,6 +59,7 @@ public class TestDiskError extends TestCase {
|
|
|
Path fileName = new Path("/test.txt"+i);
|
|
|
DFSTestUtil.createFile(fs, fileName, 1024, (short)2, 1L);
|
|
|
DFSTestUtil.waitReplication(fs, fileName, (short)2);
|
|
|
+ fs.delete(fileName, true);
|
|
|
}
|
|
|
} finally {
|
|
|
// restore its old permission
|
|
@@ -62,4 +68,78 @@ public class TestDiskError extends TestCase {
|
|
|
cluster.shutdown();
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ public void testReplicationError() throws Exception {
|
|
|
+ // bring up a cluster of 1
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
|
|
|
+ cluster.waitActive();
|
|
|
+ FileSystem fs = cluster.getFileSystem();
|
|
|
+
|
|
|
+ try {
|
|
|
+ // create a file of replication factor of 1
|
|
|
+ final Path fileName = new Path("/test.txt");
|
|
|
+ final int fileLen = 1;
|
|
|
+ DFSTestUtil.createFile(fs, fileName, 1, (short)1, 1L);
|
|
|
+ DFSTestUtil.waitReplication(fs, fileName, (short)1);
|
|
|
+
|
|
|
+ // get the block belonged to the created file
|
|
|
+ LocatedBlocks blocks = cluster.getNameNode().namesystem.getBlockLocations(
|
|
|
+ fileName.toString(), 0, (long)fileLen);
|
|
|
+ assertEquals(blocks.locatedBlockCount(), 1);
|
|
|
+ LocatedBlock block = blocks.get(0);
|
|
|
+
|
|
|
+ // bring up a second datanode
|
|
|
+ cluster.startDataNodes(conf, 1, true, null, null);
|
|
|
+ cluster.waitActive();
|
|
|
+ final int sndNode = 1;
|
|
|
+ DataNode datanode = cluster.getDataNodes().get(sndNode);
|
|
|
+
|
|
|
+ // replicate the block to the second datanode
|
|
|
+ InetSocketAddress target = datanode.getSelfAddr();
|
|
|
+ Socket s = new Socket(target.getAddress(), target.getPort());
|
|
|
+ //write the header.
|
|
|
+ DataOutputStream out = new DataOutputStream(
|
|
|
+ s.getOutputStream());
|
|
|
+
|
|
|
+ out.writeShort( FSConstants.DATA_TRANSFER_VERSION );
|
|
|
+ out.write( FSConstants.OP_WRITE_BLOCK );
|
|
|
+ out.writeLong( block.getBlock().getBlockId());
|
|
|
+ out.writeLong( block.getBlock().getGenerationStamp() );
|
|
|
+ out.writeInt(1);
|
|
|
+ out.writeBoolean( false ); // recovery flag
|
|
|
+ Text.writeString( out, "" );
|
|
|
+ out.writeBoolean(false); // Not sending src node information
|
|
|
+ out.writeInt(0);
|
|
|
+
|
|
|
+ // write check header
|
|
|
+ out.writeByte( 1 );
|
|
|
+ out.writeInt( 512 );
|
|
|
+
|
|
|
+ out.flush();
|
|
|
+
|
|
|
+ // close the connection before sending the content of the block
|
|
|
+ out.close();
|
|
|
+
|
|
|
+ // the temporary block & meta files should be deleted
|
|
|
+ String dataDir = new File(
|
|
|
+ System.getProperty("test.build.data", "build/test/data"),
|
|
|
+ "dfs").toString() + "/data";
|
|
|
+ File dir1 = new File(new File(dataDir, "data"+(2*sndNode+1)), "tmp");
|
|
|
+ File dir2 = new File(new File(dataDir, "data"+(2*sndNode+2)), "tmp");
|
|
|
+ while (dir1.listFiles().length != 0 || dir2.listFiles().length != 0) {
|
|
|
+ Thread.sleep(100);
|
|
|
+ }
|
|
|
+
|
|
|
+ // then increase the file's replication factor
|
|
|
+ fs.setReplication(fileName, (short)2);
|
|
|
+ // replication should succeed
|
|
|
+ DFSTestUtil.waitReplication(fs, fileName, (short)1);
|
|
|
+
|
|
|
+ // clean up the file
|
|
|
+ fs.delete(fileName, false);
|
|
|
+ } finally {
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|