HADOOP-7817. RawLocalFileSystem.append() should give FSDataOutputStream with accurate .getPos() (Contributed by kanaka kumar avvaru)

This commit is contained in:
Vinayakumar B 2015-06-10 11:05:58 +05:30
parent 3107434031
commit 48ca23def1
3 changed files with 42 additions and 2 deletions

View File

@ -846,6 +846,9 @@ Release 2.8.0 - UNRELEASED
HADOOP-12073. Azure FileSystem PageBlobInputStream does not return -1 on HADOOP-12073. Azure FileSystem PageBlobInputStream does not return -1 on
EOF. (Ivan Mitic via cnauroth) 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 Release 2.7.1 - UNRELEASED
INCOMPATIBLE CHANGES INCOMPATIBLE CHANGES

View File

@ -271,11 +271,13 @@ public FSDataOutputStream append(Path f, int bufferSize,
if (!exists(f)) { if (!exists(f)) {
throw new FileNotFoundException("File " + f + " not found"); 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 + " )"); throw new IOException("Cannot append to a diretory (=" + f + " )");
} }
return new FSDataOutputStream(new BufferedOutputStream( return new FSDataOutputStream(new BufferedOutputStream(
createOutputStreamWithMode(f, true, null), bufferSize), statistics); createOutputStreamWithMode(f, true, null), bufferSize), statistics,
status.getLen());
} }
@Override @Override

View File

@ -561,4 +561,39 @@ public void testStripFragmentFromPath() throws Exception {
assertEquals("resolvePath did not strip fragment from Path", pathQualified, assertEquals("resolvePath did not strip fragment from Path", pathQualified,
resolved); 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();
}
}
} }