/** * TC5: Only one simultaneous append. * * @throws IOException an exception might be thrown */ public void testTC5() throws Exception { final Path p = new Path("/TC5/foo"); System.out.println("p=" + p); // a. Create file on Machine M1. Write half block to it. Close file. { FSDataOutputStream out = fs.create(p, false, buffersize, REPLICATION, BLOCK_SIZE); AppendTestUtil.write(out, 0, (int) (BLOCK_SIZE / 2)); out.close(); } // b. Reopen file in "append" mode on Machine M1. FSDataOutputStream out = fs.append(p); // c. On Machine M2, reopen file in "append" mode. This should fail. try { AppendTestUtil.createHdfsWithDifferentUsername(conf).append(p); fail("This should fail."); } catch (IOException ioe) { AppendTestUtil.LOG.info("GOOD: got an exception", ioe); } // d. On Machine M1, close file. out.close(); }
/** Test that all open files are closed when client dies abnormally. */ public void testDFSClientDeath() throws IOException { Configuration conf = new Configuration(); System.out.println("Testing adbornal client death."); if (simulatedStorage) { conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true); } MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null); FileSystem fs = cluster.getFileSystem(); DistributedFileSystem dfs = (DistributedFileSystem) fs; DFSClient dfsclient = dfs.dfs; try { // create a new file in home directory. Do not close it. // Path file1 = new Path("/clienttest.dat"); FSDataOutputStream stm = createFile(fs, file1, 1); System.out.println("Created file clienttest.dat"); // write to file writeFile(stm); // close the dfsclient before closing the output stream. // This should close all existing file. dfsclient.close(); // reopen file system and verify that file exists. assertTrue( file1 + " does not exist.", AppendTestUtil.createHdfsWithDifferentUsername(conf).exists(file1)); } finally { cluster.shutdown(); } }
/** * Test case that stops a writer after finalizing a block but before calling completeFile, * recovers a file from another writer, starts writing from that writer, and then has the old * lease holder call completeFile */ @Test(timeout = 60000) public void testCompleteOtherLeaseHoldersFile() throws Throwable { cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build(); try { cluster.waitActive(); NameNode preSpyNN = cluster.getNameNode(); NameNode spyNN = spy(preSpyNN); // Delay completeFile DelayAnswer delayer = new DelayAnswer(); doAnswer(delayer).when(spyNN).complete(anyString(), anyString(), (Block) anyObject()); DFSClient client = new DFSClient(null, spyNN, conf, null); file1 = new Path("/testCompleteOtherLease"); final OutputStream stm = client.create("/testCompleteOtherLease", true); // write 1/2 block AppendTestUtil.write(stm, 0, 4096); final AtomicReference<Throwable> err = new AtomicReference<Throwable>(); Thread t = new Thread() { public void run() { try { stm.close(); } catch (Throwable t) { err.set(t); } } }; t.start(); LOG.info("Waiting for close to get to latch..."); delayer.waitForCall(); // At this point, the block is finalized on the DNs, but the file // has not been completed in the NN. // Lose the leases LOG.info("Killing lease checker"); client.leasechecker.interruptAndJoin(); FileSystem fs1 = cluster.getFileSystem(); FileSystem fs2 = AppendTestUtil.createHdfsWithDifferentUsername(fs1.getConf()); LOG.info("Recovering file"); recoverFile(fs2); LOG.info("Opening file for append from new fs"); FSDataOutputStream appenderStream = fs2.append(file1); LOG.info("Writing some data from new appender"); AppendTestUtil.write(appenderStream, 0, 4096); LOG.info("Telling old close to proceed."); delayer.proceed(); LOG.info("Waiting for close to finish."); t.join(); LOG.info("Close finished."); // We expect that close will get a "Lease mismatch" // error. Throwable thrownByClose = err.get(); assertNotNull(thrownByClose); assertTrue(thrownByClose instanceof IOException); if (!thrownByClose.getMessage().contains("Lease mismatch")) throw thrownByClose; // The appender should be able to close properly appenderStream.close(); } finally { cluster.shutdown(); } }
/** * Test case that stops a writer after finalizing a block but before calling completeFile, and * then tries to recover the lease from another thread. */ @Test(timeout = 60000) public void testRecoverFinalizedBlock() throws Throwable { cluster = new MiniDFSCluster.Builder(conf).numDataNodes(5).build(); try { cluster.waitActive(); NamenodeProtocols preSpyNN = cluster.getNameNodeRpc(); NamenodeProtocols spyNN = spy(preSpyNN); // Delay completeFile GenericTestUtils.DelayAnswer delayer = new GenericTestUtils.DelayAnswer(LOG); doAnswer(delayer) .when(spyNN) .complete(anyString(), anyString(), (ExtendedBlock) anyObject(), anyLong()); DFSClient client = new DFSClient(null, spyNN, conf, null); file1 = new Path("/testRecoverFinalized"); final OutputStream stm = client.create("/testRecoverFinalized", true); // write 1/2 block AppendTestUtil.write(stm, 0, 4096); final AtomicReference<Throwable> err = new AtomicReference<Throwable>(); Thread t = new Thread() { @Override public void run() { try { stm.close(); } catch (Throwable t) { err.set(t); } } }; t.start(); LOG.info("Waiting for close to get to latch..."); delayer.waitForCall(); // At this point, the block is finalized on the DNs, but the file // has not been completed in the NN. // Lose the leases LOG.info("Killing lease checker"); client.getLeaseRenewer().interruptAndJoin(); FileSystem fs1 = cluster.getFileSystem(); FileSystem fs2 = AppendTestUtil.createHdfsWithDifferentUsername(fs1.getConf()); LOG.info("Recovering file"); recoverFile(fs2); LOG.info("Telling close to proceed."); delayer.proceed(); LOG.info("Waiting for close to finish."); t.join(); LOG.info("Close finished."); // We expect that close will get a "File is not open" error. Throwable thrownByClose = err.get(); assertNotNull(thrownByClose); assertTrue(thrownByClose instanceof LeaseExpiredException); GenericTestUtils.assertExceptionContains("File is not open for writing", thrownByClose); } finally { cluster.shutdown(); } }