HADOOP-6991. Fix SequenceFile::Reader to honor file lengths and call
openFile (cdouglas via omalley) git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1028473 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
7afa9466f7
commit
0c462b223f
|
@ -286,6 +286,9 @@ Trunk (unreleased changes)
|
|||
HADOOP-6663. BlockDecompressorStream get EOF exception when decompressing
|
||||
the file compressed from empty file. (Kang Xiao via tomwhite)
|
||||
|
||||
HADOOP-6991. Fix SequenceFile::Reader to honor file lengths and call
|
||||
openFile (cdouglas via omalley)
|
||||
|
||||
Release 0.21.1 - Unreleased
|
||||
|
||||
IMPROVEMENTS
|
||||
|
|
|
@ -951,7 +951,7 @@ public class SequenceFile {
|
|||
blockSizeOption.getValue();
|
||||
Progressable progress = progressOption == null ? null :
|
||||
progressOption.getValue();
|
||||
out = fs.create(p, false, bufferSize, replication, blockSize, progress);
|
||||
out = fs.create(p, true, bufferSize, replication, blockSize, progress);
|
||||
} else {
|
||||
out = streamOption.getValue();
|
||||
}
|
||||
|
@ -1563,14 +1563,17 @@ public class SequenceFile {
|
|||
// figure out the real values
|
||||
Path filename = null;
|
||||
FSDataInputStream file;
|
||||
long len = lenOpt == null ? Long.MAX_VALUE : lenOpt.getValue();
|
||||
final long len;
|
||||
if (fileOpt != null) {
|
||||
filename = fileOpt.getValue();
|
||||
FileSystem fs = filename.getFileSystem(conf);
|
||||
int bufSize = bufOpt == null ? getBufferSize(conf): bufOpt.getValue();
|
||||
file = fs.open(filename, bufSize);
|
||||
len = fs.getFileStatus(filename).getLen();
|
||||
len = null == lenOpt
|
||||
? fs.getFileStatus(filename).getLen()
|
||||
: lenOpt.getValue();
|
||||
file = openFile(fs, filename, bufSize, len);
|
||||
} else {
|
||||
len = null == lenOpt ? Long.MAX_VALUE : lenOpt.getValue();
|
||||
file = streamOpt.getValue();
|
||||
}
|
||||
long start = startOpt == null ? 0 : startOpt.getValue();
|
||||
|
@ -1589,9 +1592,7 @@ public class SequenceFile {
|
|||
@Deprecated
|
||||
public Reader(FileSystem fs, Path file,
|
||||
Configuration conf) throws IOException {
|
||||
initialize(file,
|
||||
fs.open(file, getBufferSize(conf)),
|
||||
0L, fs.getFileStatus(file).getLen(), conf, false);
|
||||
this(conf, file(file.makeQualified(fs)));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1607,7 +1608,7 @@ public class SequenceFile {
|
|||
@Deprecated
|
||||
public Reader(FSDataInputStream in, int buffersize,
|
||||
long start, long length, Configuration conf) throws IOException {
|
||||
initialize(null, in, start, length, conf, false);
|
||||
this(conf, stream(in), start(start), length(length));
|
||||
}
|
||||
|
||||
/** Common work of the constructors. */
|
||||
|
|
Loading…
Reference in New Issue