Browse Source

HADOOP-6233. Rename configuration keys towards API standardization and backward compatibility. Contributed by Jithendra Pandey.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@816830 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 16 years ago
parent
commit
6ef2552953

+ 3 - 0
CHANGES.txt

@@ -598,6 +598,9 @@ Trunk (unreleased changes)
     HADOOP-6271. Add recursive and non recursive create and mkdir to 
     HADOOP-6271. Add recursive and non recursive create and mkdir to 
     FileContext. (Sanjay Radia via suresh)
     FileContext. (Sanjay Radia via suresh)
 
 
+    HADOOP-6233. Rename configuration keys towards API standardization and
+    backward compatibility. (Jithendra Pandey via suresh)
+
   BUG FIXES
   BUG FIXES
     
     
     HADOOP-5379. CBZip2InputStream to throw IOException on data crc error.
     HADOOP-5379. CBZip2InputStream to throw IOException on data crc error.

+ 218 - 5
src/java/core-default.xml

@@ -25,6 +25,12 @@
 
 
 <!--- global properties -->
 <!--- global properties -->
 
 
+<property>
+  <name>hadoop.common.configuration.version</name>
+  <value>0.21.0</value>
+  <description>version of this configuration file</description>
+</property>
+
 <property>
 <property>
   <name>hadoop.tmp.dir</name>
   <name>hadoop.tmp.dir</name>
   <value>/tmp/hadoop-${user.name}</value>
   <value>/tmp/hadoop-${user.name}</value>
@@ -32,7 +38,7 @@
 </property>
 </property>
 
 
 <property>
 <property>
-  <name>hadoop.native.lib</name>
+  <name>io.native.lib.available</name>
   <value>true</value>
   <value>true</value>
   <description>Should native hadoop libraries, if present, be used.</description>
   <description>Should native hadoop libraries, if present, be used.</description>
 </property>
 </property>
@@ -127,7 +133,7 @@
 <!-- file system properties -->
 <!-- file system properties -->
 
 
 <property>
 <property>
-  <name>fs.default.name</name>
+  <name>fs.defaultFS</name>
   <value>file:///</value>
   <value>file:///</value>
   <description>The name of the default file system.  A URI whose
   <description>The name of the default file system.  A URI whose
   scheme and authority determine the FileSystem implementation.  The
   scheme and authority determine the FileSystem implementation.  The
@@ -244,7 +250,34 @@
   </description>
   </description>
 </property>
 </property>
 
 
+<property>
+  <name>fs.df.interval</name>
+  <value>60000</value>
+  <description>Disk usage statistics refresh interval in msec.</description>
+</property>
 
 
+<property>
+  <name>fs.access.token.enable</name>
+  <value>false</value>
+  <description>
+    If "true", access tokens are used as capabilities for accessing datanodes.
+    If "false", no access tokens are checked on accessing datanodes.
+  </description>
+</property>
+
+<property>
+  <name>fs.access.key.update.interval</name>
+  <value>600</value>
+  <description>
+    Interval in minutes at which namenode updates its access keys.
+  </description>
+</property>
+
+<property>
+  <name>fs.access.token.lifetime</name>
+  <value>600</value>
+  <description>The lifetime of access tokens in minutes.</description>
+</property>
 
 
 <property>
 <property>
   <name>fs.s3.block.size</name>
   <name>fs.s3.block.size</name>
@@ -459,7 +492,7 @@
 <!-- Rack Configuration -->
 <!-- Rack Configuration -->
 
 
 <property>
 <property>
-  <name>topology.node.switch.mapping.impl</name>
+	<name>net.topology.node.switch.mapping.impl</name>
   <value>org.apache.hadoop.net.ScriptBasedMapping</value>
   <value>org.apache.hadoop.net.ScriptBasedMapping</value>
   <description> The default implementation of the DNSToSwitchMapping. It
   <description> The default implementation of the DNSToSwitchMapping. It
     invokes a script specified in topology.script.file.name to resolve
     invokes a script specified in topology.script.file.name to resolve
@@ -469,7 +502,7 @@
 </property>
 </property>
 
 
 <property>
 <property>
-  <name>topology.script.file.name</name>
+  <name>net.topology.script.file.name</name>
   <value></value>
   <value></value>
   <description> The script name that should be invoked to resolve DNS names to
   <description> The script name that should be invoked to resolve DNS names to
     NetworkTopology names. Example: the script would take host.foo.bar as an
     NetworkTopology names. Example: the script would take host.foo.bar as an
@@ -478,11 +511,191 @@
 </property>
 </property>
 
 
 <property>
 <property>
-  <name>topology.script.number.args</name>
+  <name>net.topology.script.number.args</name>
   <value>100</value>
   <value>100</value>
   <description> The max number of args that the script configured with 
   <description> The max number of args that the script configured with 
     topology.script.file.name should be run with. Each arg is an
     topology.script.file.name should be run with. Each arg is an
     IP address.
     IP address.
   </description>
   </description>
 </property>
 </property>
+
+<!-- Local file system -->
+<property>
+  <name>file.stream-buffer-size</name>
+  <value>4096</value>
+  <description>The size of buffer to stream files.
+  The size of this buffer should probably be a multiple of hardware
+  page size (4096 on Intel x86), and it determines how much data is
+  buffered during read and write operations.</description>
+</property>
+
+<property>
+  <name>file.bytes-per-checksum</name>
+  <value>512</value>
+  <description>The number of bytes per checksum.  Must not be larger than
+  file.stream-buffer-size</description>
+</property>
+
+<property>
+  <name>file.client-write-packet-size</name>
+  <value>65536</value>
+  <description>Packet size for clients to write</description>
+</property>
+
+<property>
+  <name>file.blocksize</name>
+  <value>67108864</value>
+  <description>Block size</description>
+</property>
+
+<property>
+  <name>file.replication</name>
+  <value>1</value>
+  <description>Replication factor</description>
+</property>
+
+<!-- s3 File System -->
+
+<property>
+  <name>s3.stream-buffer-size</name>
+  <value>4096</value>
+  <description>The size of buffer to stream files.
+  The size of this buffer should probably be a multiple of hardware
+  page size (4096 on Intel x86), and it determines how much data is
+  buffered during read and write operations.</description>
+</property>
+
+<property>
+  <name>s3.bytes-per-checksum</name>
+  <value>512</value>
+  <description>The number of bytes per checksum.  Must not be larger than
+  s3.stream-buffer-size</description>
+</property>
+
+<property>
+  <name>s3.client-write-packet-size</name>
+  <value>65536</value>
+  <description>Packet size for clients to write</description>
+</property>
+
+<property>
+  <name>s3.blocksize</name>
+  <value>67108864</value>
+  <description>Block size</description>
+</property>
+
+<property>
+  <name>s3.replication</name>
+  <value>3</value>
+  <description>Replication factor</description>
+</property>
+
+<!-- s3native File System -->
+
+<property>
+  <name>s3native.stream-buffer-size</name>
+  <value>4096</value>
+  <description>The size of buffer to stream files.
+  The size of this buffer should probably be a multiple of hardware
+  page size (4096 on Intel x86), and it determines how much data is
+  buffered during read and write operations.</description>
+</property>
+
+<property>
+  <name>s3native.bytes-per-checksum</name>
+  <value>512</value>
+  <description>The number of bytes per checksum.  Must not be larger than
+  s3native.stream-buffer-size</description>
+</property>
+
+<property>
+  <name>s3native.client-write-packet-size</name>
+  <value>65536</value>
+  <description>Packet size for clients to write</description>
+</property>
+
+<property>
+  <name>s3native.blocksize</name>
+  <value>67108864</value>
+  <description>Block size</description>
+</property>
+
+<property>
+  <name>s3native.replication</name>
+  <value>3</value>
+  <description>Replication factor</description>
+</property>
+
+<!-- Kosmos File System -->
+
+<property>
+  <name>kfs.stream-buffer-size</name>
+  <value>4096</value>
+  <description>The size of buffer to stream files.
+  The size of this buffer should probably be a multiple of hardware
+  page size (4096 on Intel x86), and it determines how much data is
+  buffered during read and write operations.</description>
+</property>
+
+<property>
+  <name>kfs.bytes-per-checksum</name>
+  <value>512</value>
+  <description>The number of bytes per checksum.  Must not be larger than
+  kfs.stream-buffer-size</description>
+</property>
+
+<property>
+  <name>kfs.client-write-packet-size</name>
+  <value>65536</value>
+  <description>Packet size for clients to write</description>
+</property>
+
+<property>
+  <name>kfs.blocksize</name>
+  <value>67108864</value>
+  <description>Block size</description>
+</property>
+
+<property>
+  <name>kfs.replication</name>
+  <value>3</value>
+  <description>Replication factor</description>
+</property>
+
+<!-- FTP file system -->
+<property>
+  <name>ftp.stream-buffer-size</name>
+  <value>4096</value>
+  <description>The size of buffer to stream files.
+  The size of this buffer should probably be a multiple of hardware
+  page size (4096 on Intel x86), and it determines how much data is
+  buffered during read and write operations.</description>
+</property>
+
+<property>
+  <name>ftp.bytes-per-checksum</name>
+  <value>512</value>
+  <description>The number of bytes per checksum.  Must not be larger than
+  ftp.stream-buffer-size</description>
+</property>
+
+<property>
+  <name>ftp.client-write-packet-size</name>
+  <value>65536</value>
+  <description>Packet size for clients to write</description>
+</property>
+
+<property>
+  <name>ftp.blocksize</name>
+  <value>67108864</value>
+  <description>Block size</description>
+</property>
+
+<property>
+  <name>ftp.replication</name>
+  <value>3</value>
+  <description>Replication factor</description>
+</property>
+
+
 </configuration>
 </configuration>

+ 33 - 1
src/java/org/apache/hadoop/conf/Configuration.java

@@ -59,6 +59,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ReflectionUtils;
@@ -184,7 +185,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
    * the key most recently
    * the key most recently
    */
    */
   private HashMap<String, String> updatingResource;
   private HashMap<String, String> updatingResource;
-  
+ 
   /**
   /**
    * Class to keep the information about the keys which replace the deprecated
    * Class to keep the information about the keys which replace the deprecated
    * ones.
    * ones.
@@ -355,6 +356,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     //adds deprecation for oldKey1 to two new keys(newkey1, newkey2).
     //adds deprecation for oldKey1 to two new keys(newkey1, newkey2).
     //so get or set of oldKey1 will correctly populate/access values of 
     //so get or set of oldKey1 will correctly populate/access values of 
     //newkey1 and newkey2
     //newkey1 and newkey2
+    addDeprecatedKeys();
   }
   }
   
   
   private Properties properties;
   private Properties properties;
@@ -1742,4 +1744,34 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     }
     }
   }
   }
 
 
+  //Load deprecated keys in common
+  private static void addDeprecatedKeys() {
+    Configuration.addDeprecation("topology.script.file.name", 
+               new String[]{CommonConfigurationKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY});
+    Configuration.addDeprecation("topology.script.number.args", 
+               new String[]{CommonConfigurationKeys.NET_TOPOLOGY_SCRIPT_NUMBER_ARGS_KEY});
+    Configuration.addDeprecation("hadoop.configured.node.mapping", 
+               new String[]{CommonConfigurationKeys.NET_TOPOLOGY_CONFIGURED_NODE_MAPPING_KEY});
+    Configuration.addDeprecation("topology.node.switch.mapping.impl", 
+               new String[]{CommonConfigurationKeys.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY});
+    Configuration.addDeprecation("dfs.umask", 
+               new String[]{CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY},
+               "dfs.umask is deprecated, use " + 
+               CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY + 
+               " with octal or symbolic specifications.");
+    Configuration.addDeprecation("dfs.df.interval", 
+               new String[]{CommonConfigurationKeys.FS_DF_INTERVAL_KEY});
+    Configuration.addDeprecation("dfs.client.buffer.dir", 
+               new String[]{CommonConfigurationKeys.FS_CLIENT_BUFFER_DIR_KEY});
+    Configuration.addDeprecation("hadoop.native.lib", 
+               new String[]{CommonConfigurationKeys.IO_NATIVE_LIB_AVAILABLE_KEY});
+    Configuration.addDeprecation("dfs.access.token.enable", 
+               new String[]{CommonConfigurationKeys.FS_ACCESS_TOKEN_ENABLE_KEY});
+    Configuration.addDeprecation("dfs.access.key.update.interval", 
+               new String[]{CommonConfigurationKeys.FS_ACCESS_KEY_UPDATE_INTERVAL_KEY});
+    Configuration.addDeprecation("dfs.access.token.lifetime", 
+               new String[]{CommonConfigurationKeys.FS_ACCESS_TOKEN_LIFETIME_KEY});
+    Configuration.addDeprecation("fs.default.name", 
+               new String[]{CommonConfigurationKeys.FS_DEFAULT_NAME_KEY});
+  }
 }
 }

+ 10 - 4
src/java/org/apache/hadoop/fs/ChecksumFileSystem.java

@@ -54,7 +54,8 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
   public void setConf(Configuration conf) {
   public void setConf(Configuration conf) {
     super.setConf(conf);
     super.setConf(conf);
     if (conf != null) {
     if (conf != null) {
-      bytesPerChecksum = conf.getInt("io.bytes.per.checksum", 512);
+      bytesPerChecksum = conf.getInt(LocalFileSystemConfigKeys.LOCAL_FS_BYTES_PER_CHECKSUM_KEY,
+		                     LocalFileSystemConfigKeys.LOCAL_FS_BYTES_PER_CHECKSUM_DEFAULT);
     }
     }
   }
   }
   
   
@@ -94,7 +95,9 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
   }
   }
 
 
   private int getSumBufferSize(int bytesPerSum, int bufferSize) {
   private int getSumBufferSize(int bytesPerSum, int bufferSize) {
-    int defaultBufferSize = getConf().getInt("io.file.buffer.size", 4096);
+    int defaultBufferSize = getConf().getInt(
+                       LocalFileSystemConfigKeys.LOCAL_FS_STREAM_BUFFER_SIZE_KEY,
+                       LocalFileSystemConfigKeys.LOCAL_FS_STREAM_BUFFER_SIZE_DEFAULT);
     int proportionalBufferSize = bufferSize / bytesPerSum;
     int proportionalBufferSize = bufferSize / bytesPerSum;
     return Math.max(bytesPerSum,
     return Math.max(bytesPerSum,
                     Math.max(proportionalBufferSize, defaultBufferSize));
                     Math.max(proportionalBufferSize, defaultBufferSize));
@@ -119,7 +122,9 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
     
     
     public ChecksumFSInputChecker(ChecksumFileSystem fs, Path file)
     public ChecksumFSInputChecker(ChecksumFileSystem fs, Path file)
       throws IOException {
       throws IOException {
-      this(fs, file, fs.getConf().getInt("io.file.buffer.size", 4096));
+      this(fs, file, fs.getConf().getInt(
+                       LocalFileSystemConfigKeys.LOCAL_FS_STREAM_BUFFER_SIZE_KEY, 
+                       LocalFileSystemConfigKeys.LOCAL_FS_STREAM_BUFFER_SIZE_DEFAULT));
     }
     }
     
     
     public ChecksumFSInputChecker(ChecksumFileSystem fs, Path file, int bufferSize)
     public ChecksumFSInputChecker(ChecksumFileSystem fs, Path file, int bufferSize)
@@ -320,7 +325,8 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
                           Configuration conf)
                           Configuration conf)
       throws IOException {
       throws IOException {
       this(fs, file, overwrite, 
       this(fs, file, overwrite, 
-           conf.getInt("io.file.buffer.size", 4096),
+           conf.getInt(LocalFileSystemConfigKeys.LOCAL_FS_STREAM_BUFFER_SIZE_KEY,
+		       LocalFileSystemConfigKeys.LOCAL_FS_STREAM_BUFFER_SIZE_DEFAULT),
            replication, blockSize, null);
            replication, blockSize, null);
     }
     }
     
     

+ 139 - 0
src/java/org/apache/hadoop/fs/CommonConfigurationKeys.java

@@ -0,0 +1,139 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs;
+
+/** 
+ * This class contains constants for configuration keys used
+ * in the common code.
+ *
+ */
+
+public class CommonConfigurationKeys {
+  
+  // The Keys
+  public static final String  IO_NATIVE_LIB_AVAILABLE_KEY =
+                                           "io.native.lib.available";
+  public static final boolean IO_NATIVE_LIB_AVAILABLE_DEFAULT = true;
+  public static final String  NET_TOPOLOGY_SCRIPT_NUMBER_ARGS_KEY =
+                                         "net.topology.script.number.args";
+  public static final int     NET_TOPOLOGY_SCRIPT_NUMBER_ARGS_DEFAULT = 100;
+
+  //FS keys
+  public static final String  FS_HOME_DIR_KEY = "fs.homeDir";
+  public static final String  FS_HOME_DIR_DEFAULT = "/user";
+  public static final String  FS_DEFAULT_NAME_KEY = "fs.defaultFS";
+  public static final String  FS_DEFAULT_NAME_DEFAULT = "file:///";
+  public static final String  FS_PERMISSIONS_UMASK_KEY = "fs.permissions.umask-mode";
+  public static final int     FS_PERMISSIONS_UMASK_DEFAULT = 0022;
+  public static final String  FS_DF_INTERVAL_KEY = "fs.df.interval"; 
+  public static final long    FS_DF_INTERVAL_DEFAULT = 60000;
+  public static final String  FS_ACCESS_TOKEN_ENABLE_KEY =
+                                       "fs.access.token.enable";
+  public static final boolean FS_ACCESS_TOKEN_ENABLE_DEFAULT = false;
+  public static final String  FS_ACCESS_KEY_UPDATE_INTERVAL_KEY =
+                                       "fs.access.key.update.interval";
+  public static final long    FS_ACCESS_KEY_UPDATE_INTERVAL_DEFAULT = 600;
+  public static final String  FS_ACCESS_TOKEN_LIFETIME_KEY =
+                                       "fs.access.token.lifetime";
+  public static final long    FS_ACCESS_TOKEN_LIFETIME_DEFAULT = 600;
+
+
+  //Defaults are not specified for following keys
+  public static final String  NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY = 
+                                         "net.topology.script.file.name";
+  public static final String  NET_TOPOLOGY_CONFIGURED_NODE_MAPPING_KEY =
+                                     "net.topology.configured.node.mapping";
+  public static final String  NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY =
+                                   "net.topology.node.switch.mapping.impl";
+
+  public static final String  FS_CLIENT_BUFFER_DIR_KEY =
+                                         "fs.client.buffer.dir";
+
+  //TBD: Code is not updated to use following keys.
+  //These keys will be used in later versions
+  //
+  public static final long    FS_LOCAL_BLOCK_SIZE_DEFAULT = 32*1024*1024;
+  public static final String  FS_AUTOMATIC_CLOSE_KEY = "fs.automatic.close";
+  public static final boolean FS_AUTOMATIC_CLOSE_DEFAULT = true;
+  public static final String  FS_FILE_IMPL_KEY = "fs.file.impl";
+  public static final String  FS_FTP_HOST_KEY = "fs.ftp.host";
+  public static final String  FS_FTP_HOST_PORT_KEY = "fs.ftp.host.port";
+  public static final String  FS_TRASH_INTERVAL_KEY = "fs.trash.interval";
+  public static final long    FS_TRASH_INTERVAL_DEFAULT = 0;
+
+  public static final String  IO_MAPFILE_BLOOM_SIZE_KEY = "io.mapfile.bloom.size";
+  public static final int     IO_MAPFILE_BLOOM_SIZE_DEFAULT = 1024*1024;
+  public static final String  IO_MAPFILE_BLOOM_ERROR_RATE_KEY = 
+                                       "io.mapfile.bloom.error.rate" ;
+  public static final float   IO_MAPFILE_BLOOM_ERROR_RATE_DEFAULT = 0.005f;
+  public static final String  IO_COMPRESSION_CODEC_LZO_CLASS_KEY = "io.compression.codec.lzo.class";
+  public static final String  IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_KEY = 
+                                       "io.compression.codec.lzo.buffersize";
+  public static final int     IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_DEFAULT = 64*1024;
+  public static final String  IO_MAP_INDEX_INTERVAL_KEY = "io.map.index.interval";
+  public static final int     IO_MAP_INDEX_INTERVAL_DEFAULT = 128;
+  public static final String  IO_MAP_INDEX_SKIP_KEY = "io.map.index.skip";
+  public static final int     IO_MAP_INDEX_SKIP_DEFAULT = 0;
+  public static final String  IO_SEQFILE_COMPRESS_BLOCKSIZE_KEY = "io.seqfile.compress.blocksize";
+  public static final int     IO_SEQFILE_COMPRESS_BLOCKSIZE_DEFAULT = 1000000;
+  public static final String  IO_SKIP_CHECKSUM_ERRORS_KEY = "io.skip.checksum.errors";
+  public static final boolean IO_SKIP_CHECKSUM_ERRORS_DEFAULT = false;
+  public static final String  IO_SORT_MB_KEY = "io.sort.mb";
+  public static final int     IO_SORT_MB_DEFAULT = 100;
+  public static final String  IO_SORT_FACTOR_KEY = "io.sort.factor";
+  public static final int     IO_SORT_FACTOR_DEFAULT = 100;
+  public static final String  IO_SERIALIZATIONS_KEY = "io.serializations";
+
+  public static final String  TFILE_IO_CHUNK_SIZE_KEY = "tfile.io.chunk.size";
+  public static final int     TFILE_IO_CHUNK_SIZE_DEFAULT = 1024*1024;
+  public static final String  TFILE_FS_INPUT_BUFFER_SIZE_KEY = "tfile.fs.input.buffer.size";
+  public static final int     TFILE_FS_INPUT_BUFFER_SIZE_DEFAULT = 256*1024;
+  public static final String  TFILE_FS_OUTPUT_BUFFER_SIZE_KEY = "tfile.fs.output.buffer.size";
+  public static final int     TFILE_FS_OUTPUT_BUFFER_SIZE_DEFAULT = 256*1024;
+
+  public static final String  IPC_PING_INTERVAL_KEY = "ipc.ping.interval";
+  public static final int     IPC_PING_INTERVAL_DEFAULT = 60000;
+  public static final String  IPC_CLIENT_PING_KEY = "ipc.client.ping";
+  public static final boolean IPC_CLIENT_PING_DEFAULT = true;
+  public static final String  IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY = 
+                                       "ipc.client.connection.maxidletime";
+  public static final int     IPC_CLIENT_CONNECTION_MAXIDLETIME_DEFAULT = 10000;
+  public static final String  IPC_CLIENT_CONNECT_MAX_RETRIES_KEY = 
+                                       "ipc.client.connect.max.retries";
+  public static final int     IPC_CLIENT_CONNECT_MAX_RETRIES_DEFAULT = 10;
+  public static final String  IPC_CLIENT_TCPNODELAY_KEY = "ipc.client.tcpnodelay";
+  public static final boolean IPC_CLIENT_TCPNODELAY_DEFAULT = false;
+  public static final String  IPC_SERVER_LISTEN_QUEUE_SIZE_KEY = 
+                                       "ipc.server.listen.queue.size";
+  public static final int     IPC_SERVER_LISTEN_QUEUE_SIZE_DEFAULT = 128;
+  public static final String  IPC_CLIENT_KILL_MAX_KEY = "ipc.client.kill.max";
+  public static final int     IPC_CLIENT_KILL_MAX_DEFAULT = 10;
+  public static final String  IPC_CLIENT_IDLETHRESHOLD_KEY = "ipc.client.idlethreshold";
+  public static final int     IPC_CLIENT_IDLETHRESHOLD_DEFAULT = 4000;
+  public static final String  IPC_SERVER_TCPNODELAY_KEY = "ipc.server.tcpnodelay";
+  public static final boolean IPC_SERVER_TCPNODELAY_DEFAULT = false;
+
+  public static final String  HADOOP_RPC_SOCKET_FACTORY_CLASS_DEFAULT_KEY = 
+                                       "hadoop.rpc.socket.factory.class.default";
+  public static final String  HADOOP_SOCKS_SERVER_KEY = "hadoop.socks.server";
+  public static final String  HADOOP_JOB_UGI_KEY = "hadoop.job.ugi";
+  public static final String  HADOOP_UTIL_HASH_TYPE_KEY = "hadoop.util.hash.type";
+  public static final String  HADOOP_UTIL_HASH_TYPE_DEFAULT = "murmur";
+}
+

+ 2 - 1
src/java/org/apache/hadoop/fs/DF.java

@@ -25,6 +25,7 @@ import java.util.EnumSet;
 import java.util.StringTokenizer;
 import java.util.StringTokenizer;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.Shell;
 
 
 /** Filesystem disk space usage statistics.  Uses the unix 'df' program.
 /** Filesystem disk space usage statistics.  Uses the unix 'df' program.
@@ -72,7 +73,7 @@ public class DF extends Shell {
   }
   }
 
 
   public DF(File path, Configuration conf) throws IOException {
   public DF(File path, Configuration conf) throws IOException {
-    this(path, conf.getLong("dfs.df.interval", DF.DF_INTERVAL_DEFAULT));
+    this(path, conf.getLong(CommonConfigurationKeys.FS_DF_INTERVAL_KEY, DF.DF_INTERVAL_DEFAULT));
   }
   }
 
 
   public DF(File path, long dfInterval) throws IOException {
   public DF(File path, long dfInterval) throws IOException {

+ 6 - 2
src/java/org/apache/hadoop/fs/FileSystem.java

@@ -44,6 +44,7 @@ import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.Options.CreateOpts;
 import org.apache.hadoop.fs.Options.CreateOpts;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.io.MultipleIOException;
 import org.apache.hadoop.io.MultipleIOException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.Progressable;
@@ -68,7 +69,10 @@ import org.apache.hadoop.util.ReflectionUtils;
  * implementation is DistributedFileSystem.
  * implementation is DistributedFileSystem.
  *****************************************************************/
  *****************************************************************/
 public abstract class FileSystem extends Configured implements Closeable {
 public abstract class FileSystem extends Configured implements Closeable {
-  public static final String FS_DEFAULT_NAME_KEY = "fs.default.name";
+  public static final String FS_DEFAULT_NAME_KEY = 
+                   CommonConfigurationKeys.FS_DEFAULT_NAME_KEY;
+  public static final String DEFAULT_FS = 
+                   CommonConfigurationKeys.FS_DEFAULT_NAME_DEFAULT;
 
 
   public static final Log LOG = LogFactory.getLog(FileSystem.class);
   public static final Log LOG = LogFactory.getLog(FileSystem.class);
 
 
@@ -104,7 +108,7 @@ public abstract class FileSystem extends Configured implements Closeable {
    * @return the uri of the default filesystem
    * @return the uri of the default filesystem
    */
    */
   public static URI getDefaultUri(Configuration conf) {
   public static URI getDefaultUri(Configuration conf) {
-    return URI.create(fixName(conf.get(FS_DEFAULT_NAME_KEY, "file:///")));
+    return URI.create(fixName(conf.get(FS_DEFAULT_NAME_KEY, DEFAULT_FS)));
   }
   }
 
 
   /** Set the default filesystem URI in a configuration.
   /** Set the default filesystem URI in a configuration.

+ 42 - 0
src/java/org/apache/hadoop/fs/LocalFileSystemConfigKeys.java

@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs;
+
+/** 
+ * This class contains constants for configuration keys used
+ * in the local file system, raw local fs and checksum fs.
+ *
+ */
+
+public class LocalFileSystemConfigKeys extends CommonConfigurationKeys {
+  public static final String  LOCAL_FS_BLOCK_SIZE_KEY = "file.blocksize";
+  public static final long    LOCAL_FS_BLOCK_SIZE_DEFAULT = 64*1024*1024;
+  public static final String  LOCAL_FS_REPLICATION_KEY = "file.replication";
+  public static final short   LOCAL_FS_REPLICATION_DEFAULT = 1;
+  public static final String  LOCAL_FS_STREAM_BUFFER_SIZE_KEY = 
+                                                     "file.stream-buffer-size";
+  public static final int     LOCAL_FS_STREAM_BUFFER_SIZE_DEFAULT = 4096;
+  public static final String  LOCAL_FS_BYTES_PER_CHECKSUM_KEY = 
+                                                     "file.bytes-per-checksum";
+  public static final int     LOCAL_FS_BYTES_PER_CHECKSUM_DEFAULT = 512;
+  public static final String  LOCAL_FS_CLIENT_WRITE_PACKET_SIZE_KEY =
+                                                     "file.client-write-packet-size";
+  public static final int     LOCAL_FS_CLIENT_WRITE_PACKET_SIZE_DEFAULT = 64*1024;
+}
+  

+ 44 - 0
src/java/org/apache/hadoop/fs/ftp/FTPFileSystemConfigKeys.java

@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.ftp;
+
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+
+/** 
+ * This class contains constants for configuration keys used
+ * in the ftp file system. 
+ *
+ */
+
+public class FTPFileSystemConfigKeys extends CommonConfigurationKeys {
+  public static final String  FTP_BLOCK_SIZE_KEY = "ftp.blocksize";
+  public static final long    FTP_BLOCK_SIZE_DEFAULT = 64*1024*1024;
+  public static final String  FTP_REPLICATION_KEY = "ftp.replication";
+  public static final short   FTP_REPLICATION_DEFAULT = 1;
+  public static final String  FTP_STREAM_BUFFER_SIZE_KEY = 
+                                                    "ftp.stream-buffer-size";
+  public static final int     FTP_STREAM_BUFFER_SIZE_DEFAULT = 4096;
+  public static final String  FTP_BYTES_PER_CHECKSUM_KEY = 
+                                                    "ftp.bytes-per-checksum";
+  public static final int     FTP_BYTES_PER_CHECKSUM_DEFAULT = 512;
+  public static final String  FTP_CLIENT_WRITE_PACKET_SIZE_KEY =
+                                                    "ftp.client-write-packet-size";
+  public static final int     FTP_CLIENT_WRITE_PACKET_SIZE_DEFAULT = 64*1024;
+}
+  

+ 44 - 0
src/java/org/apache/hadoop/fs/kfs/KFSConfigKeys.java

@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.kfs;
+
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+
+/** 
+ * This class contains constants for configuration keys used
+ * in the kfs file system. 
+ *
+ */
+
+public class KFSConfigKeys extends CommonConfigurationKeys {
+  public static final String  KFS_BLOCK_SIZE_KEY = "kfs.blocksize";
+  public static final long    KFS_BLOCK_SIZE_DEFAULT = 64*1024*1024;
+  public static final String  KFS_REPLICATION_KEY = "kfs.replication";
+  public static final short   KFS_REPLICATION_DEFAULT = 1;
+  public static final String  KFS_STREAM_BUFFER_SIZE_KEY = 
+                                                    "kfs.stream-buffer-size";
+  public static final int     KFS_STREAM_BUFFER_SIZE_DEFAULT = 4096;
+  public static final String  KFS_BYTES_PER_CHECKSUM_KEY = 
+                                                    "kfs.bytes-per-checksum";
+  public static final int     KFS_BYTES_PER_CHECKSUM_DEFAULT = 512;
+  public static final String  KFS_CLIENT_WRITE_PACKET_SIZE_KEY =
+                                                    "kfs.client-write-packet-size";
+  public static final int     KFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT = 64*1024;
+}
+  

+ 5 - 6
src/java/org/apache/hadoop/fs/permission/FsPermission.java

@@ -24,6 +24,7 @@ import java.io.IOException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableFactories;
 import org.apache.hadoop.io.WritableFactories;
 import org.apache.hadoop.io.WritableFactory;
 import org.apache.hadoop.io.WritableFactory;
@@ -183,12 +184,10 @@ public class FsPermission implements Writable {
 
 
   /** umask property label Deprecated key may be removed in version .23 */
   /** umask property label Deprecated key may be removed in version .23 */
   public static final String DEPRECATED_UMASK_LABEL = "dfs.umask"; 
   public static final String DEPRECATED_UMASK_LABEL = "dfs.umask"; 
-  public static final String UMASK_LABEL = "dfs.umaskmode";
-  public static final int DEFAULT_UMASK = 0022;
-  { Configuration.addDeprecation(DEPRECATED_UMASK_LABEL,
-      new String [] {UMASK_LABEL}, DEPRECATED_UMASK_LABEL + " is deprecated, " +
-            "use " + UMASK_LABEL + " with octal or symbolic specifications."); 
-  }
+  public static final String UMASK_LABEL = 
+                  CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY;
+  public static final int DEFAULT_UMASK = 
+                  CommonConfigurationKeys.FS_PERMISSIONS_UMASK_DEFAULT;
 
 
   /** Get the user file creation mask (umask) */
   /** Get the user file creation mask (umask) */
   public static FsPermission getUMask(Configuration conf) {
   public static FsPermission getUMask(Configuration conf) {

+ 4 - 1
src/java/org/apache/hadoop/fs/s3/Jets3tFileSystemStore.java

@@ -98,7 +98,10 @@ class Jets3tFileSystemStore implements FileSystemStore {
     }
     }
     bucket = new S3Bucket(uri.getHost());
     bucket = new S3Bucket(uri.getHost());
 
 
-    this.bufferSize = conf.getInt("io.file.buffer.size", 4096);
+    this.bufferSize = conf.getInt(
+                       S3FileSystemConfigKeys.S3_STREAM_BUFFER_SIZE_KEY,
+                       S3FileSystemConfigKeys.S3_STREAM_BUFFER_SIZE_DEFAULT
+		      );
   }
   }
 
 
   public String getVersion() throws IOException {
   public String getVersion() throws IOException {

+ 44 - 0
src/java/org/apache/hadoop/fs/s3/S3FileSystemConfigKeys.java

@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3;
+
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+
+/** 
+ * This class contains constants for configuration keys used
+ * in the s3 file system. 
+ *
+ */
+
+public class S3FileSystemConfigKeys extends CommonConfigurationKeys {
+  public static final String  S3_BLOCK_SIZE_KEY = "s3.blocksize";
+  public static final long    S3_BLOCK_SIZE_DEFAULT = 64*1024*1024;
+  public static final String  S3_REPLICATION_KEY = "s3.replication";
+  public static final short   S3_REPLICATION_DEFAULT = 1;
+  public static final String  S3_STREAM_BUFFER_SIZE_KEY = 
+                                                    "s3.stream-buffer-size";
+  public static final int     S3_STREAM_BUFFER_SIZE_DEFAULT = 4096;
+  public static final String  S3_BYTES_PER_CHECKSUM_KEY = 
+                                                    "s3.bytes-per-checksum";
+  public static final int     S3_BYTES_PER_CHECKSUM_DEFAULT = 512;
+  public static final String  S3_CLIENT_WRITE_PACKET_SIZE_KEY =
+                                                    "s3.client-write-packet-size";
+  public static final int     S3_CLIENT_WRITE_PACKET_SIZE_DEFAULT = 64*1024;
+}
+  

+ 44 - 0
src/java/org/apache/hadoop/fs/s3native/S3NativeFileSystemConfigKeys.java

@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3native;
+
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+
+/** 
+ * This class contains constants for configuration keys used
+ * in the s3 file system. 
+ *
+ */
+
+public class S3NativeFileSystemConfigKeys extends CommonConfigurationKeys {
+  public static final String  S3_NATIVE_BLOCK_SIZE_KEY = "s3native.blocksize";
+  public static final long    S3_NATIVE_BLOCK_SIZE_DEFAULT = 64*1024*1024;
+  public static final String  S3_NATIVE_REPLICATION_KEY = "s3native.replication";
+  public static final short   S3_NATIVE_REPLICATION_DEFAULT = 1;
+  public static final String  S3_NATIVE_STREAM_BUFFER_SIZE_KEY = 
+                                                    "s3native.stream-buffer-size";
+  public static final int     S3_NATIVE_STREAM_BUFFER_SIZE_DEFAULT = 4096;
+  public static final String  S3_NATIVE_BYTES_PER_CHECKSUM_KEY = 
+                                                    "s3native.bytes-per-checksum";
+  public static final int     S3_NATIVE_BYTES_PER_CHECKSUM_DEFAULT = 512;
+  public static final String  S3_NATIVE_CLIENT_WRITE_PACKET_SIZE_KEY =
+                                                    "s3native.client-write-packet-size";
+  public static final int     S3_NATIVE_CLIENT_WRITE_PACKET_SIZE_DEFAULT = 64*1024;
+}
+  

+ 4 - 1
src/java/org/apache/hadoop/io/compress/zlib/ZlibFactory.java

@@ -26,6 +26,7 @@ import org.apache.hadoop.io.compress.Decompressor;
 import org.apache.hadoop.io.compress.zlib.ZlibCompressor.CompressionLevel;
 import org.apache.hadoop.io.compress.zlib.ZlibCompressor.CompressionLevel;
 import org.apache.hadoop.io.compress.zlib.ZlibCompressor.CompressionStrategy;
 import org.apache.hadoop.io.compress.zlib.ZlibCompressor.CompressionStrategy;
 import org.apache.hadoop.util.NativeCodeLoader;
 import org.apache.hadoop.util.NativeCodeLoader;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 
 
 /**
 /**
  * A collection of factories to create the right 
  * A collection of factories to create the right 
@@ -60,7 +61,9 @@ public class ZlibFactory {
    *         and can be loaded for this job, else <code>false</code>
    *         and can be loaded for this job, else <code>false</code>
    */
    */
   public static boolean isNativeZlibLoaded(Configuration conf) {
   public static boolean isNativeZlibLoaded(Configuration conf) {
-    return nativeZlibLoaded && conf.getBoolean("hadoop.native.lib", true); 
+    return nativeZlibLoaded && conf.getBoolean(
+                          CommonConfigurationKeys.IO_NATIVE_LIB_AVAILABLE_KEY, 
+                          CommonConfigurationKeys.IO_NATIVE_LIB_AVAILABLE_DEFAULT);
   }
   }
   
   
   /**
   /**

+ 8 - 4
src/java/org/apache/hadoop/net/ScriptBasedMapping.java

@@ -26,10 +26,11 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.util.*;
 import org.apache.hadoop.util.*;
 import org.apache.hadoop.util.Shell.ShellCommandExecutor;
 import org.apache.hadoop.util.Shell.ShellCommandExecutor;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.conf.*;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 
 
 /**
 /**
  * This class implements the {@link DNSToSwitchMapping} interface using a 
  * This class implements the {@link DNSToSwitchMapping} interface using a 
- * script configured via topology.script.file.name .
+ * script configured via net.topology.script.file.name .
  */
  */
 public final class ScriptBasedMapping extends CachedDNSToSwitchMapping 
 public final class ScriptBasedMapping extends CachedDNSToSwitchMapping 
 implements Configurable
 implements Configurable
@@ -41,10 +42,13 @@ implements Configurable
   // script must accept at least this many args
   // script must accept at least this many args
   static final int MIN_ALLOWABLE_ARGS = 1;
   static final int MIN_ALLOWABLE_ARGS = 1;
   
   
-  static final int DEFAULT_ARG_COUNT = 100;
+  static final int DEFAULT_ARG_COUNT = 
+                     CommonConfigurationKeys.NET_TOPOLOGY_SCRIPT_NUMBER_ARGS_DEFAULT;
   
   
-  static final String SCRIPT_FILENAME_KEY = "topology.script.file.name";
-  static final String SCRIPT_ARG_COUNT_KEY = "topology.script.number.args";
+  static final String SCRIPT_FILENAME_KEY = 
+                     CommonConfigurationKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY ;
+  static final String SCRIPT_ARG_COUNT_KEY = 
+                     CommonConfigurationKeys.NET_TOPOLOGY_SCRIPT_NUMBER_ARGS_KEY ;
   
   
   public ScriptBasedMapping(Configuration conf) {
   public ScriptBasedMapping(Configuration conf) {
     this();
     this();

+ 9 - 4
src/java/org/apache/hadoop/security/AccessTokenHandler.java

@@ -39,6 +39,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 
 
 /**
 /**
  * AccessTokenHandler can be instantiated in 2 modes, master mode and slave
  * AccessTokenHandler can be instantiated in 2 modes, master mode and slave
@@ -49,9 +50,13 @@ import org.apache.hadoop.io.WritableUtils;
  */
  */
 public class AccessTokenHandler {
 public class AccessTokenHandler {
   private static final Log LOG = LogFactory.getLog(AccessTokenHandler.class);
   private static final Log LOG = LogFactory.getLog(AccessTokenHandler.class);
-  public static final String STRING_ENABLE_ACCESS_TOKEN = "dfs.access.token.enable";
-  public static final String STRING_ACCESS_KEY_UPDATE_INTERVAL = "dfs.access.key.update.interval";
-  public static final String STRING_ACCESS_TOKEN_LIFETIME = "dfs.access.token.lifetime";
+  public static final String STRING_ENABLE_ACCESS_TOKEN =
+                         CommonConfigurationKeys.FS_ACCESS_TOKEN_ENABLE_KEY;
+  public static final String STRING_ACCESS_KEY_UPDATE_INTERVAL =
+                         CommonConfigurationKeys.FS_ACCESS_KEY_UPDATE_INTERVAL_KEY;
+  public static final String STRING_ACCESS_TOKEN_LIFETIME =
+                         CommonConfigurationKeys.FS_ACCESS_TOKEN_LIFETIME_KEY;
+
 
 
   private final boolean isMaster;
   private final boolean isMaster;
   /*
   /*
@@ -304,4 +309,4 @@ public class AccessTokenHandler {
   synchronized void setTokenLifetime(long tokenLifetime) {
   synchronized void setTokenLifetime(long tokenLifetime) {
     this.tokenLifetime = tokenLifetime;
     this.tokenLifetime = tokenLifetime;
   }
   }
-}
+}

+ 5 - 2
src/java/org/apache/hadoop/util/NativeCodeLoader.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.util;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 
 
 /**
 /**
  * A helper to load the native hadoop code i.e. libhadoop.so.
  * A helper to load the native hadoop code i.e. libhadoop.so.
@@ -72,7 +73,8 @@ public class NativeCodeLoader {
    *         used for this job; <code>false</code> otherwise.
    *         used for this job; <code>false</code> otherwise.
    */
    */
   public boolean getLoadNativeLibraries(Configuration conf) {
   public boolean getLoadNativeLibraries(Configuration conf) {
-    return conf.getBoolean("hadoop.native.lib", true);
+    return conf.getBoolean(CommonConfigurationKeys.IO_NATIVE_LIB_AVAILABLE_KEY, 
+                           CommonConfigurationKeys.IO_NATIVE_LIB_AVAILABLE_DEFAULT);
   }
   }
   
   
   /**
   /**
@@ -83,7 +85,8 @@ public class NativeCodeLoader {
    */
    */
   public void setLoadNativeLibraries(Configuration conf, 
   public void setLoadNativeLibraries(Configuration conf, 
                                      boolean loadNativeLibraries) {
                                      boolean loadNativeLibraries) {
-    conf.setBoolean("hadoop.native.lib", loadNativeLibraries);
+    conf.setBoolean(CommonConfigurationKeys.IO_NATIVE_LIB_AVAILABLE_KEY,
+                    loadNativeLibraries);
   }
   }
 
 
 }
 }

+ 45 - 0
src/test/core/org/apache/hadoop/conf/TestDeprecatedKeys.java

@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.conf;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.log4j.Level;
+
+import junit.framework.TestCase;
+
+public class TestDeprecatedKeys extends TestCase {
+ 
+  //Tests a deprecated key
+  public void testDeprecatedKeys() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set("topology.script.file.name", "xyz");
+    String scriptFile = conf.get(CommonConfigurationKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY);
+    assertTrue(scriptFile.equals("xyz")) ;
+    int m = conf.getInt(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, CommonConfigurationKeys.FS_PERMISSIONS_UMASK_DEFAULT) ;
+    assertTrue(m == 0022) ;
+    conf.setInt("dfs.umask", 0077);
+    m = conf.getInt(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, CommonConfigurationKeys.FS_PERMISSIONS_UMASK_DEFAULT) ;
+    assertTrue(m == 0077) ;
+    conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "405");
+    String umask = conf.get(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY);
+    assertTrue(umask.equals("405"));
+  }
+}