/** * TC7: Corrupted replicas are present. * * @throws IOException an exception might be thrown */ public void testTC7() throws Exception { final short repl = 2; final Path p = new Path("/TC7/foo"); System.out.println("p=" + p); // a. Create file with replication factor of 2. Write half block of data. Close file. final int len1 = (int) (BLOCK_SIZE / 2); { FSDataOutputStream out = fs.create(p, false, buffersize, repl, BLOCK_SIZE); AppendTestUtil.write(out, 0, len1); out.close(); } DFSTestUtil.waitReplication(fs, p, repl); // b. Log into one datanode that has one replica of this block. // Find the block file on this datanode and truncate it to zero size. final LocatedBlocks locatedblocks = fs.dfs.getNamenode().getBlockLocations(p.toString(), 0L, len1); assertEquals(1, locatedblocks.locatedBlockCount()); final LocatedBlock lb = locatedblocks.get(0); final Block blk = lb.getBlock(); assertEquals(len1, lb.getBlockSize()); DatanodeInfo[] datanodeinfos = lb.getLocations(); assertEquals(repl, datanodeinfos.length); final DataNode dn = cluster.getDataNode(datanodeinfos[0].getIpcPort()); final FSDataset data = (FSDataset) dn.getFSDataset(); final RandomAccessFile raf = new RandomAccessFile(data.getBlockFile(blk), "rw"); AppendTestUtil.LOG.info("dn=" + dn + ", blk=" + blk + " (length=" + blk.getNumBytes() + ")"); assertEquals(len1, raf.length()); raf.setLength(0); raf.close(); // c. Open file in "append mode". Append a new block worth of data. Close file. final int len2 = (int) BLOCK_SIZE; { FSDataOutputStream out = fs.append(p); AppendTestUtil.write(out, len1, len2); out.close(); } // d. Reopen file and read two blocks worth of data. AppendTestUtil.check(fs, p, len1 + len2); }
/** * 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 copy on write for blocks works correctly * * @throws IOException an exception might be thrown */ @Test public void testCopyOnWrite() throws IOException { Configuration conf = new HdfsConfiguration(); if (simulatedStorage) { conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true); } MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build(); FileSystem fs = cluster.getFileSystem(); InetSocketAddress addr = new InetSocketAddress("localhost", cluster.getNameNodePort()); DFSClient client = new DFSClient(addr, conf); try { // create a new file, write to it and close it. // Path file1 = new Path("/filestatus.dat"); FSDataOutputStream stm = AppendTestUtil.createFile(fs, file1, 1); writeFile(stm); stm.close(); // Get a handle to the datanode DataNode[] dn = cluster.listDataNodes(); assertTrue("There should be only one datanode but found " + dn.length, dn.length == 1); LocatedBlocks locations = client.getNamenode().getBlockLocations(file1.toString(), 0, Long.MAX_VALUE); List<LocatedBlock> blocks = locations.getLocatedBlocks(); FSDataset dataset = (FSDataset) dn[0].data; // // Create hard links for a few of the blocks // for (int i = 0; i < blocks.size(); i = i + 2) { ExtendedBlock b = blocks.get(i).getBlock(); File f = dataset.getFile(b.getBlockPoolId(), b.getLocalBlock()); File link = new File(f.toString() + ".link"); System.out.println("Creating hardlink for File " + f + " to " + link); HardLink.createHardLink(f, link); } // // Detach all blocks. This should remove hardlinks (if any) // for (int i = 0; i < blocks.size(); i++) { ExtendedBlock b = blocks.get(i).getBlock(); System.out.println("testCopyOnWrite detaching block " + b); assertTrue( "Detaching block " + b + " should have returned true", dataset.unlinkBlock(b, 1)); } // Since the blocks were already detached earlier, these calls should // return false // for (int i = 0; i < blocks.size(); i++) { ExtendedBlock b = blocks.get(i).getBlock(); System.out.println("testCopyOnWrite detaching block " + b); assertTrue( "Detaching block " + b + " should have returned false", !dataset.unlinkBlock(b, 1)); } } finally { fs.close(); cluster.shutdown(); } }