// test closing file system before all file handles are closed. public void testFsClose() throws Exception { System.out.println("test file system close start"); final int DATANODE_NUM = 3; Configuration conf = new Configuration(); // create cluster MiniDFSCluster cluster = new MiniDFSCluster(conf, DATANODE_NUM, true, null); DistributedFileSystem dfs = null; try { cluster.waitActive(); dfs = (DistributedFileSystem) cluster.getFileSystem(); // create a new file. final String f = DIR + "foofs"; final Path fpath = new Path(f); FSDataOutputStream out = TestFileCreation.createFile(dfs, fpath, DATANODE_NUM); out.write("something".getBytes()); // close file system without closing file dfs.close(); } finally { System.out.println("testFsClose successful"); cluster.shutdown(); } }
/** * This test creates three empty files and lets their leases expire. This triggers release of the * leases. The empty files are supposed to be closed by that without causing * ConcurrentModificationException. */ public void testLeaseExpireEmptyFiles() throws Exception { final Thread.UncaughtExceptionHandler oldUEH = Thread.getDefaultUncaughtExceptionHandler(); Thread.setDefaultUncaughtExceptionHandler( new Thread.UncaughtExceptionHandler() { public void uncaughtException(Thread t, Throwable e) { if (e instanceof ConcurrentModificationException) { FSNamesystem.LOG.error("t=" + t, e); isConcurrentModificationException = true; } } }); System.out.println("testLeaseExpireEmptyFiles start"); final long leasePeriod = 1000; final int DATANODE_NUM = 3; final Configuration conf = new HdfsConfiguration(); conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000); conf.setInt("dfs.heartbeat.interval", 1); // create cluster MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(DATANODE_NUM).build(); try { cluster.waitActive(); DistributedFileSystem dfs = (DistributedFileSystem) cluster.getFileSystem(); // create a new file. TestFileCreation.createFile(dfs, new Path("/foo"), DATANODE_NUM); TestFileCreation.createFile(dfs, new Path("/foo2"), DATANODE_NUM); TestFileCreation.createFile(dfs, new Path("/foo3"), DATANODE_NUM); // 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) { } assertFalse(isConcurrentModificationException); } finally { Thread.setDefaultUncaughtExceptionHandler(oldUEH); cluster.shutdown(); } }
/** Test that that writes to an incomplete block are available to a reader */ public void testUnfinishedBlockRead() throws IOException { // create a new file in the root, write data, do no close Path file1 = new Path("/unfinished-block"); FSDataOutputStream stm = TestFileCreation.createFile(fileSystem, file1, 1); // write partial block and sync int partialBlockSize = blockSize / 2; writeFileAndSync(stm, partialBlockSize); // Make sure a client can read it before it is closed checkCanRead(fileSystem, file1, partialBlockSize); stm.close(); }
// test closing file after cluster is shutdown public void testFsCloseAfterClusterShutdown() throws IOException { System.out.println("test testFsCloseAfterClusterShutdown start"); final int DATANODE_NUM = 3; Configuration conf = new Configuration(); conf.setInt("dfs.replication.min", 3); conf.setBoolean("ipc.client.ping", false); // hdfs timeout is default 60 seconds conf.setInt("ipc.ping.interval", 10000); // hdfs timeout is now 10 second // create cluster MiniDFSCluster cluster = new MiniDFSCluster(conf, DATANODE_NUM, true, null); DistributedFileSystem dfs = null; try { cluster.waitActive(); dfs = (DistributedFileSystem) cluster.getFileSystem(); // create a new file. final String f = DIR + "dhrubashutdown"; final Path fpath = new Path(f); FSDataOutputStream out = TestFileCreation.createFile(dfs, fpath, DATANODE_NUM); out.write("something_dhruba".getBytes()); out.sync(); // ensure that block is allocated // shutdown last datanode in pipeline. cluster.stopDataNode(2); // close file. Since we have set the minReplication to 3 but have killed one // of the three datanodes, the close call will loop until the hdfsTimeout is // encountered. boolean hasException = false; try { out.close(); System.out.println("testFsCloseAfterClusterShutdown: Error here"); } catch (IOException e) { hasException = true; } assertTrue("Failed to close file after cluster shutdown", hasException); } finally { System.out.println("testFsCloseAfterClusterShutdown successful"); if (cluster != null) { cluster.shutdown(); } } }
/** * test case: if the BlockSender decides there is only one packet to send, the previous * computation of the pktSize based on transferToAllowed would result in too small a buffer to do * the buffer-copy needed for partial chunks. */ public void testUnfinishedBlockPacketBufferOverrun() throws IOException { // check that / exists Path path = new Path("/"); System.out.println("Path : \"" + path.toString() + "\""); // create a new file in the root, write data, do no close Path file1 = new Path("/unfinished-block"); final FSDataOutputStream stm = TestFileCreation.createFile(fileSystem, file1, 1); // write partial block and sync final int bytesPerChecksum = conf.getInt("io.bytes.per.checksum", 512); final int partialBlockSize = bytesPerChecksum - 1; writeFileAndSync(stm, partialBlockSize); // Make sure a client can read it before it is closed checkCanRead(fileSystem, file1, partialBlockSize); stm.close(); }
/** * Create a file, write something, fsync but not close. Then change lease period and wait for * lease recovery. Finally, read the block directly from each Datanode and verify the content. */ public void testLeaseExpireHardLimit() throws Exception { System.out.println("testLeaseExpireHardLimit start"); final long leasePeriod = 1000; final int DATANODE_NUM = 3; Configuration conf = new Configuration(); conf.setInt("heartbeat.recheck.interval", 1000); conf.setInt("dfs.heartbeat.interval", 1); // create cluster MiniDFSCluster cluster = new MiniDFSCluster(conf, DATANODE_NUM, true, null); DistributedFileSystem dfs = null; try { cluster.waitActive(); dfs = (DistributedFileSystem) cluster.getFileSystem(); // create a new file. final String f = DIR + "foo"; final Path fpath = new Path(f); FSDataOutputStream out = TestFileCreation.createFile(dfs, fpath, DATANODE_NUM); out.write("something".getBytes()); out.sync(); int actualRepl = ((DFSClient.DFSOutputStream) (out.getWrappedStream())).getNumCurrentReplicas(); assertTrue( f + " should be replicated to " + DATANODE_NUM + " datanodes.", actualRepl == DATANODE_NUM); // 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) { } LocatedBlocks locations = dfs.dfs.namenode.getBlockLocations(f, 0, Long.MAX_VALUE); assertEquals(1, locations.locatedBlockCount()); LocatedBlock locatedblock = locations.getLocatedBlocks().get(0); int successcount = 0; for (DatanodeInfo datanodeinfo : locatedblock.getLocations()) { DataNode datanode = cluster.getDataNode(datanodeinfo.ipcPort); FSDataset dataset = (FSDataset) datanode.data; Block b = dataset.getStoredBlock(locatedblock.getBlock().getBlockId()); File blockfile = dataset.findBlockFile(b.getBlockId()); System.out.println("blockfile=" + blockfile); if (blockfile != null) { BufferedReader in = new BufferedReader(new FileReader(blockfile)); assertEquals("something", in.readLine()); in.close(); successcount++; } } System.out.println("successcount=" + successcount); assertTrue(successcount > 0); } finally { IOUtils.closeStream(dfs); cluster.shutdown(); } System.out.println("testLeaseExpireHardLimit successful"); }