/** * Test a simple flush on a simple HDFS file. * * @throws IOException an exception might be thrown */ @Test public void testSimpleFlush() throws IOException { Configuration conf = new HdfsConfiguration(); if (simulatedStorage) { conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true); } fileContents = AppendTestUtil.initBuffer(AppendTestUtil.FILE_SIZE); MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build(); FileSystem fs = cluster.getFileSystem(); try { // create a new file. Path file1 = new Path("/simpleFlush.dat"); FSDataOutputStream stm = AppendTestUtil.createFile(fs, file1, 1); System.out.println("Created file simpleFlush.dat"); // write to file int mid = AppendTestUtil.FILE_SIZE / 2; stm.write(fileContents, 0, mid); stm.hflush(); System.out.println("Wrote and Flushed first part of file."); // write the remainder of the file stm.write(fileContents, mid, AppendTestUtil.FILE_SIZE - mid); System.out.println("Written second part of file"); stm.hflush(); stm.hflush(); System.out.println("Wrote and Flushed second part of file."); // verify that full blocks are sane checkFile(fs, file1, 1); stm.close(); System.out.println("Closed file."); // verify that entire file is good AppendTestUtil.checkFullFile(fs, file1, AppendTestUtil.FILE_SIZE, fileContents, "Read 2"); } 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(); } }
/** * Append to a partial CRC chunk and the first write does not fill up the partial CRC trunk * * * @throws IOException */ public void testAppendToPartialChunk() throws IOException { final Path p = new Path("/partialChunk/foo"); final int fileLen = 513; System.out.println("p=" + p); byte[] fileContents = AppendTestUtil.initBuffer(fileLen); // create a new file. FSDataOutputStream stm = AppendTestUtil.createFile(fs, p, 1); // create 1 byte file stm.write(fileContents, 0, 1); stm.close(); System.out.println("Wrote 1 byte and closed the file " + p); // append to file stm = fs.append(p); // Append to a partial CRC trunk stm.write(fileContents, 1, 1); stm.hflush(); // The partial CRC trunk is not full yet and close the file stm.close(); System.out.println("Append 1 byte and closed the file " + p); // write the remainder of the file stm = fs.append(p); // ensure getPos is set to reflect existing size of the file assertEquals(2, stm.getPos()); // append to a partial CRC trunk stm.write(fileContents, 2, 1); // The partial chunk is not full yet, force to send a packet to DN stm.hflush(); System.out.println("Append and flush 1 byte"); // The partial chunk is not full yet, force to send another packet to DN stm.write(fileContents, 3, 2); stm.hflush(); System.out.println("Append and flush 2 byte"); // fill up the partial chunk and close the file stm.write(fileContents, 5, fileLen - 5); stm.close(); System.out.println("Flush 508 byte and closed the file " + p); // verify that entire file is good AppendTestUtil.checkFullFile( fs, p, fileLen, fileContents, "Failed to append to a partial chunk"); }
/** * Test that file data can be flushed. * * @throws IOException an exception might be thrown */ @Test public void testComplexFlush() throws IOException { Configuration conf = new HdfsConfiguration(); if (simulatedStorage) { conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true); } fileContents = AppendTestUtil.initBuffer(AppendTestUtil.FILE_SIZE); MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build(); FileSystem fs = cluster.getFileSystem(); try { // create a new file. Path file1 = new Path("/complexFlush.dat"); FSDataOutputStream stm = AppendTestUtil.createFile(fs, file1, 1); System.out.println("Created file complexFlush.dat"); int start = 0; for (start = 0; (start + 29) < AppendTestUtil.FILE_SIZE; ) { stm.write(fileContents, start, 29); stm.hflush(); start += 29; } stm.write(fileContents, start, AppendTestUtil.FILE_SIZE - start); // verify that full blocks are sane checkFile(fs, file1, 1); stm.close(); // verify that entire file is good AppendTestUtil.checkFullFile(fs, file1, AppendTestUtil.FILE_SIZE, fileContents, "Read 2"); } 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(); } }
// // verify that the data written to the full blocks are sane // private void checkFile(FileSystem fileSys, Path name, int repl) throws IOException { boolean done = false; // wait till all full blocks are confirmed by the datanodes. while (!done) { try { Thread.sleep(1000); } catch (InterruptedException e) {; } done = true; BlockLocation[] locations = fileSys.getFileBlockLocations(fileSys.getFileStatus(name), 0, AppendTestUtil.FILE_SIZE); if (locations.length < AppendTestUtil.NUM_BLOCKS) { System.out.println("Number of blocks found " + locations.length); done = false; continue; } for (int idx = 0; idx < AppendTestUtil.NUM_BLOCKS; idx++) { if (locations[idx].getHosts().length < repl) { System.out.println("Block index " + idx + " not yet replciated."); done = false; break; } } } byte[] expected = new byte[AppendTestUtil.NUM_BLOCKS * AppendTestUtil.BLOCK_SIZE]; if (simulatedStorage) { for (int i = 0; i < expected.length; i++) { expected[i] = SimulatedFSDataset.DEFAULT_DATABYTE; } } else { System.arraycopy(fileContents, 0, expected, 0, expected.length); } // do a sanity check. Read the file AppendTestUtil.checkFullFile( fileSys, name, AppendTestUtil.NUM_BLOCKS * AppendTestUtil.BLOCK_SIZE, expected, "Read 1"); }
/** * 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(); } }
// 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); } } }