|
@@ -23,10 +23,8 @@ import java.net.*;
|
|
|
import java.util.Random;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
-import org.apache.hadoop.fs.FsShell;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
-import org.apache.hadoop.util.StringUtils;
|
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.FSDataInputStream;
|
|
|
import org.apache.hadoop.dfs.FSConstants.DatanodeReportType;
|
|
@@ -47,16 +45,11 @@ public class TestFileCreation extends TestCase {
|
|
|
// entire file is written, the first two blocks definitely get flushed to
|
|
|
// the datanodes.
|
|
|
|
|
|
- private static String TEST_ROOT_DIR =
|
|
|
- new Path(System.getProperty("test.build.data","/tmp"))
|
|
|
- .toString().replace(' ', '+');
|
|
|
-
|
|
|
//
|
|
|
// creates a file but does not close it
|
|
|
//
|
|
|
private FSDataOutputStream createFile(FileSystem fileSys, Path name, int repl)
|
|
|
throws IOException {
|
|
|
- // create and write a file that contains three blocks of data
|
|
|
FSDataOutputStream stm = fileSys.create(name, true,
|
|
|
fileSys.getConf().getInt("io.file.buffer.size", 4096),
|
|
|
(short)repl, (long)blockSize);
|
|
@@ -116,9 +109,9 @@ public class TestFileCreation extends TestCase {
|
|
|
|
|
|
private void checkData(byte[] actual, int from, byte[] expected, String message) {
|
|
|
for (int idx = 0; idx < actual.length; idx++) {
|
|
|
- this.assertEquals(message+" byte "+(from+idx)+" differs. expected "+
|
|
|
- expected[from+idx]+" actual "+actual[idx],
|
|
|
- expected[from+idx], actual[idx]);
|
|
|
+ assertEquals(message+" byte "+(from+idx)+" differs. expected "+
|
|
|
+ expected[from+idx]+" actual "+actual[idx],
|
|
|
+ expected[from+idx], actual[idx]);
|
|
|
actual[idx] = 0;
|
|
|
}
|
|
|
}
|
|
@@ -266,6 +259,78 @@ public class TestFileCreation extends TestCase {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Test that the filesystem removes the last block from a file if its
|
|
|
+ * lease expires.
|
|
|
+ */
|
|
|
+ public void testFileCreationError2() throws IOException {
|
|
|
+ long leasePeriod = 1000;
|
|
|
+ System.out.println("testFileCreationError2 start");
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ conf.setInt("heartbeat.recheck.interval", 1000);
|
|
|
+ conf.setInt("dfs.heartbeat.interval", 1);
|
|
|
+ if (simulatedStorage) {
|
|
|
+ conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
|
|
|
+ }
|
|
|
+ // create cluster
|
|
|
+ MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
|
|
|
+ FileSystem fs = cluster.getFileSystem();
|
|
|
+ cluster.waitActive();
|
|
|
+ InetSocketAddress addr = new InetSocketAddress("localhost",
|
|
|
+ cluster.getNameNodePort());
|
|
|
+ DFSClient client = new DFSClient(addr, conf);
|
|
|
+
|
|
|
+ try {
|
|
|
+
|
|
|
+ // create a new file.
|
|
|
+ //
|
|
|
+ Path file1 = new Path("/filestatus.dat");
|
|
|
+ createFile(fs, file1, 1);
|
|
|
+ System.out.println("testFileCreationError2: "
|
|
|
+ + "Created file filestatus.dat with one "
|
|
|
+ + " replicas.");
|
|
|
+
|
|
|
+ LocatedBlocks locations = client.namenode.getBlockLocations(
|
|
|
+ file1.toString(), 0, Long.MAX_VALUE);
|
|
|
+ System.out.println("The file has " + locations.locatedBlockCount() +
|
|
|
+ " blocks.");
|
|
|
+
|
|
|
+ // add another block to the file
|
|
|
+ LocatedBlock location = client.namenode.addBlock(file1.toString(),
|
|
|
+ null);
|
|
|
+ System.out.println("Added block " + location.getBlock());
|
|
|
+
|
|
|
+ locations = client.namenode.getBlockLocations(file1.toString(),
|
|
|
+ 0, Long.MAX_VALUE);
|
|
|
+ System.out.println("The file now has " + locations.locatedBlockCount() +
|
|
|
+ " blocks.");
|
|
|
+
|
|
|
+ // set the soft and hard limit to be 1 second so that the
|
|
|
+ // namenode triggers lease recovery
|
|
|
+ cluster.setLeasePeriod(leasePeriod, leasePeriod);
|
|
|
+
|
|
|
+ // wait for the lease to expire
|
|
|
+ try {
|
|
|
+ Thread.sleep(5 * leasePeriod);
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ }
|
|
|
+
|
|
|
+ // verify that the last block was cleaned up.
|
|
|
+ locations = client.namenode.getBlockLocations(file1.toString(),
|
|
|
+ 0, Long.MAX_VALUE);
|
|
|
+ System.out.println("locations = " + locations.locatedBlockCount());
|
|
|
+ assertTrue("Error blocks were not cleaned up",
|
|
|
+ locations.locatedBlockCount() == 0);
|
|
|
+ System.out.println("testFileCreationError2 successful");
|
|
|
+ } finally {
|
|
|
+ try {
|
|
|
+ fs.close();
|
|
|
+ } catch (Exception e) {
|
|
|
+ }
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Test that file data becomes available before file is closed.
|
|
|
*/
|