瀏覽代碼

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 年之前
父節點
當前提交
6ef2552953

+ 3 - 0
CHANGES.txt

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

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

@@ -25,6 +25,12 @@
 
 <!--- global properties -->
 
+<property>
+  <name>hadoop.common.configuration.version</name>
+  <value>0.21.0</value>
+  <description>version of this configuration file</description>
+</property>
+
 <property>
   <name>hadoop.tmp.dir</name>
   <value>/tmp/hadoop-${user.name}</value>
@@ -32,7 +38,7 @@
 </property>
 
 <property>
-  <name>hadoop.native.lib</name>
+  <name>io.native.lib.available</name>
   <value>true</value>
   <description>Should native hadoop libraries, if present, be used.</description>
 </property>
@@ -127,7 +133,7 @@
 <!-- file system properties -->
 
 <property>
-  <name>fs.default.name</name>
+  <name>fs.defaultFS</name>
   <value>file:///</value>
   <description>The name of the default file system.  A URI whose
   scheme and authority determine the FileSystem implementation.  The
@@ -244,7 +250,34 @@
   </description>
 </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>
   <name>fs.s3.block.size</name>
@@ -459,7 +492,7 @@
 <!-- Rack Configuration -->
 
 <property>
-  <name>topology.node.switch.mapping.impl</name>
+	<name>net.topology.node.switch.mapping.impl</name>
   <value>org.apache.hadoop.net.ScriptBasedMapping</value>
   <description> The default implementation of the DNSToSwitchMapping. It
     invokes a script specified in topology.script.file.name to resolve
@@ -469,7 +502,7 @@
 </property>
 
 <property>
-  <name>topology.script.file.name</name>
+  <name>net.topology.script.file.name</name>
   <value></value>
   <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
@@ -478,11 +511,191 @@
 </property>
 
 <property>
-  <name>topology.script.number.args</name>
+  <name>net.topology.script.number.args</name>
   <value>100</value>
   <description> The max number of args that the script configured with 
     topology.script.file.name should be run with. Each arg is an
     IP address.
   </description>
 </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>

+ 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.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.util.ReflectionUtils;
@@ -184,7 +185,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
    * the key most recently
    */
   private HashMap<String, String> updatingResource;
-  
+ 
   /**
    * Class to keep the information about the keys which replace the deprecated
    * ones.
@@ -355,6 +356,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     //adds deprecation for oldKey1 to two new keys(newkey1, newkey2).
     //so get or set of oldKey1 will correctly populate/access values of 
     //newkey1 and newkey2
+    addDeprecatedKeys();
   }
   
   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) {
     super.setConf(conf);
     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) {
-    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;
     return Math.max(bytesPerSum,
                     Math.max(proportionalBufferSize, defaultBufferSize));
@@ -119,7 +122,9 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
     
     public ChecksumFSInputChecker(ChecksumFileSystem fs, Path file)
       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)
@@ -320,7 +325,8 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
                           Configuration conf)
       throws IOException {
       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);
     }
     

+ 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.util.Shell;
 
 /** 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 {
-    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 {

+ 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.Rename;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.io.MultipleIOException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Progressable;
@@ -68,7 +69,10 @@ import org.apache.hadoop.util.ReflectionUtils;
  * implementation is DistributedFileSystem.
  *****************************************************************/
 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);
 
@@ -104,7 +108,7 @@ public abstract class FileSystem extends Configured implements Closeable {
    * @return the uri of the default filesystem
    */
   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.

+ 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.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableFactories;
 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 */
   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) */
   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());
 
-    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 {

+ 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.CompressionStrategy;
 import org.apache.hadoop.util.NativeCodeLoader;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 
 /**
  * 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>
    */
   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.Shell.ShellCommandExecutor;
 import org.apache.hadoop.conf.*;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 
 /**
  * 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 
 implements Configurable
@@ -41,10 +42,13 @@ implements Configurable
   // script must accept at least this many args
   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) {
     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.hadoop.io.Text;
 import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 
 /**
  * AccessTokenHandler can be instantiated in 2 modes, master mode and slave
@@ -49,9 +50,13 @@ import org.apache.hadoop.io.WritableUtils;
  */
 public class AccessTokenHandler {
   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;
   /*
@@ -304,4 +309,4 @@ public class AccessTokenHandler {
   synchronized void setTokenLifetime(long 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.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 
 /**
  * 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.
    */
   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, 
                                      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"));
+  }
+}