/** * Test that appends to files at random offsets. * * @throws IOException an exception might be thrown */ public void testComplexAppend() throws IOException { fileContents = AppendTestUtil.initBuffer(AppendTestUtil.FILE_SIZE); Configuration conf = new HdfsConfiguration(); conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 2000); conf.setInt("dfs.heartbeat.interval", 2); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY, 2); conf.setInt(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 30000); conf.setInt("dfs.datanode.socket.write.timeout", 30000); conf.setInt("dfs.datanode.handler.count", 50); conf.setBoolean("dfs.support.append", true); MiniDFSCluster cluster = new MiniDFSCluster(conf, numDatanodes, true, null); cluster.waitActive(); FileSystem fs = cluster.getFileSystem(); try { // create a bunch of test files with random replication factors. // Insert them into a linked list. // for (int i = 0; i < numberOfFiles; i++) { short replication = (short) (AppendTestUtil.nextInt(numDatanodes) + 1); Path testFile = new Path("/" + i + ".dat"); FSDataOutputStream stm = AppendTestUtil.createFile(fs, testFile, replication); stm.close(); testFiles.add(testFile); } // Create threads and make them run workload concurrently. workload = new Workload[numThreads]; for (int i = 0; i < numThreads; i++) { workload[i] = new Workload(cluster, i); workload[i].start(); } // wait for all transactions to get over for (int i = 0; i < numThreads; i++) { try { System.out.println("Waiting for thread " + i + " to complete..."); workload[i].join(); System.out.println("Waiting for thread " + i + " complete."); } catch (InterruptedException e) { i--; // retry } } } finally { fs.close(); cluster.shutdown(); } // If any of the worker thread failed in their job, indicate that // this test failed. // assertTrue("testComplexAppend Worker encountered exceptions.", globalStatus); }
/** * Creates one file, writes a few bytes to it and then closed it. Reopens the same file for * appending, write all blocks and then close. Verify that all data exists in file. * * @throws IOException an exception might be thrown */ public void testSimpleAppend() throws IOException { final Configuration conf = new HdfsConfiguration(); if (simulatedStorage) { conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true); } conf.setInt("dfs.datanode.handler.count", 50); conf.setBoolean("dfs.support.append", true); fileContents = AppendTestUtil.initBuffer(AppendTestUtil.FILE_SIZE); MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null); FileSystem fs = cluster.getFileSystem(); try { { // test appending to a file. // create a new file. Path file1 = new Path("/simpleAppend.dat"); FSDataOutputStream stm = AppendTestUtil.createFile(fs, file1, 1); System.out.println("Created file simpleAppend.dat"); // write to file int mid = 186; // io.bytes.per.checksum bytes System.out.println("Writing " + mid + " bytes to file " + file1); stm.write(fileContents, 0, mid); stm.close(); System.out.println("Wrote and Closed first part of file."); // write to file int mid2 = 607; // io.bytes.per.checksum bytes System.out.println("Writing " + mid + " bytes to file " + file1); stm = fs.append(file1); stm.write(fileContents, mid, mid2 - mid); stm.close(); System.out.println("Wrote and Closed second part of file."); // write the remainder of the file stm = fs.append(file1); // ensure getPos is set to reflect existing size of the file assertTrue(stm.getPos() > 0); System.out.println( "Writing " + (AppendTestUtil.FILE_SIZE - mid2) + " bytes to file " + file1); stm.write(fileContents, mid2, AppendTestUtil.FILE_SIZE - mid2); System.out.println("Written second part of file"); stm.close(); System.out.println("Wrote and Closed second part of file."); // verify that entire file is good AppendTestUtil.checkFullFile(fs, file1, AppendTestUtil.FILE_SIZE, fileContents, "Read 2"); } { // test appending to an non-existing file. FSDataOutputStream out = null; try { out = fs.append(new Path("/non-existing.dat")); fail("Expected to have FileNotFoundException"); } catch (java.io.FileNotFoundException fnfe) { System.out.println("Good: got " + fnfe); fnfe.printStackTrace(System.out); } finally { IOUtils.closeStream(out); } } { // test append permission. // set root to all writable Path root = new Path("/"); fs.setPermission(root, new FsPermission((short) 0777)); fs.close(); // login as a different user final UserGroupInformation superuser = UserGroupInformation.getCurrentUser(); String username = "******"; String group = "testappendgroup"; assertFalse(superuser.getShortUserName().equals(username)); assertFalse(Arrays.asList(superuser.getGroupNames()).contains(group)); UserGroupInformation appenduser = UserGroupInformation.createUserForTesting(username, new String[] {group}); fs = DFSTestUtil.getFileSystemAs(appenduser, conf); // create a file Path dir = new Path(root, getClass().getSimpleName()); Path foo = new Path(dir, "foo.dat"); FSDataOutputStream out = null; int offset = 0; try { out = fs.create(foo); int len = 10 + AppendTestUtil.nextInt(100); out.write(fileContents, offset, len); offset += len; } finally { IOUtils.closeStream(out); } // change dir and foo to minimal permissions. fs.setPermission(dir, new FsPermission((short) 0100)); fs.setPermission(foo, new FsPermission((short) 0200)); // try append, should success out = null; try { out = fs.append(foo); int len = 10 + AppendTestUtil.nextInt(100); out.write(fileContents, offset, len); offset += len; } finally { IOUtils.closeStream(out); } // change dir and foo to all but no write on foo. fs.setPermission(foo, new FsPermission((short) 0577)); fs.setPermission(dir, new FsPermission((short) 0777)); // try append, should fail out = null; try { out = fs.append(foo); fail("Expected to have AccessControlException"); } catch (AccessControlException ace) { System.out.println("Good: got " + ace); ace.printStackTrace(System.out); } finally { IOUtils.closeStream(out); } } } catch (IOException e) { System.out.println("Exception :" + e); throw e; } catch (Throwable e) { System.out.println("Throwable :" + e); e.printStackTrace(); throw new IOException("Throwable : " + e); } finally { fs.close(); 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 } } }
// create a bunch of files. Write to them and then verify. public void run() { System.out.println("Workload " + id + " starting... "); for (int i = 0; i < numAppendsPerThread; i++) { // pick a file at random and remove it from pool Path testfile; synchronized (testFiles) { if (testFiles.size() == 0) { System.out.println("Completed write to almost all files."); return; } int index = AppendTestUtil.nextInt(testFiles.size()); testfile = testFiles.remove(index); } long len = 0; int sizeToAppend = 0; try { FileSystem fs = cluster.getFileSystem(); // add a random number of bytes to file len = fs.getFileStatus(testfile).getLen(); // if file is already full, then pick another file if (len >= AppendTestUtil.FILE_SIZE) { System.out.println("File " + testfile + " is full."); continue; } // do small size appends so that we can trigger multiple // appends to the same file. // int left = (int) (AppendTestUtil.FILE_SIZE - len) / 3; if (left <= 0) { left = 1; } sizeToAppend = AppendTestUtil.nextInt(left); System.out.println( "Workload thread " + id + " appending " + sizeToAppend + " bytes " + " to file " + testfile + " of size " + len); FSDataOutputStream stm = fs.append(testfile); stm.write(fileContents, (int) len, sizeToAppend); stm.close(); // wait for the file size to be reflected in the namenode metadata while (fs.getFileStatus(testfile).getLen() != (len + sizeToAppend)) { try { System.out.println( "Workload thread " + id + " file " + testfile + " size " + fs.getFileStatus(testfile).getLen() + " expected size " + (len + sizeToAppend) + " waiting for namenode metadata update."); Thread.sleep(5000); } catch (InterruptedException e) {; } } assertTrue( "File " + testfile + " size is " + fs.getFileStatus(testfile).getLen() + " but expected " + (len + sizeToAppend), fs.getFileStatus(testfile).getLen() == (len + sizeToAppend)); AppendTestUtil.checkFullFile( fs, testfile, (int) (len + sizeToAppend), fileContents, "Read 2"); } catch (Throwable e) { globalStatus = false; if (e != null && e.toString() != null) { System.out.println("Workload exception " + id + " testfile " + testfile + " " + e); e.printStackTrace(); } assertTrue( "Workload exception " + id + " testfile " + testfile + " expected size " + (len + sizeToAppend), false); } // Add testfile back to the pool of files. synchronized (testFiles) { testFiles.add(testfile); } } }
/** Test replace datanode on failure. */ @Test public void testReplaceDatanodeOnFailure() throws Exception { final Configuration conf = new HdfsConfiguration(); // do not consider load factor when selecting a data node conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, false); // always replace a datanode ReplaceDatanodeOnFailure.write(Policy.ALWAYS, true, conf); final String[] racks = new String[REPLICATION]; Arrays.fill(racks, RACK0); final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).racks(racks).numDataNodes(REPLICATION).build(); try { cluster.waitActive(); final DistributedFileSystem fs = cluster.getFileSystem(); final Path dir = new Path(DIR); final int NUM_WRITERS = 10; final int FIRST_BATCH = 5; final SlowWriter[] slowwriters = new SlowWriter[NUM_WRITERS]; for (int i = 1; i <= slowwriters.length; i++) { // create slow writers in different speed slowwriters[i - 1] = new SlowWriter(fs, new Path(dir, "file" + i), i * 200L); } for (int i = 0; i < FIRST_BATCH; i++) { slowwriters[i].start(); } // Let slow writers write something. // Some of them are too slow and will be not yet started. sleepSeconds(3); // start new datanodes cluster.startDataNodes(conf, 2, true, null, new String[] {RACK1, RACK1}); cluster.waitActive(); // wait for first block reports for up to 10 seconds cluster.waitFirstBRCompleted(0, 10000); // stop an old datanode MiniDFSCluster.DataNodeProperties dnprop = cluster.stopDataNode(AppendTestUtil.nextInt(REPLICATION)); for (int i = FIRST_BATCH; i < slowwriters.length; i++) { slowwriters[i].start(); } waitForBlockReplication(slowwriters); // check replication and interrupt. for (SlowWriter s : slowwriters) { s.checkReplication(); s.interruptRunning(); } // close files for (SlowWriter s : slowwriters) { s.joinAndClose(); } // Verify the file LOG.info("Verify the file"); for (int i = 0; i < slowwriters.length; i++) { LOG.info( slowwriters[i].filepath + ": length=" + fs.getFileStatus(slowwriters[i].filepath).getLen()); FSDataInputStream in = null; try { in = fs.open(slowwriters[i].filepath); for (int j = 0, x; (x = in.read()) != -1; j++) { Assert.assertEquals(j, x); } } finally { IOUtils.closeStream(in); } } } finally { if (cluster != null) { cluster.shutdown(); } } }