You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by "Johannes Herr (JIRA)" <ji...@apache.org> on 2016/06/10 10:31:21 UTC
[jira] [Commented] (HADOOP-7844) Flush/sync do not work on Hadoop
LocalFileSystem
[ https://issues.apache.org/jira/browse/HADOOP-7844?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15324202#comment-15324202 ]
Johannes Herr commented on HADOOP-7844:
---------------------------------------
This is still live. (Tested with Hadoop 2.5.1)
To reproduce:
{code:java}
public static void main(String[] args) throws IOException {
FileSystem fs = FileSystem.getLocal(new Configuration());
Path file = new Path("test.txt");
try (FSDataOutputStream stream = fs.create(file)) {
stream.write("foo\nbar\n".getBytes());
stream.hflush();
stream.hsync();
stream.flush();
new Thread() {
@Override
public void run() {
while (true) {
try {
byte[] bytes = Files.readAllBytes(Paths.get("test.txt"));
System.out.println("content = " + Arrays.toString(bytes));
Thread.sleep(1000);
} catch (InterruptedException | IOException e) {
e.printStackTrace();
}
}
}
}.start();
System.out.println("waiting");
System.in.read();
System.out.println("closing");
}
}
{code}
This write some bytes to a local file and tries to get it to disc. The file is closed when pressing return.
Until the file is closed the written bytes are not present. Output is:
{noformat}
waiting
content = []
content = []
content = []
content = []
closing
content = [102, 111, 111, 10, 98, 97, 114, 10]
content = [102, 111, 111, 10, 98, 97, 114, 10]
content = [102, 111, 111, 10, 98, 97, 114, 10]
content = [102, 111, 111, 10, 98, 97, 114, 10]
content = [102, 111, 111, 10, 98, 97, 114, 10]
content = [102, 111, 111, 10, 98, 97, 114, 10]
{noformat}
Using *.getRaw()* solves it.
HADOOP-8861 did not solve the issue because it just added a call to *flush()* if the _wrappedStream_ is not _Syncable_. However _ChecksumFSOutputSummer_ inherits an empty *flush()* and therefore does nothing.
> Flush/sync do not work on Hadoop LocalFileSystem
> ------------------------------------------------
>
> Key: HADOOP-7844
> URL: https://issues.apache.org/jira/browse/HADOOP-7844
> Project: Hadoop Common
> Issue Type: Bug
> Affects Versions: 0.20.1
> Reporter: Nathan Marz
>
> They seem to be no-ops. This is really easy to reproduce, just open a file using FileSystem/getLocal(new Configuration()), write data to the output stream, and then try to flush/sync. I also tried creating the output stream with a buffer size of 1, but that had no effect.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)
---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org