Return-Path: Delivered-To: apmail-hadoop-hdfs-commits-archive@minotaur.apache.org Received: (qmail 11632 invoked from network); 31 Aug 2010 22:10:03 -0000 Received: from unknown (HELO mail.apache.org) (140.211.11.3) by 140.211.11.9 with SMTP; 31 Aug 2010 22:10:03 -0000 Received: (qmail 13165 invoked by uid 500); 31 Aug 2010 22:10:03 -0000 Delivered-To: apmail-hadoop-hdfs-commits-archive@hadoop.apache.org Received: (qmail 13142 invoked by uid 500); 31 Aug 2010 22:10:03 -0000 Mailing-List: contact hdfs-commits-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hdfs-dev@hadoop.apache.org Delivered-To: mailing list hdfs-commits@hadoop.apache.org Received: (qmail 13134 invoked by uid 99); 31 Aug 2010 22:10:03 -0000 Received: from Unknown (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 31 Aug 2010 22:10:03 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=10.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 31 Aug 2010 22:09:43 +0000 Received: by eris.apache.org (Postfix, from userid 65534) id CF1502388AB8; Tue, 31 Aug 2010 22:08:23 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r991360 [4/4] - in /hadoop/hdfs/branches/HDFS-1052: ./ src/c++/libhdfs/ src/contrib/hdfsproxy/ src/contrib/hdfsproxy/src/java/org/apache/hadoop/hdfsproxy/ src/contrib/thriftfs/src/java/org/apache/hadoop/thriftfs/ src/java/ src/java/org/apac... Date: Tue, 31 Aug 2010 22:08:22 -0000 To: hdfs-commits@hadoop.apache.org From: suresh@apache.org X-Mailer: svnmailer-1.0.8 Message-Id: <20100831220823.CF1502388AB8@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestPipelines.java URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestPipelines.java?rev=991360&r1=991359&r2=991360&view=diff ============================================================================== --- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestPipelines.java (original) +++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestPipelines.java Tue Aug 31 22:08:21 2010 @@ -87,11 +87,15 @@ public class TestPipelines { @Test public void pipeline_01() throws IOException { final String METHOD_NAME = GenericTestUtils.getMethodName(); - LOG.debug("Running " + METHOD_NAME); + if(LOG.isDebugEnabled()) { + LOG.debug("Running " + METHOD_NAME); + } Path filePath = new Path("/" + METHOD_NAME + ".dat"); DFSTestUtil.createFile(fs, filePath, FILE_SIZE, REPL_FACTOR, rand.nextLong()); - LOG.debug("Invoking append but doing nothing otherwise..."); + if(LOG.isDebugEnabled()) { + LOG.debug("Invoking append but doing nothing otherwise..."); + } FSDataOutputStream ofs = fs.append(filePath); ofs.writeBytes("Some more stuff to write"); ((DFSOutputStream) ofs.getWrappedStream()).hflush(); @@ -130,7 +134,9 @@ public class TestPipelines { System.arraycopy(toWrite, 0, ret, (ret.length - bytesToWrite), bytesToWriteNext); written += bytesToWriteNext; - LOG.debug("Written: " + bytesToWriteNext + "; Total: " + written); + if(LOG.isDebugEnabled()) { + LOG.debug("Written: " + bytesToWriteNext + "; Total: " + written); + } bytesToWrite -= bytesToWriteNext; } return ret; Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestBlockReport.java URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestBlockReport.java?rev=991360&r1=991359&r2=991360&view=diff ============================================================================== --- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestBlockReport.java (original) +++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestBlockReport.java Tue Aug 31 22:08:21 2010 @@ -108,19 +108,27 @@ public class TestBlockReport { ArrayList blocks = prepareForRide(filePath, METHOD_NAME, FILE_SIZE); - LOG.debug("Number of blocks allocated " + blocks.size()); + if(LOG.isDebugEnabled()) { + LOG.debug("Number of blocks allocated " + blocks.size()); + } long[] oldLengths = new long[blocks.size()]; int tempLen; for (int i = 0; i < blocks.size(); i++) { Block b = blocks.get(i); - LOG.debug("Block " + b.getBlockName() + " before\t" + "Size " + - b.getNumBytes()); + if(LOG.isDebugEnabled()) { + LOG.debug("Block " + b.getBlockName() + " before\t" + "Size " + + b.getNumBytes()); + } oldLengths[i] = b.getNumBytes(); - LOG.debug("Setting new length"); + if(LOG.isDebugEnabled()) { + LOG.debug("Setting new length"); + } tempLen = rand.nextInt(BLOCK_SIZE); b.set(b.getBlockId(), tempLen, b.getGenerationStamp()); - LOG.debug("Block " + b.getBlockName() + " after\t " + "Size " + - b.getNumBytes()); + if(LOG.isDebugEnabled()) { + LOG.debug("Block " + b.getBlockName() + " after\t " + "Size " + + b.getNumBytes()); + } } cluster.getNameNode().blockReport( cluster.getDataNodes().get(DN_N0).dnRegistration, @@ -129,8 +137,10 @@ public class TestBlockReport { List blocksAfterReport = DFSTestUtil.getAllBlocks(fs.open(filePath)); - LOG.debug("After mods: Number of blocks allocated " + - blocksAfterReport.size()); + if(LOG.isDebugEnabled()) { + LOG.debug("After mods: Number of blocks allocated " + + blocksAfterReport.size()); + } for (int i = 0; i < blocksAfterReport.size(); i++) { Block b = blocksAfterReport.get(i).getBlock(); @@ -178,10 +188,14 @@ public class TestBlockReport { } ArrayList blocks = locatedToBlocks(lBlocks, removedIndex); - LOG.debug("Number of blocks allocated " + lBlocks.size()); + if(LOG.isDebugEnabled()) { + LOG.debug("Number of blocks allocated " + lBlocks.size()); + } for (Block b : blocks2Remove) { - LOG.debug("Removing the block " + b.getBlockName()); + if(LOG.isDebugEnabled()) { + LOG.debug("Removing the block " + b.getBlockName()); + } for (File f : findAllFiles(dataDir, new MyFileFilter(b.getBlockName(), true))) { cluster.getDataNodes().get(DN_N0).getFSDataset().unfinalizeBlock(b); @@ -231,7 +245,9 @@ public class TestBlockReport { cluster.getNameNode().blockReport( cluster.getDataNodes().get(DN_N0).dnRegistration, new BlockListAsLongs(blocks, null).getBlockListAsLongs()); - LOG.debug("Got the command: " + dnCmd); + if(LOG.isDebugEnabled()) { + LOG.debug("Got the command: " + dnCmd); + } printStats(); assertEquals("Wrong number of CorruptedReplica+PendingDeletion " + @@ -313,12 +329,15 @@ public class TestBlockReport { Block corruptedBlock = blocks.get(randIndex); String secondNode = cluster.getDataNodes().get(DN_N1). getDatanodeRegistration().getStorageID(); - LOG.debug("Working with " + secondNode); - LOG.debug("BlockGS before " + blocks.get(randIndex).getGenerationStamp()); + if(LOG.isDebugEnabled()) { + LOG.debug("Working with " + secondNode); + LOG.debug("BlockGS before " + blocks.get(randIndex).getGenerationStamp()); + } corruptBlockGS(corruptedBlock); - LOG.debug("BlockGS after " + blocks.get(randIndex).getGenerationStamp()); - - LOG.debug("Done corrupting GS of " + corruptedBlock.getBlockName()); + if(LOG.isDebugEnabled()) { + LOG.debug("BlockGS after " + blocks.get(randIndex).getGenerationStamp()); + LOG.debug("Done corrupting GS of " + corruptedBlock.getBlockName()); + } cluster.getNameNode().blockReport( cluster.getDataNodes().get(DN_N1).dnRegistration, new BlockListAsLongs(blocks, null).getBlockListAsLongs()); @@ -338,7 +357,9 @@ public class TestBlockReport { randIndex--; corruptedBlock = blocks.get(randIndex); corruptBlockLen(corruptedBlock); - LOG.debug("Done corrupting length of " + corruptedBlock.getBlockName()); + if(LOG.isDebugEnabled()) { + LOG.debug("Done corrupting length of " + corruptedBlock.getBlockName()); + } cluster.getNameNode().blockReport( cluster.getDataNodes().get(DN_N1).dnRegistration, new BlockListAsLongs(blocks, null).getBlockListAsLongs()); @@ -448,11 +469,15 @@ public class TestBlockReport { final boolean tooLongWait = false; final int TIMEOUT = 40000; - LOG.debug("Wait for datanode " + DN_N1 + " to appear"); + if(LOG.isDebugEnabled()) { + LOG.debug("Wait for datanode " + DN_N1 + " to appear"); + } while (cluster.getDataNodes().size() <= DN_N1) { waitTil(20); } - LOG.debug("Total number of DNs " + cluster.getDataNodes().size()); + if(LOG.isDebugEnabled()) { + LOG.debug("Total number of DNs " + cluster.getDataNodes().size()); + } // Look about specified DN for the replica of the block from 1st DN Replica r; r = ((FSDataset) cluster.getDataNodes().get(DN_N1).getFSDataset()). @@ -465,25 +490,33 @@ public class TestBlockReport { fetchReplicaInfo(bl.getBlockId()); long waiting_period = System.currentTimeMillis() - start; if (count++ % 10 == 0) - LOG.debug("Has been waiting for " + waiting_period + " ms."); + if(LOG.isDebugEnabled()) { + LOG.debug("Has been waiting for " + waiting_period + " ms."); + } if (waiting_period > TIMEOUT) assertTrue("Was waiting too long to get ReplicaInfo from a datanode", tooLongWait); } HdfsConstants.ReplicaState state = r.getState(); - LOG.debug("Replica state before the loop " + state.getValue()); + if(LOG.isDebugEnabled()) { + LOG.debug("Replica state before the loop " + state.getValue()); + } start = System.currentTimeMillis(); while (state != HdfsConstants.ReplicaState.TEMPORARY) { waitTil(100); state = r.getState(); - LOG.debug("Keep waiting for " + bl.getBlockName() + - " is in state " + state.getValue()); + if(LOG.isDebugEnabled()) { + LOG.debug("Keep waiting for " + bl.getBlockName() + + " is in state " + state.getValue()); + } if (System.currentTimeMillis() - start > TIMEOUT) assertTrue("Was waiting too long for a replica to become TEMPORARY", tooLongWait); } - LOG.debug("Replica state after the loop " + state.getValue()); + if(LOG.isDebugEnabled()) { + LOG.debug("Replica state after the loop " + state.getValue()); + } } // Helper methods from here below... @@ -497,21 +530,27 @@ public class TestBlockReport { REPL_FACTOR = 2; blocks = prepareForRide(filePath, METHOD_NAME, fileSize); } catch (IOException e) { - LOG.debug("Caught exception ", e); + if(LOG.isDebugEnabled()) { + LOG.debug("Caught exception ", e); + } } return blocks; } private void startDNandWait(Path filePath, boolean waitReplicas) throws IOException { - LOG.debug("Before next DN start: " + cluster.getDataNodes().size()); + if(LOG.isDebugEnabled()) { + LOG.debug("Before next DN start: " + cluster.getDataNodes().size()); + } cluster.startDataNodes(conf, 1, true, null, null); ArrayList datanodes = cluster.getDataNodes(); assertEquals(datanodes.size(), 2); - LOG.debug("New datanode " - + cluster.getDataNodes().get(datanodes.size() - 1) - .getDatanodeRegistration() + " has been started"); + if(LOG.isDebugEnabled()) { + LOG.debug("New datanode " + + cluster.getDataNodes().get(datanodes.size() - 1) + .getDatanodeRegistration() + " has been started"); + } if (waitReplicas) DFSTestUtil.waitReplication(fs, filePath, REPL_FACTOR); } @@ -530,16 +569,18 @@ public class TestBlockReport { private void printStats() { NameNodeAdapter.refreshBlockCounts(cluster.getNameNode()); - LOG.debug("Missing " + cluster.getNamesystem().getMissingBlocksCount()); - LOG.debug("Corrupted " + cluster.getNamesystem().getCorruptReplicaBlocks()); - LOG.debug("Under-replicated " + cluster.getNamesystem(). - getUnderReplicatedBlocks()); - LOG.debug("Pending delete " + cluster.getNamesystem(). - getPendingDeletionBlocks()); - LOG.debug("Pending replications " + cluster.getNamesystem(). - getPendingReplicationBlocks()); - LOG.debug("Excess " + cluster.getNamesystem().getExcessBlocks()); - LOG.debug("Total " + cluster.getNamesystem().getBlocksTotal()); + if(LOG.isDebugEnabled()) { + LOG.debug("Missing " + cluster.getNamesystem().getMissingBlocksCount()); + LOG.debug("Corrupted " + cluster.getNamesystem().getCorruptReplicaBlocks()); + LOG.debug("Under-replicated " + cluster.getNamesystem(). + getUnderReplicatedBlocks()); + LOG.debug("Pending delete " + cluster.getNamesystem(). + getPendingDeletionBlocks()); + LOG.debug("Pending replications " + cluster.getNamesystem(). + getPendingReplicationBlocks()); + LOG.debug("Excess " + cluster.getNamesystem().getExcessBlocks()); + LOG.debug("Total " + cluster.getNamesystem().getBlocksTotal()); + } } private ArrayList locatedToBlocks(final List locatedBlks, @@ -547,7 +588,9 @@ public class TestBlockReport { ArrayList newList = new ArrayList(); for (int i = 0; i < locatedBlks.size(); i++) { if (positionsToRemove != null && positionsToRemove.contains(i)) { - LOG.debug(i + " block to be omitted"); + if(LOG.isDebugEnabled()) { + LOG.debug(i + " block to be omitted"); + } continue; } newList.add(new Block(locatedBlks.get(i).getBlock())); @@ -611,8 +654,10 @@ public class TestBlockReport { assertTrue("Old and new length shouldn't be the same", block.getNumBytes() != newLen); block.setNumBytes(newLen); - LOG.debug("Length of " + block.getBlockName() + - " is changed to " + newLen + " from " + oldLen); + if(LOG.isDebugEnabled()) { + LOG.debug("Length of " + block.getBlockName() + + " is changed to " + newLen + " from " + oldLen); + } } private void corruptBlockGS(final Block block) @@ -625,8 +670,10 @@ public class TestBlockReport { assertTrue("Old and new GS shouldn't be the same", block.getGenerationStamp() != newGS); block.setGenerationStamp(newGS); - LOG.debug("Generation stamp of " + block.getBlockName() + - " is changed to " + block.getGenerationStamp() + " from " + oldGS); + if(LOG.isDebugEnabled()) { + LOG.debug("Generation stamp of " + block.getBlockName() + + " is changed to " + block.getGenerationStamp() + " from " + oldGS); + } } private Block findBlock(Path path, long size) throws IOException { Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java?rev=991360&r1=991359&r2=991360&view=diff ============================================================================== --- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java (original) +++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java Tue Aug 31 22:08:21 2010 @@ -770,14 +770,18 @@ public class NNThroughputBenchmark { dnRegistration, DF_CAPACITY, DF_USED, DF_CAPACITY - DF_USED, 0, 0); if(cmds != null) { for (DatanodeCommand cmd : cmds ) { - LOG.debug("sendHeartbeat Name-node reply: " + cmd.getAction()); + if(LOG.isDebugEnabled()) { + LOG.debug("sendHeartbeat Name-node reply: " + cmd.getAction()); + } } } } boolean addBlock(Block blk) { if(nrBlocks == blocks.size()) { - LOG.debug("Cannot add block: datanode capacity = " + blocks.size()); + if(LOG.isDebugEnabled()) { + LOG.debug("Cannot add block: datanode capacity = " + blocks.size()); + } return false; } blocks.set(nrBlocks, blk); Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestLargeDirectoryDelete.java URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestLargeDirectoryDelete.java?rev=991360&r1=991359&r2=991360&view=diff ============================================================================== --- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestLargeDirectoryDelete.java (original) +++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestLargeDirectoryDelete.java Tue Aug 31 22:08:21 2010 @@ -198,7 +198,9 @@ public class TestLargeDirectoryDelete { try { wait(); } catch (InterruptedException e) { - LOG.debug("Ignoring " + e, e); + if(LOG.isDebugEnabled()) { + LOG.debug("Ignoring " + e, e); + } } } } Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNameNodeJspHelper.java URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNameNodeJspHelper.java?rev=991360&r1=991359&r2=991360&view=diff ============================================================================== --- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNameNodeJspHelper.java (original) +++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNameNodeJspHelper.java Tue Aug 31 22:08:21 2010 @@ -24,12 +24,13 @@ import java.io.IOException; import javax.servlet.http.HttpServletRequest; -import junit.framework.Assert; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.security.UserGroupInformation; import org.junit.After; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -60,5 +61,20 @@ public class TestNameNodeJspHelper { //tokenString returned must be null because security is disabled Assert.assertEquals(null, tokenString); } - + + @Test + public void tesSecurityModeText() { + conf.set(DFSConfigKeys.HADOOP_SECURITY_AUTHENTICATION, "kerberos"); + UserGroupInformation.setConfiguration(conf); + String securityOnOff = NamenodeJspHelper.getSecurityModeText(); + Assert.assertTrue("security mode doesn't match. Should be ON", + securityOnOff.contains("ON")); + //Security is enabled + conf.set(DFSConfigKeys.HADOOP_SECURITY_AUTHENTICATION, "simple"); + UserGroupInformation.setConfiguration(conf); + + securityOnOff = NamenodeJspHelper.getSecurityModeText(); + Assert.assertTrue("security mode doesn't match. Should be OFF", + securityOnOff.contains("OFF")); + } } Modified: hadoop/hdfs/branches/HDFS-1052/src/test/unit/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/unit/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java?rev=991360&r1=991359&r2=991360&view=diff ============================================================================== --- hadoop/hdfs/branches/HDFS-1052/src/test/unit/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java (original) +++ hadoop/hdfs/branches/HDFS-1052/src/test/unit/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java Tue Aug 31 22:08:21 2010 @@ -151,7 +151,9 @@ public class TestBlockRecovery { */ @Test public void testFinalizedReplicas () throws IOException { - LOG.debug("Running " + GenericTestUtils.getMethodName()); + if(LOG.isDebugEnabled()) { + LOG.debug("Running " + GenericTestUtils.getMethodName()); + } ReplicaRecoveryInfo replica1 = new ReplicaRecoveryInfo(BLOCK_ID, REPLICA_LEN1, GEN_STAMP-1, ReplicaState.FINALIZED); ReplicaRecoveryInfo replica2 = new ReplicaRecoveryInfo(BLOCK_ID, @@ -186,7 +188,9 @@ public class TestBlockRecovery { */ @Test public void testFinalizedRbwReplicas() throws IOException { - LOG.debug("Running " + GenericTestUtils.getMethodName()); + if(LOG.isDebugEnabled()) { + LOG.debug("Running " + GenericTestUtils.getMethodName()); + } // rbw and finalized replicas have the same length ReplicaRecoveryInfo replica1 = new ReplicaRecoveryInfo(BLOCK_ID, @@ -223,7 +227,9 @@ public class TestBlockRecovery { */ @Test public void testFinalizedRwrReplicas() throws IOException { - LOG.debug("Running " + GenericTestUtils.getMethodName()); + if(LOG.isDebugEnabled()) { + LOG.debug("Running " + GenericTestUtils.getMethodName()); + } // rbw and finalized replicas have the same length ReplicaRecoveryInfo replica1 = new ReplicaRecoveryInfo(BLOCK_ID, @@ -261,7 +267,9 @@ public class TestBlockRecovery { */ @Test public void testRBWReplicas() throws IOException { - LOG.debug("Running " + GenericTestUtils.getMethodName()); + if(LOG.isDebugEnabled()) { + LOG.debug("Running " + GenericTestUtils.getMethodName()); + } ReplicaRecoveryInfo replica1 = new ReplicaRecoveryInfo(BLOCK_ID, REPLICA_LEN1, GEN_STAMP-1, ReplicaState.RBW); ReplicaRecoveryInfo replica2 = new ReplicaRecoveryInfo(BLOCK_ID, @@ -283,7 +291,9 @@ public class TestBlockRecovery { */ @Test public void testRBW_RWRReplicas() throws IOException { - LOG.debug("Running " + GenericTestUtils.getMethodName()); + if(LOG.isDebugEnabled()) { + LOG.debug("Running " + GenericTestUtils.getMethodName()); + } ReplicaRecoveryInfo replica1 = new ReplicaRecoveryInfo(BLOCK_ID, REPLICA_LEN1, GEN_STAMP-1, ReplicaState.RBW); ReplicaRecoveryInfo replica2 = new ReplicaRecoveryInfo(BLOCK_ID, @@ -305,7 +315,9 @@ public class TestBlockRecovery { */ @Test public void testRWRReplicas() throws IOException { - LOG.debug("Running " + GenericTestUtils.getMethodName()); + if(LOG.isDebugEnabled()) { + LOG.debug("Running " + GenericTestUtils.getMethodName()); + } ReplicaRecoveryInfo replica1 = new ReplicaRecoveryInfo(BLOCK_ID, REPLICA_LEN1, GEN_STAMP-1, ReplicaState.RWR); ReplicaRecoveryInfo replica2 = new ReplicaRecoveryInfo(BLOCK_ID, @@ -339,7 +351,9 @@ public class TestBlockRecovery { @Test public void testRecoveryInProgressException() throws IOException, InterruptedException { - LOG.debug("Running " + GenericTestUtils.getMethodName()); + if(LOG.isDebugEnabled()) { + LOG.debug("Running " + GenericTestUtils.getMethodName()); + } DataNode spyDN = spy(dn); doThrow(new RecoveryInProgressException("Replica recovery is in progress")). when(spyDN).initReplicaRecovery(any(RecoveringBlock.class)); @@ -357,7 +371,9 @@ public class TestBlockRecovery { */ @Test public void testErrorReplicas() throws IOException, InterruptedException { - LOG.debug("Running " + GenericTestUtils.getMethodName()); + if(LOG.isDebugEnabled()) { + LOG.debug("Running " + GenericTestUtils.getMethodName()); + } DataNode spyDN = spy(dn); doThrow(new IOException()). when(spyDN).initReplicaRecovery(any(RecoveringBlock.class)); @@ -374,7 +390,9 @@ public class TestBlockRecovery { */ @Test public void testZeroLenReplicas() throws IOException, InterruptedException { - LOG.debug("Running " + GenericTestUtils.getMethodName()); + if(LOG.isDebugEnabled()) { + LOG.debug("Running " + GenericTestUtils.getMethodName()); + } DataNode spyDN = spy(dn); doReturn(new ReplicaRecoveryInfo(block.getBlockId(), 0, block.getGenerationStamp(), ReplicaState.FINALIZED)).when(spyDN). @@ -405,7 +423,9 @@ public class TestBlockRecovery { */ @Test public void testFailedReplicaUpdate() throws IOException { - LOG.debug("Running " + GenericTestUtils.getMethodName()); + if(LOG.isDebugEnabled()) { + LOG.debug("Running " + GenericTestUtils.getMethodName()); + } DataNode spyDN = spy(dn); doThrow(new IOException()).when(spyDN).updateReplicaUnderRecovery( block, RECOVERY_ID, block.getNumBytes()); @@ -424,7 +444,9 @@ public class TestBlockRecovery { */ @Test public void testNoReplicaUnderRecovery() throws IOException { - LOG.debug("Running " + GenericTestUtils.getMethodName()); + if(LOG.isDebugEnabled()) { + LOG.debug("Running " + GenericTestUtils.getMethodName()); + } dn.data.createRbw(block); try { dn.syncBlock(rBlock, initBlockRecords(dn)); @@ -444,7 +466,9 @@ public class TestBlockRecovery { */ @Test public void testNotMatchedReplicaID() throws IOException { - LOG.debug("Running " + GenericTestUtils.getMethodName()); + if(LOG.isDebugEnabled()) { + LOG.debug("Running " + GenericTestUtils.getMethodName()); + } ReplicaInPipelineInterface replicaInfo = dn.data.createRbw(block); BlockWriteStreams streams = null; try { Modified: hadoop/hdfs/branches/HDFS-1052/src/test/unit/org/apache/hadoop/hdfs/server/namenode/TestNNLeaseRecovery.java URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/unit/org/apache/hadoop/hdfs/server/namenode/TestNNLeaseRecovery.java?rev=991360&r1=991359&r2=991360&view=diff ============================================================================== --- hadoop/hdfs/branches/HDFS-1052/src/test/unit/org/apache/hadoop/hdfs/server/namenode/TestNNLeaseRecovery.java (original) +++ hadoop/hdfs/branches/HDFS-1052/src/test/unit/org/apache/hadoop/hdfs/server/namenode/TestNNLeaseRecovery.java Tue Aug 31 22:08:21 2010 @@ -105,7 +105,9 @@ public class TestNNLeaseRecovery { */ @Test public void testInternalReleaseLease_allCOMPLETE () throws IOException { - LOG.debug("Running " + GenericTestUtils.getMethodName()); + if(LOG.isDebugEnabled()) { + LOG.debug("Running " + GenericTestUtils.getMethodName()); + } LeaseManager.Lease lm = mock(LeaseManager.Lease.class); Path file = spy(new Path("/test.dat")); DatanodeDescriptor dnd = mock(DatanodeDescriptor.class); @@ -127,7 +129,9 @@ public class TestNNLeaseRecovery { */ @Test(expected=IOException.class) public void testInternalReleaseLease_UNKNOWN_COMM () throws IOException { - LOG.debug("Running " + GenericTestUtils.getMethodName()); + if(LOG.isDebugEnabled()) { + LOG.debug("Running " + GenericTestUtils.getMethodName()); + } LeaseManager.Lease lm = mock(LeaseManager.Lease.class); Path file = spy(new Path("/" + GenericTestUtils.getMethodName() + "_test.dat")); @@ -151,7 +155,9 @@ public class TestNNLeaseRecovery { */ @Test(expected=AlreadyBeingCreatedException.class) public void testInternalReleaseLease_COMM_COMM () throws IOException { - LOG.debug("Running " + GenericTestUtils.getMethodName()); + if(LOG.isDebugEnabled()) { + LOG.debug("Running " + GenericTestUtils.getMethodName()); + } LeaseManager.Lease lm = mock(LeaseManager.Lease.class); Path file = spy(new Path("/" + GenericTestUtils.getMethodName() + "_test.dat")); @@ -174,7 +180,9 @@ public class TestNNLeaseRecovery { */ @Test public void testInternalReleaseLease_0blocks () throws IOException { - LOG.debug("Running " + GenericTestUtils.getMethodName()); + if(LOG.isDebugEnabled()) { + LOG.debug("Running " + GenericTestUtils.getMethodName()); + } LeaseManager.Lease lm = mock(LeaseManager.Lease.class); Path file = spy(new Path("/" + GenericTestUtils.getMethodName() + "_test.dat")); @@ -196,7 +204,9 @@ public class TestNNLeaseRecovery { */ @Test(expected=AlreadyBeingCreatedException.class) public void testInternalReleaseLease_1blocks () throws IOException { - LOG.debug("Running " + GenericTestUtils.getMethodName()); + if(LOG.isDebugEnabled()) { + LOG.debug("Running " + GenericTestUtils.getMethodName()); + } LeaseManager.Lease lm = mock(LeaseManager.Lease.class); Path file = spy(new Path("/" + GenericTestUtils.getMethodName() + "_test.dat")); @@ -219,7 +229,9 @@ public class TestNNLeaseRecovery { */ @Test public void testInternalReleaseLease_COMM_CONSTRUCTION () throws IOException { - LOG.debug("Running " + GenericTestUtils.getMethodName()); + if(LOG.isDebugEnabled()) { + LOG.debug("Running " + GenericTestUtils.getMethodName()); + } LeaseManager.Lease lm = mock(LeaseManager.Lease.class); Path file = spy(new Path("/" + GenericTestUtils.getMethodName() + "_test.dat")); @@ -237,7 +249,9 @@ public class TestNNLeaseRecovery { @Test public void testCommitBlockSynchronization_BlockNotFound () throws IOException { - LOG.debug("Running " + GenericTestUtils.getMethodName()); + if(LOG.isDebugEnabled()) { + LOG.debug("Running " + GenericTestUtils.getMethodName()); + } long recoveryId = 2002; long newSize = 273487234; Path file = @@ -261,7 +275,9 @@ public class TestNNLeaseRecovery { @Test public void testCommitBlockSynchronization_notUR () throws IOException { - LOG.debug("Running " + GenericTestUtils.getMethodName()); + if(LOG.isDebugEnabled()) { + LOG.debug("Running " + GenericTestUtils.getMethodName()); + } long recoveryId = 2002; long newSize = 273487234; Path file = @@ -287,7 +303,9 @@ public class TestNNLeaseRecovery { @Test public void testCommitBlockSynchronization_WrongGreaterRecoveryID() throws IOException { - LOG.debug("Running " + GenericTestUtils.getMethodName()); + if(LOG.isDebugEnabled()) { + LOG.debug("Running " + GenericTestUtils.getMethodName()); + } long recoveryId = 2002; long newSize = 273487234; Path file = @@ -313,7 +331,9 @@ public class TestNNLeaseRecovery { @Test public void testCommitBlockSynchronization_WrongLesserRecoveryID() throws IOException { - LOG.debug("Running " + GenericTestUtils.getMethodName()); + if(LOG.isDebugEnabled()) { + LOG.debug("Running " + GenericTestUtils.getMethodName()); + } long recoveryId = 2002; long newSize = 273487234; Path file = @@ -339,7 +359,9 @@ public class TestNNLeaseRecovery { @Test public void testCommitBlockSynchronization_EqualRecoveryID() throws IOException { - LOG.debug("Running " + GenericTestUtils.getMethodName()); + if(LOG.isDebugEnabled()) { + LOG.debug("Running " + GenericTestUtils.getMethodName()); + } long recoveryId = 2002; long newSize = 273487234; Path file = @@ -360,6 +382,7 @@ public class TestNNLeaseRecovery { recoveryId, newSize, true, false, new DatanodeID[1]); } catch (NullPointerException ioe) { // It is fine to get NPE here because the datanodes array is empty + LOG.info("Exception ", ioe); recoveryChecked = true; } assertTrue("commitBlockSynchronization had to throw NPE here", recoveryChecked); Propchange: hadoop/hdfs/branches/HDFS-1052/src/webapps/datanode/ ------------------------------------------------------------------------------ --- svn:mergeinfo (original) +++ svn:mergeinfo Tue Aug 31 22:08:21 2010 @@ -2,4 +2,4 @@ /hadoop/core/trunk/src/webapps/datanode:776175-784663 /hadoop/hdfs/branches/HDFS-265/src/webapps/datanode:796829-820463 /hadoop/hdfs/branches/branch-0.21/src/webapps/datanode:820487 -/hadoop/hdfs/trunk/src/webapps/datanode:987665-990214 +/hadoop/hdfs/trunk/src/webapps/datanode:987665-991358 Propchange: hadoop/hdfs/branches/HDFS-1052/src/webapps/hdfs/ ------------------------------------------------------------------------------ --- svn:mergeinfo (original) +++ svn:mergeinfo Tue Aug 31 22:08:21 2010 @@ -2,4 +2,4 @@ /hadoop/core/trunk/src/webapps/hdfs:776175-784663 /hadoop/hdfs/branches/HDFS-265/src/webapps/hdfs:796829-820463 /hadoop/hdfs/branches/branch-0.21/src/webapps/hdfs:820487 -/hadoop/hdfs/trunk/src/webapps/hdfs:987665-990214 +/hadoop/hdfs/trunk/src/webapps/hdfs:987665-991358 Modified: hadoop/hdfs/branches/HDFS-1052/src/webapps/hdfs/dfshealth.jsp URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/webapps/hdfs/dfshealth.jsp?rev=991360&r1=991359&r2=991360&view=diff ============================================================================== --- hadoop/hdfs/branches/HDFS-1052/src/webapps/hdfs/dfshealth.jsp (original) +++ hadoop/hdfs/branches/HDFS-1052/src/webapps/hdfs/dfshealth.jsp Tue Aug 31 22:08:21 2010 @@ -47,6 +47,7 @@

Cluster Summary

+ <%= NamenodeJspHelper.getSecurityModeText()%> <%= NamenodeJspHelper.getSafeModeText(fsn)%> <%= NamenodeJspHelper.getInodeLimitText(fsn)%> Propchange: hadoop/hdfs/branches/HDFS-1052/src/webapps/secondary/ ------------------------------------------------------------------------------ --- svn:mergeinfo (original) +++ svn:mergeinfo Tue Aug 31 22:08:21 2010 @@ -2,4 +2,4 @@ /hadoop/core/trunk/src/webapps/secondary:776175-784663 /hadoop/hdfs/branches/HDFS-265/src/webapps/secondary:796829-820463 /hadoop/hdfs/branches/branch-0.21/src/webapps/secondary:820487 -/hadoop/hdfs/trunk/src/webapps/secondary:987665-990214 +/hadoop/hdfs/trunk/src/webapps/secondary:987665-991358