HADOOP-7817. RawLocalFileSystem.append() should give FSDataOutputStream with accurate .getPos() (Contributed by kanaka kumar avvaru)
This commit is contained in:
parent
3107434031
commit
48ca23def1
|
@ -846,6 +846,9 @@ Release 2.8.0 - UNRELEASED
|
|||
HADOOP-12073. Azure FileSystem PageBlobInputStream does not return -1 on
|
||||
EOF. (Ivan Mitic via cnauroth)
|
||||
|
||||
HADOOP-7817. RawLocalFileSystem.append() should give FSDataOutputStream
|
||||
with accurate .getPos() (kanaka kumar avvaru via vinayakumarb)
|
||||
|
||||
Release 2.7.1 - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
|
|
@ -271,11 +271,13 @@ public class RawLocalFileSystem extends FileSystem {
|
|||
if (!exists(f)) {
|
||||
throw new FileNotFoundException("File " + f + " not found");
|
||||
}
|
||||
if (getFileStatus(f).isDirectory()) {
|
||||
FileStatus status = getFileStatus(f);
|
||||
if (status.isDirectory()) {
|
||||
throw new IOException("Cannot append to a diretory (=" + f + " )");
|
||||
}
|
||||
return new FSDataOutputStream(new BufferedOutputStream(
|
||||
createOutputStreamWithMode(f, true, null), bufferSize), statistics);
|
||||
createOutputStreamWithMode(f, true, null), bufferSize), statistics,
|
||||
status.getLen());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -561,4 +561,39 @@ public class TestLocalFileSystem {
|
|||
assertEquals("resolvePath did not strip fragment from Path", pathQualified,
|
||||
resolved);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAppendSetsPosCorrectly() throws Exception {
|
||||
FileSystem fs = fileSys.getRawFileSystem();
|
||||
Path file = new Path(TEST_ROOT_DIR, "test-append");
|
||||
|
||||
fs.delete(file, true);
|
||||
FSDataOutputStream out = fs.create(file);
|
||||
|
||||
try {
|
||||
out.write("text1".getBytes());
|
||||
} finally {
|
||||
out.close();
|
||||
}
|
||||
|
||||
// Verify the position
|
||||
out = fs.append(file);
|
||||
try {
|
||||
assertEquals(5, out.getPos());
|
||||
out.write("text2".getBytes());
|
||||
} finally {
|
||||
out.close();
|
||||
}
|
||||
|
||||
// Verify the content
|
||||
FSDataInputStream in = fs.open(file);
|
||||
try {
|
||||
byte[] buf = new byte[in.available()];
|
||||
in.readFully(buf);
|
||||
assertEquals("text1text2", new String(buf));
|
||||
} finally {
|
||||
in.close();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue