public void testAbandonBlock() throws IOException { MiniDFSCluster cluster = new MiniDFSCluster(CONF, 2, true, null); FileSystem fs = cluster.getFileSystem(); String src = FILE_NAME_PREFIX + "foo"; FSDataOutputStream fout = null; try { // start writing a a file but not close it fout = fs.create(new Path(src), true, 4096, (short) 1, 512L); for (int i = 0; i < 1024; i++) { fout.write(123); } fout.sync(); // try reading the block by someone final DFSClient dfsclient = new DFSClient(NameNode.getAddress(CONF), CONF); LocatedBlocks blocks = dfsclient.namenode.getBlockLocations(src, 0, 1); LocatedBlock b = blocks.get(0); try { dfsclient.namenode.abandonBlock(b.getBlock(), src, "someone"); // previous line should throw an exception. assertTrue(false); } catch (IOException ioe) { LOG.info("GREAT! " + StringUtils.stringifyException(ioe)); } } finally { try { fout.close(); } catch (Exception e) { } try { fs.close(); } catch (Exception e) { } try { cluster.shutdown(); } catch (Exception e) { } } }
/** Make sure that the quota is decremented correctly when a block is abandoned */ public void testQuotaUpdatedWhenBlockAbandoned() throws IOException { MiniDFSCluster cluster = new MiniDFSCluster(CONF, 2, true, null); FileSystem fs = cluster.getFileSystem(); DistributedFileSystem dfs = (DistributedFileSystem) fs; try { // Setting diskspace quota to 3MB dfs.setQuota(new Path("/"), FSConstants.QUOTA_DONT_SET, 3 * 1024 * 1024); // Start writing a file with 2 replicas to ensure each datanode has one. // Block Size is 1MB. String src = FILE_NAME_PREFIX + "test_quota1"; FSDataOutputStream fout = fs.create(new Path(src), true, 4096, (short) 2, 1024 * 1024); for (int i = 0; i < 1024; i++) { fout.writeByte(123); } // Shutdown one datanode, causing the block abandonment. cluster.getDataNodes().get(0).shutdown(); // Close the file, new block will be allocated with 2MB pending size. try { fout.close(); } catch (QuotaExceededException e) { fail("Unexpected quota exception when closing fout"); } } finally { try { fs.close(); } catch (Exception e) { } try { cluster.shutdown(); } catch (Exception e) { } } }