HADOOP-12663. Remove Hard-Coded Values From FileSystem.java. (BELUGA BEHR via stevel)
This commit is contained in:
parent
21bededd2f
commit
3fcdbe076f
|
@ -656,7 +656,9 @@ Release 2.9.0 - UNRELEASED
|
||||||
|
|
||||||
HADOOP-12566. Add NullGroupMapping. (Daniel Templeton via kasha)
|
HADOOP-12566. Add NullGroupMapping. (Daniel Templeton via kasha)
|
||||||
|
|
||||||
OPTIMIZATIONS
|
HADOOP-12663. Remove Hard-Coded Values From FileSystem.java.
|
||||||
|
(BELUGA BEHR via stevel)
|
||||||
|
|
||||||
|
|
||||||
BUG FIXES
|
BUG FIXES
|
||||||
|
|
||||||
|
|
|
@ -74,6 +74,8 @@ import org.apache.htrace.core.TraceScope;
|
||||||
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
|
|
||||||
|
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.*;
|
||||||
|
|
||||||
/****************************************************************
|
/****************************************************************
|
||||||
* An abstract base class for a fairly generic filesystem. It
|
* An abstract base class for a fairly generic filesystem. It
|
||||||
* may be implemented as a distributed filesystem, or as a "local"
|
* may be implemented as a distributed filesystem, or as a "local"
|
||||||
|
@ -730,9 +732,9 @@ public abstract class FileSystem extends Configured implements Closeable {
|
||||||
conf.getInt("io.bytes.per.checksum", 512),
|
conf.getInt("io.bytes.per.checksum", 512),
|
||||||
64 * 1024,
|
64 * 1024,
|
||||||
getDefaultReplication(),
|
getDefaultReplication(),
|
||||||
conf.getInt("io.file.buffer.size", 4096),
|
conf.getInt(IO_FILE_BUFFER_SIZE_KEY, IO_FILE_BUFFER_SIZE_DEFAULT),
|
||||||
false,
|
false,
|
||||||
CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_DEFAULT,
|
FS_TRASH_INTERVAL_DEFAULT,
|
||||||
DataChecksum.Type.CRC32);
|
DataChecksum.Type.CRC32);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -772,7 +774,8 @@ public abstract class FileSystem extends Configured implements Closeable {
|
||||||
* @param f the file to open
|
* @param f the file to open
|
||||||
*/
|
*/
|
||||||
public FSDataInputStream open(Path f) throws IOException {
|
public FSDataInputStream open(Path f) throws IOException {
|
||||||
return open(f, getConf().getInt("io.file.buffer.size", 4096));
|
return open(f, getConf().getInt(IO_FILE_BUFFER_SIZE_KEY,
|
||||||
|
IO_FILE_BUFFER_SIZE_DEFAULT));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -793,7 +796,8 @@ public abstract class FileSystem extends Configured implements Closeable {
|
||||||
public FSDataOutputStream create(Path f, boolean overwrite)
|
public FSDataOutputStream create(Path f, boolean overwrite)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
return create(f, overwrite,
|
return create(f, overwrite,
|
||||||
getConf().getInt("io.file.buffer.size", 4096),
|
getConf().getInt(IO_FILE_BUFFER_SIZE_KEY,
|
||||||
|
IO_FILE_BUFFER_SIZE_DEFAULT),
|
||||||
getDefaultReplication(f),
|
getDefaultReplication(f),
|
||||||
getDefaultBlockSize(f));
|
getDefaultBlockSize(f));
|
||||||
}
|
}
|
||||||
|
@ -808,7 +812,8 @@ public abstract class FileSystem extends Configured implements Closeable {
|
||||||
public FSDataOutputStream create(Path f, Progressable progress)
|
public FSDataOutputStream create(Path f, Progressable progress)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
return create(f, true,
|
return create(f, true,
|
||||||
getConf().getInt("io.file.buffer.size", 4096),
|
getConf().getInt(IO_FILE_BUFFER_SIZE_KEY,
|
||||||
|
IO_FILE_BUFFER_SIZE_DEFAULT),
|
||||||
getDefaultReplication(f),
|
getDefaultReplication(f),
|
||||||
getDefaultBlockSize(f), progress);
|
getDefaultBlockSize(f), progress);
|
||||||
}
|
}
|
||||||
|
@ -822,7 +827,8 @@ public abstract class FileSystem extends Configured implements Closeable {
|
||||||
public FSDataOutputStream create(Path f, short replication)
|
public FSDataOutputStream create(Path f, short replication)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
return create(f, true,
|
return create(f, true,
|
||||||
getConf().getInt("io.file.buffer.size", 4096),
|
getConf().getInt(IO_FILE_BUFFER_SIZE_KEY,
|
||||||
|
IO_FILE_BUFFER_SIZE_DEFAULT),
|
||||||
replication,
|
replication,
|
||||||
getDefaultBlockSize(f));
|
getDefaultBlockSize(f));
|
||||||
}
|
}
|
||||||
|
@ -838,11 +844,9 @@ public abstract class FileSystem extends Configured implements Closeable {
|
||||||
public FSDataOutputStream create(Path f, short replication,
|
public FSDataOutputStream create(Path f, short replication,
|
||||||
Progressable progress) throws IOException {
|
Progressable progress) throws IOException {
|
||||||
return create(f, true,
|
return create(f, true,
|
||||||
getConf().getInt(
|
getConf().getInt(IO_FILE_BUFFER_SIZE_KEY,
|
||||||
CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY,
|
IO_FILE_BUFFER_SIZE_DEFAULT),
|
||||||
CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT),
|
replication, getDefaultBlockSize(f), progress);
|
||||||
replication,
|
|
||||||
getDefaultBlockSize(f), progress);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -1151,19 +1155,22 @@ public abstract class FileSystem extends Configured implements Closeable {
|
||||||
if (exists(f)) {
|
if (exists(f)) {
|
||||||
return false;
|
return false;
|
||||||
} else {
|
} else {
|
||||||
create(f, false, getConf().getInt("io.file.buffer.size", 4096)).close();
|
create(f, false, getConf().getInt(IO_FILE_BUFFER_SIZE_KEY,
|
||||||
|
IO_FILE_BUFFER_SIZE_DEFAULT)).close();
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Append to an existing file (optional operation).
|
* Append to an existing file (optional operation).
|
||||||
* Same as append(f, getConf().getInt("io.file.buffer.size", 4096), null)
|
* Same as append(f, getConf().getInt(IO_FILE_BUFFER_SIZE_KEY,
|
||||||
|
* IO_FILE_BUFFER_SIZE_DEFAULT), null)
|
||||||
* @param f the existing file to be appended.
|
* @param f the existing file to be appended.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public FSDataOutputStream append(Path f) throws IOException {
|
public FSDataOutputStream append(Path f) throws IOException {
|
||||||
return append(f, getConf().getInt("io.file.buffer.size", 4096), null);
|
return append(f, getConf().getInt(IO_FILE_BUFFER_SIZE_KEY,
|
||||||
|
IO_FILE_BUFFER_SIZE_DEFAULT), null);
|
||||||
}
|
}
|
||||||
/**
|
/**
|
||||||
* Append to an existing file (optional operation).
|
* Append to an existing file (optional operation).
|
||||||
|
|
Loading…
Reference in New Issue