You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by st...@apache.org on 2009/11/28 21:06:08 UTC

svn commit: r885143 [18/18] - in /hadoop/hdfs/branches/HDFS-326: ./ .eclipse.templates/ .eclipse.templates/.launches/ conf/ ivy/ lib/ src/ant/org/apache/hadoop/ant/ src/ant/org/apache/hadoop/ant/condition/ src/c++/ src/c++/libhdfs/ src/c++/libhdfs/docs...

Modified: hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java?rev=885143&r1=885142&r2=885143&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java (original)
+++ hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java Sat Nov 28 20:05:56 2009
@@ -25,6 +25,7 @@
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.permission.*;
 
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.EditLogFileInputStream;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
@@ -83,7 +84,7 @@
   public void testEditLog() throws IOException {
 
     // start a cluster 
-    Configuration conf = new Configuration();
+    Configuration conf = new HdfsConfiguration();
     MiniDFSCluster cluster = null;
     FileSystem fileSys = null;
 

Modified: hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestFileLimit.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestFileLimit.java?rev=885143&r1=885142&r2=885143&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestFileLimit.java (original)
+++ hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestFileLimit.java Sat Nov 28 20:05:56 2009
@@ -26,8 +26,10 @@
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 
 
 /**
@@ -73,9 +75,9 @@
    * Test that file data becomes available before file is closed.
    */
   public void testFileLimit() throws IOException {
-    Configuration conf = new Configuration();
+    Configuration conf = new HdfsConfiguration();
     int maxObjects = 5;
-    conf.setLong("dfs.max.objects", maxObjects);
+    conf.setLong(DFSConfigKeys.DFS_NAMENODE_MAX_OBJECTS_KEY, maxObjects);
     conf.setLong("dfs.blockreport.intervalMsec", 1000L);
     conf.setInt("dfs.heartbeat.interval", 1);
     int currentNodes = 0;

Modified: hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestFsck.java?rev=885143&r1=885142&r2=885143&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestFsck.java (original)
+++ hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestFsck.java Sat Nov 28 20:05:56 2009
@@ -37,6 +37,7 @@
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.tools.DFSck;
@@ -70,7 +71,7 @@
     MiniDFSCluster cluster = null;
     FileSystem fs = null;
     try {
-      Configuration conf = new Configuration();
+      Configuration conf = new HdfsConfiguration();
       conf.setLong("dfs.blockreport.intervalMsec", 10000L);
       cluster = new MiniDFSCluster(conf, 4, true, null);
       fs = cluster.getFileSystem();
@@ -106,7 +107,7 @@
     MiniDFSCluster cluster = null;
     FileSystem fs = null;
     try {
-      Configuration conf = new Configuration();
+      Configuration conf = new HdfsConfiguration();
       conf.setLong("dfs.blockreport.intervalMsec", 10000L);
       cluster = new MiniDFSCluster(conf, 4, true, null);
       fs = cluster.getFileSystem();
@@ -125,7 +126,7 @@
   /** Test fsck with permission set on inodes */
   public void testFsckPermission() throws Exception {
     final DFSTestUtil util = new DFSTestUtil(getClass().getSimpleName(), 20, 3, 8*1024);
-    final Configuration conf = new Configuration();
+    final Configuration conf = new HdfsConfiguration();
     conf.setLong("dfs.blockreport.intervalMsec", 10000L);
 
     MiniDFSCluster cluster = null;
@@ -160,7 +161,7 @@
     MiniDFSCluster cluster = null;
     FileSystem fs = null;
     try {
-      Configuration conf = new Configuration();
+      Configuration conf = new HdfsConfiguration();
       conf.setLong("dfs.blockreport.intervalMsec", 10000L);
       conf.setInt("dfs.datanode.directoryscan.interval", 1);
       cluster = new MiniDFSCluster(conf, 4, true, null);
@@ -182,7 +183,8 @@
       File baseDir = new File(System.getProperty("test.build.data",
                                                  "build/test/data"),"dfs/data");
       for (int i=0; i<8; i++) {
-        File blockFile = new File(baseDir, "data" +(i+1)+ "/current/" + block);
+        File blockFile = new File(baseDir, "data" +(i+1) + 
+            MiniDFSCluster.FINALIZED_DIR_NAME + block);
         if(blockFile.exists()) {
           assertTrue(blockFile.delete());
         }
@@ -219,7 +221,7 @@
     MiniDFSCluster cluster = null;
     FileSystem fs = null;
     try {
-      Configuration conf = new Configuration();
+      Configuration conf = new HdfsConfiguration();
       conf.setLong("dfs.blockreport.intervalMsec", 10000L);
       cluster = new MiniDFSCluster(conf, 4, true, null);
       String topDir = "/srcdat";
@@ -265,7 +267,7 @@
   }
 
   public void testCorruptBlock() throws Exception {
-    Configuration conf = new Configuration();
+    Configuration conf = new HdfsConfiguration();
     conf.setLong("dfs.blockreport.intervalMsec", 1000);
     FileSystem fs = null;
     DFSClient dfsClient = null;
@@ -294,8 +296,8 @@
     File baseDir = new File(System.getProperty("test.build.data",
                                                "build/test/data"),"dfs/data");
     for (int i=0; i < 6; i++) {
-      File blockFile = new File(baseDir, "data" + (i+1) + "/current/" +
-                                block);
+      File blockFile = new File(baseDir, "data" + (i+1) + 
+          MiniDFSCluster.FINALIZED_DIR_NAME + block);
       if (blockFile.exists()) {
         RandomAccessFile raFile = new RandomAccessFile(blockFile, "rw");
         FileChannel channel = raFile.getChannel();
@@ -348,7 +350,7 @@
     MiniDFSCluster cluster = null;
     try {
       // bring up a one-node cluster
-      Configuration conf = new Configuration();
+      Configuration conf = new HdfsConfiguration();
       cluster = new MiniDFSCluster(conf, 1, true, null);
       String fileName = "/test.txt";
       Path filePath = new Path(fileName);

Modified: hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestHeartbeatHandling.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestHeartbeatHandling.java?rev=885143&r1=885142&r2=885143&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestHeartbeatHandling.java (original)
+++ hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestHeartbeatHandling.java Sat Nov 28 20:05:56 2009
@@ -1,8 +1,26 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package org.apache.hadoop.hdfs.server.namenode;
 
 import java.util.ArrayList;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
@@ -10,6 +28,7 @@
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 
 import junit.framework.TestCase;
 
@@ -23,7 +42,7 @@
    * observes the max limit
    */
   public void testHeartbeat() throws Exception {
-    final Configuration conf = new Configuration();
+    final Configuration conf = new HdfsConfiguration();
     final MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
     try {
       cluster.waitActive();
@@ -32,7 +51,7 @@
       DatanodeDescriptor dd = namesystem.getDatanode(nodeReg);
       
       final int REMAINING_BLOCKS = 1;
-      final int MAX_REPLICATE_LIMIT = conf.getInt("dfs.max-repl-streams", 2);
+      final int MAX_REPLICATE_LIMIT = conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 2);
       final int MAX_INVALIDATE_LIMIT = FSNamesystem.BLOCK_INVALIDATE_CHUNK;
       final int MAX_INVALIDATE_BLOCKS = 2*MAX_INVALIDATE_LIMIT+REMAINING_BLOCKS;
       final int MAX_REPLICATE_BLOCKS = 2*MAX_REPLICATE_LIMIT+REMAINING_BLOCKS;

Modified: hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNNThroughputBenchmark.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNNThroughputBenchmark.java?rev=885143&r1=885142&r2=885143&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNNThroughputBenchmark.java (original)
+++ hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNNThroughputBenchmark.java Sat Nov 28 20:05:56 2009
@@ -23,6 +23,8 @@
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 
 public class TestNNThroughputBenchmark extends TestCase {
 
@@ -30,9 +32,9 @@
    * This test runs all benchmarks defined in {@link NNThroughputBenchmark}.
    */
   public void testNNThroughput() throws Exception {
-    Configuration conf = new Configuration();
+    Configuration conf = new HdfsConfiguration();
     FileSystem.setDefaultUri(conf, "hdfs://localhost:" + 0);
-    conf.set("dfs.http.address", "0.0.0.0:0");
+    conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
     NameNode.format(conf);
     String[] args = new String[] {"-op", "all"};
     NNThroughputBenchmark.runBenchmark(conf, Arrays.asList(args));

Modified: hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java?rev=885143&r1=885142&r2=885143&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java (original)
+++ hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java Sat Nov 28 20:05:56 2009
@@ -21,11 +21,13 @@
 import java.io.*;
 import java.util.Random;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 
 /**
  * This class tests various combinations of dfs.name.dir 
@@ -93,7 +95,7 @@
   @SuppressWarnings("deprecation")
   SecondaryNameNode startSecondaryNameNode(Configuration conf
                                           ) throws IOException {
-    conf.set("dfs.secondary.http.address", "0.0.0.0:0");
+    conf.set(DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY, "0.0.0.0:0");
     return new SecondaryNameNode(conf);
   }
 
@@ -126,11 +128,11 @@
     File checkpointNameAndEdits = new File(base_dir, "second_name_and_edits");
     
     // Start namenode with same dfs.name.dir and dfs.name.edits.dir
-    conf = new Configuration();
-    conf.set("dfs.name.dir", nameAndEdits.getPath());
-    conf.set("dfs.name.edits.dir", nameAndEdits.getPath());
-    conf.set("fs.checkpoint.dir", checkpointNameAndEdits.getPath());
-    conf.set("fs.checkpoint.edits.dir", checkpointNameAndEdits.getPath());
+    conf = new HdfsConfiguration();
+    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, nameAndEdits.getPath());
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, nameAndEdits.getPath());
+    conf.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY, checkpointNameAndEdits.getPath());
+    conf.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_EDITS_DIR_KEY, checkpointNameAndEdits.getPath());
     replication = (short)conf.getInt("dfs.replication", 3);
     // Manage our own dfs directories
     cluster = new MiniDFSCluster(0, conf, NUM_DATA_NODES, true, false, true, null,
@@ -151,17 +153,17 @@
     }
 
     // Start namenode with additional dfs.name.dir and dfs.name.edits.dir
-    conf =  new Configuration();
+    conf =  new HdfsConfiguration();
     assertTrue(newNameDir.mkdir());
     assertTrue(newEditsDir.mkdir());
 
-    conf.set("dfs.name.dir", nameAndEdits.getPath() +
+    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, nameAndEdits.getPath() +
               "," + newNameDir.getPath());
-    conf.set("dfs.name.edits.dir", nameAndEdits.getPath() + 
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, nameAndEdits.getPath() + 
              "," + newEditsDir.getPath());
-    conf.set("fs.checkpoint.dir", checkpointNameDir.getPath() +
+    conf.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY, checkpointNameDir.getPath() +
              "," + checkpointNameAndEdits.getPath());
-    conf.set("fs.checkpoint.edits.dir", checkpointEditsDir.getPath() +
+    conf.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_EDITS_DIR_KEY, checkpointEditsDir.getPath() +
              "," + checkpointNameAndEdits.getPath());
     replication = (short)conf.getInt("dfs.replication", 3);
     // Manage our own dfs directories. Do not format.
@@ -201,11 +203,11 @@
         new File(checkpointNameDir, FILE_EDITS));
     new File(checkpointNameAndEdits, FILE_IMAGE).renameTo(
         new File(checkpointEditsDir, FILE_IMAGE));
-    conf =  new Configuration();
-    conf.set("dfs.name.dir", newNameDir.getPath());
-    conf.set("dfs.name.edits.dir", newEditsDir.getPath());
-    conf.set("fs.checkpoint.dir", checkpointNameDir.getPath());
-    conf.set("fs.checkpoint.edits.dir", checkpointEditsDir.getPath());
+    conf =  new HdfsConfiguration();
+    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, newNameDir.getPath());
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, newEditsDir.getPath());
+    conf.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY, checkpointNameDir.getPath());
+    conf.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_EDITS_DIR_KEY, checkpointEditsDir.getPath());
     replication = (short)conf.getInt("dfs.replication", 3);
     cluster = new MiniDFSCluster(0, conf, NUM_DATA_NODES, false, false, true,
                                   null, null, null, null);
@@ -236,14 +238,14 @@
     // from old dir
     assertTrue(FileUtil.fullyDelete(new File(nameAndEdits, "current")));
     assertTrue(FileUtil.fullyDelete(new File(checkpointNameAndEdits, "current")));
-    conf = new Configuration();
-    conf.set("dfs.name.dir", nameAndEdits.getPath() +
+    conf = new HdfsConfiguration();
+    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, nameAndEdits.getPath() +
               "," + newNameDir.getPath());
-    conf.set("dfs.name.edits.dir", nameAndEdits +
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, nameAndEdits +
               "," + newEditsDir.getPath());
-    conf.set("fs.checkpoint.dir", checkpointNameDir.getPath() +
+    conf.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY, checkpointNameDir.getPath() +
         "," + checkpointNameAndEdits.getPath());
-    conf.set("fs.checkpoint.edits.dir", checkpointEditsDir.getPath() +
+    conf.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_EDITS_DIR_KEY, checkpointEditsDir.getPath() +
         "," + checkpointNameAndEdits.getPath());
     replication = (short)conf.getInt("dfs.replication", 3);
     cluster = new MiniDFSCluster(0, conf, NUM_DATA_NODES, false, false, true,
@@ -290,9 +292,9 @@
     File nameAndEdits = new File(base_dir, "name_and_edits");
     
     // Start namenode with same dfs.name.dir and dfs.name.edits.dir
-    conf = new Configuration();
-    conf.set("dfs.name.dir", nameAndEdits.getPath());
-    conf.set("dfs.name.edits.dir", nameAndEdits.getPath());
+    conf = new HdfsConfiguration();
+    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, nameAndEdits.getPath());
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, nameAndEdits.getPath());
     replication = (short)conf.getInt("dfs.replication", 3);
     // Manage our own dfs directories
     cluster = new MiniDFSCluster(0, conf, NUM_DATA_NODES, true, false, true, null,
@@ -310,13 +312,13 @@
     }
 
     // Start namenode with additional dfs.name.dir and dfs.name.edits.dir
-    conf =  new Configuration();
+    conf =  new HdfsConfiguration();
     assertTrue(newNameDir.mkdir());
     assertTrue(newEditsDir.mkdir());
 
-    conf.set("dfs.name.dir", nameAndEdits.getPath() +
+    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, nameAndEdits.getPath() +
               "," + newNameDir.getPath());
-    conf.set("dfs.name.edits.dir", nameAndEdits.getPath() +
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, nameAndEdits.getPath() +
               "," + newEditsDir.getPath());
     replication = (short)conf.getInt("dfs.replication", 3);
     // Manage our own dfs directories. Do not format.
@@ -338,9 +340,9 @@
     
     // Now remove common directory both have and start namenode with 
     // separate name and edits dirs
-    conf =  new Configuration();
-    conf.set("dfs.name.dir", newNameDir.getPath());
-    conf.set("dfs.name.edits.dir", newEditsDir.getPath());
+    conf =  new HdfsConfiguration();
+    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, newNameDir.getPath());
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, newEditsDir.getPath());
     replication = (short)conf.getInt("dfs.replication", 3);
     cluster = new MiniDFSCluster(0, conf, NUM_DATA_NODES, false, false, true,
                                   null, null, null, null);
@@ -360,10 +362,10 @@
     }
 
     // Add old shared directory for name and edits along with latest name
-    conf = new Configuration();
-    conf.set("dfs.name.dir", newNameDir.getPath() + "," + 
+    conf = new HdfsConfiguration();
+    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, newNameDir.getPath() + "," + 
              nameAndEdits.getPath());
-    conf.set("dfs.name.edits.dir", nameAndEdits.getPath());
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, nameAndEdits.getPath());
     replication = (short)conf.getInt("dfs.replication", 3);
     try {
       cluster = new MiniDFSCluster(0, conf, NUM_DATA_NODES, false, false, true,
@@ -377,9 +379,9 @@
     }
 
     // Add old shared directory for name and edits along with latest edits
-    conf = new Configuration();
-    conf.set("dfs.name.dir", nameAndEdits.getPath());
-    conf.set("dfs.name.edits.dir", newEditsDir.getPath() +
+    conf = new HdfsConfiguration();
+    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, nameAndEdits.getPath());
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, newEditsDir.getPath() +
              "," + nameAndEdits.getPath());
     replication = (short)conf.getInt("dfs.replication", 3);
     try {

Modified: hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java?rev=885143&r1=885142&r2=885143&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java (original)
+++ hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java Sat Nov 28 20:05:56 2009
@@ -23,6 +23,7 @@
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.DF;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@@ -44,7 +45,7 @@
    * Then, it updates the block with new information and verifies again. 
    */
   public void testVolumeSize() throws Exception {
-    Configuration conf = new Configuration();
+    Configuration conf = new HdfsConfiguration();
     MiniDFSCluster cluster = null;
 
     // Set aside fifth of the total capacity as reserved

Modified: hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNodeCount.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNodeCount.java?rev=885143&r1=885142&r2=885143&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNodeCount.java (original)
+++ hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNodeCount.java Sat Nov 28 20:05:56 2009
@@ -1,3 +1,20 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package org.apache.hadoop.hdfs.server.namenode;
 
 import java.util.Collection;
@@ -7,6 +24,7 @@
 import org.apache.hadoop.fs.FileSystem;
 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.MiniDFSCluster.DataNodeProperties;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -21,7 +39,7 @@
 public class TestNodeCount extends TestCase {
   public void testNodeCount() throws Exception {
     // start a mini dfs cluster of 2 nodes
-    final Configuration conf = new Configuration();
+    final Configuration conf = new HdfsConfiguration();
     final short REPLICATION_FACTOR = (short)2;
     final MiniDFSCluster cluster = 
       new MiniDFSCluster(conf, REPLICATION_FACTOR, true, null);

Modified: hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestOverReplicatedBlocks.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestOverReplicatedBlocks.java?rev=885143&r1=885142&r2=885143&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestOverReplicatedBlocks.java (original)
+++ hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestOverReplicatedBlocks.java Sat Nov 28 20:05:56 2009
@@ -24,11 +24,13 @@
 import org.apache.hadoop.fs.FileSystem;
 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.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.DFSConfigKeys;
 
 import junit.framework.TestCase;
 
@@ -39,9 +41,9 @@
    * corrupt ones.
    */
   public void testProcesOverReplicateBlock() throws IOException {
-    Configuration conf = new Configuration();
+    Configuration conf = new HdfsConfiguration();
     conf.setLong("dfs.blockreport.intervalMsec", 1000L);
-    conf.set("dfs.replication.pending.timeout.sec", Integer.toString(2));
+    conf.set(DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY, Integer.toString(2));
     MiniDFSCluster cluster = new MiniDFSCluster(conf, 3, true, null);
     FileSystem fs = cluster.getFileSystem();
 
@@ -56,7 +58,8 @@
       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/current/dncp_block_verification.log.curr");
+          "dfs/data/data1" + MiniDFSCluster.FINALIZED_DIR_NAME + 
+          "dncp_block_verification.log.curr");
       //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);

Modified: hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestReplicationPolicy.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestReplicationPolicy.java?rev=885143&r1=885142&r2=885143&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestReplicationPolicy.java (original)
+++ hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestReplicationPolicy.java Sat Nov 28 20:05:56 2009
@@ -29,16 +29,19 @@
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 
 import junit.framework.TestCase;
 
 public class TestReplicationPolicy extends TestCase {
   private static final int BLOCK_SIZE = 1024;
   private static final int NUM_OF_DATANODES = 6;
-  private static final Configuration CONF = new Configuration();
+  private static final Configuration CONF = new HdfsConfiguration();
   private static final NetworkTopology cluster;
   private static final NameNode namenode;
-  private static final ReplicationTargetChooser replicator;
+  private static final BlockPlacementPolicy replicator;
+  private static final String filename = "/dummyfile.txt";
   private static final DatanodeDescriptor dataNodes[] = 
     new DatanodeDescriptor[] {
       new DatanodeDescriptor(new DatanodeID("h1:5020"), "/d1/r1"),
@@ -55,7 +58,7 @@
   static {
     try {
       FileSystem.setDefaultUri(CONF, "hdfs://localhost:0");
-      CONF.set("dfs.http.address", "0.0.0.0:0");
+      CONF.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
       NameNode.format(CONF);
       namenode = new NameNode(CONF);
       NameNode.startService(namenode);
@@ -92,30 +95,30 @@
         FSConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 4); // overloaded
 
     DatanodeDescriptor[] targets;
-    targets = replicator.chooseTarget(
-                                      0, dataNodes[0], null, BLOCK_SIZE);
+    targets = replicator.chooseTarget(filename,
+                                      0, dataNodes[0], BLOCK_SIZE);
     assertEquals(targets.length, 0);
     
-    targets = replicator.chooseTarget(
-                                      1, dataNodes[0], null, BLOCK_SIZE);
+    targets = replicator.chooseTarget(filename,
+                                      1, dataNodes[0], BLOCK_SIZE);
     assertEquals(targets.length, 1);
     assertEquals(targets[0], dataNodes[0]);
     
-    targets = replicator.chooseTarget(
-                                      2, dataNodes[0], null, BLOCK_SIZE);
+    targets = replicator.chooseTarget(filename,
+                                      2, dataNodes[0], BLOCK_SIZE);
     assertEquals(targets.length, 2);
     assertEquals(targets[0], dataNodes[0]);
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     
-    targets = replicator.chooseTarget(
-                                      3, dataNodes[0], null, BLOCK_SIZE);
+    targets = replicator.chooseTarget(filename,
+                                      3, dataNodes[0], BLOCK_SIZE);
     assertEquals(targets.length, 3);
     assertEquals(targets[0], dataNodes[0]);
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     assertTrue(cluster.isOnSameRack(targets[1], targets[2]));
 
-    targets = replicator.chooseTarget(
-                                     4, dataNodes[0], null, BLOCK_SIZE);
+    targets = replicator.chooseTarget(filename,
+                                     4, dataNodes[0], BLOCK_SIZE);
     assertEquals(targets.length, 4);
     assertEquals(targets[0], dataNodes[0]);
     assertTrue(cluster.isOnSameRack(targets[1], targets[2]) ||
@@ -138,41 +141,47 @@
   public void testChooseTarget2() throws Exception { 
     HashMap<Node, Node> excludedNodes;
     DatanodeDescriptor[] targets;
+    BlockPlacementPolicyDefault repl = (BlockPlacementPolicyDefault)replicator;
+    List<DatanodeDescriptor> chosenNodes = new ArrayList<DatanodeDescriptor>();
     
     excludedNodes = new HashMap<Node, Node>();
     excludedNodes.put(dataNodes[1], dataNodes[1]); 
-    targets = replicator.chooseTarget(
-                                      0, dataNodes[0], excludedNodes, BLOCK_SIZE);
+    targets = repl.chooseTarget(
+                                0, dataNodes[0], chosenNodes, excludedNodes, BLOCK_SIZE);
     assertEquals(targets.length, 0);
     
     excludedNodes.clear();
+    chosenNodes.clear();
     excludedNodes.put(dataNodes[1], dataNodes[1]); 
-    targets = replicator.chooseTarget(
-                                      1, dataNodes[0], excludedNodes, BLOCK_SIZE);
+    targets = repl.chooseTarget(
+                                1, dataNodes[0], chosenNodes, excludedNodes, BLOCK_SIZE);
     assertEquals(targets.length, 1);
     assertEquals(targets[0], dataNodes[0]);
     
     excludedNodes.clear();
+    chosenNodes.clear();
     excludedNodes.put(dataNodes[1], dataNodes[1]); 
-    targets = replicator.chooseTarget(
-                                      2, dataNodes[0], excludedNodes, BLOCK_SIZE);
+    targets = repl.chooseTarget(
+                                2, dataNodes[0], chosenNodes, excludedNodes, BLOCK_SIZE);
     assertEquals(targets.length, 2);
     assertEquals(targets[0], dataNodes[0]);
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     
     excludedNodes.clear();
+    chosenNodes.clear();
     excludedNodes.put(dataNodes[1], dataNodes[1]); 
-    targets = replicator.chooseTarget(
-                                      3, dataNodes[0], excludedNodes, BLOCK_SIZE);
+    targets = repl.chooseTarget(
+                                3, dataNodes[0], chosenNodes, excludedNodes, BLOCK_SIZE);
     assertEquals(targets.length, 3);
     assertEquals(targets[0], dataNodes[0]);
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     assertTrue(cluster.isOnSameRack(targets[1], targets[2]));
     
     excludedNodes.clear();
+    chosenNodes.clear();
     excludedNodes.put(dataNodes[1], dataNodes[1]); 
-    targets = replicator.chooseTarget(
-                                      4, dataNodes[0], excludedNodes, BLOCK_SIZE);
+    targets = repl.chooseTarget(
+                                4, dataNodes[0], chosenNodes, excludedNodes, BLOCK_SIZE);
     assertEquals(targets.length, 4);
     assertEquals(targets[0], dataNodes[0]);
     for(int i=1; i<4; i++) {
@@ -198,30 +207,30 @@
         (FSConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0); // no space
         
     DatanodeDescriptor[] targets;
-    targets = replicator.chooseTarget(
-                                      0, dataNodes[0], null, BLOCK_SIZE);
+    targets = replicator.chooseTarget(filename,
+                                      0, dataNodes[0], BLOCK_SIZE);
     assertEquals(targets.length, 0);
     
-    targets = replicator.chooseTarget(
-                                      1, dataNodes[0], null, BLOCK_SIZE);
+    targets = replicator.chooseTarget(filename,
+                                      1, dataNodes[0], BLOCK_SIZE);
     assertEquals(targets.length, 1);
     assertEquals(targets[0], dataNodes[1]);
     
-    targets = replicator.chooseTarget(
-                                      2, dataNodes[0], null, BLOCK_SIZE);
+    targets = replicator.chooseTarget(filename,
+                                      2, dataNodes[0], BLOCK_SIZE);
     assertEquals(targets.length, 2);
     assertEquals(targets[0], dataNodes[1]);
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     
-    targets = replicator.chooseTarget(
-                                      3, dataNodes[0], null, BLOCK_SIZE);
+    targets = replicator.chooseTarget(filename,
+                                      3, dataNodes[0], BLOCK_SIZE);
     assertEquals(targets.length, 3);
     assertEquals(targets[0], dataNodes[1]);
     assertTrue(cluster.isOnSameRack(targets[1], targets[2]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     
-    targets = replicator.chooseTarget(
-                                      4, dataNodes[0], null, BLOCK_SIZE);
+    targets = replicator.chooseTarget(filename,
+                                      4, dataNodes[0], BLOCK_SIZE);
     assertEquals(targets.length, 4);
     assertEquals(targets[0], dataNodes[1]);
     for(int i=1; i<4; i++) {
@@ -253,23 +262,23 @@
     }
       
     DatanodeDescriptor[] targets;
-    targets = replicator.chooseTarget(
-                                      0, dataNodes[0], null, BLOCK_SIZE);
+    targets = replicator.chooseTarget(filename,
+                                      0, dataNodes[0], BLOCK_SIZE);
     assertEquals(targets.length, 0);
     
-    targets = replicator.chooseTarget(
-                                      1, dataNodes[0], null, BLOCK_SIZE);
+    targets = replicator.chooseTarget(filename,
+                                      1, dataNodes[0], BLOCK_SIZE);
     assertEquals(targets.length, 1);
     assertFalse(cluster.isOnSameRack(targets[0], dataNodes[0]));
     
-    targets = replicator.chooseTarget(
-                                      2, dataNodes[0], null, BLOCK_SIZE);
+    targets = replicator.chooseTarget(filename,
+                                      2, dataNodes[0], BLOCK_SIZE);
     assertEquals(targets.length, 2);
     assertFalse(cluster.isOnSameRack(targets[0], dataNodes[0]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     
-    targets = replicator.chooseTarget(
-                                      3, dataNodes[0], null, BLOCK_SIZE);
+    targets = replicator.chooseTarget(filename,
+                                      3, dataNodes[0], BLOCK_SIZE);
     assertEquals(targets.length, 3);
     for(int i=0; i<3; i++) {
       assertFalse(cluster.isOnSameRack(targets[i], dataNodes[0]));
@@ -293,21 +302,21 @@
    */
   public void testChooseTarget5() throws Exception {
     DatanodeDescriptor[] targets;
-    targets = replicator.chooseTarget(
-                                      0, NODE, null, BLOCK_SIZE);
+    targets = replicator.chooseTarget(filename,
+                                      0, NODE, BLOCK_SIZE);
     assertEquals(targets.length, 0);
     
-    targets = replicator.chooseTarget(
-                                      1, NODE, null, BLOCK_SIZE);
+    targets = replicator.chooseTarget(filename,
+                                      1, NODE, BLOCK_SIZE);
     assertEquals(targets.length, 1);
     
-    targets = replicator.chooseTarget(
-                                      2, NODE, null, BLOCK_SIZE);
+    targets = replicator.chooseTarget(filename,
+                                      2, NODE, BLOCK_SIZE);
     assertEquals(targets.length, 2);
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     
-    targets = replicator.chooseTarget(
-                                      3, NODE, null, BLOCK_SIZE);
+    targets = replicator.chooseTarget(filename,
+                                      3, NODE, BLOCK_SIZE);
     assertEquals(targets.length, 3);
     assertTrue(cluster.isOnSameRack(targets[1], targets[2]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));    
@@ -325,23 +334,23 @@
     chosenNodes.add(dataNodes[0]);    
     DatanodeDescriptor[] targets;
     
-    targets = replicator.chooseTarget(
-                                      0, dataNodes[0], chosenNodes, null, BLOCK_SIZE);
+    targets = replicator.chooseTarget(filename,
+                                      0, dataNodes[0], chosenNodes, BLOCK_SIZE);
     assertEquals(targets.length, 0);
     
-    targets = replicator.chooseTarget(
-                                      1, dataNodes[0], chosenNodes, null, BLOCK_SIZE);
+    targets = replicator.chooseTarget(filename,
+                                      1, dataNodes[0], chosenNodes, BLOCK_SIZE);
     assertEquals(targets.length, 1);
     assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0]));
     
-    targets = replicator.chooseTarget(
-                                      2, dataNodes[0], chosenNodes, null, BLOCK_SIZE);
+    targets = replicator.chooseTarget(filename,
+                                      2, dataNodes[0], chosenNodes, BLOCK_SIZE);
     assertEquals(targets.length, 2);
     assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     
-    targets = replicator.chooseTarget(
-                                      3, dataNodes[0], chosenNodes, null, BLOCK_SIZE);
+    targets = replicator.chooseTarget(filename,
+                                      3, dataNodes[0], chosenNodes, BLOCK_SIZE);
     assertEquals(targets.length, 3);
     assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[2]));
@@ -360,17 +369,17 @@
     chosenNodes.add(dataNodes[1]);
 
     DatanodeDescriptor[] targets;
-    targets = replicator.chooseTarget(
-                                      0, dataNodes[0], chosenNodes, null, BLOCK_SIZE);
+    targets = replicator.chooseTarget(filename,
+                                      0, dataNodes[0], chosenNodes, BLOCK_SIZE);
     assertEquals(targets.length, 0);
     
-    targets = replicator.chooseTarget(
-                                      1, dataNodes[0], chosenNodes, null, BLOCK_SIZE);
+    targets = replicator.chooseTarget(filename,
+                                      1, dataNodes[0], chosenNodes, BLOCK_SIZE);
     assertEquals(targets.length, 1);
     assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0]));
     
-    targets = replicator.chooseTarget(
-                                      2, dataNodes[0], chosenNodes, null, BLOCK_SIZE);
+    targets = replicator.chooseTarget(filename,
+                                      2, dataNodes[0], chosenNodes, BLOCK_SIZE);
     assertEquals(targets.length, 2);
     assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0]));
     assertFalse(cluster.isOnSameRack(dataNodes[0], targets[1]));
@@ -389,29 +398,29 @@
     chosenNodes.add(dataNodes[2]);
     
     DatanodeDescriptor[] targets;
-    targets = replicator.chooseTarget(
-                                      0, dataNodes[0], chosenNodes, null, BLOCK_SIZE);
+    targets = replicator.chooseTarget(filename,
+                                      0, dataNodes[0], chosenNodes, BLOCK_SIZE);
     assertEquals(targets.length, 0);
     
-    targets = replicator.chooseTarget(
-                                      1, dataNodes[0], chosenNodes, null, BLOCK_SIZE);
+    targets = replicator.chooseTarget(filename,
+                                      1, dataNodes[0], chosenNodes, BLOCK_SIZE);
     assertEquals(targets.length, 1);
     assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0]));
     assertFalse(cluster.isOnSameRack(dataNodes[2], targets[0]));
     
-    targets = replicator.chooseTarget(
-                               1, dataNodes[2], chosenNodes, null, BLOCK_SIZE);
+    targets = replicator.chooseTarget(filename,
+                               1, dataNodes[2], chosenNodes, BLOCK_SIZE);
     assertEquals(targets.length, 1);
     assertTrue(cluster.isOnSameRack(dataNodes[2], targets[0]));
     assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0]));
 
-    targets = replicator.chooseTarget(
-                                      2, dataNodes[0], chosenNodes, null, BLOCK_SIZE);
+    targets = replicator.chooseTarget(filename,
+                                      2, dataNodes[0], chosenNodes, BLOCK_SIZE);
     assertEquals(targets.length, 2);
     assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0]));
     
-    targets = replicator.chooseTarget(
-                               2, dataNodes[2], chosenNodes, null, BLOCK_SIZE);
+    targets = replicator.chooseTarget(filename,
+                               2, dataNodes[2], chosenNodes, BLOCK_SIZE);
     assertEquals(targets.length, 2);
     assertTrue(cluster.isOnSameRack(dataNodes[2], targets[0]));
   }

Modified: hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStartup.java?rev=885143&r1=885142&r2=885143&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStartup.java (original)
+++ hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStartup.java Sat Nov 28 20:05:56 2009
@@ -1,3 +1,20 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.File;
@@ -16,7 +33,9 @@
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSImage.NameNodeDirType;
@@ -54,7 +73,7 @@
 
 
   protected void setUp() throws Exception {
-    config = new Configuration();
+    config = new HdfsConfiguration();
     String baseDir = System.getProperty("test.build.data", "/tmp");
 
     hdfsDir = new File(baseDir, "dfs");
@@ -62,10 +81,10 @@
       throw new IOException("Could not delete hdfs directory '" + hdfsDir + "'");
     }
     LOG.info("--hdfsdir is " + hdfsDir.getAbsolutePath());
-    config.set("dfs.name.dir", new File(hdfsDir, "name").getPath());
-    config.set("dfs.data.dir", new File(hdfsDir, "data").getPath());
+    config.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, new File(hdfsDir, "name").getPath());
+    config.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, new File(hdfsDir, "data").getPath());
 
-    config.set("fs.checkpoint.dir",new File(hdfsDir, "secondary").getPath());
+    config.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY,new File(hdfsDir, "secondary").getPath());
     //config.set("fs.default.name", "hdfs://"+ NAME_NODE_HOST + "0");
     
     FileSystem.setDefaultUri(config, "hdfs://"+NAME_NODE_HOST + "0");
@@ -212,11 +231,11 @@
   public void testChkpointStartup2() throws IOException{
     LOG.info("--starting checkpointStartup2 - same directory for checkpoint");
     // different name dirs
-    config.set("dfs.name.dir", new File(hdfsDir, "name").getPath());
-    config.set("dfs.name.edits.dir", new File(hdfsDir, "edits").getPath());
+    config.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, new File(hdfsDir, "name").getPath());
+    config.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, new File(hdfsDir, "edits").getPath());
     // same checkpoint dirs
-    config.set("fs.checkpoint.edits.dir", new File(hdfsDir, "chkpt").getPath());
-    config.set("fs.checkpoint.dir", new File(hdfsDir, "chkpt").getPath());
+    config.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_EDITS_DIR_KEY, new File(hdfsDir, "chkpt").getPath());
+    config.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY, new File(hdfsDir, "chkpt").getPath());
 
     createCheckPoint();
 
@@ -234,11 +253,11 @@
     //setUpConfig();
     LOG.info("--starting testStartup Recovery");
     // different name dirs
-    config.set("dfs.name.dir", new File(hdfsDir, "name").getPath());
-    config.set("dfs.name.edits.dir", new File(hdfsDir, "edits").getPath());
+    config.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, new File(hdfsDir, "name").getPath());
+    config.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, new File(hdfsDir, "edits").getPath());
     // same checkpoint dirs
-    config.set("fs.checkpoint.edits.dir", new File(hdfsDir, "chkpt_edits").getPath());
-    config.set("fs.checkpoint.dir", new File(hdfsDir, "chkpt").getPath());
+    config.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_EDITS_DIR_KEY, new File(hdfsDir, "chkpt_edits").getPath());
+    config.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY, new File(hdfsDir, "chkpt").getPath());
 
     createCheckPoint();
     corruptNameNodeFiles();
@@ -255,11 +274,11 @@
     //setUpConfig();
     LOG.info("--starting SecondNN startup test");
     // different name dirs
-    config.set("dfs.name.dir", new File(hdfsDir, "name").getPath());
-    config.set("dfs.name.edits.dir", new File(hdfsDir, "name").getPath());
+    config.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, new File(hdfsDir, "name").getPath());
+    config.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, new File(hdfsDir, "name").getPath());
     // same checkpoint dirs
-    config.set("fs.checkpoint.edits.dir", new File(hdfsDir, "chkpt_edits").getPath());
-    config.set("fs.checkpoint.dir", new File(hdfsDir, "chkpt").getPath());
+    config.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_EDITS_DIR_KEY, new File(hdfsDir, "chkpt_edits").getPath());
+    config.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY, new File(hdfsDir, "chkpt").getPath());
 
     LOG.info("--starting NN ");
     MiniDFSCluster cluster = null;

Modified: hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java?rev=885143&r1=885142&r2=885143&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java (original)
+++ hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java Sat Nov 28 20:05:56 2009
@@ -40,11 +40,13 @@
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSImage.NameNodeDirType;
 import org.apache.hadoop.hdfs.server.namenode.FSImage.NameNodeFile;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 
 
 /**
@@ -78,7 +80,7 @@
   
  
   protected void setUp() throws Exception {
-    config = new Configuration();
+    config = new HdfsConfiguration();
     String baseDir = System.getProperty("test.build.data",  "build/test/data");
     
     hdfsDir = new File(baseDir, "dfs");
@@ -100,17 +102,17 @@
     System.out.println("configuring hdfsdir is " + hdfsDir.getAbsolutePath() + 
         "; dfs_name_dir = "+ dfs_name_dir + ";dfs_name_edits_dir(only)=" + path3.getPath());
     
-    config.set("dfs.name.dir", dfs_name_dir);
-    config.set("dfs.name.edits.dir", dfs_name_dir + "," + path3.getPath());
+    config.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, dfs_name_dir);
+    config.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, dfs_name_dir + "," + path3.getPath());
 
-    config.set("fs.checkpoint.dir",new File(hdfsDir, "secondary").getPath());
+    config.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY,new File(hdfsDir, "secondary").getPath());
  
     FileSystem.setDefaultUri(config, "hdfs://"+NAME_NODE_HOST + "0");
     
-    config.set("dfs.secondary.http.address", "0.0.0.0:0");
+    config.set(DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY, "0.0.0.0:0");
     
     // set the restore feature on
-    config.setBoolean("dfs.name.dir.restore", true);
+    config.setBoolean(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_RESTORE_KEY, true);
   }
 
   /**
@@ -348,7 +350,7 @@
       // now run DFSAdmnin command
 
       String cmd = "-fs NAMENODE -restoreFailedStorage false";
-      String namenode = config.get("fs.default.name", "file:///");
+      String namenode = config.get(DFSConfigKeys.FS_DEFAULT_NAME_KEY, "file:///");
       CommandExecutor executor = new TestHDFSCLI.DFSAdminCmdExecutor(namenode);
       executor.executeCommand(cmd);
       restore = fsi.getRestoreFailedStorage();

Modified: hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestUnderReplicatedBlocks.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestUnderReplicatedBlocks.java?rev=885143&r1=885142&r2=885143&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestUnderReplicatedBlocks.java (original)
+++ hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestUnderReplicatedBlocks.java Sat Nov 28 20:05:56 2009
@@ -1,3 +1,20 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package org.apache.hadoop.hdfs.server.namenode;
 
 import org.apache.hadoop.conf.Configuration;
@@ -5,6 +22,7 @@
 import org.apache.hadoop.fs.FsShell;
 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;
 
@@ -12,7 +30,7 @@
 
 public class TestUnderReplicatedBlocks extends TestCase {
   public void testSetrepIncWithUnderReplicatedBlocks() throws Exception {
-    Configuration conf = new Configuration();
+    Configuration conf = new HdfsConfiguration();
     final short REPLICATION_FACTOR = 2;
     final String FILE_NAME = "/testFile";
     final Path FILE_PATH = new Path(FILE_NAME);

Modified: hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/metrics/TestNNMetricFilesInGetListingOps.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/metrics/TestNNMetricFilesInGetListingOps.java?rev=885143&r1=885142&r2=885143&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/metrics/TestNNMetricFilesInGetListingOps.java (original)
+++ hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/metrics/TestNNMetricFilesInGetListingOps.java Sat Nov 28 20:05:56 2009
@@ -28,17 +28,19 @@
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 
 /**
  * Test case for FilesInGetListingOps metric in Namenode
  */
 public class TestNNMetricFilesInGetListingOps extends TestCase {
-  private static final Configuration CONF = new Configuration();
+  private static final Configuration CONF = new HdfsConfiguration();
   static {
-    CONF.setLong("dfs.block.size", 100);
-    CONF.setInt("io.bytes.per.checksum", 1);
+    CONF.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 100);
+    CONF.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 1);
     CONF.setLong("dfs.heartbeat.interval", 1L);
-    CONF.setInt("dfs.replication.interval", 1);
+    CONF.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1);
   }
      
   private MiniDFSCluster cluster;

Modified: hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java?rev=885143&r1=885142&r2=885143&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java (original)
+++ hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java Sat Nov 28 20:05:56 2009
@@ -30,17 +30,20 @@
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.namenode.BlockManager;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 
 /**
  * Test for metrics published by the Namenode
  */
 public class TestNameNodeMetrics extends TestCase {
-  private static final Configuration CONF = new Configuration();
+  private static final Configuration CONF = new HdfsConfiguration();
   static {
-    CONF.setLong("dfs.block.size", 100);
-    CONF.setInt("io.bytes.per.checksum", 1);
+    CONF.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 100);
+    CONF.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 1);
     CONF.setLong("dfs.heartbeat.interval", 1L);
-    CONF.setInt("dfs.replication.interval", 1);
+    CONF.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1);
   }
   
   private MiniDFSCluster cluster;
@@ -53,7 +56,7 @@
   protected void setUp() throws Exception {
     cluster = new MiniDFSCluster(CONF, 3, true, null);
     cluster.waitActive();
-    namesystem = cluster.getNameNode().getNamesystem();
+    namesystem = cluster.getNamesystem();
     fs = (DistributedFileSystem) cluster.getFileSystem();
     metrics = namesystem.getFSNamesystemMetrics();
   }
@@ -106,7 +109,8 @@
     createFile(file, 100, (short)2);
     
     // Corrupt first replica of the block
-    LocatedBlock block = namesystem.getBlockLocations(file, 0, 1).get(0);
+    LocatedBlock block = NameNodeAdapter.getBlockLocations(
+        cluster.getNameNode(), file, 0, 1).get(0);
     namesystem.markBlockAsCorrupt(block.getBlock(), block.getLocations()[0]);
     updateMetrics();
     assertEquals(1, metrics.corruptBlocks.get());
@@ -140,7 +144,8 @@
     createFile(file, 100, (short)1);
     
     // Corrupt the only replica of the block to result in a missing block
-    LocatedBlock block = namesystem.getBlockLocations(file, 0, 1).get(0);
+    LocatedBlock block = NameNodeAdapter.getBlockLocations(
+        cluster.getNameNode(), file, 0, 1).get(0);
     namesystem.markBlockAsCorrupt(block.getBlock(), block.getLocations()[0]);
     updateMetrics();
     assertEquals(1, metrics.underReplicatedBlocks.get());

Modified: hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java?rev=885143&r1=885142&r2=885143&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java (original)
+++ hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java Sat Nov 28 20:05:56 2009
@@ -41,6 +41,7 @@
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.FSConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 
 /**
  * Test function of OfflineImageViewer by:
@@ -101,7 +102,7 @@
     MiniDFSCluster cluster = null;
     File orig = null;
     try {
-      Configuration conf = new Configuration();
+      Configuration conf = new HdfsConfiguration();
       cluster = new MiniDFSCluster(conf, 4, true, null);
       FileSystem hdfs = cluster.getFileSystem();
       

Modified: hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/security/TestPermission.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/security/TestPermission.java?rev=885143&r1=885142&r2=885143&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/security/TestPermission.java (original)
+++ hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/security/TestPermission.java Sat Nov 28 20:05:56 2009
@@ -25,6 +25,8 @@
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
 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.fs.*;
 import org.apache.hadoop.fs.permission.*;
@@ -65,9 +67,9 @@
   }
 
   public void testCreate() throws Exception {
-    Configuration conf = new Configuration();
-    conf.setBoolean("dfs.permissions", true);
-    conf.setInt(FsPermission.UMASK_LABEL, 0);
+    Configuration conf = new HdfsConfiguration();
+    conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
+    conf.set(FsPermission.UMASK_LABEL, "000");
     MiniDFSCluster cluster = null;
     FileSystem fs = null;
 
@@ -95,7 +97,7 @@
       checkPermission(fs, "/b1/b2", inheritPerm);
       checkPermission(fs, "/b1/b2/b3.txt", filePerm);
       
-      conf.setInt(FsPermission.UMASK_LABEL, 0022);
+      conf.set(FsPermission.UMASK_LABEL, "022");
       FsPermission permission = 
         FsPermission.createImmutable((short)0666);
       FileSystem.mkdirs(fs, new Path("/c1"), new FsPermission(permission));
@@ -118,8 +120,8 @@
   }
 
   public void testFilePermision() throws Exception {
-    Configuration conf = new Configuration();
-    conf.setBoolean("dfs.permissions", true);
+    Configuration conf = new HdfsConfiguration();
+    conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
     MiniDFSCluster cluster = new MiniDFSCluster(conf, 3, true, null);
     cluster.waitActive();
 

Modified: hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/tools/TestJMXGet.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/tools/TestJMXGet.java?rev=885143&r1=885142&r2=885143&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/tools/TestJMXGet.java (original)
+++ hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/tools/TestJMXGet.java Sat Nov 28 20:05:56 2009
@@ -29,6 +29,7 @@
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.tools.JMXGet;
@@ -61,7 +62,7 @@
 
 
   protected void setUp() throws Exception {
-    config = new Configuration();
+    config = new HdfsConfiguration();
   }
 
   /**

Propchange: hadoop/hdfs/branches/HDFS-326/src/webapps/datanode/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Sat Nov 28 20:05:56 2009
@@ -1,3 +1,5 @@
 /hadoop/core/branches/branch-0.19/hdfs/src/webapps/datanode:713112
 /hadoop/core/trunk/src/webapps/datanode:776175-784663
-/hadoop/hdfs/trunk/src/webapps/datanode:804973-807690
+/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:804973-884907

Propchange: hadoop/hdfs/branches/HDFS-326/src/webapps/hdfs/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Sat Nov 28 20:05:56 2009
@@ -1,3 +1,5 @@
 /hadoop/core/branches/branch-0.19/hdfs/src/webapps/hdfs:713112
 /hadoop/core/trunk/src/webapps/hdfs:776175-784663
-/hadoop/hdfs/trunk/src/webapps/hdfs:804973-807690
+/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:804973-884907

Propchange: hadoop/hdfs/branches/HDFS-326/src/webapps/secondary/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Sat Nov 28 20:05:56 2009
@@ -1,3 +1,5 @@
 /hadoop/core/branches/branch-0.19/hdfs/src/webapps/secondary:713112
 /hadoop/core/trunk/src/webapps/secondary:776175-784663
-/hadoop/hdfs/trunk/src/webapps/secondary:804973-807690
+/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:804973-884907