private void testDataNodeRedirect(Path path) throws IOException { // Create the file if (hdfs.exists(path)) { hdfs.delete(path, true); } FSDataOutputStream out = hdfs.create(path, (short) 1); out.writeBytes("0123456789"); out.close(); // Get the path's block location so we can determine // if we were redirected to the right DN. FileStatus status = hdfs.getFileStatus(path); BlockLocation[] locations = hdfs.getFileBlockLocations(status, 0, 10); String locationName = locations[0].getNames()[0]; // Connect to the NN to get redirected URL u = hftpFs.getNamenodeURL( "/data" + ServletUtil.encodePath(path.toUri().getPath()), "ugi=userx,groupy"); HttpURLConnection conn = (HttpURLConnection) u.openConnection(); HttpURLConnection.setFollowRedirects(true); conn.connect(); conn.getInputStream(); boolean checked = false; // Find the datanode that has the block according to locations // and check that the URL was redirected to this DN's info port for (DataNode node : cluster.getDataNodes()) { DatanodeRegistration dnR = node.dnRegistration; if (dnR.getName().equals(locationName)) { checked = true; assertEquals(dnR.getInfoPort(), conn.getURL().getPort()); } } assertTrue( "The test never checked that location of " + "the block and hftp desitnation are the same", checked); }
/** 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) { } } }