瀏覽代碼

HDFS-2410. Further cleanup of hardcoded configuration keys and values. Contributed by Suresh Srinivas.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1204753 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 13 年之前
父節點
當前提交
5293760973
共有 53 個文件被更改,包括 252 次插入199 次删除
  1. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  2. 5 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  3. 8 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
  4. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol.java
  5. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol2.java
  6. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/permission/TestStickyBit.java
  7. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/AppendTestUtil.java
  8. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java
  9. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/FileAppendTest4.java
  10. 51 51
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
  11. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockMissingException.java
  12. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java
  13. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java
  14. 9 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSAddressConfig.java
  15. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
  16. 4 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java
  17. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferProtocol.java
  18. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeDeath.java
  19. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
  20. 10 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
  21. 6 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFSInputChecker.java
  22. 5 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFSOutputSummer.java
  23. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend2.java
  24. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend3.java
  25. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileConcurrentReader.java
  26. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
  27. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreationDelete.java
  28. 7 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSServerPorts.java
  29. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInjectionForSimulatedStorage.java
  30. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLargeBlock.java
  31. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery.java
  32. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery2.java
  33. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestModTime.java
  34. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMultiThreadedHflush.java
  35. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
  36. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java
  37. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadWhileWriting.java
  38. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRenameWhileOpen.java
  39. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplication.java
  40. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSetTimes.java
  41. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSmallBlock.java
  42. 5 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
  43. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlocksWithNotEnoughRacks.java
  44. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
  45. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRestart.java
  46. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
  47. 5 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java
  48. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileLimit.java
  49. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetaSave.java
  50. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java
  51. 12 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
  52. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/security/TestPermission.java
  53. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestJMXGet.java

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -63,6 +63,9 @@ Trunk (unreleased changes)
 
     HDFS-2572. Remove unnecessary double-check in DN#getHostName. (harsh)
 
+    HDFS-2410. Further cleanup of hardcoded configuration keys and values.
+    (suresh)
+
   OPTIMIZATIONS
     HDFS-2477. Optimize computing the diff between a block report and the
                namenode state. (Tomasz Nykiel via hairong)

+ 5 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -37,6 +37,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_KE
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTPS_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_KEYTAB_FILE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_PLUGINS_KEY;
@@ -48,6 +49,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_STARTUP_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_STORAGEID_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_FEDERATION_NAMESERVICES;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HTTPS_ENABLE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEBHDFS_ENABLED_DEFAULT;
 
@@ -460,11 +462,11 @@ public class DataNode extends Configured
     if(LOG.isDebugEnabled()) {
       LOG.debug("Datanode listening on " + infoHost + ":" + tmpInfoPort);
     }
-    if (conf.getBoolean("dfs.https.enable", false)) {
+    if (conf.getBoolean(DFS_HTTPS_ENABLE_KEY, false)) {
       boolean needClientAuth = conf.getBoolean(DFS_CLIENT_HTTPS_NEED_AUTH_KEY,
                                                DFS_CLIENT_HTTPS_NEED_AUTH_DEFAULT);
       InetSocketAddress secInfoSocAddr = NetUtils.createSocketAddr(conf.get(
-          "dfs.datanode.https.address", infoHost + ":" + 0));
+          DFS_DATANODE_HTTPS_ADDRESS_KEY, infoHost + ":" + 0));
       Configuration sslConf = new HdfsConfiguration(false);
       sslConf.addResource(conf.get("dfs.https.server.keystore.resource",
           "ssl-server.xml"));
@@ -505,7 +507,7 @@ public class DataNode extends Configured
 
   private void initIpcServer(Configuration conf) throws IOException {
     InetSocketAddress ipcAddr = NetUtils.createSocketAddr(
-        conf.get("dfs.datanode.ipc.address"));
+        conf.get(DFS_DATANODE_IPC_ADDRESS_KEY));
     
     // Add all the RPC protocols that the Datanode implements    
     ClientDatanodeProtocolServerSideTranslatorR23 

+ 8 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java

@@ -17,6 +17,10 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTPS_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_DEFAULT;
+
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.security.PrivilegedExceptionAction;
@@ -145,7 +149,7 @@ public class NameNodeHttpServer {
             }
           };
 
-          boolean certSSL = conf.getBoolean("dfs.https.enable", false);
+          boolean certSSL = conf.getBoolean(DFSConfigKeys.DFS_HTTPS_ENABLE_KEY, false);
           boolean useKrb = UserGroupInformation.isSecurityEnabled();
           if (certSSL || useKrb) {
             boolean needClientAuth = conf.getBoolean(
@@ -156,14 +160,14 @@ public class NameNodeHttpServer {
                     DFSConfigKeys.DFS_NAMENODE_HTTPS_ADDRESS_DEFAULT));
             Configuration sslConf = new HdfsConfiguration(false);
             if (certSSL) {
-              sslConf.addResource(conf.get(
-                  "dfs.https.server.keystore.resource", "ssl-server.xml"));
+              sslConf.addResource(conf.get(DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_KEY,
+                  DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_DEFAULT));
             }
             httpServer.addSslListener(secInfoSocAddr, sslConf, needClientAuth,
                 useKrb);
             // assume same ssl port for all datanodes
             InetSocketAddress datanodeSslPort = NetUtils.createSocketAddr(conf
-                .get("dfs.datanode.https.address", infoHost + ":" + 50475));
+                .get(DFS_DATANODE_HTTPS_ADDRESS_KEY, infoHost + ":" + 50475));
             httpServer.setAttribute("datanode.https.port", datanodeSslPort
                 .getPort());
           }

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

@@ -29,6 +29,7 @@ import org.apache.hadoop.fi.DataTransferTestUtil.SleepAction;
 import org.apache.hadoop.fi.DataTransferTestUtil.VerificationAction;
 import org.apache.hadoop.fi.FiTestUtil;
 import org.apache.hadoop.fi.FiTestUtil.Action;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -56,8 +57,9 @@ public class TestFiDataTransferProtocol {
 
   static private FSDataOutputStream createFile(FileSystem fs, Path p
       ) throws IOException {
-    return fs.create(p, true, fs.getConf().getInt("io.file.buffer.size", 4096),
-        REPLICATION, BLOCKSIZE);
+    return fs.create(p, true,
+        fs.getConf().getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY,
+            4096), REPLICATION, BLOCKSIZE);
   }
 
   {

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol2.java

@@ -30,6 +30,7 @@ import org.apache.hadoop.fi.DataTransferTestUtil.DataTransferTest;
 import org.apache.hadoop.fi.DataTransferTestUtil.SleepAction;
 import org.apache.hadoop.fi.DataTransferTestUtil.VerificationAction;
 import org.apache.hadoop.fi.FiTestUtil;
+import static org.apache.hadoop.fs.CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -65,8 +66,8 @@ public class TestFiDataTransferProtocol2 {
 
   static private FSDataOutputStream createFile(FileSystem fs, Path p
       ) throws IOException {
-    return fs.create(p, true, fs.getConf().getInt("io.file.buffer.size", 4096),
-        REPLICATION, BLOCKSIZE);
+    return fs.create(p, true, fs.getConf()
+        .getInt(IO_FILE_BUFFER_SIZE_KEY, 4096), REPLICATION, BLOCKSIZE);
   }
 
   {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/permission/TestStickyBit.java

@@ -163,7 +163,7 @@ public class TestStickyBit extends TestCase {
     try {
       Configuration conf = new HdfsConfiguration();
       conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
-      conf.setBoolean("dfs.support.append", true);
+      conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(4).build();
 
       FileSystem hdfs = cluster.getFileSystem();

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

@@ -143,8 +143,8 @@ public class AppendTestUtil {
   public static FSDataOutputStream createFile(FileSystem fileSys, Path name, int repl)
       throws IOException {
     return fileSys.create(name, true,
-        fileSys.getConf().getInt("io.file.buffer.size", 4096),
-        (short) repl, (long) BLOCK_SIZE);
+        fileSys.getConf().getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+        (short) repl, BLOCK_SIZE);
   }
 
   /**

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

@@ -30,6 +30,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.net.NetUtils;
@@ -148,7 +149,7 @@ public class BlockReaderTestUtil {
       sock, targetAddr.toString()+ ":" + block.getBlockId(), block,
       testBlock.getBlockToken(), 
       offset, lenToRead,
-      conf.getInt("io.file.buffer.size", 4096),
+      conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
       true, "");
   }
 

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/FileAppendTest4.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.junit.AfterClass;
@@ -66,7 +67,7 @@ public class FileAppendTest4 {
   }
 
   @AfterClass
-  public static void tearDown() throws IOException {
+  public static void tearDown() {
     cluster.shutdown();
   }
   
@@ -91,7 +92,7 @@ public class FileAppendTest4 {
             new Path("foo"+ oldFileLen +"_"+ flushedBytes1 +"_"+ flushedBytes2);
           LOG.info("Creating file " + p);
           FSDataOutputStream out = fs.create(p, false, 
-              conf.getInt("io.file.buffer.size", 4096), 
+              conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096), 
               REPLICATION, BLOCK_SIZE);
           out.write(contents, 0, oldFileLen);
           out.close();

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

@@ -39,9 +39,9 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
+import static 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.ClientProtocol;
@@ -324,8 +324,8 @@ public class MiniDFSCluster {
    * Servers will be started on free ports.
    * <p>
    * The caller must manage the creation of NameNode and DataNode directories
-   * and have already set {@link DFSConfigKeys#DFS_NAMENODE_NAME_DIR_KEY} and 
-   * {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} in the given conf.
+   * and have already set {@link #DFS_NAMENODE_NAME_DIR_KEY} and 
+   * {@link #DFS_DATANODE_DATA_DIR_KEY} in the given conf.
    * 
    * @param conf the base configuration to use in starting the servers.  This
    *          will be modified as necessary.
@@ -399,8 +399,8 @@ public class MiniDFSCluster {
    * @param format if true, format the NameNode and DataNodes before starting 
    *          up
    * @param manageDfsDirs if true, the data directories for servers will be
-   *          created and {@link DFSConfigKeys#DFS_NAMENODE_NAME_DIR_KEY} and 
-   *          {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be set in 
+   *          created and {@link #DFS_NAMENODE_NAME_DIR_KEY} and 
+   *          {@link #DFS_DATANODE_DATA_DIR_KEY} will be set in 
    *          the conf
    * @param operation the operation with which to start the servers.  If null
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
@@ -431,8 +431,8 @@ public class MiniDFSCluster {
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @param format if true, format the NameNode and DataNodes before starting up
    * @param manageDfsDirs if true, the data directories for servers will be
-   *          created and {@link DFSConfigKeys#DFS_NAMENODE_NAME_DIR_KEY} and 
-   *          {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be set in 
+   *          created and {@link #DFS_NAMENODE_NAME_DIR_KEY} and 
+   *          {@link #DFS_DATANODE_DATA_DIR_KEY} will be set in 
    *          the conf
    * @param operation the operation with which to start the servers.  If null
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
@@ -465,11 +465,11 @@ public class MiniDFSCluster {
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @param format if true, format the NameNode and DataNodes before starting up
    * @param manageNameDfsDirs if true, the data directories for servers will be
-   *          created and {@link DFSConfigKeys#DFS_NAMENODE_NAME_DIR_KEY} and 
-   *          {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be set in 
+   *          created and {@link #DFS_NAMENODE_NAME_DIR_KEY} and 
+   *          {@link #DFS_DATANODE_DATA_DIR_KEY} will be set in 
    *          the conf
    * @param manageDataDfsDirs if true, the data directories for datanodes will
-   *          be created and {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} 
+   *          be created and {@link #DFS_DATANODE_DATA_DIR_KEY} 
    *          set to same in the conf
    * @param operation the operation with which to start the servers.  If null
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
@@ -527,15 +527,15 @@ public class MiniDFSCluster {
       }
 
       // disable service authorization, as it does not work with tunnelled RPC
-      conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION,
+      conf.setBoolean(HADOOP_SECURITY_AUTHORIZATION,
                       false);
     }
     
-    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(DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY, 3); // 3 second
-    conf.setClass(DFSConfigKeys.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY, 
+    int replication = conf.getInt(DFS_REPLICATION_KEY, 3);
+    conf.setInt(DFS_REPLICATION_KEY, Math.min(replication, numDataNodes));
+    conf.setInt(DFS_NAMENODE_SAFEMODE_EXTENSION_KEY, 0);
+    conf.setInt(DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY, 3); // 3 second
+    conf.setClass(NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY, 
                    StaticMapping.class, DNSToSwitchMapping.class);
     
     Collection<String> nameserviceIds = DFSUtil.getNameServiceIds(conf);
@@ -543,8 +543,8 @@ public class MiniDFSCluster {
       federation = true;
   
     if (!federation) {
-      conf.set(DFSConfigKeys.FS_DEFAULT_NAME_KEY, "127.0.0.1:" + nameNodePort);
-      conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "127.0.0.1:"
+      conf.set(FS_DEFAULT_NAME_KEY, "127.0.0.1:" + nameNodePort);
+      conf.set(DFS_NAMENODE_HTTP_ADDRESS_KEY, "127.0.0.1:"
           + nameNodeHttpPort);
       NameNode nn = createNameNode(0, conf, numDataNodes, manageNameDfsDirs,
           format, operation, clusterId);
@@ -588,7 +588,7 @@ public class MiniDFSCluster {
       initFederatedNamenodeAddress(conf, nameserviceId, nnPort);
       nnPort = nnPort == 0 ? 0 : nnPort + 2;
     }
-    conf.set(DFSConfigKeys.DFS_FEDERATION_NAMESERVICES, nameserviceIdList);
+    conf.set(DFS_FEDERATION_NAMESERVICES, nameserviceIdList);
   }
 
   /* For federated namenode initialize the address:port */
@@ -596,11 +596,11 @@ public class MiniDFSCluster {
       String nameserviceId, int nnPort) {
     // Set nameserviceId specific key
     String key = DFSUtil.getNameServiceIdKey(
-        DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, nameserviceId);
+        DFS_NAMENODE_HTTP_ADDRESS_KEY, nameserviceId);
     conf.set(key, "127.0.0.1:0");
 
     key = DFSUtil.getNameServiceIdKey(
-        DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY, nameserviceId);
+        DFS_NAMENODE_RPC_ADDRESS_KEY, nameserviceId);
     conf.set(key, "127.0.0.1:" + nnPort);
   }
   
@@ -621,10 +621,10 @@ public class MiniDFSCluster {
       StartupOption operation, String clusterId)
       throws IOException {
     if (manageNameDfsDirs) {
-      conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+      conf.set(DFS_NAMENODE_NAME_DIR_KEY,
           fileAsURI(new File(base_dir, "name" + (2*nnIndex + 1)))+","+
           fileAsURI(new File(base_dir, "name" + (2*nnIndex + 2))));
-      conf.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY,
+      conf.set(DFS_NAMENODE_CHECKPOINT_DIR_KEY,
           fileAsURI(new File(base_dir, "namesecondary" + (2*nnIndex + 1)))+","+
           fileAsURI(new File(base_dir, "namesecondary" + (2*nnIndex + 2))));
     }
@@ -649,17 +649,17 @@ public class MiniDFSCluster {
       int numDataNodes, boolean manageNameDfsDirs, boolean format,
       StartupOption operation, String clusterId, String nameserviceId)
       throws IOException {
-    conf.set(DFSConfigKeys.DFS_FEDERATION_NAMESERVICE_ID, nameserviceId);
+    conf.set(DFS_FEDERATION_NAMESERVICE_ID, nameserviceId);
     NameNode nn = createNameNode(nnIndex, conf, numDataNodes, manageNameDfsDirs,
         format, operation, clusterId);
     conf.set(DFSUtil.getNameServiceIdKey(
-        DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY, nameserviceId), NameNode
+        DFS_NAMENODE_RPC_ADDRESS_KEY, nameserviceId), NameNode
         .getHostPortString(nn.getNameNodeAddress()));
     conf.set(DFSUtil.getNameServiceIdKey(
-        DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, nameserviceId), NameNode
+        DFS_NAMENODE_HTTP_ADDRESS_KEY, nameserviceId), NameNode
         .getHostPortString(nn.getHttpAddress()));
     DFSUtil.setGenericConf(conf, nameserviceId, 
-        DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY);
+        DFS_NAMENODE_HTTP_ADDRESS_KEY);
     nameNodes[nnIndex] = new NameNodeInfo(nn, new Configuration(conf));
   }
 
@@ -739,7 +739,7 @@ public class MiniDFSCluster {
    *          will be modified as necessary.
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @param manageDfsDirs if true, the data directories for DataNodes will be
-   *          created and {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be set 
+   *          created and {@link #DFS_DATANODE_DATA_DIR_KEY} will be set 
    *          in the conf
    * @param operation the operation with which to start the DataNodes.  If null
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
@@ -771,7 +771,7 @@ public class MiniDFSCluster {
    *          will be modified as necessary.
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @param manageDfsDirs if true, the data directories for DataNodes will be
-   *          created and {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be 
+   *          created and {@link #DFS_DATANODE_DATA_DIR_KEY} will be 
    *          set in the conf
    * @param operation the operation with which to start the DataNodes.  If null
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
@@ -805,7 +805,7 @@ public class MiniDFSCluster {
    *          will be modified as necessary.
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @param manageDfsDirs if true, the data directories for DataNodes will be
-   *          created and {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be 
+   *          created and {@link #DFS_DATANODE_DATA_DIR_KEY} will be 
    *          set in the conf
    * @param operation the operation with which to start the DataNodes.  If null
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
@@ -823,12 +823,12 @@ public class MiniDFSCluster {
                              long[] simulatedCapacities,
                              boolean setupHostsFile,
                              boolean checkDataNodeAddrConfig) throws IOException {
-    conf.set(DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY, "127.0.0.1");
+    conf.set(DFS_DATANODE_HOST_NAME_KEY, "127.0.0.1");
 
     int curDatanodesNum = dataNodes.size();
     // for mincluster's the default initialDelay for BRs is 0
-    if (conf.get(DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY) == null) {
-      conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY, 0);
+    if (conf.get(DFS_BLOCKREPORT_INITIAL_DELAY_KEY) == null) {
+      conf.setLong(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.
@@ -875,8 +875,8 @@ public class MiniDFSCluster {
                                 + i + ": " + dir1 + " or " + dir2);
         }
         String dirs = fileAsURI(dir1) + "," + fileAsURI(dir2);
-        dnConf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dirs);
-        conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dirs);
+        dnConf.set(DFS_DATANODE_DATA_DIR_KEY, dirs);
+        conf.set(DFS_DATANODE_DATA_DIR_KEY, dirs);
       }
       if (simulatedCapacities != null) {
         dnConf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
@@ -905,7 +905,7 @@ public class MiniDFSCluster {
       DataNode dn = DataNode.instantiateDataNode(dnArgs, dnConf);
       if(dn == null)
         throw new IOException("Cannot start DataNode in "
-            + dnConf.get(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY));
+            + dnConf.get(DFS_DATANODE_DATA_DIR_KEY));
       //since the HDFS does things based on IP:port, we need to add the mapping
       //for IP:port to rackId
       String ipAddr = dn.getSelfAddr().getAddress().getHostAddress();
@@ -1321,7 +1321,7 @@ public class MiniDFSCluster {
     Configuration newconf = new HdfsConfiguration(conf); // save cloned config
     if (keepPort) {
       InetSocketAddress addr = dnprop.datanode.getSelfAddr();
-      conf.set("dfs.datanode.address", addr.getAddress().getHostAddress() + ":"
+      conf.set(DFS_DATANODE_ADDRESS_KEY, addr.getAddress().getHostAddress() + ":"
           + addr.getPort());
     }
     dataNodes.add(new DataNodeProperties(DataNode.createDataNode(args, conf),
@@ -1448,10 +1448,10 @@ public class MiniDFSCluster {
   /**
    * @return a http URL
    */
-  public String getHttpUri(int nnIndex) throws IOException {
+  public String getHttpUri(int nnIndex) {
     return "http://"
         + nameNodes[nnIndex].conf
-            .get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY);
+            .get(DFS_NAMENODE_HTTP_ADDRESS_KEY);
   }
   
   /**
@@ -1460,7 +1460,7 @@ public class MiniDFSCluster {
   public HftpFileSystem getHftpFileSystem(int nnIndex) throws IOException {
     String uri = "hftp://"
         + nameNodes[nnIndex].conf
-            .get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY);
+            .get(DFS_NAMENODE_HTTP_ADDRESS_KEY);
     try {
       return (HftpFileSystem)FileSystem.get(new URI(uri), conf);
     } catch (URISyntaxException e) {
@@ -1910,9 +1910,9 @@ public class MiniDFSCluster {
     nameNodes = newlist;
     String nameserviceId = NAMESERVICE_ID_PREFIX + (nnIndex + 1);
     
-    String nameserviceIds = conf.get(DFSConfigKeys.DFS_FEDERATION_NAMESERVICES);
+    String nameserviceIds = conf.get(DFS_FEDERATION_NAMESERVICES);
     nameserviceIds += "," + nameserviceId;
-    conf.set(DFSConfigKeys.DFS_FEDERATION_NAMESERVICES, nameserviceIds);
+    conf.set(DFS_FEDERATION_NAMESERVICES, nameserviceIds);
     
     initFederatedNamenodeAddress(conf, nameserviceId, namenodePort);
     createFederatedNameNode(nnIndex, conf, numDataNodes, true, true, null,
@@ -1945,28 +1945,28 @@ public class MiniDFSCluster {
   private void setupDatanodeAddress(Configuration conf, boolean setupHostsFile,
                            boolean checkDataNodeAddrConfig) throws IOException {
     if (setupHostsFile) {
-      String hostsFile = conf.get(DFSConfigKeys.DFS_HOSTS, "").trim();
+      String hostsFile = conf.get(DFS_HOSTS, "").trim();
       if (hostsFile.length() == 0) {
         throw new IOException("Parameter dfs.hosts is not setup in conf");
       }
       // Setup datanode in the include file, if it is defined in the conf
       String address = "127.0.0.1:" + getFreeSocketPort();
       if (checkDataNodeAddrConfig) {
-        conf.setIfUnset("dfs.datanode.address", address);
+        conf.setIfUnset(DFS_DATANODE_ADDRESS_KEY, address);
       } else {
-        conf.set("dfs.datanode.address", address);
+        conf.set(DFS_DATANODE_ADDRESS_KEY, address);
       }
       addToFile(hostsFile, address);
       LOG.info("Adding datanode " + address + " to hosts file " + hostsFile);
     } else {
       if (checkDataNodeAddrConfig) {
-        conf.setIfUnset("dfs.datanode.address", "127.0.0.1:0");
-        conf.setIfUnset("dfs.datanode.http.address", "127.0.0.1:0");
-        conf.setIfUnset("dfs.datanode.ipc.address", "127.0.0.1:0");
+        conf.setIfUnset(DFS_DATANODE_ADDRESS_KEY, "127.0.0.1:0");
+        conf.setIfUnset(DFS_DATANODE_HTTP_ADDRESS_KEY, "127.0.0.1:0");
+        conf.setIfUnset(DFS_DATANODE_IPC_ADDRESS_KEY, "127.0.0.1:0");
       } else {
-        conf.set("dfs.datanode.address", "127.0.0.1:0");
-        conf.set("dfs.datanode.http.address", "127.0.0.1:0");
-        conf.set("dfs.datanode.ipc.address", "127.0.0.1:0");
+        conf.set(DFS_DATANODE_ADDRESS_KEY, "127.0.0.1:0");
+        conf.set(DFS_DATANODE_HTTP_ADDRESS_KEY, "127.0.0.1:0");
+        conf.set(DFS_DATANODE_IPC_ADDRESS_KEY, "127.0.0.1:0");
       }
     }
   }

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockMissingException.java

@@ -24,6 +24,7 @@ import junit.framework.TestCase;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -80,9 +81,9 @@ public class TestBlockMissingException extends TestCase {
   //
   private void createOldFile(FileSystem fileSys, Path name, int repl, int numBlocks, long blocksize)
     throws IOException {
-    FSDataOutputStream stm = fileSys.create(name, true,
-                                            fileSys.getConf().getInt("io.file.buffer.size", 4096),
-                                            (short)repl, blocksize);
+    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
+        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+        (short) repl, blocksize);
     // fill data into file
     final byte[] b = new byte[(int)blocksize];
     for (int i = 0; i < numBlocks; i++) {

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

@@ -40,7 +40,7 @@ public class TestClientProtocolForPipelineRecovery {
   @Test public void testGetNewStamp() throws IOException {
     int numDataNodes = 1;
     Configuration conf = new HdfsConfiguration();
-    conf.setBoolean("dfs.support.append", true);
+    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDataNodes).build();
     try {
       cluster.waitActive();

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

@@ -28,6 +28,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ChecksumException;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -80,7 +81,7 @@ public class TestClientReportBadBlock {
         .build();
     cluster.waitActive();
     dfs = (DistributedFileSystem) cluster.getFileSystem();
-    buffersize = conf.getInt("io.file.buffer.size", 4096);
+    buffersize = conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096);
   }
 
   @After

+ 9 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSAddressConfig.java

@@ -28,6 +28,9 @@ import java.io.IOException;
 import java.util.ArrayList;
 import junit.framework.TestCase;
 import org.apache.hadoop.conf.Configuration;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
@@ -62,9 +65,9 @@ public class TestDFSAddressConfig extends TestCase {
       assertNotNull("Should have been able to stop simulated datanode", dnp);
     }
 
-    conf.unset("dfs.datanode.address");
-    conf.unset("dfs.datanode.http.address");
-    conf.unset("dfs.datanode.ipc.address");
+    conf.unset(DFS_DATANODE_ADDRESS_KEY);
+    conf.unset(DFS_DATANODE_HTTP_ADDRESS_KEY);
+    conf.unset(DFS_DATANODE_IPC_ADDRESS_KEY);
 
     cluster.startDataNodes(conf, 1, true, StartupOption.REGULAR,
                            null, null, null, false, true);
@@ -87,9 +90,9 @@ public class TestDFSAddressConfig extends TestCase {
       assertNotNull("Should have been able to stop simulated datanode", dnp);
     }
 
-    conf.set("dfs.datanode.address","0.0.0.0:0");
-    conf.set("dfs.datanode.http.address","0.0.0.0:0");
-    conf.set("dfs.datanode.ipc.address","0.0.0.0:0");
+    conf.set(DFS_DATANODE_ADDRESS_KEY, "0.0.0.0:0");
+    conf.set(DFS_DATANODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
+    conf.set(DFS_DATANODE_IPC_ADDRESS_KEY, "0.0.0.0:0");
 
     cluster.startDataNodes(conf, 1, true, StartupOption.REGULAR,
                            null, null, null, false, true);

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

@@ -43,6 +43,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.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileSystem;
@@ -141,7 +142,7 @@ public class TestDFSClientRetries extends TestCase {
     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);
+    conf.setInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, bufferSize);
 
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
     

+ 4 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java

@@ -22,14 +22,13 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Random;
 
-import javax.security.auth.login.LoginException;
-
 import junit.framework.AssertionFailedError;
 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.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -202,7 +201,7 @@ public class TestDFSPermission extends TestCase {
     switch (op) {
     case CREATE:
       FSDataOutputStream out = fs.create(name, permission, true, 
-          conf.getInt("io.file.buffer.size", 4096),
+          conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
           fs.getDefaultReplication(), fs.getDefaultBlockSize(), null);
       out.close();
       break;
@@ -520,8 +519,7 @@ public class TestDFSPermission extends TestCase {
     }
 
     /* Perform an operation and verify if the permission checking is correct */
-    void verifyPermission(UserGroupInformation ugi) throws LoginException,
-        IOException {
+    void verifyPermission(UserGroupInformation ugi) throws IOException {
       if (this.ugi != ugi) {
         setRequiredPermissions(ugi);
         this.ugi = ugi;
@@ -564,8 +562,7 @@ public class TestDFSPermission extends TestCase {
     }
 
     /* Set the permissions required to pass the permission checking */
-    protected void setRequiredPermissions(UserGroupInformation ugi)
-        throws IOException {
+    protected void setRequiredPermissions(UserGroupInformation ugi) {
       if (SUPERUSER.equals(ugi)) {
         requiredAncestorPermission = SUPER_MASK;
         requiredParentPermission = SUPER_MASK;

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

@@ -208,7 +208,7 @@ public class TestDataTransferProtocol extends TestCase {
   @Test public void testOpWrite() throws IOException {
     int numDataNodes = 1;
     Configuration conf = new HdfsConfiguration();
-    conf.setBoolean("dfs.support.append", true);
+    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDataNodes).build();
     try {
       cluster.waitActive();

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeDeath.java

@@ -25,6 +25,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -123,9 +124,9 @@ public class TestDatanodeDeath extends TestCase {
   static private FSDataOutputStream createFile(FileSystem fileSys, Path name, short repl)
     throws IOException {
     // create and write a file that contains three blocks of data
-    FSDataOutputStream stm = fileSys.create(name, true,
-                                            fileSys.getConf().getInt("io.file.buffer.size", 4096),
-                                            repl, (long)blockSize);
+    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
+        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096), repl,
+        blockSize);
     return stm;
   }
 

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java

@@ -30,6 +30,7 @@ import java.util.Random;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -115,9 +116,9 @@ public class TestDecommission {
   private void writeFile(FileSystem fileSys, Path name, int repl)
     throws IOException {
     // create and write a file that contains three blocks of data
-    FSDataOutputStream stm = fileSys.create(name, true, 
-                                            fileSys.getConf().getInt("io.file.buffer.size", 4096),
-                                            (short)repl, (long)blockSize);
+    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
+        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+        (short) repl, blockSize);
     byte[] buffer = new byte[fileSize];
     Random rand = new Random(seed);
     rand.nextBytes(buffer);
@@ -246,7 +247,7 @@ public class TestDecommission {
    * Wait till node is fully decommissioned.
    */
   private void waitNodeState(DatanodeInfo node,
-                             AdminStates state) throws IOException {
+                             AdminStates state) {
     boolean done = state == node.getAdminState();
     while (!done) {
       LOG.info("Waiting for node " + node + " to change state to "

+ 10 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java

@@ -31,6 +31,7 @@ import java.util.Random;
 
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileChecksum;
@@ -88,17 +89,17 @@ public class TestDistributedFileSystem {
   @Test
   public void testDFSClose() throws Exception {
     Configuration conf = getTestConfiguration();
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
-    FileSystem fileSys = cluster.getFileSystem();
-
+    MiniDFSCluster cluster = null;
     try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
+      FileSystem fileSys = cluster.getFileSystem();
+      
       // create two files
       fileSys.create(new Path("/test/dfsclose/file-0"));
       fileSys.create(new Path("/test/dfsclose/file-1"));
 
       fileSys.close();
-    }
-    finally {
+    } finally {
       if (cluster != null) {cluster.shutdown();}
     }
   }
@@ -106,10 +107,10 @@ public class TestDistributedFileSystem {
   @Test
   public void testDFSSeekExceptions() throws IOException {
     Configuration conf = getTestConfiguration();
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
-    FileSystem fileSys = cluster.getFileSystem();
-
+    MiniDFSCluster cluster = null;
     try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
+      FileSystem fileSys = cluster.getFileSystem();
       String file = "/test/fileclosethenseek/file-0";
       Path path = new Path(file);
       // create file
@@ -455,7 +456,7 @@ public class TestDistributedFileSystem {
 
     final Path dir = new Path("/filechecksum");
     final int block_size = 1024;
-    final int buffer_size = conf.getInt("io.file.buffer.size", 4096);
+    final int buffer_size = conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096);
     conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 512);
 
     //try different number of blocks

+ 6 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFSInputChecker.java

@@ -27,6 +27,7 @@ import junit.framework.TestCase;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ChecksumException;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -54,7 +55,7 @@ public class TestFSInputChecker extends TestCase {
   private void writeFile(FileSystem fileSys, Path name) throws IOException {
     // create and write a file that contains three blocks of data
     FSDataOutputStream stm = fileSys.create(name, new FsPermission((short)0777),
-        true, fileSys.getConf().getInt("io.file.buffer.size", 4096),
+        true, fileSys.getConf().getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
         NUM_OF_DATANODES, BLOCK_SIZE, null);
     stm.write(expected);
     stm.close();
@@ -325,8 +326,10 @@ public class TestFSInputChecker extends TestCase {
   throws IOException {
     Path file = new Path("try.dat");
     writeFile(fileSys, file);
-    stm = fileSys.open(file,
-        fileSys.getConf().getInt("io.file.buffer.size", 4096));
+    stm = fileSys.open(
+        file,
+        fileSys.getConf().getInt(
+            CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096));
     checkSeekAndRead();
     stm.close();
     cleanupFile(fileSys, file);

+ 5 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFSOutputSummer.java

@@ -21,6 +21,8 @@ import junit.framework.TestCase;
 import java.io.*;
 import java.util.Random;
 import org.apache.hadoop.conf.Configuration;
+import static org.apache.hadoop.fs.CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY;
+
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -43,7 +45,7 @@ public class TestFSOutputSummer extends TestCase {
   /* create a file, write all data at once */
   private void writeFile1(Path name) throws Exception {
     FSDataOutputStream stm = fileSys.create(name, true, 
-               fileSys.getConf().getInt("io.file.buffer.size", 4096),
+               fileSys.getConf().getInt(IO_FILE_BUFFER_SIZE_KEY, 4096),
                NUM_OF_DATANODES, BLOCK_SIZE);
     stm.write(expected);
     stm.close();
@@ -54,7 +56,7 @@ public class TestFSOutputSummer extends TestCase {
   /* create a file, write data chunk by chunk */
   private void writeFile2(Path name) throws Exception {
     FSDataOutputStream stm = fileSys.create(name, true, 
-               fileSys.getConf().getInt("io.file.buffer.size", 4096),
+               fileSys.getConf().getInt(IO_FILE_BUFFER_SIZE_KEY, 4096),
                NUM_OF_DATANODES, BLOCK_SIZE);
     int i=0;
     for( ;i<FILE_SIZE-BYTES_PER_CHECKSUM; i+=BYTES_PER_CHECKSUM) {
@@ -69,7 +71,7 @@ public class TestFSOutputSummer extends TestCase {
   /* create a file, write data with vairable amount of data */
   private void writeFile3(Path name) throws Exception {
     FSDataOutputStream stm = fileSys.create(name, true, 
-        fileSys.getConf().getInt("io.file.buffer.size", 4096),
+        fileSys.getConf().getInt(IO_FILE_BUFFER_SIZE_KEY, 4096),
         NUM_OF_DATANODES, BLOCK_SIZE);
     stm.write(expected, 0, HALF_CHUNK_SIZE);
     stm.write(expected, HALF_CHUNK_SIZE, BYTES_PER_CHECKSUM+2);

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

@@ -85,7 +85,7 @@ public class TestFileAppend2 extends TestCase {
       conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
     }
     conf.setInt(DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_KEY, 50);
-    conf.setBoolean("dfs.support.append", true);
+    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
     fileContents = AppendTestUtil.initBuffer(AppendTestUtil.FILE_SIZE);
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     FileSystem fs = cluster.getFileSystem();
@@ -306,7 +306,7 @@ public class TestFileAppend2 extends TestCase {
               fileContents, "Read 2");
         } catch (Throwable e) {
           globalStatus = false;
-          if (e != null && e.toString() != null) {
+          if (e.toString() != null) {
             System.out.println("Workload exception " + id + 
                                " testfile " + testfile +
                                " " + e);
@@ -338,7 +338,7 @@ public class TestFileAppend2 extends TestCase {
     conf.setInt(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 30000);
     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);
+    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
 
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
                                                .numDataNodes(numDatanodes)

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend3.java

@@ -27,6 +27,7 @@ import junit.framework.TestSuite;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -68,8 +69,8 @@ public class TestFileAppend3 extends junit.framework.TestCase {
         AppendTestUtil.LOG.info("setUp()");
         conf = new HdfsConfiguration();
         conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 512);
-        conf.setBoolean("dfs.support.append", true);
-        buffersize = conf.getInt("io.file.buffer.size", 4096);
+        conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
+        buffersize = conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096);
         cluster = new MiniDFSCluster.Builder(conf).numDataNodes(DATANODE_NUM).build();
         fs = (DistributedFileSystem)cluster.getFileSystem();
       }

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileConcurrentReader.java

@@ -311,8 +311,9 @@ public class TestFileConcurrentReader extends junit.framework.TestCase {
     final int writeSize,
     Configuration conf
   ) throws IOException {
-    conf.setBoolean("dfs.support.append", syncType == SyncType.APPEND);
-    conf.setBoolean("dfs.datanode.transferTo.allowed", transferToAllowed);
+    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, syncType == SyncType.APPEND);
+    conf.setBoolean(DFSConfigKeys.DFS_DATANODE_TRANSFERTO_ALLOWED_KEY,
+        transferToAllowed);
     init(conf);
 
     final Path file = new Path("/block-being-written-to");

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

@@ -28,6 +28,7 @@ import java.util.EnumSet;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -75,9 +76,9 @@ public class TestFileCreation extends junit.framework.TestCase {
   public static FSDataOutputStream createFile(FileSystem fileSys, Path name, int repl)
     throws IOException {
     System.out.println("createFile: Created " + name + " with " + repl + " replica.");
-    FSDataOutputStream stm = fileSys.create(name, true,
-                                            fileSys.getConf().getInt("io.file.buffer.size", 4096),
-                                            (short)repl, (long)blockSize);
+    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
+        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+        (short) repl, blockSize);
     return stm;
   }
 
@@ -638,7 +639,6 @@ public class TestFileCreation extends junit.framework.TestCase {
       out = createNonRecursive(fs, path, 1, createFlag);
       out.close();
       // Create a file when parent dir exists as file, should fail
-      expectedException = null;
       try {
         createNonRecursive(fs, new Path(path, "Create"), 1, createFlag);
       } catch (IOException e) {
@@ -704,7 +704,7 @@ public class TestFileCreation extends junit.framework.TestCase {
         + " replica.");
     FSDataOutputStream stm = ((DistributedFileSystem) fs).createNonRecursive(
         name, FsPermission.getDefault(), flag, fs.getConf().getInt(
-            "io.file.buffer.size", 4096), (short) repl, (long) blockSize, null);
+            CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096), (short) repl,  blockSize, null);
     return stm;
   }
   

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

@@ -43,7 +43,7 @@ public class TestFileCreationDelete extends junit.framework.TestCase {
     conf.setInt("ipc.client.connection.maxidletime", MAX_IDLE_TIME);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
     conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
-    conf.setBoolean("dfs.support.append", true);
+    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
 
     // create cluster
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();

+ 7 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSServerPorts.java

@@ -308,23 +308,23 @@ public class TestHDFSServerPorts extends TestCase {
       // start data-node on the same port as name-node
       Configuration conf2 = new HdfsConfiguration(config);
       conf2.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, new File(hdfsDir, "data").getPath());
-      conf2.set("dfs.datanode.address",
+      conf2.set(DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY,
                 FileSystem.getDefaultUri(config).getAuthority());
-      conf2.set("dfs.datanode.http.address", THIS_HOST);
+      conf2.set(DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY, THIS_HOST);
       boolean started = canStartDataNode(conf2);
       assertFalse(started); // should fail
 
       // bind http server to the same port as name-node
-      conf2.set("dfs.datanode.address", THIS_HOST);
-      conf2.set("dfs.datanode.http.address", 
+      conf2.set(DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY, THIS_HOST);
+      conf2.set(DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY,
                 config.get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY));
       started = canStartDataNode(conf2);
       assertFalse(started); // should fail
     
       // both ports are different from the name-node ones
-      conf2.set("dfs.datanode.address", THIS_HOST);
-      conf2.set("dfs.datanode.http.address", THIS_HOST);
-      conf2.set("dfs.datanode.ipc.address", THIS_HOST);
+      conf2.set(DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY, THIS_HOST);
+      conf2.set(DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY, THIS_HOST);
+      conf2.set(DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY, THIS_HOST);
       started = canStartDataNode(conf2);
       assertTrue(started); // should start now
     } finally {

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInjectionForSimulatedStorage.java

@@ -27,6 +27,7 @@ import java.net.*;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -53,9 +54,9 @@ public class TestInjectionForSimulatedStorage extends TestCase {
   private void writeFile(FileSystem fileSys, Path name, int repl)
                                                 throws IOException {
     // create and write a file that contains three blocks of data
-    FSDataOutputStream stm = fileSys.create(name, true,
-          fileSys.getConf().getInt("io.file.buffer.size", 4096),
-                                      (short)repl, (long)blockSize);
+    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
+        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+        (short) repl, blockSize);
     byte[] buffer = new byte[filesize];
     for (int i=0; i<buffer.length; i++) {
       buffer[i] = '1';

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLargeBlock.java

@@ -23,6 +23,7 @@ import java.util.Arrays;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -55,9 +56,9 @@ public class TestLargeBlock {
   static FSDataOutputStream createFile(FileSystem fileSys, Path name, int repl,
                                        final long blockSize)
     throws IOException {
-    FSDataOutputStream stm = fileSys.create(name, true,
-        fileSys.getConf().getInt("io.file.buffer.size", 4096),
-        (short)repl, blockSize);
+    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
+        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+        (short) repl, blockSize);
     LOG.info("createFile: Created " + name + " with " + repl + " replica.");
     return stm;
   }

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

@@ -69,7 +69,7 @@ public class TestLeaseRecovery extends junit.framework.TestCase {
     final int ORG_FILE_SIZE = 3000; 
     Configuration conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
-    conf.setBoolean("dfs.support.append", true);
+    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
     MiniDFSCluster cluster = null;
 
     try {

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

@@ -30,6 +30,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
@@ -72,7 +73,8 @@ public class TestLeaseRecovery2 {
   static private MiniDFSCluster cluster;
   static private DistributedFileSystem dfs;
   final static private Configuration conf = new HdfsConfiguration();
-  final static private int BUF_SIZE = conf.getInt("io.file.buffer.size", 4096);
+  final static private int BUF_SIZE = conf.getInt(
+      CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096);
   
   final static private long SHORT_LEASE_PERIOD = 1000L;
   final static private long LONG_LEASE_PERIOD = 60*60*SHORT_LEASE_PERIOD;

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestModTime.java

@@ -24,6 +24,7 @@ import java.net.*;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -46,9 +47,9 @@ public class TestModTime extends TestCase {
   private void writeFile(FileSystem fileSys, Path name, int repl)
     throws IOException {
     // create and write a file that contains three blocks of data
-    FSDataOutputStream stm = fileSys.create(name, true, 
-                                            fileSys.getConf().getInt("io.file.buffer.size", 4096),
-                                            (short)repl, (long)blockSize);
+    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
+        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+        (short) repl, blockSize);
     byte[] buffer = new byte[fileSize];
     Random rand = new Random(seed);
     rand.nextBytes(buffer);

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMultiThreadedHflush.java

@@ -27,6 +27,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -65,9 +66,9 @@ public class TestMultiThreadedHflush {
    */ 
   private FSDataOutputStream createFile(FileSystem fileSys, Path name, int repl)
     throws IOException {
-    FSDataOutputStream stm = fileSys.create(name, true,
-                                            fileSys.getConf().getInt("io.file.buffer.size", 4096),
-                                            (short)repl, (long)blockSize);
+    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
+        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+        (short) repl, blockSize);
     return stm;
   }
   

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

@@ -42,7 +42,7 @@ public class TestPread extends TestCase {
   private void writeFile(FileSystem fileSys, Path name) throws IOException {
     // create and write a file that contains three blocks of data
     DataOutputStream stm = fileSys.create(name, true, 4096, (short)1,
-                                          (long)blockSize);
+                                          blockSize);
     // test empty file open and read
     stm.close();
     FSDataInputStream in = fileSys.open(name);
@@ -61,7 +61,7 @@ public class TestPread extends TestCase {
       assertTrue("Cannot delete file", false);
     
     // now create the real file
-    stm = fileSys.create(name, true, 4096, (short)1, (long)blockSize);
+    stm = fileSys.create(name, true, 4096, (short)1, blockSize);
     Random rand = new Random(seed);
     rand.nextBytes(buffer);
     stm.write(buffer);
@@ -206,7 +206,7 @@ public class TestPread extends TestCase {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 4096);
     conf.setLong(DFSConfigKeys.DFS_CLIENT_READ_PREFETCH_SIZE_KEY, 4096);
     if (simulatedStorage) {
-      conf.setBoolean("dfs.datanode.simulateddatastorage", true);
+      conf.setBoolean(DFSConfigKeys.DFS_DATANODE_SIMULATEDDATASTORAGE_KEY, true);
     }
     if (disableTransferTo) {
       conf.setBoolean("dfs.datanode.transferTo.allowed", false);

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

@@ -84,7 +84,7 @@ public class TestQuota {
     // Space quotas
     final int DEFAULT_BLOCK_SIZE = 512;
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
-    conf.setBoolean("dfs.support.append", true);
+    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     final FileSystem fs = cluster.getFileSystem();
     assertTrue("Not a HDFS: "+fs.getUri(),
@@ -533,7 +533,7 @@ public class TestQuota {
     // set a smaller block size so that we can test with smaller 
     // diskspace quotas
     conf.set(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, "512");
-    conf.setBoolean("dfs.support.append", true);
+    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     final FileSystem fs = cluster.getFileSystem();
     assertTrue("Not a HDFS: "+fs.getUri(),

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

@@ -24,6 +24,7 @@ import java.security.PrivilegedExceptionAction;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -77,7 +78,7 @@ public class TestReadWhileWriting {
       //   Do not close file yet.
       {
         final FSDataOutputStream out = fs.create(p, true,
-            fs.getConf().getInt("io.file.buffer.size", 4096),
+            fs.getConf().getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
             (short)3, BLOCK_SIZE);
         write(out, 0, half);
 

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

@@ -54,7 +54,7 @@ public class TestRenameWhileOpen extends junit.framework.TestCase {
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
     conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY, 1);
-    conf.setBoolean("dfs.support.append", true);
+    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
 
     // create cluster
     System.out.println("Test 1*****************************");
@@ -140,7 +140,7 @@ public class TestRenameWhileOpen extends junit.framework.TestCase {
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
     conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY, 1);
-    conf.setBoolean("dfs.support.append", true);
+    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
     System.out.println("Test 2************************************");
 
     // create cluster
@@ -215,7 +215,7 @@ public class TestRenameWhileOpen extends junit.framework.TestCase {
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
     conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY, 1);
-    conf.setBoolean("dfs.support.append", true);
+    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
     System.out.println("Test 3************************************");
 
     // create cluster
@@ -280,7 +280,7 @@ public class TestRenameWhileOpen extends junit.framework.TestCase {
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
     conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY, 1);
-    conf.setBoolean("dfs.support.append", true);
+    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
     System.out.println("Test 4************************************");
 
     // create cluster

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplication.java

@@ -31,6 +31,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -60,9 +61,9 @@ public class TestReplication extends TestCase {
   private void writeFile(FileSystem fileSys, Path name, int repl)
     throws IOException {
     // create and write a file that contains three blocks of data
-    FSDataOutputStream stm = fileSys.create(name, true,
-                                            fileSys.getConf().getInt("io.file.buffer.size", 4096),
-                                            (short)repl, (long)blockSize);
+    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
+        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+        (short) repl, blockSize);
     byte[] buffer = new byte[fileSize];
     Random rand = new Random(seed);
     rand.nextBytes(buffer);

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

@@ -24,6 +24,7 @@ import java.net.*;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -49,9 +50,9 @@ public class TestSetTimes extends TestCase {
 
   private FSDataOutputStream writeFile(FileSystem fileSys, Path name, int repl)
     throws IOException {
-    FSDataOutputStream stm = fileSys.create(name, true, 
-                                            fileSys.getConf().getInt("io.file.buffer.size", 4096),
-                                            (short)repl, (long)blockSize);
+    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
+        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+        (short) repl, blockSize);
     byte[] buffer = new byte[fileSize];
     Random rand = new Random(seed);
     rand.nextBytes(buffer);

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSmallBlock.java

@@ -21,6 +21,7 @@ import junit.framework.TestCase;
 import java.io.*;
 import java.util.Random;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -40,9 +41,9 @@ public class TestSmallBlock extends TestCase {
 
   private void writeFile(FileSystem fileSys, Path name) throws IOException {
     // create and write a file that contains three blocks of data
-    FSDataOutputStream stm = fileSys.create(name, true, 
-                                            fileSys.getConf().getInt("io.file.buffer.size", 4096),
-                                            (short)1, (long)blockSize);
+    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
+        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+        (short) 1, blockSize);
     byte[] buffer = new byte[fileSize];
     Random rand = new Random(seed);
     rand.nextBytes(buffer);
@@ -92,7 +93,7 @@ public class TestSmallBlock extends TestCase {
   public void testSmallBlock() throws IOException {
     Configuration conf = new HdfsConfiguration();
     if (simulatedStorage) {
-      conf.setBoolean("dfs.datanode.simulateddatastorage", true);
+      conf.setBoolean(DFSConfigKeys.DFS_DATANODE_SIMULATEDDATASTORAGE_KEY, true);
     }
     conf.set(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, "1");
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();

+ 5 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java

@@ -31,6 +31,7 @@ import java.util.Random;
 
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -123,7 +124,7 @@ public class TestBlockTokenWithDFS {
   private static FSDataOutputStream writeFile(FileSystem fileSys, Path name,
       short repl, long blockSize) throws IOException {
     FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
-        .getInt("io.file.buffer.size", 4096), repl, blockSize);
+        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096), repl, blockSize);
     return stm;
   }
 
@@ -173,7 +174,7 @@ public class TestBlockTokenWithDFS {
   }
 
   // get a conf for testing
-  private static Configuration getConf(int numDataNodes) throws IOException {
+  private static Configuration getConf(int numDataNodes) {
     Configuration conf = new Configuration();
     conf.setBoolean(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
@@ -181,7 +182,8 @@ public class TestBlockTokenWithDFS {
     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);
+    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY,
+        DFSConfigKeys.DFS_SUPPORT_APPEND_DEFAULT);
     return conf;
   }
 

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

@@ -391,7 +391,7 @@ public class TestBlocksWithNotEnoughRacks {
     Path excludeFile = new Path(dir, "exclude");
     assertTrue(localFileSys.mkdirs(dir));
     DFSTestUtil.writeFile(localFileSys, excludeFile, "");
-    conf.set("dfs.hosts.exclude", excludeFile.toUri().getPath());
+    conf.set(DFSConfigKeys.DFS_HOSTS_EXCLUDE, excludeFile.toUri().getPath());
 
     // Two blocks and four racks
     String racks[] = {"/rack1", "/rack1", "/rack2", "/rack2"};
@@ -441,7 +441,7 @@ public class TestBlocksWithNotEnoughRacks {
     Path excludeFile = new Path(dir, "exclude");
     assertTrue(localFileSys.mkdirs(dir));
     DFSTestUtil.writeFile(localFileSys, excludeFile, "");
-    conf.set("dfs.hosts.exclude", excludeFile.toUri().getPath());
+    conf.set(DFSConfigKeys.DFS_HOSTS_EXCLUDE, excludeFile.toUri().getPath());
 
     // All hosts are on two racks, only one host on /rack2
     String racks[] = {"/rack1", "/rack2", "/rack1", "/rack1", "/rack1"};

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java

@@ -65,9 +65,9 @@ import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 public class SimulatedFSDataset  implements FSDatasetInterface, Configurable{
   
   public static final String CONFIG_PROPERTY_SIMULATED =
-                                    "dfs.datanode.simulateddatastorage";
+      DFSConfigKeys.DFS_DATANODE_SIMULATEDDATASTORAGE_KEY;
   public static final String CONFIG_PROPERTY_CAPACITY =
-                            "dfs.datanode.simulateddatastorage.capacity";
+      DFSConfigKeys.DFS_DATANODE_SIMULATEDDATASTORAGE_CAPACITY_KEY;
   
   public static final long DEFAULT_CAPACITY = 2L<<40; // 1 terabyte
   public static final byte DEFAULT_DATABYTE = 9; // 1 terabyte
@@ -135,7 +135,7 @@ public class SimulatedFSDataset  implements FSDatasetInterface, Configurable{
       }
     }
     
-    synchronized SimulatedInputStream getIStream() throws IOException {
+    synchronized SimulatedInputStream getIStream() {
       if (!finalized) {
         // throw new IOException("Trying to read an unfinalized block");
          return new SimulatedInputStream(oStream.getLength(), DEFAULT_DATABYTE);
@@ -362,7 +362,7 @@ public class SimulatedFSDataset  implements FSDatasetInterface, Configurable{
   private SimulatedStorage storage = null;
   private String storageId;
   
-  public SimulatedFSDataset(Configuration conf) throws IOException {
+  public SimulatedFSDataset(Configuration conf) {
     setConf(conf);
   }
   

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRestart.java

@@ -73,7 +73,7 @@ public class TestDatanodeRestart {
     Configuration conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 1024L);
     conf.setInt(DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY, 512);
-    conf.setBoolean("dfs.support.append", true);
+    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     cluster.waitActive();
     try {
@@ -137,7 +137,7 @@ public class TestDatanodeRestart {
     Configuration conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 1024L);
     conf.setInt(DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY, 512);
-    conf.setBoolean("dfs.support.append", true);
+    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     cluster.waitActive();
     try {

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java

@@ -34,6 +34,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileContext;
@@ -99,9 +100,9 @@ public class TestCheckpoint extends TestCase {
 
   static void writeFile(FileSystem fileSys, Path name, int repl)
     throws IOException {
-    FSDataOutputStream stm = fileSys.create(name, true,
-                                            fileSys.getConf().getInt("io.file.buffer.size", 4096),
-                                            (short)repl, (long)blockSize);
+    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
+        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+        (short) repl, blockSize);
     byte[] buffer = new byte[TestCheckpoint.fileSize];
     Random rand = new Random(TestCheckpoint.seed);
     rand.nextBytes(buffer);

+ 5 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java

@@ -28,6 +28,7 @@ import java.util.List;
 import java.util.Random;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -117,7 +118,8 @@ public class TestDecommissioningStatus {
       throws IOException {
     // create and write a file that contains three blocks of data
     FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
-        .getInt("io.file.buffer.size", 4096), repl, (long) blockSize);
+        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096), repl,
+        blockSize);
     byte[] buffer = new byte[fileSize];
     Random rand = new Random(seed);
     rand.nextBytes(buffer);
@@ -129,7 +131,8 @@ public class TestDecommissioningStatus {
       short repl) throws IOException {
     // create and write a file that contains three blocks of data
     FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
-        .getInt("io.file.buffer.size", 4096), repl, (long) blockSize);
+        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096), repl,
+        blockSize);
     byte[] buffer = new byte[fileSize];
     Random rand = new Random(seed);
     rand.nextBytes(buffer);

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileLimit.java

@@ -23,6 +23,7 @@ import java.util.Random;
 import junit.framework.TestCase;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -44,9 +45,9 @@ public class TestFileLimit extends TestCase {
   // creates a zero file.
   private void createFile(FileSystem fileSys, Path name)
     throws IOException {
-    FSDataOutputStream stm = fileSys.create(name, true,
-                                            fileSys.getConf().getInt("io.file.buffer.size", 4096),
-                                            (short)1, (long)blockSize);
+    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
+        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+        (short) 1, blockSize);
     byte[] buffer = new byte[1024];
     Random rand = new Random(seed);
     rand.nextBytes(buffer);

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetaSave.java

@@ -29,6 +29,7 @@ import java.lang.InterruptedException;
 import java.util.Random;
 import static org.junit.Assert.assertTrue;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
@@ -50,7 +51,8 @@ public class TestMetaSave {
 
   private void createFile(FileSystem fileSys, Path name) throws IOException {
     FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
-        .getInt("io.file.buffer.size", 4096), (short) 2, (long) blockSize);
+        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+        (short) 2, blockSize);
     byte[] buffer = new byte[1024];
     Random rand = new Random(seed);
     rand.nextBytes(buffer);

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java

@@ -24,6 +24,7 @@ import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
@@ -59,9 +60,9 @@ public class TestNameEditsConfigs extends TestCase {
 
   private void writeFile(FileSystem fileSys, Path name, int repl)
     throws IOException {
-    FSDataOutputStream stm = fileSys.create(name, true,
-                                            fileSys.getConf().getInt("io.file.buffer.size", 4096),
-                                            (short)repl, (long)BLOCK_SIZE);
+    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
+        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+        (short) repl, BLOCK_SIZE);
     byte[] buffer = new byte[FILE_SIZE];
     Random rand = new Random(SEED);
     rand.nextBytes(buffer);
@@ -96,7 +97,7 @@ public class TestNameEditsConfigs extends TestCase {
     int replication = fileSys.getFileStatus(name).getReplication();
     assertEquals("replication for " + name, repl, replication);
     long size = fileSys.getContentSummary(name).getLength();
-    assertEquals("file size for " + name, size, (long)FILE_SIZE);
+    assertEquals("file size for " + name, size, FILE_SIZE);
   }
 
   private void cleanupFile(FileSystem fileSys, Path name)

+ 12 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java

@@ -34,6 +34,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.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
@@ -76,9 +77,9 @@ public class TestStartup extends TestCase {
 
   private void writeFile(FileSystem fileSys, Path name, int repl)
   throws IOException {
-    FSDataOutputStream stm = fileSys.create(name, true,
-        fileSys.getConf().getInt("io.file.buffer.size", 4096),
-        (short)repl, (long)blockSize);
+    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
+        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+        (short) repl, blockSize);
     byte[] buffer = new byte[fileSize];
     Random rand = new Random(seed);
     rand.nextBytes(buffer);
@@ -233,11 +234,13 @@ public class TestStartup extends TestCase {
       sd = it.next();
 
       if(sd.getStorageDirType().isOfType(NameNodeDirType.IMAGE)) {
-        File imf = img.getStorage().getStorageFile(sd, NameNodeFile.IMAGE, 0);
+        img.getStorage();
+        File imf = NNStorage.getStorageFile(sd, NameNodeFile.IMAGE, 0);
         LOG.info("--image file " + imf.getAbsolutePath() + "; len = " + imf.length() + "; expected = " + expectedImgSize);
         assertEquals(expectedImgSize, imf.length());	
       } else if(sd.getStorageDirType().isOfType(NameNodeDirType.EDITS)) {
-        File edf = img.getStorage().getStorageFile(sd, NameNodeFile.EDITS, 0);
+        img.getStorage();
+        File edf = NNStorage.getStorageFile(sd, NameNodeFile.EDITS, 0);
         LOG.info("-- edits file " + edf.getAbsolutePath() + "; len = " + edf.length()  + "; expected = " + expectedEditsSize);
         assertEquals(expectedEditsSize, edf.length());	
       } else {
@@ -342,8 +345,10 @@ public class TestStartup extends TestCase {
       FSImage image = nn.getFSImage();
       StorageDirectory sd = image.getStorage().getStorageDir(0); //only one
       assertEquals(sd.getStorageDirType(), NameNodeDirType.IMAGE_AND_EDITS);
-      File imf = image.getStorage().getStorageFile(sd, NameNodeFile.IMAGE, 0);
-      File edf = image.getStorage().getStorageFile(sd, NameNodeFile.EDITS, 0);
+      image.getStorage();
+      File imf = NNStorage.getStorageFile(sd, NameNodeFile.IMAGE, 0);
+      image.getStorage();
+      File edf = NNStorage.getStorageFile(sd, NameNodeFile.EDITS, 0);
       LOG.info("--image file " + imf.getAbsolutePath() + "; len = " + imf.length());
       LOG.info("--edits file " + edf.getAbsolutePath() + "; len = " + edf.length());
 

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/security/TestPermission.java

@@ -124,7 +124,7 @@ public class TestPermission extends TestCase {
 
       FsPermission filePerm = new FsPermission((short)0444);
       FSDataOutputStream out = fs.create(new Path("/b1/b2/b3.txt"), filePerm,
-          true, conf.getInt("io.file.buffer.size", 4096),
+          true, conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
           fs.getDefaultReplication(), fs.getDefaultBlockSize(), null);
       out.write(123);
       out.close();
@@ -223,7 +223,7 @@ public class TestPermission extends TestCase {
       userfs.mkdirs(RENAME_PATH);
       assertTrue(canRename(userfs, RENAME_PATH, CHILD_DIR1));
     } finally {
-      if(cluster != null) cluster.shutdown();
+      cluster.shutdown();
     }
   }
 

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestJMXGet.java

@@ -25,6 +25,7 @@ import java.util.Random;
 import junit.framework.TestCase;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
@@ -51,8 +52,8 @@ public class TestJMXGet extends TestCase {
   private void writeFile(FileSystem fileSys, Path name, int repl)
   throws IOException {
     FSDataOutputStream stm = fileSys.create(name, true,
-        fileSys.getConf().getInt("io.file.buffer.size", 4096),
-        (short)repl, (long)blockSize);
+        fileSys.getConf().getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+        (short)repl, blockSize);
     byte[] buffer = new byte[fileSize];
     Random rand = new Random(seed);
     rand.nextBytes(buffer);