/* * Recover file. * Try and open file in append mode. * Doing this, we get a hold of the file that crashed writer * was writing to. Once we have it, close it. This will * allow subsequent reader to see up to last sync. * NOTE: This is the same algorithm that HBase uses for file recovery * @param fs * @throws Exception */ private void recoverFile(final FileSystem fs) throws Exception { LOG.info("Recovering File Lease"); // set the soft limit to be 1 second so that the // namenode triggers lease recovery upon append request cluster.setLeasePeriod(1000, FSConstants.LEASE_HARDLIMIT_PERIOD); // Trying recovery int tries = 60; boolean recovered = false; FSDataOutputStream out = null; while (!recovered && tries-- > 0) { try { out = fs.append(file1); LOG.info("Successfully opened for appends"); recovered = true; } catch (IOException e) { LOG.info("Failed open for append, waiting on lease recovery"); try { Thread.sleep(1000); } catch (InterruptedException ex) { // ignore it and try again } } } if (out != null) { out.close(); } if (!recovered) { fail("Recovery should take < 1 min"); } LOG.info("Past out lease recovery"); }
void waitLeaseRecovery(MiniDFSCluster cluster) { cluster.setLeasePeriod(LEASE_PERIOD, LEASE_PERIOD); // wait for the lease to expire try { Thread.sleep(2 * 3000); // 2 heartbeat intervals } catch (InterruptedException e) { } }
/** * TC11: Racing rename * * @throws IOException an exception might be thrown */ public void testTC11() throws Exception { final Path p = new Path("/TC11/foo"); System.out.println("p=" + p); // a. Create file and write one block of data. Close file. final int len1 = (int) BLOCK_SIZE; { FSDataOutputStream out = fs.create(p, false, buffersize, REPLICATION, BLOCK_SIZE); AppendTestUtil.write(out, 0, len1); out.close(); } // b. Reopen file in "append" mode. Append half block of data. FSDataOutputStream out = fs.append(p); final int len2 = (int) BLOCK_SIZE / 2; AppendTestUtil.write(out, len1, len2); out.hflush(); // c. Rename file to file.new. final Path pnew = new Path(p + ".new"); assertTrue(fs.rename(p, pnew)); // d. Close file handle that was opened in (b). try { out.close(); fail("close() should throw an exception"); } catch (Exception e) { AppendTestUtil.LOG.info("GOOD!", e); } // wait for the lease recovery cluster.setLeasePeriod(1000, 1000); AppendTestUtil.sleep(5000); // check block sizes final long len = fs.getFileStatus(pnew).getLen(); final LocatedBlocks locatedblocks = fs.dfs.getNamenode().getBlockLocations(pnew.toString(), 0L, len); final int numblock = locatedblocks.locatedBlockCount(); for (int i = 0; i < numblock; i++) { final LocatedBlock lb = locatedblocks.get(i); final Block blk = lb.getBlock(); final long size = lb.getBlockSize(); if (i < numblock - 1) { assertEquals(BLOCK_SIZE, size); } for (DatanodeInfo datanodeinfo : lb.getLocations()) { final DataNode dn = cluster.getDataNode(datanodeinfo.getIpcPort()); final Block metainfo = dn.data.getStoredBlock(blk.getBlockId()); assertEquals(size, metainfo.getNumBytes()); } } }
/** * 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(); } }
/** * 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"); }
/** 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); DistributedFileSystem dfs = null; try { cluster.waitActive(); dfs = (DistributedFileSystem) cluster.getFileSystem(); DFSClient client = dfs.dfs; // create a new file. // Path file1 = new Path("/filestatus.dat"); createFile(dfs, 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( "testFileCreationError2: " + "The file has " + locations.locatedBlockCount() + " blocks."); // add another block to the file LocatedBlock location = client.namenode.addBlock(file1.toString(), client.clientName); System.out.println("testFileCreationError2: " + "Added block " + location.getBlock()); locations = client.namenode.getBlockLocations(file1.toString(), 0, Long.MAX_VALUE); int count = locations.locatedBlockCount(); System.out.println("testFileCreationError2: " + "The file now has " + count + " 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 synchronized. locations = client.namenode.getBlockLocations(file1.toString(), 0, Long.MAX_VALUE); System.out.println( "testFileCreationError2: " + "locations = " + locations.locatedBlockCount()); assertEquals(0, locations.locatedBlockCount()); System.out.println("testFileCreationError2 successful"); } finally { IOUtils.closeStream(dfs); cluster.shutdown(); } }
public void testBlockSynchronization() throws Exception { final long softLease = 1000; final long hardLease = 60 * 60 * 1000; final short repl = 3; final Configuration conf = new Configuration(); final int bufferSize = conf.getInt("io.file.buffer.size", 4096); conf.setLong("dfs.block.size", BLOCK_SIZE); conf.setInt("dfs.heartbeat.interval", 1); // conf.setInt("io.bytes.per.checksum", 16); MiniDFSCluster cluster = null; byte[] actual = new byte[FILE_SIZE]; try { cluster = new MiniDFSCluster(conf, 5, true, null); cluster.waitActive(); // create a file DistributedFileSystem dfs = (DistributedFileSystem) cluster.getFileSystem(); // create a random file name String filestr = "/foo" + AppendTestUtil.nextInt(); System.out.println("filestr=" + filestr); Path filepath = new Path(filestr); FSDataOutputStream stm = dfs.create(filepath, true, bufferSize, repl, BLOCK_SIZE); assertTrue(dfs.dfs.exists(filestr)); // write random number of bytes into it. int size = AppendTestUtil.nextInt(FILE_SIZE); System.out.println("size=" + size); stm.write(buffer, 0, size); // sync file AppendTestUtil.LOG.info("sync"); stm.sync(); AppendTestUtil.LOG.info("leasechecker.interrupt()"); dfs.dfs.leaseChecker.interrupt(); // set the soft limit to be 1 second so that the // namenode triggers lease recovery on next attempt to write-for-open. cluster.setLeasePeriod(softLease, hardLease); // try to re-open the file before closing the previous handle. This // should fail but will trigger lease recovery. { Configuration conf2 = new Configuration(conf); String username = UserGroupInformation.getCurrentUGI().getUserName() + "_1"; UnixUserGroupInformation.saveToConf( conf2, UnixUserGroupInformation.UGI_PROPERTY_NAME, new UnixUserGroupInformation(username, new String[] {"supergroup"})); FileSystem dfs2 = FileSystem.get(conf2); boolean done = false; for (int i = 0; i < 10 && !done; i++) { AppendTestUtil.LOG.info("i=" + i); try { dfs2.create(filepath, false, bufferSize, repl, BLOCK_SIZE); fail("Creation of an existing file should never succeed."); } catch (IOException ioe) { final String message = ioe.getMessage(); if (message.contains("file exists")) { AppendTestUtil.LOG.info("done", ioe); done = true; } else if (message.contains(AlreadyBeingCreatedException.class.getSimpleName())) { AppendTestUtil.LOG.info("GOOD! got " + message); } else { AppendTestUtil.LOG.warn("UNEXPECTED IOException", ioe); } } if (!done) { AppendTestUtil.LOG.info("sleep " + 5000 + "ms"); try { Thread.sleep(5000); } catch (InterruptedException e) { } } } assertTrue(done); } AppendTestUtil.LOG.info( "Lease for file " + filepath + " is recovered. " + "Validating its contents now..."); // verify that file-size matches assertTrue( "File should be " + size + " bytes, but is actually " + " found to be " + dfs.getFileStatus(filepath).getLen() + " bytes", dfs.getFileStatus(filepath).getLen() == size); // verify that there is enough data to read. System.out.println("File size is good. Now validating sizes from datanodes..."); FSDataInputStream stmin = dfs.open(filepath); stmin.readFully(0, actual, 0, size); stmin.close(); } finally { try { if (cluster != null) { cluster.shutdown(); } } catch (Exception e) { // ignore } } }