Quellcode durchsuchen

Merging change r1083043 from trunk to federation branch

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hdfs/branches/HDFS-1052@1089247 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas vor 14 Jahren
Ursprung
Commit
c727cbbfad
73 geänderte Dateien mit 233 neuen und 184 gelöschten Zeilen
  1. 3 0
      CHANGES.txt
  2. 12 9
      src/java/org/apache/hadoop/hdfs/DFSClient.java
  3. 6 1
      src/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  4. 5 2
      src/java/org/apache/hadoop/hdfs/DFSOutputStream.java
  5. 7 5
      src/java/org/apache/hadoop/hdfs/HftpFileSystem.java
  6. 3 1
      src/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
  7. 3 1
      src/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java
  8. 1 0
      src/java/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java
  9. 8 7
      src/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  10. 6 5
      src/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java
  11. 2 1
      src/java/org/apache/hadoop/hdfs/server/datanode/DatanodeJspHelper.java
  12. 1 1
      src/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
  13. 8 4
      src/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java
  14. 3 1
      src/java/org/apache/hadoop/hdfs/server/datanode/SecureDataNodeStarter.java
  15. 3 1
      src/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
  16. 11 8
      src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java
  17. 5 3
      src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  18. 11 8
      src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  19. 4 2
      src/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
  20. 4 2
      src/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java
  21. 1 1
      src/test/aop/org/apache/hadoop/fs/TestFiRename.java
  22. 2 2
      src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol.java
  23. 4 4
      src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol2.java
  24. 2 2
      src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiPipelineClose.java
  25. 1 1
      src/test/hdfs/org/apache/hadoop/cli/TestHDFSCLI.java
  26. 1 1
      src/test/hdfs/org/apache/hadoop/fs/loadGenerator/TestLoadGenerator.java
  27. 1 1
      src/test/hdfs/org/apache/hadoop/hdfs/BlockReaderTestUtil.java
  28. 5 5
      src/test/hdfs/org/apache/hadoop/hdfs/MiniDFSCluster.java
  29. 2 2
      src/test/hdfs/org/apache/hadoop/hdfs/TestCrcCorruption.java
  30. 8 7
      src/test/hdfs/org/apache/hadoop/hdfs/TestDFSClientRetries.java
  31. 1 1
      src/test/hdfs/org/apache/hadoop/hdfs/TestDFSFinalize.java
  32. 1 1
      src/test/hdfs/org/apache/hadoop/hdfs/TestDFSRollback.java
  33. 2 2
      src/test/hdfs/org/apache/hadoop/hdfs/TestDFSStorageStateRecovery.java
  34. 1 1
      src/test/hdfs/org/apache/hadoop/hdfs/TestDFSUpgrade.java
  35. 1 1
      src/test/hdfs/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
  36. 1 1
      src/test/hdfs/org/apache/hadoop/hdfs/TestDataTransferProtocol.java
  37. 3 3
      src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeBlockScanner.java
  38. 2 2
      src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeDeath.java
  39. 1 1
      src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeReport.java
  40. 4 4
      src/test/hdfs/org/apache/hadoop/hdfs/TestFileAppend2.java
  41. 1 1
      src/test/hdfs/org/apache/hadoop/hdfs/TestFileConcurrentReader.java
  42. 5 5
      src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreation.java
  43. 2 2
      src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreationClient.java
  44. 1 1
      src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreationDelete.java
  45. 1 1
      src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreationEmpty.java
  46. 1 1
      src/test/hdfs/org/apache/hadoop/hdfs/TestInjectionForSimulatedStorage.java
  47. 1 1
      src/test/hdfs/org/apache/hadoop/hdfs/TestLeaseRecovery2.java
  48. 2 1
      src/test/hdfs/org/apache/hadoop/hdfs/TestPread.java
  49. 1 1
      src/test/hdfs/org/apache/hadoop/hdfs/TestReadWhileWriting.java
  50. 4 4
      src/test/hdfs/org/apache/hadoop/hdfs/TestRenameWhileOpen.java
  51. 2 2
      src/test/hdfs/org/apache/hadoop/hdfs/TestReplication.java
  52. 3 3
      src/test/hdfs/org/apache/hadoop/hdfs/TestSetTimes.java
  53. 2 2
      src/test/hdfs/org/apache/hadoop/hdfs/TestSetrepIncreasing.java
  54. 1 1
      src/test/hdfs/org/apache/hadoop/hdfs/UpgradeUtilities.java
  55. 2 1
      src/test/hdfs/org/apache/hadoop/hdfs/server/common/TestDistributedUpgrade.java
  56. 1 1
      src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java
  57. 1 1
      src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
  58. 3 3
      src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
  59. 1 1
      src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java
  60. 3 3
      src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
  61. 3 3
      src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBlockTokenWithDFS.java
  62. 2 2
      src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBlocksWithNotEnoughRacks.java
  63. 1 1
      src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
  64. 2 2
      src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCorruptFilesJsp.java
  65. 1 1
      src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java
  66. 2 2
      src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestFileLimit.java
  67. 9 9
      src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
  68. 2 1
      src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestHDFSConcat.java
  69. 10 10
      src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestListCorruptFileBlocks.java
  70. 9 9
      src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java
  71. 2 1
      src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java
  72. 1 1
      src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestOverReplicatedBlocks.java
  73. 1 1
      src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/metrics/TestNNMetricFilesInGetListingOps.java

+ 3 - 0
CHANGES.txt

@@ -304,6 +304,9 @@ Trunk (unreleased changes)
     HDFS-1445. Batch the calls in DataStorage to FileUtil.createHardLink().
     (Matt Foley via jghoman)
 
+    HDFS-1763. Replace hard-coded option strings with variables from
+    DFSConfigKeys. (eli)
+
   OPTIMIZATIONS
 
     HDFS-1458. Improve checkpoint performance by avoiding unnecessary image

+ 12 - 9
src/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -119,7 +119,6 @@ import org.apache.hadoop.util.StringUtils;
 public class DFSClient implements FSConstants, java.io.Closeable {
   public static final Log LOG = LogFactory.getLog(DFSClient.class);
   public static final long SERVER_DEFAULTS_VALIDITY_PERIOD = 60 * 60 * 1000L; // 1 hour
-  public static final int MAX_BLOCK_ACQUIRE_FAILURES = 3;
   static final int TCP_WINDOW_SIZE = 128 * 1024; // 128 KB
   final ClientProtocol namenode;
   private final ClientProtocol rpcNamenode;
@@ -241,12 +240,14 @@ public class DFSClient implements FSConstants, java.io.Closeable {
     throws IOException {
     this.conf = conf;
     this.stats = stats;
-    this.socketTimeout = conf.getInt(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 
-                                     HdfsConstants.READ_TIMEOUT);
+    this.socketTimeout = 
+      conf.getInt(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 
+                  HdfsConstants.READ_TIMEOUT);
     this.socketFactory = NetUtils.getSocketFactory(conf, ClientProtocol.class);
     // dfs.write.packet.size is an internal config variable
-    this.writePacketSize = conf.getInt(DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY, 
-                                       DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT);
+    this.writePacketSize = 
+      conf.getInt(DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY, 
+                  DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT);
     // The hdfsTimeout is currently the same as the ipc timeout 
     this.hdfsTimeout = Client.getTimeout(conf);
 
@@ -256,7 +257,9 @@ public class DFSClient implements FSConstants, java.io.Closeable {
     this.clientName = "DFSClient_" + taskId + "_" +
                       r.nextInt() + "_" + Thread.currentThread().getId(); 
     defaultBlockSize = conf.getLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
-    defaultReplication = (short) conf.getInt("dfs.replication", 3);
+    defaultReplication = (short) 
+      conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, 
+                  DFSConfigKeys.DFS_REPLICATION_DEFAULT);
 
     if (nameNodeAddr != null && rpcNamenode == null) {
       this.rpcNamenode = createRPCNamenode(nameNodeAddr, conf, ugi);
@@ -276,8 +279,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * to retrieve block locations when reading.
    */
   int getMaxBlockAcquireFailures() {
-    return conf.getInt("dfs.client.max.block.acquire.failures",
-                       MAX_BLOCK_ACQUIRE_FAILURES);
+    return conf.getInt(DFSConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY,
+                       DFSConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_DEFAULT);
   }
 
   /**
@@ -286,7 +289,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    */
   int getDatanodeWriteTimeout(int numNodes) {
     int confTime =
-        conf.getInt("dfs.datanode.socket.write.timeout",
+        conf.getInt(DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY,
                     HdfsConstants.WRITE_TIMEOUT);
 
     return (confTime > 0) ?

+ 6 - 1
src/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -144,7 +144,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY = "dfs.client.max.block.acquire.failures";
   public static final int     DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_DEFAULT = 3;
   public static final String  DFS_BALANCER_MOVEDWINWIDTH_KEY = "dfs.balancer.movedWinWidth";
-  public static final int     DFS_BALANCER_MOVEDWINWIDTH_DEFAULT = 5400*1000;
+  public static final long    DFS_BALANCER_MOVEDWINWIDTH_DEFAULT = 5400*1000L;
   public static final String  DFS_DATANODE_ADDRESS_KEY = "dfs.datanode.address";
   public static final String  DFS_DATANODE_ADDRESS_DEFAULT = "0.0.0.0:50010";
   public static final String  DFS_DATANODE_DATA_DIR_PERMISSION_KEY = "dfs.datanode.data.dir.perm";
@@ -189,6 +189,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final boolean DFS_SUPPORT_APPEND_DEFAULT = true;
   public static final String  DFS_HTTPS_ENABLE_KEY = "dfs.https.enable";
   public static final boolean DFS_HTTPS_ENABLE_DEFAULT = false;
+  public static final String  DFS_HTTPS_PORT_KEY = "dfs.https.port";
+  public static final int     DFS_HTTPS_PORT_DEFAULT = 50470;
   public static final String  DFS_DEFAULT_CHUNK_VIEW_SIZE_KEY = "dfs.default.chunk.view.size";
   public static final int     DFS_DEFAULT_CHUNK_VIEW_SIZE_DEFAULT = 32*1024;
   public static final String  DFS_DATANODE_HTTPS_ADDRESS_KEY = "dfs.datanode.https.address";
@@ -213,6 +215,9 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_BLOCKREPORT_INITIAL_DELAY_DEFAULT = 0;
   public static final String  DFS_BLOCK_INVALIDATE_LIMIT_KEY = "dfs.block.invalidate.limit";
   public static final int     DFS_BLOCK_INVALIDATE_LIMIT_DEFAULT = 1000;
+  public static final String  DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED_KEY = "dfs.corruptfilesreturned.max";
+  public static final int     DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED = 500;
+
 
   // property for fsimage compression
   public static final String DFS_IMAGE_COMPRESS_KEY = "dfs.image.compress";

+ 5 - 2
src/java/org/apache/hadoop/hdfs/DFSOutputStream.java

@@ -832,7 +832,8 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
     private DatanodeInfo[] nextBlockOutputStream(String client) throws IOException {
       LocatedBlock lb = null;
       DatanodeInfo[] nodes = null;
-      int count = conf.getInt("dfs.client.block.write.retries", 3);
+      int count = conf.getInt(DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_RETRIES_KEY,
+                              DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_RETRIES_DEFAULT);
       boolean success = false;
       do {
         hasError = false;
@@ -965,7 +966,9 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
     private LocatedBlock locateFollowingBlock(long start,
         DatanodeInfo[] excludedNodes) 
         throws IOException, UnresolvedLinkException {
-      int retries = conf.getInt("dfs.client.block.write.locateFollowingBlock.retries", 5);
+      int retries = 
+        conf.getInt(DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY,
+                    DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_DEFAULT);
       long sleeptime = 400;
       while (true) {
         long localstart = System.currentTimeMillis();

+ 7 - 5
src/java/org/apache/hadoop/hdfs/HftpFileSystem.java

@@ -49,6 +49,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -84,7 +85,6 @@ public class HftpFileSystem extends FileSystem {
     HttpURLConnection.setFollowRedirects(true);
   }
 
-  private static final int DEFAULT_PORT = 50470;
   private String nnHttpUrl;
   private URI hdfsURI;
   protected InetSocketAddress nnAddr;
@@ -116,7 +116,7 @@ public class HftpFileSystem extends FileSystem {
 
   @Override
   protected int getDefaultPort() {
-    return DEFAULT_PORT;
+    return DFSConfigKeys.DFS_HTTPS_PORT_DEFAULT;
   }
 
   @Override
@@ -139,14 +139,16 @@ public class HftpFileSystem extends FileSystem {
     this.ugi = UserGroupInformation.getCurrentUser(); 
     nnAddr = NetUtils.createSocketAddr(name.toString());
    
-    nnHttpUrl = buildUri("https://", NetUtils.normalizeHostName(name.getHost()), 
-        conf.getInt("dfs.https.port", DEFAULT_PORT));
+    nnHttpUrl = buildUri("https://", 
+                         NetUtils.normalizeHostName(name.getHost()), 
+                         conf.getInt(DFSConfigKeys.DFS_HTTPS_PORT_KEY, 
+                                     DFSConfigKeys.DFS_HTTPS_PORT_DEFAULT));
 
     // if one uses RPC port different from the Default one,  
     // one should specify what is the setvice name for this delegation token
     // otherwise it is hostname:RPC_PORT
     String key = HftpFileSystem.HFTP_SERVICE_NAME_KEY+
-    SecurityUtil.buildDTServiceName(name, DEFAULT_PORT);
+    SecurityUtil.buildDTServiceName(name, DFSConfigKeys.DFS_HTTPS_PORT_DEFAULT);
     if(LOG.isDebugEnabled()) {
       LOG.debug("Trying to find DT for " + name + " using key=" + key + 
           "; conf=" + conf.get(key, ""));

+ 3 - 1
src/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java

@@ -1467,7 +1467,9 @@ public class Balancer {
     public int run(String[] args) {
       final long startTime = Util.now();
       final Configuration conf = getConf();
-      WIN_WIDTH = conf.getLong("dfs.balancer.movedWinWidth", WIN_WIDTH);
+      WIN_WIDTH = conf.getLong(
+          DFSConfigKeys.DFS_BALANCER_MOVEDWINWIDTH_KEY, 
+          DFSConfigKeys.DFS_BALANCER_MOVEDWINWIDTH_DEFAULT);
 
       try {
         checkReplicationPolicyCompatibility(conf);

+ 3 - 1
src/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java

@@ -42,6 +42,7 @@ import java.util.regex.Pattern;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
@@ -144,7 +145,8 @@ class BlockPoolSliceScanner {
     this.datanode = datanode;
     this.dataset = dataset;
     this.blockPoolId  = bpid;
-    scanPeriod = conf.getInt("dfs.datanode.scan.period.hours", 0);
+    scanPeriod = conf.getInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, 
+                             DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_DEFAULT);
     if ( scanPeriod <= 0 ) {
       scanPeriod = DEFAULT_SCAN_PERIOD_HOURS;
     }

+ 1 - 0
src/java/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java

@@ -29,6 +29,7 @@ import javax.servlet.http.HttpServletResponse;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.DataNode.BPOfferService;

+ 8 - 7
src/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -436,7 +436,7 @@ public class DataNode extends Configured
   private void initConfig(Configuration conf) {
     this.socketTimeout =  conf.getInt(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY,
                                       HdfsConstants.READ_TIMEOUT);
-    this.socketWriteTimeout = conf.getInt("dfs.datanode.socket.write.timeout",
+    this.socketWriteTimeout = conf.getInt(DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY,
                                           HdfsConstants.WRITE_TIMEOUT);
     /* Based on results on different platforms, we might need set the default 
      * to false on some of them. */
@@ -446,15 +446,15 @@ public class DataNode extends Configured
                                        DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT);
 
     this.blockReportInterval =
-      conf.getLong("dfs.blockreport.intervalMsec", BLOCKREPORT_INTERVAL);
-    this.initialBlockReportDelay = conf.getLong("dfs.blockreport.initialDelay",
+      conf.getLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, BLOCKREPORT_INTERVAL);
+    this.initialBlockReportDelay = conf.getLong(DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY,
                                             BLOCKREPORT_INITIAL_DELAY)* 1000L; 
     if (this.initialBlockReportDelay >= blockReportInterval) {
       this.initialBlockReportDelay = 0;
       LOG.info("dfs.blockreport.initialDelay is greater than " +
         "dfs.blockreport.intervalMsec." + " Setting initial delay to 0 msec:");
     }
-    this.heartBeatInterval = conf.getLong("dfs.heartbeat.interval", HEARTBEAT_INTERVAL) * 1000L;
+    this.heartBeatInterval = conf.getLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, HEARTBEAT_INTERVAL) * 1000L;
 
     // do we need to sync block file contents to disk when blockfile is closed?
     this.syncOnClose = conf.getBoolean(DFSConfigKeys.DFS_DATANODE_SYNCONCLOSE_KEY, 
@@ -516,9 +516,10 @@ public class DataNode extends Configured
     InetSocketAddress ipcAddr = NetUtils.createSocketAddr(
         conf.get("dfs.datanode.ipc.address"));
     ipcServer = RPC.getServer(DataNode.class, this, ipcAddr.getHostName(),
-        ipcAddr.getPort(), conf.getInt("dfs.datanode.handler.count", 3), false,
-        conf, blockPoolTokenSecretManager);
-    
+                              ipcAddr.getPort(), 
+                              conf.getInt(DFS_DATANODE_HANDLER_COUNT_KEY, 
+                                          DFS_DATANODE_HANDLER_COUNT_DEFAULT), 
+                              false, conf, blockPoolTokenSecretManager);
     // set service-level authorization security policy
     if (conf.getBoolean(
         CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, false)) {

+ 6 - 5
src/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java

@@ -56,8 +56,7 @@ class DataXceiverServer implements Runnable, FSConstants {
    * Enforcing the limit is required in order to avoid data-node
    * running out of memory.
    */
-  static final int MAX_XCEIVER_COUNT = 256;
-  int maxXceiverCount = MAX_XCEIVER_COUNT;
+  int maxXceiverCount = DFSConfigKeys.DFS_DATANODE_MAX_XCIEVERS_DEFAULT;
 
   /** A manager to make sure that cluster balancing does not
    * take too much resources.
@@ -115,10 +114,12 @@ class DataXceiverServer implements Runnable, FSConstants {
     this.ss = ss;
     this.datanode = datanode;
     
-    this.maxXceiverCount = conf.getInt("dfs.datanode.max.xcievers",
-        MAX_XCEIVER_COUNT);
+    this.maxXceiverCount = 
+      conf.getInt(DFSConfigKeys.DFS_DATANODE_MAX_XCIEVERS_KEY,
+                  DFSConfigKeys.DFS_DATANODE_MAX_XCIEVERS_DEFAULT);
     
-    this.estimateBlockSize = conf.getLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
+    this.estimateBlockSize = 
+      conf.getLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
     
     //set up parameter for cluster balancing
     this.balanceThrottler = new BlockBalanceThrottler(

+ 2 - 1
src/java/org/apache/hadoop/hdfs/server/datanode/DatanodeJspHelper.java

@@ -72,7 +72,8 @@ public class DatanodeJspHelper {
    * @return the number of bytes to chunk in
    */
   private static int getDefaultChunkSize(Configuration conf) {
-    return conf.getInt("dfs.default.chunk.view.size", 32 * 1024);
+    return conf.getInt(DFSConfigKeys.DFS_DEFAULT_CHUNK_VIEW_SIZE_KEY,
+                       DFSConfigKeys.DFS_DEFAULT_CHUNK_VIEW_SIZE_DEFAULT);
   }
 
   static void generateDirectoryStructure(JspWriter out, 

+ 1 - 1
src/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java

@@ -220,7 +220,7 @@ public class DirectoryScanner implements Runnable {
 
   DirectoryScanner(FSDataset dataset, Configuration conf) {
     this.dataset = dataset;
-    int interval = conf.getInt("dfs.datanode.directoryscan.interval",
+    int interval = conf.getInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY,
         DEFAULT_SCAN_INTERVAL);
     scanPeriodMsecs = interval * 1000L; //msec
     int threads = 

+ 8 - 4
src/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java

@@ -536,7 +536,8 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
     private final long reserved;
     
     FSVolume(File currentDir, Configuration conf) throws IOException {
-      this.reserved = conf.getLong("dfs.datanode.du.reserved", 0);
+      this.reserved = conf.getLong(DFSConfigKeys.DFS_DATANODE_DU_RESERVED_KEY,
+                                   DFSConfigKeys.DFS_DATANODE_DU_RESERVED_DEFAULT);
       this.currentDir = currentDir; 
       File parent = currentDir.getParentFile();
       this.usage = new DF(parent, conf);
@@ -1147,9 +1148,12 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
    * An FSDataset has a directory where it loads its data files.
    */
   public FSDataset(DataStorage storage, Configuration conf) throws IOException {
-    this.maxBlocksPerDir = conf.getInt("dfs.datanode.numblocks", 64);
-    this.supportAppends = conf.getBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY,
-                                      DFSConfigKeys.DFS_SUPPORT_APPEND_DEFAULT);
+    this.maxBlocksPerDir = 
+      conf.getInt(DFSConfigKeys.DFS_DATANODE_NUMBLOCKS_KEY,
+                  DFSConfigKeys.DFS_DATANODE_NUMBLOCKS_DEFAULT);
+    this.supportAppends = 
+      conf.getBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY,
+                      DFSConfigKeys.DFS_SUPPORT_APPEND_DEFAULT);
     // The number of volumes required for operation is the total number 
     // of volumes minus the number of failed volumes we can tolerate.
     final int volFailuresTolerated =

+ 3 - 1
src/java/org/apache/hadoop/hdfs/server/datanode/SecureDataNodeStarter.java

@@ -25,6 +25,8 @@ import java.nio.channels.ServerSocketChannel;
 import org.apache.commons.daemon.Daemon;
 import org.apache.commons.daemon.DaemonContext;
 import org.apache.hadoop.conf.Configuration;
+
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.http.HttpServer;
 import org.mortbay.jetty.nio.SelectChannelConnector;
@@ -68,7 +70,7 @@ public class SecureDataNodeStarter implements Daemon {
     
     // Obtain secure port for data streaming to datanode
     InetSocketAddress socAddr = DataNode.getStreamingAddr(conf);
-    int socketWriteTimeout = conf.getInt("dfs.datanode.socket.write.timeout",
+    int socketWriteTimeout = conf.getInt(DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY,
         HdfsConstants.WRITE_TIMEOUT);
     
     ServerSocket ss = (socketWriteTimeout > 0) ? 

+ 3 - 1
src/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java

@@ -22,6 +22,7 @@ import java.net.InetSocketAddress;
 import java.net.SocketTimeoutException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
@@ -138,7 +139,8 @@ public class BackupNode extends NameNode {
   protected void initialize(Configuration conf) throws IOException {
     // Trash is disabled in BackupNameNode,
     // but should be turned back on if it ever becomes active.
-    conf.setLong("fs.trash.interval", 0L);
+    conf.setLong(CommonConfigurationKeys.FS_TRASH_INTERVAL_KEY, 
+                 CommonConfigurationKeys.FS_TRASH_INTERVAL_DEFAULT);
     NamespaceInfo nsInfo = handshake(conf);
     super.initialize(conf);
     // Backup node should never do lease recovery,

+ 11 - 8
src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java

@@ -33,6 +33,7 @@ import java.util.TreeSet;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -53,7 +54,6 @@ public class BlockManager {
   // Default initial capacity and load factor of map
   public static final int DEFAULT_INITIAL_MAP_CAPACITY = 16;
   public static final float DEFAULT_MAP_LOAD_FACTOR = 0.75f;
-  public static final int DEFAULT_MAX_CORRUPT_FILES_RETURNED = 500;
 
   private final FSNamesystem namesystem;
 
@@ -130,9 +130,9 @@ public class BlockManager {
   BlockManager(FSNamesystem fsn, Configuration conf, int capacity)
       throws IOException {
     namesystem = fsn;
-    pendingReplications = new PendingReplicationBlocks(
-        conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY,
-            DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_DEFAULT) * 1000L);
+    pendingReplications = new PendingReplicationBlocks(conf.getInt(
+      DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY,
+      DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_DEFAULT) * 1000L);
     setConfigurationParameters(conf);
     blocksMap = new BlocksMap(capacity, DEFAULT_MAP_LOAD_FACTOR);
   }
@@ -143,10 +143,13 @@ public class BlockManager {
                          namesystem,
                          namesystem.clusterMap);
 
-    this.maxCorruptFilesReturned = conf.getInt("dfs.corruptfilesreturned.max",
-        DEFAULT_MAX_CORRUPT_FILES_RETURNED);
-    this.defaultReplication = conf.getInt("dfs.replication", 3);
-    this.maxReplication = conf.getInt("dfs.replication.max", 512);
+    this.maxCorruptFilesReturned = conf.getInt(
+      DFSConfigKeys.DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED_KEY,
+      DFSConfigKeys.DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED);
+    this.defaultReplication = conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, 
+                                          DFSConfigKeys.DFS_REPLICATION_DEFAULT);
+    this.maxReplication = conf.getInt(DFSConfigKeys.DFS_REPLICATION_MAX_KEY, 
+                                      DFSConfigKeys.DFS_REPLICATION_MAX_DEFAULT);
     this.minReplication = conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY,
                                       DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_DEFAULT);
     if (minReplication <= 0)

+ 5 - 3
src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -352,8 +352,10 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     replthread.start();
 
     this.dnthread = new Daemon(new DecommissionManager(this).new Monitor(
-        conf.getInt("dfs.namenode.decommission.interval", 30),
-        conf.getInt("dfs.namenode.decommission.nodes.per.interval", 5)));
+        conf.getInt(DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY, 
+                    DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_DEFAULT),
+        conf.getInt(DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_NODES_PER_INTERVAL_KEY, 
+                    DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_NODES_PER_INTERVAL_DEFAULT)));
     dnthread.start();
 
     this.dnsToSwitchMapping = ReflectionUtils.newInstance(
@@ -496,7 +498,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
         conf.getLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE),
         conf.getInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, DEFAULT_BYTES_PER_CHECKSUM),
         conf.getInt(DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY, DEFAULT_WRITE_PACKET_SIZE),
-        (short) conf.getInt("dfs.replication", DEFAULT_REPLICATION_FACTOR),
+        (short) conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, DEFAULT_REPLICATION_FACTOR),
         conf.getInt("io.file.buffer.size", DEFAULT_FILE_BUFFER_SIZE));
     this.maxFsObjects = conf.getLong(DFSConfigKeys.DFS_NAMENODE_MAX_OBJECTS_KEY, 
                                      DFSConfigKeys.DFS_NAMENODE_MAX_OBJECTS_DEFAULT);

+ 11 - 8
src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -17,8 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SUPPORT_ALLOW_FORMAT_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SUPPORT_ALLOW_FORMAT_DEFAULT;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 
 import java.io.File;
 import java.io.IOException;
@@ -371,7 +370,9 @@ public class NameNode implements NamenodeProtocols, FSConstants {
     UserGroupInformation.setConfiguration(conf);
     SecurityUtil.login(conf, DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY,
         DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY, socAddr.getHostName());
-    int handlerCount = conf.getInt("dfs.namenode.handler.count", 10);
+    int handlerCount = 
+      conf.getInt(DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_KEY, 
+                  DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_DEFAULT);
 
     NameNode.initMetrics(conf, this.getRole());
     loadNamesystem(conf);
@@ -440,7 +441,9 @@ public class NameNode implements NamenodeProtocols, FSConstants {
   }
 
   private void startTrashEmptier(Configuration conf) throws IOException {
-    long trashInterval = conf.getLong("fs.trash.interval", 0);
+    long trashInterval 
+      = conf.getLong(CommonConfigurationKeys.FS_TRASH_INTERVAL_KEY, 
+                     CommonConfigurationKeys.FS_TRASH_INTERVAL_DEFAULT);
     if(trashInterval == 0)
       return;
     this.emptier = new Thread(new Trash(conf).getEmptier(), "Trash Emptier");
@@ -1412,12 +1415,12 @@ public class NameNode implements NamenodeProtocols, FSConstants {
   private static boolean format(Configuration conf,
                                 boolean isConfirmationNeeded)
       throws IOException {
-    if (!conf.getBoolean(DFS_NAMENODE_SUPPORT_ALLOW_FORMAT_KEY, 
-                         DFS_NAMENODE_SUPPORT_ALLOW_FORMAT_DEFAULT)) {
-      throw new IOException("The option " + DFS_NAMENODE_SUPPORT_ALLOW_FORMAT_KEY
+    if (!conf.getBoolean(DFSConfigKeys.DFS_NAMENODE_SUPPORT_ALLOW_FORMAT_KEY, 
+                         DFSConfigKeys.DFS_NAMENODE_SUPPORT_ALLOW_FORMAT_DEFAULT)) {
+      throw new IOException("The option " + DFSConfigKeys.DFS_NAMENODE_SUPPORT_ALLOW_FORMAT_KEY
                              + " is set to false for this filesystem, so it "
                              + "cannot be formatted. You will need to set "
-                             + DFS_NAMENODE_SUPPORT_ALLOW_FORMAT_KEY +" parameter "
+                             + DFSConfigKeys.DFS_NAMENODE_SUPPORT_ALLOW_FORMAT_KEY +" parameter "
                              + "to true in order to format this filesystem");
     }
     

+ 4 - 2
src/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java

@@ -36,6 +36,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.BlockReader;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -485,7 +486,7 @@ public class NamenodeFsck {
         chosenNode = bestNode(dfs, lblock.getLocations(), deadNodes);
         targetAddr = NetUtils.createSocketAddr(chosenNode.getName());
       }  catch (IOException ie) {
-        if (failures >= DFSClient.MAX_BLOCK_ACQUIRE_FAILURES) {
+        if (failures >= DFSConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_DEFAULT) {
           throw new IOException("Could not obtain block " + lblock);
         }
         LOG.info("Could not obtain block from any node:  " + ie);
@@ -618,7 +619,8 @@ public class NamenodeFsck {
     final short replication;
     
     private Result(Configuration conf) {
-      this.replication = (short)conf.getInt("dfs.replication", 3);
+      this.replication = (short)conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, 
+                                            DFSConfigKeys.DFS_REPLICATION_DEFAULT);
     }
     
     /**

+ 4 - 2
src/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java

@@ -37,6 +37,7 @@ import javax.servlet.jsp.JspWriter;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
@@ -823,8 +824,9 @@ class NamenodeJspHelper {
         doc.endTag();
       }
       
-      doc.startTag("dfs_replication");
-      doc.pcdata(""+conf.getInt("dfs.replication", 3));
+      doc.startTag(DFSConfigKeys.DFS_REPLICATION_KEY);
+      doc.pcdata(""+conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, 
+                                DFSConfigKeys.DFS_REPLICATION_DEFAULT));
       doc.endTag();
       
       doc.startTag("num_missing_blocks");

+ 1 - 1
src/test/aop/org/apache/hadoop/fs/TestFiRename.java

@@ -60,7 +60,7 @@ public class TestFiRename {
   
   private static Configuration CONF = new Configuration();
   static {
-    CONF.setLong("dfs.block.size", 1);
+    CONF.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 1);
     CONF.setInt("io.bytes.per.checksum", 1);
   }
 

+ 2 - 2
src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol.java

@@ -52,8 +52,8 @@ public class TestFiDataTransferProtocol {
 
   static final Configuration conf = new HdfsConfiguration();
   static {
-    conf.setInt("dfs.datanode.handler.count", 1);
-    conf.setInt("dfs.replication", REPLICATION);
+    conf.setInt(DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_KEY, 1);
+    conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, REPLICATION);
     conf.setInt(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 5000);
   }
 

+ 4 - 4
src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol2.java

@@ -53,10 +53,10 @@ public class TestFiDataTransferProtocol2 {
 
   static final Configuration conf = new Configuration();
   static {
-    conf.setInt("dfs.datanode.handler.count", 1);
-    conf.setInt("dfs.replication", REPLICATION);
+    conf.setInt(DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_KEY, 1);
+    conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, REPLICATION);
     conf.setInt(DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY, PACKET_SIZE);
-    conf.setInt("dfs.socket.timeout", 5000);
+    conf.setInt(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 5000);
   }
 
   static final byte[] bytes = new byte[MAX_N_PACKET * PACKET_SIZE];
@@ -284,4 +284,4 @@ public class TestFiDataTransferProtocol2 {
     final String methodName = FiTestUtil.getMethodName();
     runTest34_35(methodName, 0);
   }
-}
+}

+ 2 - 2
src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiPipelineClose.java

@@ -43,8 +43,8 @@ public class TestFiPipelineClose {
 
   static final Configuration conf = new HdfsConfiguration();
   static {
-    conf.setInt("dfs.datanode.handler.count", 1);
-    conf.setInt("dfs.replication", REPLICATION);
+    conf.setInt(DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_KEY, 1);
+    conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, REPLICATION);
     conf.setInt(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 5000);
   }
 

+ 1 - 1
src/test/hdfs/org/apache/hadoop/cli/TestHDFSCLI.java

@@ -45,7 +45,7 @@ public class TestHDFSCLI extends CLITestHelper {
         HDFSPolicyProvider.class, PolicyProvider.class);
     
     // Many of the tests expect a replication value of 1 in the output
-    conf.setInt("dfs.replication", 1);
+    conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 1);
     
     // Build racks and hosts configuration to test dfsAdmin -printTopology
     String [] racks =  {"/rack1", "/rack1", "/rack2", "/rack2",

+ 1 - 1
src/test/hdfs/org/apache/hadoop/fs/loadGenerator/TestLoadGenerator.java

@@ -56,7 +56,7 @@ public class TestLoadGenerator extends Configured implements Tool {
   static {
     CONF.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
     CONF.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, DEFAULT_BLOCK_SIZE);
-    CONF.setLong("dfs.heartbeat.interval", 1L);
+    CONF.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
   }
 
   /** Test if the structure generator works fine */ 

+ 1 - 1
src/test/hdfs/org/apache/hadoop/hdfs/BlockReaderTestUtil.java

@@ -49,7 +49,7 @@ public class BlockReaderTestUtil {
    */
   public BlockReaderTestUtil(int replicationFactor) throws Exception {
     conf = new HdfsConfiguration();
-    conf.setInt("dfs.replication", replicationFactor);
+    conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, replicationFactor);
     cluster = new MiniDFSCluster.Builder(conf).format(true).build();
     cluster.waitActive();
   }

+ 5 - 5
src/test/hdfs/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -494,10 +494,10 @@ public class MiniDFSCluster {
                       false);
     }
     
-    int replication = conf.getInt("dfs.replication", 3);
-    conf.setInt("dfs.replication", Math.min(replication, numDataNodes));
+    int replication = conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, 3);
+    conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, Math.min(replication, numDataNodes));
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_EXTENSION_KEY, 0);
-    conf.setInt("dfs.namenode.decommission.interval", 3); // 3 second
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY, 3); // 3 second
     conf.setClass(DFSConfigKeys.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY, 
                    StaticMapping.class, DNSToSwitchMapping.class);
     
@@ -742,8 +742,8 @@ public class MiniDFSCluster {
 
     int curDatanodesNum = dataNodes.size();
     // for mincluster's the default initialDelay for BRs is 0
-    if (conf.get("dfs.blockreport.initialDelay") == null) {
-      conf.setLong("dfs.blockreport.initialDelay", 0);
+    if (conf.get(DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY) == null) {
+      conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY, 0);
     }
     // If minicluster's name node is null assume that the conf has been
     // set with the right address:port of the name node.

+ 2 - 2
src/test/hdfs/org/apache/hadoop/hdfs/TestCrcCorruption.java

@@ -205,7 +205,7 @@ public class TestCrcCorruption {
     //
     System.out.println("TestCrcCorruption with default parameters");
     Configuration conf1 = new HdfsConfiguration();
-    conf1.setInt("dfs.blockreport.intervalMsec", 3 * 1000);
+    conf1.setInt(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 3 * 1000);
     DFSTestUtil util1 = new DFSTestUtil("TestCrcCorruption", 40, 3, 8*1024);
     thistest(conf1, util1);
 
@@ -249,7 +249,7 @@ public class TestCrcCorruption {
     Path file = new Path("/testFile");
 
     Configuration conf = new Configuration();
-    conf.setInt("dfs.replication", numDataNodes);
+    conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, numDataNodes);
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDataNodes).build();
 
     try {

+ 8 - 7
src/test/hdfs/org/apache/hadoop/hdfs/TestDFSClientRetries.java

@@ -42,6 +42,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -83,11 +84,11 @@ public class TestDFSClientRetries extends TestCase {
     
     final int writeTimeout = 100; //milliseconds.
     // set a very short write timeout for datanode, so that tests runs fast.
-    conf.setInt("dfs.datanode.socket.write.timeout", writeTimeout); 
+    conf.setInt(DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY, writeTimeout); 
     // set a smaller block size
     final int blockSize = 10*1024*1024;
     conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
-    conf.setInt("dfs.client.max.block.acquire.failures", 1);
+    conf.setInt(DFSConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY, 1);
     // set a small buffer size
     final int bufferSize = 4096;
     conf.setInt("io.file.buffer.size", bufferSize);
@@ -136,8 +137,7 @@ public class TestDFSClientRetries extends TestCase {
     final String exceptionMsg = "Nope, not replicated yet...";
     final int maxRetries = 1; // Allow one retry (total of two calls)
     Configuration conf = new HdfsConfiguration();
-   conf.setInt(DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY, 
-                maxRetries);
+    conf.setInt(DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY, maxRetries);
     
     NameNode mockNN = mock(NameNode.class);
     Answer<Object> answer = new ThrowsException(new IOException()) {
@@ -380,9 +380,10 @@ public class TestDFSClientRetries extends TestCase {
     int bufferSize = 4096;
     
     Configuration conf = new HdfsConfiguration();
-    conf.setInt("dfs.datanode.max.xcievers",xcievers);
-    conf.setInt("dfs.client.max.block.acquire.failures", retries);
-    conf.setInt("dfs.client.retry.window.base", timeWin);
+    conf.setInt(DFSConfigKeys.DFS_DATANODE_MAX_XCIEVERS_KEY,xcievers);
+    conf.setInt(DFSConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY, 
+                retries);
+    conf.setInt(DFSConfigKeys.DFS_CLIENT_RETRY_WINDOW_BASE, timeWin);
 
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(replicationFactor).build();
     cluster.waitActive();

+ 1 - 1
src/test/hdfs/org/apache/hadoop/hdfs/TestDFSFinalize.java

@@ -89,7 +89,7 @@ public class TestDFSFinalize extends TestCase {
        * not changed.
        */
       conf = new HdfsConfiguration();
-      conf.setInt("dfs.datanode.scan.period.hours", -1);
+      conf.setInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1);
       conf = UpgradeUtilities.initializeStorageStateConf(numDirs, conf);
       String[] nameNodeDirs = conf.getStrings(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY);
       String[] dataNodeDirs = conf.getStrings(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY);

+ 1 - 1
src/test/hdfs/org/apache/hadoop/hdfs/TestDFSRollback.java

@@ -128,7 +128,7 @@ public class TestDFSRollback extends TestCase {
     StorageInfo storageInfo = null;
     for (int numDirs = 1; numDirs <= 2; numDirs++) {
       conf = new HdfsConfiguration();
-      conf.setInt("dfs.datanode.scan.period.hours", -1);      
+      conf.setInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1);      
       conf = UpgradeUtilities.initializeStorageStateConf(numDirs, conf);
       String[] nameNodeDirs = conf.getStrings(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY);
       String[] dataNodeDirs = conf.getStrings(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY);

+ 2 - 2
src/test/hdfs/org/apache/hadoop/hdfs/TestDFSStorageStateRecovery.java

@@ -311,7 +311,7 @@ public class TestDFSStorageStateRecovery extends TestCase {
 
     for (int numDirs = 1; numDirs <= 2; numDirs++) {
       conf = new HdfsConfiguration();
-      conf.setInt("dfs.datanode.scan.period.hours", -1);      
+      conf.setInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1);      
       conf = UpgradeUtilities.initializeStorageStateConf(numDirs, conf);
       for (int i = 0; i < NUM_NN_TEST_CASES; i++) {
         boolean[] testCase = testCases[i];
@@ -355,7 +355,7 @@ public class TestDFSStorageStateRecovery extends TestCase {
     // First setup the datanode storage directory
     for (int numDirs = 1; numDirs <= 2; numDirs++) {
       conf = new HdfsConfiguration();
-      conf.setInt("dfs.datanode.scan.period.hours", -1);      
+      conf.setInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1);      
       conf = UpgradeUtilities.initializeStorageStateConf(numDirs, conf);
       for (int i = 0; i < NUM_DN_TEST_CASES; i++) {
         boolean[] testCase = testCases[i];

+ 1 - 1
src/test/hdfs/org/apache/hadoop/hdfs/TestDFSUpgrade.java

@@ -150,7 +150,7 @@ public class TestDFSUpgrade extends TestCase {
     StorageInfo storageInfo = null;
     for (int numDirs = 1; numDirs <= 2; numDirs++) {
       conf = new HdfsConfiguration();
-      conf.setInt("dfs.datanode.scan.period.hours", -1);      
+      conf.setInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1);      
       conf = UpgradeUtilities.initializeStorageStateConf(numDirs, conf);
       String[] nameNodeDirs = conf.getStrings(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY);
       String[] dataNodeDirs = conf.getStrings(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY);

+ 1 - 1
src/test/hdfs/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java

@@ -184,7 +184,7 @@ public class TestDFSUpgradeFromImage extends TestCase {
       if (System.getProperty("test.build.data") == null) { // to allow test to be run outside of Ant
         System.setProperty("test.build.data", "build/test/data");
       }
-      conf.setInt("dfs.datanode.scan.period.hours", -1); // block scanning off
+      conf.setInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1); // block scanning off
       cluster = new MiniDFSCluster.Builder(conf)
                                   .numDataNodes(numDataNodes)
                                   .format(false)

+ 1 - 1
src/test/hdfs/org/apache/hadoop/hdfs/TestDataTransferProtocol.java

@@ -324,7 +324,7 @@ public class TestDataTransferProtocol extends TestCase {
     int numDataNodes = 1;
     
     Configuration conf = new HdfsConfiguration();
-    conf.setInt("dfs.replication", numDataNodes); 
+    conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, numDataNodes); 
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDataNodes).build();
     try {
     cluster.waitActive();

+ 3 - 3
src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeBlockScanner.java

@@ -148,7 +148,7 @@ public class TestDatanodeBlockScanner extends TestCase {
 
   public void testBlockCorruptionPolicy() throws IOException {
     Configuration conf = new HdfsConfiguration();
-    conf.setLong("dfs.blockreport.intervalMsec", 1000L);
+    conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 1000L);
     Random random = new Random();
     FileSystem fs = null;
     DFSClient dfsClient = null;
@@ -258,9 +258,9 @@ public class TestDatanodeBlockScanner extends TestCase {
                                              int numCorruptReplicas) 
                                              throws IOException {
     Configuration conf = new HdfsConfiguration();
-    conf.setLong("dfs.blockreport.intervalMsec", 30L);
+    conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 30L);
     conf.setLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 30);
-    conf.setLong("dfs.heartbeat.interval", 30L);
+    conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 30L);
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, false);
     FileSystem fs = null;
     DFSClient dfsClient = null;

+ 2 - 2
src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeDeath.java

@@ -285,7 +285,7 @@ public class TestDatanodeDeath extends TestCase {
   private void complexTest() throws IOException {
     Configuration conf = new HdfsConfiguration();
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 2000);
-    conf.setInt("dfs.heartbeat.interval", 2);
+    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 2);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY, 2);
     conf.setInt(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 5000);
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
@@ -341,7 +341,7 @@ public class TestDatanodeDeath extends TestCase {
   private void simpleTest(int datanodeToKill) throws IOException {
     Configuration conf = new HdfsConfiguration();
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 2000);
-    conf.setInt("dfs.heartbeat.interval", 1);
+    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY, 2);
     conf.setInt(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 5000);
     int myMaxNodes = 5;

+ 1 - 1
src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeReport.java

@@ -41,7 +41,7 @@ public class TestDatanodeReport extends TestCase {
   public void testDatanodeReport() throws Exception {
     conf.setInt(
         DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 500); // 0.5s
-    conf.setLong("dfs.heartbeat.interval", 1L);
+    conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
     MiniDFSCluster cluster = 
       new MiniDFSCluster.Builder(conf).numDataNodes(NUM_OF_DATANODES).build();
     try {

+ 4 - 4
src/test/hdfs/org/apache/hadoop/hdfs/TestFileAppend2.java

@@ -83,7 +83,7 @@ public class TestFileAppend2 extends TestCase {
     if (simulatedStorage) {
       conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
     }
-    conf.setInt("dfs.datanode.handler.count", 50);
+    conf.setInt(DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_KEY, 50);
     conf.setBoolean("dfs.support.append", true);
     fileContents = AppendTestUtil.initBuffer(AppendTestUtil.FILE_SIZE);
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
@@ -332,11 +332,11 @@ public class TestFileAppend2 extends TestCase {
     fileContents = AppendTestUtil.initBuffer(AppendTestUtil.FILE_SIZE);
     Configuration conf = new HdfsConfiguration();
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 2000);
-    conf.setInt("dfs.heartbeat.interval", 2);
+    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 2);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY, 2);
     conf.setInt(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 30000);
-    conf.setInt("dfs.datanode.socket.write.timeout", 30000);
-    conf.setInt("dfs.datanode.handler.count", 50);
+    conf.setInt(DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY, 30000);
+    conf.setInt(DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_KEY, 50);
     conf.setBoolean("dfs.support.append", true);
 
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)

+ 1 - 1
src/test/hdfs/org/apache/hadoop/hdfs/TestFileConcurrentReader.java

@@ -196,7 +196,7 @@ public class TestFileConcurrentReader extends junit.framework.TestCase {
     final int writeSize = 10 * blockSize;
     Configuration conf = new Configuration();
     
-    conf.setLong("dfs.block.size", blockSize);
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
     init(conf);
 
     final int requiredSuccessfulOpens = 100;

+ 5 - 5
src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreation.java

@@ -102,7 +102,7 @@ public class TestFileCreation extends junit.framework.TestCase {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, FSConstants.DEFAULT_BLOCK_SIZE);
     conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, FSConstants.DEFAULT_BYTES_PER_CHECKSUM);
     conf.setInt(DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY, FSConstants.DEFAULT_WRITE_PACKET_SIZE);
-    conf.setInt("dfs.replication", FSConstants.DEFAULT_REPLICATION_FACTOR + 1);
+    conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, FSConstants.DEFAULT_REPLICATION_FACTOR + 1);
     conf.setInt("io.file.buffer.size", FSConstants.DEFAULT_FILE_BUFFER_SIZE);
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
                      .numDataNodes(FSConstants.DEFAULT_REPLICATION_FACTOR + 1)
@@ -269,7 +269,7 @@ public class TestFileCreation extends junit.framework.TestCase {
   public void testFileCreationError1() throws IOException {
     Configuration conf = new HdfsConfiguration();
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
-    conf.setInt("dfs.heartbeat.interval", 1);
+    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
     if (simulatedStorage) {
       conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
     }
@@ -343,7 +343,7 @@ public class TestFileCreation extends junit.framework.TestCase {
     System.out.println("testFileCreationError2 start");
     Configuration conf = new HdfsConfiguration();
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
-    conf.setInt("dfs.heartbeat.interval", 1);
+    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
     if (simulatedStorage) {
       conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
     }
@@ -412,7 +412,7 @@ public class TestFileCreation extends junit.framework.TestCase {
     final int MAX_IDLE_TIME = 2000; // 2s
     conf.setInt("ipc.client.connection.maxidletime", MAX_IDLE_TIME);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
-    conf.setInt("dfs.heartbeat.interval", 1);
+    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
     if (simulatedStorage) {
       conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
     }
@@ -762,7 +762,7 @@ public class TestFileCreation extends junit.framework.TestCase {
 
     Configuration conf = new HdfsConfiguration();
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
-    conf.setInt("dfs.heartbeat.interval", 1);
+    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
 
     // create cluster
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(DATANODE_NUM).build();

+ 2 - 2
src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreationClient.java

@@ -48,8 +48,8 @@ public class TestFileCreationClient extends junit.framework.TestCase {
   public void testClientTriggeredLeaseRecovery() throws Exception {
     final int REPLICATION = 3;
     Configuration conf = new HdfsConfiguration();
-    conf.setInt("dfs.datanode.handler.count", 1);
-    conf.setInt("dfs.replication", REPLICATION);
+    conf.setInt(DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_KEY, 1);
+    conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, REPLICATION);
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION).build();
 
     try {

+ 1 - 1
src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreationDelete.java

@@ -41,7 +41,7 @@ public class TestFileCreationDelete extends junit.framework.TestCase {
     final int MAX_IDLE_TIME = 2000; // 2s
     conf.setInt("ipc.client.connection.maxidletime", MAX_IDLE_TIME);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
-    conf.setInt("dfs.heartbeat.interval", 1);
+    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
     conf.setBoolean("dfs.support.append", true);
 
     // create cluster

+ 1 - 1
src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreationEmpty.java

@@ -52,7 +52,7 @@ public class TestFileCreationEmpty extends junit.framework.TestCase {
 
     final Configuration conf = new HdfsConfiguration();
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
-    conf.setInt("dfs.heartbeat.interval", 1);
+    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
 
     // create cluster
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(DATANODE_NUM).build();

+ 1 - 1
src/test/hdfs/org/apache/hadoop/hdfs/TestInjectionForSimulatedStorage.java

@@ -134,7 +134,7 @@ public class TestInjectionForSimulatedStorage extends TestCase {
     
     try {
       Configuration conf = new HdfsConfiguration();
-      conf.set("dfs.replication", Integer.toString(numDataNodes));
+      conf.set(DFSConfigKeys.DFS_REPLICATION_KEY, Integer.toString(numDataNodes));
       conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, checksumSize);
       conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
       //first time format

+ 1 - 1
src/test/hdfs/org/apache/hadoop/hdfs/TestLeaseRecovery2.java

@@ -70,7 +70,7 @@ public class TestLeaseRecovery2 {
   @BeforeClass
   public static void startUp() throws IOException {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
-    conf.setInt("dfs.heartbeat.interval", 1);
+    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
 
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(5).build();
     cluster.waitActive();

+ 2 - 1
src/test/hdfs/org/apache/hadoop/hdfs/TestPread.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 
 /**
@@ -164,7 +165,7 @@ public class TestPread extends TestCase {
       return;
     }
     int numBlocks = 1;
-    assertTrue(numBlocks <= DFSClient.MAX_BLOCK_ACQUIRE_FAILURES);
+    assertTrue(numBlocks <= DFSConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_DEFAULT);
     byte[] expected = new byte[numBlocks * blockSize];
     Random rand = new Random(seed);
     rand.nextBytes(expected);

+ 1 - 1
src/test/hdfs/org/apache/hadoop/hdfs/TestReadWhileWriting.java

@@ -56,7 +56,7 @@ public class TestReadWhileWriting {
     final Configuration conf = new HdfsConfiguration();
     //enable append
     conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
-    conf.setLong("dfs.heartbeat.interval", 1);
+    conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
 
     // create cluster
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();

+ 4 - 4
src/test/hdfs/org/apache/hadoop/hdfs/TestRenameWhileOpen.java

@@ -51,7 +51,7 @@ public class TestRenameWhileOpen extends junit.framework.TestCase {
     final int MAX_IDLE_TIME = 2000; // 2s
     conf.setInt("ipc.client.connection.maxidletime", MAX_IDLE_TIME);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
-    conf.setInt("dfs.heartbeat.interval", 1);
+    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY, 1);
     conf.setBoolean("dfs.support.append", true);
 
@@ -137,7 +137,7 @@ public class TestRenameWhileOpen extends junit.framework.TestCase {
     final int MAX_IDLE_TIME = 2000; // 2s
     conf.setInt("ipc.client.connection.maxidletime", MAX_IDLE_TIME);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
-    conf.setInt("dfs.heartbeat.interval", 1);
+    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY, 1);
     conf.setBoolean("dfs.support.append", true);
     System.out.println("Test 2************************************");
@@ -212,7 +212,7 @@ public class TestRenameWhileOpen extends junit.framework.TestCase {
     final int MAX_IDLE_TIME = 2000; // 2s
     conf.setInt("ipc.client.connection.maxidletime", MAX_IDLE_TIME);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
-    conf.setInt("dfs.heartbeat.interval", 1);
+    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY, 1);
     conf.setBoolean("dfs.support.append", true);
     System.out.println("Test 3************************************");
@@ -277,7 +277,7 @@ public class TestRenameWhileOpen extends junit.framework.TestCase {
     final int MAX_IDLE_TIME = 2000; // 2s
     conf.setInt("ipc.client.connection.maxidletime", MAX_IDLE_TIME);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
-    conf.setInt("dfs.heartbeat.interval", 1);
+    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY, 1);
     conf.setBoolean("dfs.support.append", true);
     System.out.println("Test 4************************************");

+ 2 - 2
src/test/hdfs/org/apache/hadoop/hdfs/TestReplication.java

@@ -311,7 +311,7 @@ public class TestReplication extends TestCase {
     
     try {
       Configuration conf = new HdfsConfiguration();
-      conf.set("dfs.replication", Integer.toString(numDataNodes));
+      conf.set(DFSConfigKeys.DFS_REPLICATION_KEY, Integer.toString(numDataNodes));
       //first time format
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDataNodes).build();
       cluster.waitActive();
@@ -371,7 +371,7 @@ public class TestReplication extends TestCase {
       
       LOG.info("Restarting minicluster after deleting a replica and corrupting 2 crcs");
       conf = new HdfsConfiguration();
-      conf.set("dfs.replication", Integer.toString(numDataNodes));
+      conf.set(DFSConfigKeys.DFS_REPLICATION_KEY, Integer.toString(numDataNodes));
       conf.set(DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY, Integer.toString(2));
       conf.set("dfs.datanode.block.write.timeout.sec", Integer.toString(5));
       conf.set(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY, "0.75f"); // only 3 copies exist

+ 3 - 3
src/test/hdfs/org/apache/hadoop/hdfs/TestSetTimes.java

@@ -81,7 +81,7 @@ public class TestSetTimes extends TestCase {
     final int MAX_IDLE_TIME = 2000; // 2s
     conf.setInt("ipc.client.connection.maxidletime", MAX_IDLE_TIME);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
-    conf.setInt("dfs.heartbeat.interval", 1);
+    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
 
 
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
@@ -197,8 +197,8 @@ public class TestSetTimes extends TestCase {
     // parameter initialization
     conf.setInt("ipc.client.connection.maxidletime", MAX_IDLE_TIME);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
-    conf.setInt("dfs.heartbeat.interval", 1);
-    conf.setInt("dfs.datanode.handler.count", 50);
+    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
+    conf.setInt(DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_KEY, 50);
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
                                                .numDataNodes(numDatanodes)
                                                .build();

+ 2 - 2
src/test/hdfs/org/apache/hadoop/hdfs/TestSetrepIncreasing.java

@@ -30,8 +30,8 @@ public class TestSetrepIncreasing extends TestCase {
     if (simulatedStorage) {
       conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
     }
-    conf.set("dfs.replication", "" + fromREP);
-    conf.setLong("dfs.blockreport.intervalMsec", 1000L);
+    conf.set(DFSConfigKeys.DFS_REPLICATION_KEY, "" + fromREP);
+    conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 1000L);
     conf.set(DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY, Integer.toString(2));
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(10).build();
     FileSystem fs = cluster.getFileSystem();

+ 1 - 1
src/test/hdfs/org/apache/hadoop/hdfs/UpgradeUtilities.java

@@ -187,7 +187,7 @@ public class UpgradeUtilities {
     conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, nameNodeDirs.toString());
     conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, nameNodeDirs.toString());
     conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dataNodeDirs.toString());
-    conf.setInt("dfs.blockreport.intervalMsec", 10000);
+    conf.setInt(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 10000);
     return conf;
   }
   

+ 2 - 1
src/test/hdfs/org/apache/hadoop/hdfs/server/common/TestDistributedUpgrade.java

@@ -26,6 +26,7 @@ import junit.framework.TestCase;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 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.hdfs.TestDFSUpgradeFromImage;
@@ -111,7 +112,7 @@ public class TestDistributedUpgrade extends TestCase {
     if (System.getProperty("test.build.data") == null) { // to test to be run outside of ant
       System.setProperty("test.build.data", "build/test/data");
     }
-    conf.setInt("dfs.datanode.scan.period.hours", -1); // block scanning off
+    conf.setInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1); // block scanning off
 
     log("NameNode start in regular mode when dustributed upgrade is required", numDirs);
     startNameNodeShouldFail(StartupOption.REGULAR);

+ 1 - 1
src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java

@@ -93,7 +93,7 @@ public class TestBlockReplacement extends TestCase {
     
     CONF.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
     CONF.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, DEFAULT_BLOCK_SIZE/2);
-    CONF.setLong("dfs.blockreport.intervalMsec",500);
+    CONF.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY,500);
     cluster = new MiniDFSCluster.Builder(CONF).numDataNodes(REPLICATION_FACTOR)
                                               .racks(INITIAL_RACKS).build();
 

+ 1 - 1
src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java

@@ -59,7 +59,7 @@ public class TestDirectoryScanner extends TestCase {
   static {
     CONF.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 100);
     CONF.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 1);
-    CONF.setLong("dfs.heartbeat.interval", 1L);
+    CONF.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
   }
 
   /** create a file with a length of <code>fileLen</code> */

+ 3 - 3
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java

@@ -104,7 +104,7 @@ public class NNThroughputBenchmark {
     config = conf;
     // We do not need many handlers, since each thread simulates a handler
     // by calling name-node methods directly
-    config.setInt("dfs.namenode.handler.count", 1);
+    config.setInt(DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_KEY, 1);
     // set exclude file
     config.set(DFSConfigKeys.DFS_HOSTS_EXCLUDE,
       "${hadoop.tmp.dir}/dfs/hosts/exclude");
@@ -208,7 +208,7 @@ public class NNThroughputBenchmark {
 
     OperationStatsBase() {
       baseDir = BASE_DIR_NAME + "/" + getOpName();
-      replication = (short) config.getInt("dfs.replication", 3);
+      replication = (short) config.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, 3);
       numOpsRequired = 10;
       numThreads = 3;
       logLevel = Level.ERROR;
@@ -908,7 +908,7 @@ public class NNThroughputBenchmark {
       this.blocksPerReport = 100;
       this.blocksPerFile = 10;
       // set heartbeat interval to 3 min, so that expiration were 40 min
-      config.setLong("dfs.heartbeat.interval", 3 * 60);
+      config.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 3 * 60);
       parseArguments(args);
       // adjust replication to the number of data-nodes
       this.replication = (short)Math.min((int)replication, getNumDatanodes());

+ 1 - 1
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java

@@ -104,7 +104,7 @@ public class OfflineEditsViewerHelper {
     config.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY,
       Util.fileAsURI(new File(dfsDir, "namesecondary1")).toString());
     // blocksize for concat (file size must be multiple of blocksize)
-    config.setLong("dfs.blocksize", blockSize);
+    config.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
     // for security to work (fake JobTracker user)
     config.set("hadoop.security.auth_to_local",
       "RULE:[2:$1@$0](JobTracker@.*FOO.COM)s/@.*//" + "DEFAULT");

+ 3 - 3
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java

@@ -108,9 +108,9 @@ public class TestBackupNode extends TestCase {
     Path file2 = new Path("checkpoint2.dat");
 
     Configuration conf = new HdfsConfiguration();
-    short replication = (short)conf.getInt("dfs.replication", 3);
-    conf.set("dfs.blockreport.initialDelay", "0");
-    conf.setInt("dfs.datanode.scan.period.hours", -1); // disable block scanner
+    short replication = (short)conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, 3);
+    conf.set(DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY, "0");
+    conf.setInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1); // disable block scanner
     int numDatanodes = Math.max(3, replication);
     MiniDFSCluster cluster = null;
     FileSystem fileSys = null;

+ 3 - 3
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBlockTokenWithDFS.java

@@ -164,10 +164,10 @@ public class TestBlockTokenWithDFS extends TestCase {
   private static Configuration getConf(int numDataNodes) throws IOException {
     Configuration conf = new Configuration();
     conf.setBoolean(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
-    conf.setLong("dfs.block.size", BLOCK_SIZE);
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
     conf.setInt("io.bytes.per.checksum", BLOCK_SIZE);
-    conf.setInt("dfs.heartbeat.interval", 1);
-    conf.setInt("dfs.replication", numDataNodes);
+    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
+    conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, numDataNodes);
     conf.setInt("ipc.client.connect.max.retries", 0);
     conf.setBoolean("dfs.support.append", true);
     return conf;

+ 2 - 2
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBlocksWithNotEnoughRacks.java

@@ -50,7 +50,7 @@ public class TestBlocksWithNotEnoughRacks extends TestCase {
   //The block should be replicated to the new rack
   public void testSufficientlyReplicatedBlocksWithNotEnoughRacks() throws Exception {
     Configuration conf = new HdfsConfiguration();
-    conf.setLong("dfs.heartbeat.interval", 1L);
+    conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1);
     conf.set(DFSConfigKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY, "xyz");
     final short REPLICATION_FACTOR = 3;
@@ -103,7 +103,7 @@ public class TestBlocksWithNotEnoughRacks extends TestCase {
 
   public void testUnderReplicatedNotEnoughRacks() throws Exception {
     Configuration conf = new HdfsConfiguration();
-    conf.setLong("dfs.heartbeat.interval", 1L);
+    conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY, 1);
     conf.set(DFSConfigKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY, "xyz");

+ 1 - 1
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java

@@ -645,7 +645,7 @@ public class TestCheckpoint extends TestCase {
 
     Configuration conf = new HdfsConfiguration();
     conf.set(DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY, "0.0.0.0:0");
-    replication = (short)conf.getInt("dfs.replication", 3);
+    replication = (short)conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, 3);
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
                                                .numDataNodes(numDatanodes).build();
     cluster.waitActive();

+ 2 - 2
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCorruptFilesJsp.java

@@ -50,9 +50,9 @@ public class TestCorruptFilesJsp  {
 
       Configuration conf = new HdfsConfiguration();
       // datanode scans directories
-      conf.setInt("dfs.datanode.directoryscan.interval", 1);
+      conf.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY, 1);
       // datanode sends block reports
-      conf.setInt("dfs.blockreport.intervalMsec", 3 * 1000);
+      conf.setInt(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 3 * 1000);
       cluster = new MiniDFSCluster.Builder(conf).build();
       cluster.waitActive();
 

+ 1 - 1
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java

@@ -75,7 +75,7 @@ public class TestDecommissioningStatus {
     conf.set(DFSConfigKeys.DFS_HOSTS_EXCLUDE, excludeFile.toUri().getPath());
     conf
         .setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 2000);
-    conf.setInt("dfs.heartbeat.interval", 1);
+    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY,
         4);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1000);

+ 2 - 2
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestFileLimit.java

@@ -78,8 +78,8 @@ public class TestFileLimit extends TestCase {
     Configuration conf = new HdfsConfiguration();
     int maxObjects = 5;
     conf.setLong(DFSConfigKeys.DFS_NAMENODE_MAX_OBJECTS_KEY, maxObjects);
-    conf.setLong("dfs.blockreport.intervalMsec", 1000L);
-    conf.setInt("dfs.heartbeat.interval", 1);
+    conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 1000L);
+    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
     int currentNodes = 0;
     
     if (simulatedStorage) {

+ 9 - 9
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestFsck.java

@@ -94,7 +94,7 @@ public class TestFsck extends TestCase {
       Configuration conf = new HdfsConfiguration();
       final long precision = 1L;
       conf.setLong(DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, precision);
-      conf.setLong("dfs.blockreport.intervalMsec", 10000L);
+      conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 10000L);
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(4).build();
       fs = cluster.getFileSystem();
       final String fileName = "/srcdat";
@@ -164,7 +164,7 @@ public class TestFsck extends TestCase {
     FileSystem fs = null;
     try {
       Configuration conf = new HdfsConfiguration();
-      conf.setLong("dfs.blockreport.intervalMsec", 10000L);
+      conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 10000L);
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(4).build();
       fs = cluster.getFileSystem();
       util.createFiles(fs, "/srcdat");
@@ -183,7 +183,7 @@ public class TestFsck extends TestCase {
   public void testFsckPermission() throws Exception {
     final DFSTestUtil util = new DFSTestUtil(getClass().getSimpleName(), 20, 3, 8*1024);
     final Configuration conf = new HdfsConfiguration();
-    conf.setLong("dfs.blockreport.intervalMsec", 10000L);
+    conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 10000L);
 
     MiniDFSCluster cluster = null;
     try {
@@ -233,8 +233,8 @@ public class TestFsck extends TestCase {
     FileSystem fs = null;
     try {
       Configuration conf = new HdfsConfiguration();
-      conf.setLong("dfs.blockreport.intervalMsec", 10000L);
-      conf.setInt("dfs.datanode.directoryscan.interval", 1);
+      conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 10000L);
+      conf.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY, 1);
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(4).build();
       String topDir = "/srcdat";
       fs = cluster.getFileSystem();
@@ -289,7 +289,7 @@ public class TestFsck extends TestCase {
     FileSystem fs = null;
     try {
       Configuration conf = new HdfsConfiguration();
-      conf.setLong("dfs.blockreport.intervalMsec", 10000L);
+      conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 10000L);
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(4).build();
       String topDir = "/srcdat";
       String randomString = "HADOOP  ";
@@ -335,7 +335,7 @@ public class TestFsck extends TestCase {
 
   public void testCorruptBlock() throws Exception {
     Configuration conf = new HdfsConfiguration();
-    conf.setLong("dfs.blockreport.intervalMsec", 1000);
+    conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 1000);
     FileSystem fs = null;
     DFSClient dfsClient = null;
     LocatedBlocks blocks = null;
@@ -446,8 +446,8 @@ public class TestFsck extends TestCase {
   /** check if option -list-corruptfiles of fsck command works properly */
   public void testFsckListCorruptFilesBlocks() throws Exception {
     Configuration conf = new Configuration();
-    conf.setLong("dfs.blockreport.intervalMsec", 1000);
-    conf.setInt("dfs.datanode.directoryscan.interval", 1);
+    conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 1000);
+    conf.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY, 1);
     FileSystem fs = null;
 
     MiniDFSCluster cluster = null;

+ 2 - 1
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestHDFSConcat.java

@@ -33,6 +33,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -60,7 +61,7 @@ public class TestHDFSConcat {
 
   static {
     conf = new Configuration();
-    conf.setLong("dfs.blocksize", blockSize);
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
   }
   
   @Before

+ 10 - 10
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestListCorruptFileBlocks.java

@@ -62,8 +62,8 @@ public class TestListCorruptFileBlocks {
     
     try {
       Configuration conf = new HdfsConfiguration();
-      conf.setInt("dfs.datanode.directoryscan.interval", 1); // datanode scans directories
-      conf.setInt("dfs.blockreport.intervalMsec", 3 * 1000); // datanode sends block reports
+      conf.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY, 1); // datanode scans directories
+      conf.setInt(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 3 * 1000); // datanode sends block reports
       cluster = new MiniDFSCluster.Builder(conf).build();
       FileSystem fs = cluster.getFileSystem();
 
@@ -137,9 +137,9 @@ public class TestListCorruptFileBlocks {
     try {
       Configuration conf = new HdfsConfiguration();
       // datanode scans directories
-      conf.setInt("dfs.datanode.directoryscan.interval", 1);
+      conf.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY, 1);
       // datanode sends block reports
-      conf.setInt("dfs.blockreport.intervalMsec", 3 * 1000);
+      conf.setInt(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 3 * 1000);
       // never leave safemode automatically
       conf.setFloat(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY,
                     1.5f);
@@ -262,8 +262,8 @@ public class TestListCorruptFileBlocks {
   @Test
   public void testlistCorruptFileBlocks() throws Exception {
     Configuration conf = new Configuration();
-    conf.setLong("dfs.blockreport.intervalMsec", 1000);
-    conf.setInt("dfs.datanode.directoryscan.interval", 1); // datanode scans
+    conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 1000);
+    conf.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY, 1); // datanode scans
                                                            // directories
     FileSystem fs = null;
 
@@ -369,8 +369,8 @@ public class TestListCorruptFileBlocks {
   @Test
   public void testlistCorruptFileBlocksDFS() throws Exception {
     Configuration conf = new Configuration();
-    conf.setLong("dfs.blockreport.intervalMsec", 1000);
-    conf.setInt("dfs.datanode.directoryscan.interval", 1); // datanode scans
+    conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 1000);
+    conf.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY, 1); // datanode scans
                                                            // directories
     FileSystem fs = null;
 
@@ -442,8 +442,8 @@ public class TestListCorruptFileBlocks {
     MiniDFSCluster cluster = null;
     try {
       Configuration conf = new HdfsConfiguration();
-      conf.setInt("dfs.datanode.directoryscan.interval", 15); // datanode scans directories
-      conf.setInt("dfs.blockreport.intervalMsec", 3 * 1000); // datanode sends block reports
+      conf.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY, 15); // datanode scans directories
+      conf.setInt(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 3 * 1000); // datanode sends block reports
       cluster = new MiniDFSCluster.Builder(conf).build();
       FileSystem fs = cluster.getFileSystem();
       final int maxCorruptFileBlocks = 

+ 9 - 9
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java

@@ -133,7 +133,7 @@ public class TestNameEditsConfigs extends TestCase {
     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);
+    replication = (short)conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, 3);
     // Manage our own dfs directories
     cluster = new MiniDFSCluster.Builder(conf)
                                 .numDataNodes(NUM_DATA_NODES)
@@ -167,7 +167,7 @@ public class TestNameEditsConfigs extends TestCase {
              "," + checkpointNameAndEdits.getPath());
     conf.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_EDITS_DIR_KEY, checkpointEditsDir.getPath() +
              "," + checkpointNameAndEdits.getPath());
-    replication = (short)conf.getInt("dfs.replication", 3);
+    replication = (short)conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, 3);
     // Manage our own dfs directories. Do not format.
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATA_NODES)
                                               .format(false)
@@ -213,7 +213,7 @@ public class TestNameEditsConfigs extends TestCase {
     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);
+    replication = (short)conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, 3);
     cluster = new MiniDFSCluster.Builder(conf)
                                 .numDataNodes(NUM_DATA_NODES)
                                 .format(false)
@@ -256,7 +256,7 @@ public class TestNameEditsConfigs extends TestCase {
         "," + checkpointNameAndEdits.getPath());
     conf.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_EDITS_DIR_KEY, checkpointEditsDir.getPath() +
         "," + checkpointNameAndEdits.getPath());
-    replication = (short)conf.getInt("dfs.replication", 3);
+    replication = (short)conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, 3);
     cluster = new MiniDFSCluster.Builder(conf)
                                 .numDataNodes(NUM_DATA_NODES)
                                 .format(false)
@@ -307,7 +307,7 @@ public class TestNameEditsConfigs extends TestCase {
     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);
+    replication = (short)conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, 3);
     // Manage our own dfs directories
     cluster = new MiniDFSCluster.Builder(conf)
                                 .numDataNodes(NUM_DATA_NODES)
@@ -334,7 +334,7 @@ public class TestNameEditsConfigs extends TestCase {
               "," + newNameDir.getPath());
     conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, nameAndEdits.getPath() +
               "," + newEditsDir.getPath());
-    replication = (short)conf.getInt("dfs.replication", 3);
+    replication = (short)conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, 3);
     // Manage our own dfs directories. Do not format.
     cluster = new MiniDFSCluster.Builder(conf)
                                 .numDataNodes(NUM_DATA_NODES)
@@ -360,7 +360,7 @@ public class TestNameEditsConfigs extends TestCase {
     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);
+    replication = (short)conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, 3);
     cluster = new MiniDFSCluster.Builder(conf)
                                 .numDataNodes(NUM_DATA_NODES)
                                 .format(false)
@@ -386,7 +386,7 @@ public class TestNameEditsConfigs extends TestCase {
     conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, newNameDir.getPath() + "," + 
              nameAndEdits.getPath());
     conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, nameAndEdits.getPath());
-    replication = (short)conf.getInt("dfs.replication", 3);
+    replication = (short)conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, 3);
     try {
       cluster = new MiniDFSCluster.Builder(conf)
                                   .numDataNodes(NUM_DATA_NODES)
@@ -406,7 +406,7 @@ public class TestNameEditsConfigs extends TestCase {
     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);
+    replication = (short)conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, 3);
     try {
       cluster = new MiniDFSCluster.Builder(conf)
                                   .numDataNodes(NUM_DATA_NODES)

+ 2 - 1
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java

@@ -24,6 +24,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;
 import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
@@ -51,7 +52,7 @@ public class TestNamenodeCapacityReport extends TestCase {
 
     // Set aside fifth of the total capacity as reserved
     long reserved = 10000;
-    conf.setLong("dfs.datanode.du.reserved", reserved);
+    conf.setLong(DFSConfigKeys.DFS_DATANODE_DU_RESERVED_KEY, reserved);
     
     try {
       cluster = new MiniDFSCluster.Builder(conf).build();

+ 1 - 1
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestOverReplicatedBlocks.java

@@ -43,7 +43,7 @@ public class TestOverReplicatedBlocks extends TestCase {
    */
   public void testProcesOverReplicateBlock() throws IOException {
     Configuration conf = new HdfsConfiguration();
-    conf.setLong("dfs.blockreport.intervalMsec", 1000L);
+    conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 1000L);
     conf.set(
         DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY,
         Integer.toString(2));

+ 1 - 1
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/metrics/TestNNMetricFilesInGetListingOps.java

@@ -40,7 +40,7 @@ public class TestNNMetricFilesInGetListingOps extends TestCase {
   static {
     CONF.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 100);
     CONF.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 1);
-    CONF.setLong("dfs.heartbeat.interval", 1L);
+    CONF.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
     CONF.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1);
   }