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
This commit is contained in:
Suresh Srinivas 2009-09-19 00:26:01 +00:00
parent fa48d9ea17
commit 6ef2552953
19 changed files with 709 additions and 31 deletions

View File

@ -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.

View File

@ -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>

View File

@ -59,6 +59,7 @@
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 @@ private String handleDeprecation(String name) {
//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 void write(DataOutput out) throws IOException {
} }
} }
//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});
}
} }

View File

@ -54,7 +54,8 @@ public ChecksumFileSystem(FileSystem fs) {
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 int getBytesPerSum() {
} }
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 @@ private static class ChecksumFSInputChecker extends FSInputChecker {
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 ChecksumFSOutputSummer(ChecksumFileSystem fs,
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);
} }

View File

@ -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";
}

View File

@ -25,6 +25,7 @@
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 @@ protected static OSType getOSType(String osName) {
} }
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 {

View File

@ -44,6 +44,7 @@
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 @@
* 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 static FileSystem get(Configuration conf) throws IOException {
* @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.

View File

@ -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;
}

View File

@ -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;
}

View File

@ -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;
}

View File

@ -24,6 +24,7 @@
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 FsPermission applyUMask(FsPermission umask) {
/** 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 String UMASK_LABEL =
public static final int DEFAULT_UMASK = 0022; CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY;
{ Configuration.addDeprecation(DEPRECATED_UMASK_LABEL, public static final int DEFAULT_UMASK =
new String [] {UMASK_LABEL}, DEPRECATED_UMASK_LABEL + " is deprecated, " + CommonConfigurationKeys.FS_PERMISSIONS_UMASK_DEFAULT;
"use " + UMASK_LABEL + " with octal or symbolic specifications.");
}
/** 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) {

View File

@ -98,7 +98,10 @@ public void initialize(URI uri, Configuration conf) throws IOException {
} }
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 {

View File

@ -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;
}

View File

@ -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;
}

View File

@ -26,6 +26,7 @@
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);
} }
/** /**

View File

@ -26,10 +26,11 @@
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 @@ public ScriptBasedMapping() {
// 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_FILENAME_KEY =
static final String SCRIPT_ARG_COUNT_KEY = "topology.script.number.args"; 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();

View File

@ -39,6 +39,7 @@
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 @@
*/ */
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_ENABLE_ACCESS_TOKEN =
public static final String STRING_ACCESS_KEY_UPDATE_INTERVAL = "dfs.access.key.update.interval"; CommonConfigurationKeys.FS_ACCESS_TOKEN_ENABLE_KEY;
public static final String STRING_ACCESS_TOKEN_LIFETIME = "dfs.access.token.lifetime"; 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 @@ static boolean isTokenExpired(AccessToken token) throws IOException {
synchronized void setTokenLifetime(long tokenLifetime) { synchronized void setTokenLifetime(long tokenLifetime) {
this.tokenLifetime = tokenLifetime; this.tokenLifetime = tokenLifetime;
} }
} }

View File

@ -21,6 +21,7 @@
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 static boolean isNativeCodeLoaded() {
* 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 boolean getLoadNativeLibraries(Configuration conf) {
*/ */
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);
} }
} }

View File

@ -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"));
}
}