/** * tests that the distributed block fixer obeys the limit on how many jobs to submit * simultaneously. */ @Test public void testMaxPendingJobs() throws Exception { LOG.info("Test testMaxPendingJobs started."); int stripeLength = 3; mySetup(stripeLength); long[] crcs1 = new long[3]; int[] seeds1 = new int[3]; long[] crcs2 = new long[3]; int[] seeds2 = new int[3]; Path dirPath1 = new Path("/user/dhruba/raidtestrs/1"); Path[] files1 = TestRaidDfs.createTestFiles( dirPath1, fileSizes, blockSizes, crcs1, seeds1, fileSys, (short) 1); Path dirPath2 = new Path("/user/dhruba/raidtestrs/2"); Path[] files2 = TestRaidDfs.createTestFiles( dirPath2, fileSizes, blockSizes, crcs2, seeds2, fileSys, (short) 1); Path destPath = new Path("/destraidrs/user/dhruba/raidtestrs"); LOG.info("Test testMaxPendingJobs created test files"); Configuration localConf = this.getRaidNodeConfig(conf, false); localConf.setLong("raid.blockfix.maxpendingjobs", 1L); try { cnode = RaidNode.createRaidNode(null, localConf); TestRaidDfs.waitForDirRaided(LOG, fileSys, dirPath1, destPath); TestRaidDfs.waitForDirRaided(LOG, fileSys, dirPath2, destPath); cnode.stop(); cnode.join(); DistributedFileSystem dfs = (DistributedFileSystem) fileSys; String[] corruptFiles = DFSUtil.getCorruptFiles(dfs); assertEquals("no corrupt files expected", 0, corruptFiles.length); assertEquals( "filesFixed() should return 0 before fixing files", 0, cnode.blockIntegrityMonitor.getNumFilesFixed()); // corrupt directory 1 this.corruptFiles( dirPath1, crcs1, rsCorruptFileIdx1, dfs, files1, rsNumCorruptBlocksInFiles1); cnode = RaidNode.createRaidNode(null, localConf); DistBlockIntegrityMonitor blockFixer = (DistBlockIntegrityMonitor) cnode.blockIntegrityMonitor; long start = System.currentTimeMillis(); while (blockFixer.jobsRunning() < 1 && System.currentTimeMillis() - start < 60000) { LOG.info("Test testDirBlockFix waiting for fixing job 1 to start"); Thread.sleep(1000); } assertEquals("job not running", 1, blockFixer.jobsRunning()); // corrupt directory 2 this.corruptFiles( dirPath2, crcs2, rsCorruptFileIdx2, dfs, files2, rsNumCorruptBlocksInFiles2); // wait until both files are fixed while (blockFixer.getNumFilesFixed() < 6 && System.currentTimeMillis() - start < 240000) { // make sure the block fixer does not start a second job while // the first one is still running assertTrue("too many jobs running", blockFixer.jobsRunning() <= 1); Thread.sleep(1000); } TestBlockFixer.verifyMetrics( fileSys, cnode, false, 6L, getTotal(rsNumCorruptBlocksInFiles1) + getTotal(rsNumCorruptBlocksInFiles2)); dfs = getDFS(conf, dfs); for (int i = 0; i < fileSizes.length; i++) { assertTrue( "file " + files1[i] + " not fixed", TestRaidDfs.validateFile(dfs, files1[i], fileSizes[i], crcs1[i])); } for (int i = 0; i < fileSizes.length; i++) { assertTrue( "file " + files2[i] + " not fixed", TestRaidDfs.validateFile(dfs, files2[i], fileSizes[i], crcs2[i])); } } catch (Exception e) { LOG.info("Test testMaxPendingJobs exception " + e + StringUtils.stringifyException(e)); throw e; } finally { myTearDown(); } }
/** tests that we can have 2 concurrent jobs fixing files (dist block fixer) */ @Test public void testConcurrentJobs() throws Exception { LOG.info("Test testConcurrentJobs started."); int stripeLength = 3; mySetup(stripeLength); long[] crcs1 = new long[3]; int[] seeds1 = new int[3]; long[] crcs2 = new long[3]; int[] seeds2 = new int[3]; Path dirPath1 = new Path("/user/dhruba/raidtestrs/1"); Path[] files1 = TestRaidDfs.createTestFiles( dirPath1, fileSizes, blockSizes, crcs1, seeds1, fileSys, (short) 1); Path dirPath2 = new Path("/user/dhruba/raidtestrs/2"); Path[] files2 = TestRaidDfs.createTestFiles( dirPath2, fileSizes, blockSizes, crcs2, seeds2, fileSys, (short) 1); Path destPath = new Path("/destraidrs/user/dhruba/raidtestrs"); LOG.info("Test testConcurrentJobs created test files"); Configuration localConf = this.getRaidNodeConfig(conf, false); localConf.setLong(BlockIntegrityMonitor.BLOCKCHECK_INTERVAL, 15000L); localConf.setLong(DistBlockIntegrityMonitor.RAIDNODE_BLOCK_FIX_SUBMISSION_INTERVAL_KEY, 15000L); localConf.setLong( DistBlockIntegrityMonitor.RAIDNODE_BLOCK_FIX_SCAN_SUBMISSION_INTERVAL_KEY, 3600000); try { cnode = RaidNode.createRaidNode(null, localConf); TestRaidDfs.waitForDirRaided(LOG, fileSys, dirPath1, destPath); TestRaidDfs.waitForDirRaided(LOG, fileSys, dirPath2, destPath); cnode.stop(); cnode.join(); DistributedFileSystem dfs = (DistributedFileSystem) fileSys; String[] corruptFiles = DFSUtil.getCorruptFiles(dfs); assertEquals("no corrupt files expected", 0, corruptFiles.length); assertEquals( "filesFixed() should return 0 before fixing files", 0, cnode.blockIntegrityMonitor.getNumFilesFixed()); // corrupt directory 1 this.corruptFiles( dirPath1, crcs1, rsCorruptFileIdx1, dfs, files1, rsNumCorruptBlocksInFiles1); cnode = RaidNode.createRaidNode(null, localConf); DistBlockIntegrityMonitor blockFixer = (DistBlockIntegrityMonitor) cnode.blockIntegrityMonitor; long start = System.currentTimeMillis(); // All files are HIGH-PRI corrupt files while (blockFixer.jobsRunning() < 1 && System.currentTimeMillis() - start < 60000) { LOG.info("Test testDirBlockFix waiting for fixing job 1 to start"); Thread.sleep(1000); } assertEquals("job 1 not running", 1, blockFixer.jobsRunning()); // Corrupt directory 2 this.corruptFiles( dirPath2, crcs2, rsCorruptFileIdx2, dfs, files2, rsNumCorruptBlocksInFiles2); // 1 LOW-PRI file and 2 HIGH-PRI files while (blockFixer.jobsRunning() < 3 && System.currentTimeMillis() - start < 60000) { LOG.info("Test testDirBlockFix waiting for fixing job 2 and 3 to start"); Thread.sleep(1000); } assertTrue("more than 3 jobs are running", blockFixer.jobsRunning() >= 3); while (blockFixer.getNumFilesFixed() < 6 && System.currentTimeMillis() - start < 240000) { LOG.info("Test testDirBlockFix waiting for files to be fixed."); Thread.sleep(1000); } TestBlockFixer.verifyMetrics( fileSys, cnode, false, 6L, getTotal(rsNumCorruptBlocksInFiles1) + getTotal(rsNumCorruptBlocksInFiles2)); dfs = getDFS(conf, dfs); for (int i = 0; i < fileSizes.length; i++) { assertTrue( "file " + files1[i] + " not fixed", TestRaidDfs.validateFile(dfs, files1[i], fileSizes[i], crcs1[i])); } for (int i = 0; i < fileSizes.length; i++) { assertTrue( "file " + files2[i] + " not fixed", TestRaidDfs.validateFile(dfs, files2[i], fileSizes[i], crcs2[i])); } } catch (Exception e) { LOG.info("Test testConcurrentJobs exception " + e, e); throw e; } finally { myTearDown(); } }