Return-Path: X-Original-To: apmail-hadoop-hdfs-commits-archive@minotaur.apache.org Delivered-To: apmail-hadoop-hdfs-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 23C482A8D for ; Thu, 5 May 2011 05:41:13 +0000 (UTC) Received: (qmail 78265 invoked by uid 500); 5 May 2011 05:41:12 -0000 Delivered-To: apmail-hadoop-hdfs-commits-archive@hadoop.apache.org Received: (qmail 76355 invoked by uid 500); 5 May 2011 05:41:10 -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 75851 invoked by uid 99); 5 May 2011 05:41:07 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 05 May 2011 05:41:07 +0000 X-ASF-Spam-Status: No, hits=-1998.0 required=5.0 tests=ALL_TRUSTED,FB_GET_MEDS 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; Thu, 05 May 2011 05:41:04 +0000 Received: by eris.apache.org (Postfix, from userid 65534) id 787A92388C2A; Thu, 5 May 2011 05:40:17 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1099687 [15/15] - in /hadoop/hdfs/branches/HDFS-1073: ./ bin/ src/c++/libhdfs/ src/contrib/hdfsproxy/ src/contrib/hdfsproxy/src/java/org/apache/hadoop/hdfsproxy/ src/java/ src/java/org/apache/hadoop/hdfs/ src/java/org/apache/hadoop/hdfs/pr... Date: Thu, 05 May 2011 05:40:13 -0000 To: hdfs-commits@hadoop.apache.org From: todd@apache.org X-Mailer: svnmailer-1.0.8 Message-Id: <20110505054017.787A92388C2A@eris.apache.org> Modified: hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestHeartbeatHandling.java URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestHeartbeatHandling.java?rev=1099687&r1=1099686&r2=1099687&view=diff ============================================================================== --- hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestHeartbeatHandling.java (original) +++ hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestHeartbeatHandling.java Thu May 5 05:40:07 2011 @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hdfs.server.namenode; +import java.io.IOException; import java.util.ArrayList; import junit.framework.TestCase; @@ -27,6 +28,7 @@ import org.apache.hadoop.hdfs.HdfsConfig import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.server.common.GenerationStamp; +import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils; import org.apache.hadoop.hdfs.server.protocol.BlockCommand; import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand; import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol; @@ -37,9 +39,10 @@ import org.apache.hadoop.hdfs.server.pro */ public class TestHeartbeatHandling extends TestCase { /** - * Test if {@link FSNamesystem#handleHeartbeat(DatanodeRegistration, long, long, long, int, int)} - * can pick up replication and/or invalidate requests and - * observes the max limit + * Test if + * {@link FSNamesystem#handleHeartbeat(DatanodeRegistration, long, long, long, long, int, int)} + * can pick up replication and/or invalidate requests and observes the max + * limit */ public void testHeartbeat() throws Exception { final Configuration conf = new HdfsConfiguration(); @@ -47,11 +50,15 @@ public class TestHeartbeatHandling exten try { cluster.waitActive(); final FSNamesystem namesystem = cluster.getNamesystem(); - final DatanodeRegistration nodeReg = cluster.getDataNodes().get(0).dnRegistration; + final String poolId = namesystem.getBlockPoolId(); + final DatanodeRegistration nodeReg = + DataNodeTestUtils.getDNRegistrationForBP(cluster.getDataNodes().get(0), poolId); + DatanodeDescriptor dd = namesystem.getDatanode(nodeReg); final int REMAINING_BLOCKS = 1; - final int MAX_REPLICATE_LIMIT = conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 2); + final int MAX_REPLICATE_LIMIT = + conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 2); final int MAX_INVALIDATE_LIMIT = DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_DEFAULT; final int MAX_INVALIDATE_BLOCKS = 2*MAX_INVALIDATE_LIMIT+REMAINING_BLOCKS; final int MAX_REPLICATE_BLOCKS = 2*MAX_REPLICATE_LIMIT+REMAINING_BLOCKS; @@ -64,43 +71,36 @@ public class TestHeartbeatHandling exten dd.addBlockToBeReplicated( new Block(i, 0, GenerationStamp.FIRST_VALID_STAMP), ONE_TARGET); } - - DatanodeCommand[] cmds = namesystem.handleHeartbeat( - nodeReg, dd.getCapacity(), dd.getDfsUsed(), dd.getRemaining(), 0, 0); + DatanodeCommand[]cmds = sendHeartBeat(nodeReg, dd, namesystem); assertEquals(1, cmds.length); assertEquals(DatanodeProtocol.DNA_TRANSFER, cmds[0].getAction()); assertEquals(MAX_REPLICATE_LIMIT, ((BlockCommand)cmds[0]).getBlocks().length); - + ArrayList blockList = new ArrayList(MAX_INVALIDATE_BLOCKS); for (int i=0; i 0)); @@ -162,8 +163,9 @@ public class TestListCorruptFileBlocks { + " corrupt files. Expecting None.", badFiles.size() == 0); // Now deliberately corrupt one block - File data_dir = new File(System.getProperty("test.build.data"), - "dfs/data/data1/current/finalized"); + File storageDir = MiniDFSCluster.getStorageDir(0, 0); + File data_dir = MiniDFSCluster.getFinalizedDir(storageDir, + cluster.getNamesystem().getBlockPoolId()); assertTrue("data directory does not exist", data_dir.exists()); File[] blocks = data_dir.listFiles(); assertTrue("Blocks do not exist in data-dir", (blocks != null) && @@ -207,7 +209,7 @@ public class TestListCorruptFileBlocks { badFiles.size() == 1); // restart namenode - cluster.restartNameNode(); + cluster.restartNameNode(0); fs = cluster.getFileSystem(); // wait until replication queues have been initialized @@ -279,23 +281,24 @@ public class TestListCorruptFileBlocks { int numCorrupt = corruptFileBlocks.size(); assertTrue(numCorrupt == 0); // delete the blocks - File baseDir = new File(System.getProperty("test.build.data", - "build/test/data"), "dfs/data"); - for (int i = 0; i < 8; i++) { - File data_dir = new File(baseDir, "data" + (i + 1) - + MiniDFSCluster.FINALIZED_DIR_NAME); - File[] blocks = data_dir.listFiles(); - if (blocks == null) - continue; - // assertTrue("Blocks do not exist in data-dir", (blocks != null) && - // (blocks.length > 0)); - for (int idx = 0; idx < blocks.length; idx++) { - if (!blocks[idx].getName().startsWith("blk_")) { + String bpid = cluster.getNamesystem().getBlockPoolId(); + for (int i = 0; i < 4; i++) { + for (int j = 0; j <= 1; j++) { + File storageDir = MiniDFSCluster.getStorageDir(i, j); + File data_dir = MiniDFSCluster.getFinalizedDir(storageDir, bpid); + File[] blocks = data_dir.listFiles(); + if (blocks == null) continue; + // assertTrue("Blocks do not exist in data-dir", (blocks != null) && + // (blocks.length > 0)); + for (int idx = 0; idx < blocks.length; idx++) { + if (!blocks[idx].getName().startsWith("blk_")) { + continue; + } + LOG.info("Deliberately removing file " + blocks[idx].getName()); + assertTrue("Cannot remove file.", blocks[idx].delete()); + // break; } - LOG.info("Deliberately removing file " + blocks[idx].getName()); - assertTrue("Cannot remove file.", blocks[idx].delete()); - // break; } } @@ -380,17 +383,16 @@ public class TestListCorruptFileBlocks { DFSTestUtil util = new DFSTestUtil("testGetCorruptFiles", 3, 1, 1024); util.createFiles(fs, "/corruptData"); - final NameNode namenode = cluster.getNameNode(); RemoteIterator corruptFileBlocks = dfs.listCorruptFileBlocks(new Path("/corruptData")); int numCorrupt = countPaths(corruptFileBlocks); assertTrue(numCorrupt == 0); // delete the blocks - File baseDir = new File(System.getProperty("test.build.data", - "build/test/data"), "dfs/data"); - for (int i = 0; i < 8; i++) { - File data_dir = new File(baseDir, "data" + (i + 1) - + MiniDFSCluster.FINALIZED_DIR_NAME); + String bpid = cluster.getNamesystem().getBlockPoolId(); + // For loop through number of datadirectories per datanode (2) + for (int i = 0; i < 2; i++) { + File storageDir = MiniDFSCluster.getStorageDir(0, i); + File data_dir = MiniDFSCluster.getFinalizedDir(storageDir, bpid); File[] blocks = data_dir.listFiles(); if (blocks == null) continue; @@ -461,19 +463,22 @@ public class TestListCorruptFileBlocks { badFiles.size() == 0); // Now deliberately blocks from all files - File baseDir = new File(System.getProperty("test.build.data", - "build/test/data"),"dfs/data"); - for (int i=0; i<8; i++) { - File data_dir = new File(baseDir, "data" +(i+1)+ MiniDFSCluster.FINALIZED_DIR_NAME); - File[] blocks = data_dir.listFiles(); - if (blocks == null) - continue; - - for (int idx = 0; idx < blocks.length; idx++) { - if (!blocks[idx].getName().startsWith("blk_")) { + final String bpid = cluster.getNamesystem().getBlockPoolId(); + for (int i=0; i<4; i++) { + for (int j=0; j<=1; j++) { + File storageDir = MiniDFSCluster.getStorageDir(i, j); + File data_dir = MiniDFSCluster.getFinalizedDir(storageDir, bpid); + LOG.info("Removing files from " + data_dir); + File[] blocks = data_dir.listFiles(); + if (blocks == null) continue; + + for (int idx = 0; idx < blocks.length; idx++) { + if (!blocks[idx].getName().startsWith("blk_")) { + continue; + } + assertTrue("Cannot remove file.", blocks[idx].delete()); } - assertTrue("Cannot remove file.", blocks[idx].delete()); } } Modified: hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNNThroughputBenchmark.java URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNNThroughputBenchmark.java?rev=1099687&r1=1099686&r2=1099687&view=diff ============================================================================== --- hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNNThroughputBenchmark.java (original) +++ hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNNThroughputBenchmark.java Thu May 5 05:40:07 2011 @@ -23,6 +23,7 @@ import org.apache.hadoop.conf.Configurat import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.HdfsConfiguration; +import org.apache.hadoop.test.GenericTestUtils; import org.junit.Test; public class TestNNThroughputBenchmark { @@ -35,7 +36,7 @@ public class TestNNThroughputBenchmark { Configuration conf = new HdfsConfiguration(); FileSystem.setDefaultUri(conf, "hdfs://localhost:" + 0); conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "0.0.0.0:0"); - NameNode.format(conf); + GenericTestUtils.formatNamenode(conf); String[] args = new String[] {"-op", "all"}; NNThroughputBenchmark.runBenchmark(conf, Arrays.asList(args)); } Modified: hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNameNodeJspHelper.java URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNameNodeJspHelper.java?rev=1099687&r1=1099686&r2=1099687&view=diff ============================================================================== --- hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNameNodeJspHelper.java (original) +++ hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNameNodeJspHelper.java Thu May 5 05:40:07 2011 @@ -56,8 +56,9 @@ public class TestNameNodeJspHelper { public void testDelegationToken() throws IOException, InterruptedException { NameNode nn = cluster.getNameNode(); HttpServletRequest request = mock(HttpServletRequest.class); - String tokenString = NamenodeJspHelper - .getDelegationToken(nn, request, conf); + UserGroupInformation ugi = UserGroupInformation.createRemoteUser("auser"); + String tokenString = NamenodeJspHelper.getDelegationToken(nn, request, + conf, ugi); //tokenString returned must be null because security is disabled Assert.assertEquals(null, tokenString); } Modified: hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java?rev=1099687&r1=1099686&r2=1099687&view=diff ============================================================================== --- hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java (original) +++ hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java Thu May 5 05:40:07 2011 @@ -46,6 +46,13 @@ public class TestNameNodeMXBean { MBeanServer mbs = ManagementFactory.getPlatformMBeanServer(); ObjectName mxbeanName = new ObjectName("HadoopInfo:type=NameNodeInfo"); + // get attribute "ClusterId" + String clusterId = (String) mbs.getAttribute(mxbeanName, "ClusterId"); + Assert.assertEquals(fsn.getClusterId(), clusterId); + // get attribute "BlockPoolId" + String blockpoolId = (String) mbs.getAttribute(mxbeanName, + "BlockPoolId"); + Assert.assertEquals(fsn.getBlockPoolId(), blockpoolId); // get attribute "Version" String version = (String) mbs.getAttribute(mxbeanName, "Version"); Assert.assertEquals(fsn.getVersion(), version); Modified: hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java?rev=1099687&r1=1099686&r2=1099687&view=diff ============================================================================== --- hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java (original) +++ hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java Thu May 5 05:40:07 2011 @@ -23,6 +23,7 @@ import java.util.ArrayList; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.DF; +import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; @@ -66,8 +67,8 @@ public class TestNamenodeCapacityReport assertTrue(live.size() == 1); - long used, remaining, configCapacity, nonDFSUsed; - float percentUsed, percentRemaining; + long used, remaining, configCapacity, nonDFSUsed, bpUsed; + float percentUsed, percentRemaining, percentBpUsed; for (final DatanodeDescriptor datanode : live) { used = datanode.getDfsUsed(); @@ -76,6 +77,8 @@ public class TestNamenodeCapacityReport configCapacity = datanode.getCapacity(); percentUsed = datanode.getDfsUsedPercent(); percentRemaining = datanode.getRemainingPercent(); + bpUsed = datanode.getBlockPoolUsed(); + percentBpUsed = datanode.getBlockPoolUsedPercent(); LOG.info("Datanode configCapacity " + configCapacity + " used " + used + " non DFS used " + nonDFSUsed @@ -83,8 +86,11 @@ public class TestNamenodeCapacityReport + " percentRemaining " + percentRemaining); assertTrue(configCapacity == (used + remaining + nonDFSUsed)); - assertTrue(percentUsed == ((100.0f * (float)used)/(float)configCapacity)); - assertTrue(percentRemaining == ((100.0f * (float)remaining)/(float)configCapacity)); + assertTrue(percentUsed == DFSUtil.getPercentUsed(used, configCapacity)); + assertTrue(percentRemaining == DFSUtil.getPercentRemaining(remaining, + configCapacity)); + assertTrue(percentBpUsed == DFSUtil.getPercentUsed(bpUsed, + configCapacity)); } DF df = new DF(new File(cluster.getDataDirectory()), conf); @@ -110,6 +116,8 @@ public class TestNamenodeCapacityReport remaining = namesystem.getCapacityRemaining(); percentUsed = namesystem.getCapacityUsedPercent(); percentRemaining = namesystem.getCapacityRemainingPercent(); + bpUsed = namesystem.getBlockPoolUsedSpace(); + percentBpUsed = namesystem.getPercentBlockPoolUsed(); LOG.info("Data node directory " + cluster.getDataDirectory()); @@ -117,7 +125,8 @@ public class TestNamenodeCapacityReport + configCapacity + " reserved " + reserved + " used " + used + " remaining " + remaining + " nonDFSUsed " + nonDFSUsed + " remaining " + remaining + " percentUsed " + percentUsed - + " percentRemaining " + percentRemaining); + + " percentRemaining " + percentRemaining + " bpUsed " + bpUsed + + " percentBpUsed " + percentBpUsed); // Ensure new total capacity reported excludes the reserved space assertTrue(configCapacity == diskCapacity - reserved); @@ -126,7 +135,10 @@ public class TestNamenodeCapacityReport assertTrue(configCapacity == (used + remaining + nonDFSUsed)); // Ensure percent used is calculated based on used and present capacity - assertTrue(percentUsed == ((float)used * 100.0f)/(float)configCapacity); + assertTrue(percentUsed == DFSUtil.getPercentUsed(used, configCapacity)); + + // Ensure percent used is calculated based on used and present capacity + assertTrue(percentBpUsed == DFSUtil.getPercentUsed(bpUsed, configCapacity)); // Ensure percent used is calculated based on used and present capacity assertTrue(percentRemaining == ((float)remaining * 100.0f)/(float)configCapacity); Modified: hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNodeCount.java URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNodeCount.java?rev=1099687&r1=1099686&r2=1099687&view=diff ============================================================================== --- hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNodeCount.java (original) +++ hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNodeCount.java Thu May 5 05:40:07 2011 @@ -29,6 +29,7 @@ import org.apache.hadoop.hdfs.HdfsConfig import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties; import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.NumberReplicas; import junit.framework.TestCase; @@ -61,7 +62,7 @@ public class TestNodeCount extends TestC final Path FILE_PATH = new Path("/testfile"); DFSTestUtil.createFile(fs, FILE_PATH, 1L, REPLICATION_FACTOR, 1L); DFSTestUtil.waitReplication(fs, FILE_PATH, REPLICATION_FACTOR); - Block block = DFSTestUtil.getFirstBlock(fs, FILE_PATH); + ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, FILE_PATH); // keep a copy of all datanode descriptor DatanodeDescriptor[] datanodes = @@ -95,12 +96,13 @@ public class TestNodeCount extends TestC // check if excessive replica is detected (transient) initializeTimeout(TIMEOUT); - while (countNodes(block, namesystem).excessReplicas() == 0) { + while (countNodes(block.getLocalBlock(), namesystem).excessReplicas() == 0) { checkTimeout("excess replicas not detected"); } // find out a non-excess node - Iterator iter = namesystem.blockManager.blocksMap.nodeIterator(block); + Iterator iter = namesystem.blockManager.blocksMap + .nodeIterator(block.getLocalBlock()); DatanodeDescriptor nonExcessDN = null; while (iter.hasNext()) { DatanodeDescriptor dn = iter.next(); @@ -128,7 +130,7 @@ public class TestNodeCount extends TestC // The block should be replicated initializeTimeout(TIMEOUT); - while (countNodes(block, namesystem).liveReplicas() != REPLICATION_FACTOR) { + while (countNodes(block.getLocalBlock(), namesystem).liveReplicas() != REPLICATION_FACTOR) { checkTimeout("live replica count not correct", 1000); } @@ -138,7 +140,7 @@ public class TestNodeCount extends TestC // check if excessive replica is detected (transient) initializeTimeout(TIMEOUT); - while (countNodes(block, namesystem).excessReplicas() != 2) { + while (countNodes(block.getLocalBlock(), namesystem).excessReplicas() != 2) { checkTimeout("excess replica count not equal to 2"); } Modified: hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestOverReplicatedBlocks.java URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestOverReplicatedBlocks.java?rev=1099687&r1=1099686&r2=1099687&view=diff ============================================================================== --- hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestOverReplicatedBlocks.java (original) +++ hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestOverReplicatedBlocks.java Thu May 5 05:40:07 2011 @@ -28,8 +28,9 @@ import org.apache.hadoop.hdfs.HdfsConfig import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.TestDatanodeBlockScanner; import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties; -import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.DatanodeID; +import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils; import org.apache.hadoop.hdfs.DFSConfigKeys; import junit.framework.TestCase; @@ -43,7 +44,9 @@ public class TestOverReplicatedBlocks ex public void testProcesOverReplicateBlock() throws IOException { Configuration conf = new HdfsConfiguration(); conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 1000L); - conf.set(DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY, Integer.toString(2)); + conf.set( + DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY, + Integer.toString(2)); MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build(); FileSystem fs = cluster.getFileSystem(); @@ -53,13 +56,14 @@ public class TestOverReplicatedBlocks ex DFSTestUtil.waitReplication(fs, fileName, (short)3); // corrupt the block on datanode 0 - Block block = DFSTestUtil.getFirstBlock(fs, fileName); - assertTrue(cluster.corruptReplica(block.getBlockName(), 0)); + ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, fileName); + assertTrue(TestDatanodeBlockScanner.corruptReplica(block, 0)); DataNodeProperties dnProps = cluster.stopDataNode(0); // remove block scanner log to trigger block scanning - File scanLog = new File(System.getProperty("test.build.data"), - "dfs/data/data1" + MiniDFSCluster.FINALIZED_DIR_NAME + - "dncp_block_verification.log.curr"); + File scanLog = new File(MiniDFSCluster.getFinalizedDir( + MiniDFSCluster.getStorageDir(0, 0), + cluster.getNamesystem().getBlockPoolId()).getParent().toString() + + "/../dncp_block_verification.log.prev"); //wait for one minute for deletion to succeed; for(int i=0; !scanLog.delete(); i++) { assertTrue("Could not delete log file in one minute", i < 60); @@ -72,17 +76,21 @@ public class TestOverReplicatedBlocks ex cluster.restartDataNode(dnProps); DFSTestUtil.waitReplication(fs, fileName, (short)2); + String blockPoolId = cluster.getNamesystem().getBlockPoolId(); final DatanodeID corruptDataNode = - cluster.getDataNodes().get(2).dnRegistration; + DataNodeTestUtils.getDNRegistrationForBP( + cluster.getDataNodes().get(2), blockPoolId); + final FSNamesystem namesystem = cluster.getNamesystem(); try { namesystem.writeLock(); synchronized (namesystem.heartbeats) { // set live datanode's remaining space to be 0 // so they will be chosen to be deleted when over-replication occurs + String corruptMachineName = corruptDataNode.getName(); for (DatanodeDescriptor datanode : namesystem.heartbeats) { - if (!corruptDataNode.equals(datanode)) { - datanode.updateHeartbeat(100L, 100L, 0L, 0); + if (!corruptMachineName.equals(datanode.getName())) { + datanode.updateHeartbeat(100L, 100L, 0L, 100L, 0, 0); } } @@ -91,7 +99,8 @@ public class TestOverReplicatedBlocks ex // corrupt one won't be chosen to be excess one // without 4910 the number of live replicas would be 0: block gets lost - assertEquals(1, namesystem.blockManager.countNodes(block).liveReplicas()); + assertEquals(1, namesystem.blockManager.countNodes(block.getLocalBlock()) + .liveReplicas()); } } finally { namesystem.writeUnlock(); Modified: hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestReplicationPolicy.java URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestReplicationPolicy.java?rev=1099687&r1=1099686&r2=1099687&view=diff ============================================================================== --- hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestReplicationPolicy.java (original) +++ hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestReplicationPolicy.java Thu May 5 05:40:07 2011 @@ -27,6 +27,7 @@ import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.net.NetworkTopology; import org.apache.hadoop.net.Node; +import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.FSConstants; @@ -60,7 +61,7 @@ public class TestReplicationPolicy exten try { FileSystem.setDefaultUri(CONF, "hdfs://localhost:0"); CONF.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "0.0.0.0:0"); - NameNode.format(CONF); + GenericTestUtils.formatNamenode(CONF); namenode = new NameNode(CONF); } catch (IOException e) { e.printStackTrace(); @@ -76,7 +77,7 @@ public class TestReplicationPolicy exten for(int i=0; i it = cluster.getNameDirs().iterator(); it.hasNext(); ) { + for (Iterator it = cluster.getNameDirs(0).iterator(); it.hasNext(); ) { File dir = new File(it.next().getPath()); System.out.println(dir); } Modified: hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStartup.java URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStartup.java?rev=1099687&r1=1099686&r2=1099687&view=diff ============================================================================== --- hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStartup.java (original) +++ hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStartup.java Thu May 5 05:40:07 2011 @@ -230,11 +230,13 @@ public class TestStartup extends TestCas sd = it.next(); if(sd.getStorageDirType().isOfType(NameNodeDirType.IMAGE)) { - File imf = img.getStorage().getStorageFile(sd, NameNodeFile.IMAGE); + img.getStorage(); + File imf = NNStorage.getStorageFile(sd, NameNodeFile.IMAGE); LOG.info("--image file " + imf.getAbsolutePath() + "; len = " + imf.length() + "; expected = " + expectedImgSize); assertEquals(expectedImgSize, imf.length()); } else if(sd.getStorageDirType().isOfType(NameNodeDirType.EDITS)) { - File edf = img.getStorage().getStorageFile(sd, NameNodeFile.EDITS); + img.getStorage(); + File edf = NNStorage.getStorageFile(sd, NameNodeFile.EDITS); LOG.info("-- edits file " + edf.getAbsolutePath() + "; len = " + edf.length() + "; expected = " + expectedEditsSize); assertEquals(expectedEditsSize, edf.length()); } else { Modified: hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestUnderReplicatedBlocks.java URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestUnderReplicatedBlocks.java?rev=1099687&r1=1099686&r2=1099687&view=diff ============================================================================== --- hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestUnderReplicatedBlocks.java (original) +++ hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestUnderReplicatedBlocks.java Thu May 5 05:40:07 2011 @@ -24,7 +24,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import junit.framework.TestCase; @@ -44,10 +44,10 @@ public class TestUnderReplicatedBlocks e // remove one replica from the blocksMap so block becomes under-replicated // but the block does not get put into the under-replicated blocks queue final FSNamesystem namesystem = cluster.getNamesystem(); - Block b = DFSTestUtil.getFirstBlock(fs, FILE_PATH); - DatanodeDescriptor dn = namesystem.blockManager.blocksMap.nodeIterator(b).next(); - namesystem.blockManager.addToInvalidates(b, dn); - namesystem.blockManager.blocksMap.removeNode(b, dn); + ExtendedBlock b = DFSTestUtil.getFirstBlock(fs, FILE_PATH); + DatanodeDescriptor dn = namesystem.blockManager.blocksMap.nodeIterator(b.getLocalBlock()).next(); + namesystem.blockManager.addToInvalidates(b.getLocalBlock(), dn); + namesystem.blockManager.blocksMap.removeNode(b.getLocalBlock(), dn); // increment this file's replication factor FsShell shell = new FsShell(conf); Modified: hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java?rev=1099687&r1=1099686&r2=1099687&view=diff ============================================================================== --- hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java (original) +++ hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java Thu May 5 05:40:07 2011 @@ -128,7 +128,7 @@ public class TestOfflineImageViewer exte cluster.getNameNode().saveNamespace(); // Determine location of fsimage file - URI [] files = cluster.getNameDirs().toArray(new URI[0]); + URI [] files = cluster.getNameDirs(0).toArray(new URI[0]); orig = new File(files[0].getPath(), "current/fsimage"); if (!orig.exists()) { Modified: hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/test/GenericTestUtils.java URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/test/GenericTestUtils.java?rev=1099687&r1=1099686&r2=1099687&view=diff ============================================================================== --- hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/test/GenericTestUtils.java (original) +++ hadoop/hdfs/branches/HDFS-1073/src/test/hdfs/org/apache/hadoop/test/GenericTestUtils.java Thu May 5 05:40:07 2011 @@ -17,6 +17,12 @@ */ package org.apache.hadoop.test; +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption; +import org.apache.hadoop.hdfs.server.namenode.NameNode; + /** * Test provides some very generic helpers which might be used across the tests */ @@ -28,4 +34,17 @@ public abstract class GenericTestUtils { public static String getMethodName() { return Thread.currentThread().getStackTrace()[2].getMethodName(); } + + /** + * when formating a namenode - we must provide clusterid. + * @param conf + * @throws IOException + */ + public static void formatNamenode(Configuration conf) throws IOException { + String clusterId = StartupOption.FORMAT.getClusterId(); + if(clusterId == null || clusterId.isEmpty()) + StartupOption.FORMAT.setClusterId("testClusterID"); + + NameNode.format(conf); + } } Modified: hadoop/hdfs/branches/HDFS-1073/src/test/unit/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1073/src/test/unit/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java?rev=1099687&r1=1099686&r2=1099687&view=diff ============================================================================== --- hadoop/hdfs/branches/HDFS-1073/src/test/unit/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java (original) +++ hadoop/hdfs/branches/HDFS-1073/src/test/unit/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java Thu May 5 05:40:07 2011 @@ -27,9 +27,9 @@ import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException; import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState; import org.apache.hadoop.hdfs.server.datanode.DataNode; @@ -70,12 +70,18 @@ public class TestBlockRecovery { private DataNode dn; private Configuration conf; private final static long RECOVERY_ID = 3000L; + private final static String CLUSTER_ID = "testClusterID"; + private final static String POOL_ID = "BP-TEST"; private final static long BLOCK_ID = 1000L; private final static long GEN_STAMP = 2000L; private final static long BLOCK_LEN = 3000L; private final static long REPLICA_LEN1 = 6000L; private final static long REPLICA_LEN2 = 5000L; - private final static Block block = new Block(BLOCK_ID, BLOCK_LEN, GEN_STAMP); + private final static ExtendedBlock block = new ExtendedBlock(POOL_ID, + BLOCK_ID, BLOCK_LEN, GEN_STAMP); + + private final NamespaceInfo nsifno = + new NamespaceInfo(1,CLUSTER_ID, POOL_ID, 2, 3); static { ((Log4JLogger)FSNamesystem.LOG).getLogger().setLevel(Level.ALL); @@ -100,11 +106,14 @@ public class TestBlockRecovery { dataDir.mkdirs(); dirs.add(dataDir); DatanodeProtocol namenode = mock(DatanodeProtocol.class); - when(namenode.versionRequest()).thenReturn(new NamespaceInfo(1, 1L, 1)); + when(namenode.versionRequest()).thenReturn(new NamespaceInfo + (1, CLUSTER_ID, POOL_ID, 1L, 1)); when(namenode.sendHeartbeat(any(DatanodeRegistration.class), anyLong(), - anyLong(), anyLong(), anyInt(), anyInt())).thenReturn( - new DatanodeCommand[0]); - dn = new DataNode(conf, dirs, namenode, null); + anyLong(), anyLong(), anyLong(), anyInt(), anyInt(), anyInt())) + .thenReturn(new DatanodeCommand[0]); + dn = new DataNode(conf, dirs, null); + + DataNodeTestUtils.setBPNamenodeByIndex(dn, nsifno, POOL_ID, namenode); } /** @@ -146,12 +155,12 @@ public class TestBlockRecovery { syncList.add(record1); syncList.add(record2); - when(dn1.updateReplicaUnderRecovery((Block)anyObject(), anyLong(), - anyLong())).thenReturn(new Block(block.getBlockId(), - expectLen, block.getGenerationStamp())); - when(dn2.updateReplicaUnderRecovery((Block)anyObject(), anyLong(), - anyLong())).thenReturn(new Block(block.getBlockId(), - expectLen, block.getGenerationStamp())); + when(dn1.updateReplicaUnderRecovery((ExtendedBlock)anyObject(), anyLong(), + anyLong())).thenReturn(new ExtendedBlock(block.getBlockPoolId(), + block.getBlockId(), expectLen, block.getGenerationStamp())); + when(dn2.updateReplicaUnderRecovery((ExtendedBlock)anyObject(), anyLong(), + anyLong())).thenReturn(new ExtendedBlock(block.getBlockPoolId(), + block.getBlockId(), expectLen, block.getGenerationStamp())); dn.syncBlock(rBlock, syncList); } @@ -344,10 +353,10 @@ public class TestBlockRecovery { verify(dn2).updateReplicaUnderRecovery(block, RECOVERY_ID, minLen); } - private Collection initRecoveringBlocks() { + private Collection initRecoveringBlocks() throws IOException { Collection blocks = new ArrayList(1); DatanodeInfo[] locs = new DatanodeInfo[] { - new DatanodeInfo(dn.dnRegistration), + new DatanodeInfo(dn.getDNRegistrationForBP(block.getBlockPoolId())), mock(DatanodeInfo.class) }; RecoveringBlock rBlock = new RecoveringBlock(block, locs, RECOVERY_ID); blocks.add(rBlock); @@ -410,14 +419,16 @@ public class TestBlockRecovery { initReplicaRecovery(any(RecoveringBlock.class)); Daemon d = spyDN.recoverBlocks(initRecoveringBlocks()); d.join(); - verify(dn.namenode).commitBlockSynchronization( + DatanodeProtocol dnP = dn.getBPNamenode(POOL_ID); + verify(dnP).commitBlockSynchronization( block, RECOVERY_ID, 0, true, true, DatanodeID.EMPTY_ARRAY); } - private List initBlockRecords(DataNode spyDN) { + private List initBlockRecords(DataNode spyDN) throws IOException { List blocks = new ArrayList(1); + DatanodeRegistration dnR = dn.getDNRegistrationForBP(block.getBlockPoolId()); BlockRecord blockRecord = new BlockRecord( - new DatanodeID(dn.dnRegistration), spyDN, + new DatanodeID(dnR), spyDN, new ReplicaRecoveryInfo(block.getBlockId(), block.getNumBytes(), block.getGenerationStamp(), ReplicaState.FINALIZED)); blocks.add(blockRecord); @@ -465,8 +476,9 @@ public class TestBlockRecovery { } catch (IOException e) { e.getMessage().startsWith("Cannot recover "); } - verify(dn.namenode, never()).commitBlockSynchronization( - any(Block.class), anyLong(), anyLong(), anyBoolean(), + DatanodeProtocol namenode = dn.getBPNamenode(POOL_ID); + verify(namenode, never()).commitBlockSynchronization( + any(ExtendedBlock.class), anyLong(), anyLong(), anyBoolean(), anyBoolean(), any(DatanodeID[].class)); } @@ -492,8 +504,9 @@ public class TestBlockRecovery { } catch (IOException e) { e.getMessage().startsWith("Cannot recover "); } - verify(dn.namenode, never()).commitBlockSynchronization( - any(Block.class), anyLong(), anyLong(), anyBoolean(), + DatanodeProtocol namenode = dn.getBPNamenode(POOL_ID); + verify(namenode, never()).commitBlockSynchronization( + any(ExtendedBlock.class), anyLong(), anyLong(), anyBoolean(), anyBoolean(), any(DatanodeID[].class)); } finally { streams.close(); Modified: hadoop/hdfs/branches/HDFS-1073/src/test/unit/org/apache/hadoop/hdfs/server/namenode/TestNNLeaseRecovery.java URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1073/src/test/unit/org/apache/hadoop/hdfs/server/namenode/TestNNLeaseRecovery.java?rev=1099687&r1=1099686&r2=1099687&view=diff ============================================================================== --- hadoop/hdfs/branches/HDFS-1073/src/test/unit/org/apache/hadoop/hdfs/server/namenode/TestNNLeaseRecovery.java (original) +++ hadoop/hdfs/branches/HDFS-1073/src/test/unit/org/apache/hadoop/hdfs/server/namenode/TestNNLeaseRecovery.java Thu May 5 05:40:07 2011 @@ -75,7 +75,7 @@ public class TestNNLeaseRecovery { FileSystem.setDefaultUri(conf, "hdfs://localhost:0"); conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "0.0.0.0:0"); - NameNode.format(conf); + GenericTestUtils.formatNamenode(conf); fsn = spy(new FSNamesystem(conf)); } @@ -265,7 +265,7 @@ public class TestNNLeaseRecovery { BlockInfo lastBlock = fsn.dir.getFileINode(anyString()).getLastBlock(); try { - fsn.commitBlockSynchronization(lastBlock, + fsn.commitBlockSynchronization(fsn.getExtendedBlock(lastBlock), recoveryId, newSize, true, false, new DatanodeID[1]); } catch (IOException ioe) { assertTrue(ioe.getMessage().startsWith("Block (=")); @@ -293,7 +293,7 @@ public class TestNNLeaseRecovery { when(lastBlock.isComplete()).thenReturn(true); try { - fsn.commitBlockSynchronization(lastBlock, + fsn.commitBlockSynchronization(fsn.getExtendedBlock(lastBlock), recoveryId, newSize, true, false, new DatanodeID[1]); } catch (IOException ioe) { assertTrue(ioe.getMessage().startsWith("Unexpected block (=")); @@ -321,7 +321,7 @@ public class TestNNLeaseRecovery { when(((BlockInfoUnderConstruction)lastBlock).getBlockRecoveryId()).thenReturn(recoveryId-100); try { - fsn.commitBlockSynchronization(lastBlock, + fsn.commitBlockSynchronization(fsn.getExtendedBlock(lastBlock), recoveryId, newSize, true, false, new DatanodeID[1]); } catch (IOException ioe) { assertTrue(ioe.getMessage().startsWith("The recovery id " + recoveryId + " does not match current recovery id " + (recoveryId-100))); @@ -349,7 +349,7 @@ public class TestNNLeaseRecovery { when(((BlockInfoUnderConstruction)lastBlock).getBlockRecoveryId()).thenReturn(recoveryId+100); try { - fsn.commitBlockSynchronization(lastBlock, + fsn.commitBlockSynchronization(fsn.getExtendedBlock(lastBlock), recoveryId, newSize, true, false, new DatanodeID[1]); } catch (IOException ioe) { assertTrue(ioe.getMessage().startsWith("The recovery id " + recoveryId + " does not match current recovery id " + (recoveryId+100))); @@ -378,10 +378,11 @@ public class TestNNLeaseRecovery { boolean recoveryChecked = false; try { - fsn.commitBlockSynchronization(lastBlock, + fsn.commitBlockSynchronization(fsn.getExtendedBlock(lastBlock), 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-1073/src/webapps/datanode/ ------------------------------------------------------------------------------ --- svn:mergeinfo (original) +++ svn:mergeinfo Thu May 5 05:40:07 2011 @@ -1,5 +1,6 @@ /hadoop/core/branches/branch-0.19/hdfs/src/webapps/datanode:713112 /hadoop/core/trunk/src/webapps/datanode:776175-784663 +/hadoop/hdfs/branches/HDFS-1052/src/webapps/datanode:987665-1095512 /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:1086482-1097628 +/hadoop/hdfs/trunk/src/webapps/datanode:1086482-1099686 Propchange: hadoop/hdfs/branches/HDFS-1073/src/webapps/hdfs/ ------------------------------------------------------------------------------ --- svn:mergeinfo (original) +++ svn:mergeinfo Thu May 5 05:40:07 2011 @@ -1,5 +1,6 @@ /hadoop/core/branches/branch-0.19/hdfs/src/webapps/hdfs:713112 /hadoop/core/trunk/src/webapps/hdfs:776175-784663 +/hadoop/hdfs/branches/HDFS-1052/src/webapps/hdfs:987665-1095512 /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:1086482-1097628 +/hadoop/hdfs/trunk/src/webapps/hdfs:1086482-1099686 Modified: hadoop/hdfs/branches/HDFS-1073/src/webapps/hdfs/dfsnodelist.jsp URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1073/src/webapps/hdfs/dfsnodelist.jsp?rev=1099687&r1=1099686&r2=1099687&view=diff ============================================================================== --- hadoop/hdfs/branches/HDFS-1073/src/webapps/hdfs/dfsnodelist.jsp (original) +++ hadoop/hdfs/branches/HDFS-1073/src/webapps/hdfs/dfsnodelist.jsp Thu May 5 05:40:07 2011 @@ -46,7 +46,7 @@ String namenodeLabel = nn.getNameNodeAdd <%=namenodeRole%> Logs
Go back to DFS home
-<% nodelistjsp.generateNodesList(out, nn, request); %> +<% nodelistjsp.generateNodesList(application, out, request); %> <% out.println(ServletUtil.htmlFooter()); Modified: hadoop/hdfs/branches/HDFS-1073/src/webapps/hdfs/nn_browsedfscontent.jsp URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1073/src/webapps/hdfs/nn_browsedfscontent.jsp?rev=1099687&r1=1099686&r2=1099687&view=diff ============================================================================== --- hadoop/hdfs/branches/HDFS-1073/src/webapps/hdfs/nn_browsedfscontent.jsp (original) +++ hadoop/hdfs/branches/HDFS-1073/src/webapps/hdfs/nn_browsedfscontent.jsp Thu May 5 05:40:07 2011 @@ -33,10 +33,7 @@ <% - NameNode nn = (NameNode)application.getAttribute("name.node"); - Configuration conf = - (Configuration) application.getAttribute(JspHelper.CURRENT_CONF); - NamenodeJspHelper.redirectToRandomDataNode(nn, request, response, conf); + NamenodeJspHelper.redirectToRandomDataNode(application, request, response); %>
Propchange: hadoop/hdfs/branches/HDFS-1073/src/webapps/secondary/ ------------------------------------------------------------------------------ --- svn:mergeinfo (original) +++ svn:mergeinfo Thu May 5 05:40:07 2011 @@ -1,5 +1,6 @@ /hadoop/core/branches/branch-0.19/hdfs/src/webapps/secondary:713112 /hadoop/core/trunk/src/webapps/secondary:776175-784663 +/hadoop/hdfs/branches/HDFS-1052/src/webapps/secondary:987665-1095512 /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:1086482-1097628 +/hadoop/hdfs/trunk/src/webapps/secondary:1086482-1099686