You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-dev@hadoop.apache.org by "Tsz Wo (Nicholas), SZE (JIRA)" <ji...@apache.org> on 2008/10/09 03:03:44 UTC

[jira] Created: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

In HDFS, sync() not yet guarantees data available to the new readers
--------------------------------------------------------------------

                 Key: HADOOP-4379
                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
             Project: Hadoop Core
          Issue Type: New Feature
          Components: dfs
            Reporter: Tsz Wo (Nicholas), SZE


In the append design doc in https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc, it says
* A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file

However, this feature is not yet implemented.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

dhruba borthakur updated HADOOP-4379:
-------------------------------------

    Attachment: fsyncConcurrentReaders5.txt

Merged patch with latest trunk.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Philip Zeyliger (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12721422#action_12721422 ] 

Philip Zeyliger commented on HADOOP-4379:
-----------------------------------------

I'm traveling overseas and will return on June 29th.  For urgent
matters, please contact Amr Awadallah.


> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders11_20.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hadoop-stack-namenode-aa0-000-12.u.powerset.com.log.gz, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Doug Judd (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12667462#action_12667462 ] 

Doug Judd commented on HADOOP-4379:
-----------------------------------

I tried the test again and still no luck.  To recap, here's how the log file is created:

out_stream.write(header, 0, 7);
out_stream.sync()
out_stream.write(data, 0, amount);
out_stream.sync()
[...]

After the test finished, I shut down the Hypertable servers.  This time the listing shows the files to be 0 bytes in length (as opposed to 7 bytes with the previous patch):

[doug@motherlode000 aol-basic]$ hadoop fs -ls /hypertable/servers/10.0.30.1*_38060/log/range_txn
-rw-r--r--   3 doug supergroup          0 2009-01-26 11:40 /hypertable/servers/10.0.30.102_38060/log/range_txn/0.log
-rw-r--r--   3 doug supergroup          0 2009-01-26 11:40 /hypertable/servers/10.0.30.104_38060/log/range_txn/0.log
-rw-r--r--   3 doug supergroup          0 2009-01-26 11:40 /hypertable/servers/10.0.30.106_38060/log/range_txn/0.log
-rw-r--r--   3 doug supergroup          0 2009-01-26 11:40 /hypertable/servers/10.0.30.108_38060/log/range_txn/0.log
-rw-r--r--   3 doug supergroup          0 2009-01-26 11:40 /hypertable/servers/10.0.30.110_38060/log/range_txn/0.log
-rw-r--r--   3 doug supergroup          0 2009-01-26 11:40 /hypertable/servers/10.0.30.112_38060/log/range_txn/0.log
-rw-r--r--   3 doug supergroup          0 2009-01-26 11:40 /hypertable/servers/10.0.30.114_38060/log/range_txn/0.log
-rw-r--r--   3 doug supergroup          0 2009-01-26 11:40 /hypertable/servers/10.0.30.116_38060/log/range_txn/0.log

When the RangeServer starts up again, it discovers that the log file (range_txn/0.log) does exist, so it starts the recovery process.  However, it only sees the 7 byte header.  All of the subsequent log appends do not appear in the log file.  So the system starts up without recovering any of the data.

BTW, in this particular circumstance, there no other writer writing to the file when the range server comes up and reads it.  Here's the high-level of what's going on:

RangeServer opens an FSDataOutputStream to the log and starts appending to it
RangeServer is killed with 'kill -9"
RangeServer comes up again and reads the log

In your above note you said, "A reader checks to see if the file is being written to by another writer. if so, it fetches the size of the last block from the primary datanode."  This is not the case with our test, there is no writer writing to the log when we try to read it.

- Doug


> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, Reader.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "stack (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12708275#action_12708275 ] 

stack commented on HADOOP-4379:
-------------------------------

Tried the patch on loaded cluster.  First attempt got file lease after about 30 seconds or so.  Second test fell into a loop:

{code}
2009-05-12 00:51:16,031 [HMaster] INFO org.apache.hadoop.hdfs.DFSClient: Could not complete file /hbasetrunk2/.logs/aa0-000-14.u.powerset.com_1242088970041_60021/hlog.dat.1242089388713 retrying...
2009-05-12 00:51:16,441 [HMaster] INFO org.apache.hadoop.hdfs.DFSClient: Could not complete file /hbasetrunk2/.logs/aa0-000-14.u.powerset.com_1242088970041_60021/hlog.dat.1242089388713 retrying...
...
{code}

Sending Dhruba hdfs logs.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

dhruba borthakur updated HADOOP-4379:
-------------------------------------

    Attachment: fsyncConcurrentReaders6.patch

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "stack (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12710549#action_12710549 ] 

stack commented on HADOOP-4379:
-------------------------------

My testing was done with hadoop 0.20.0 (I amended patch to work with 0.20 branch). 

Patch itself looks good.  Minor comments are

+ Is there anything that can be done about duplicated test condition in BlockManager.java and DatanodeDescriptor.java.  Can the test be grouped up into a method that is used in both places?
+ The if part of the if/else in DatanodeDescriptor.java is empty.  Put a comment here?  As is, its hard to read.

I haven't played much with killing writer application and the local datanode.  Will try that now.  Will also try confirm that we're reading up to the last sync before the kill.  Will report back.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Jim Kellerman (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12680272#action_12680272 ] 

Jim Kellerman commented on HADOOP-4379:
---------------------------------------

Also, were you just testing Reader and Writer on an unloaded cluster, or using some other application 
against a cluster that was under load?


> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2, 0.20.0
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

dhruba borthakur updated HADOOP-4379:
-------------------------------------

    Attachment: Writer.java

An updated version of the writer than invokes SequenceFile.Writer.syncFs().

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, Reader.java, Reader.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Tsz Wo (Nicholas), SZE (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Tsz Wo (Nicholas), SZE updated HADOOP-4379:
-------------------------------------------

    Description: 
In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
* A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file

However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

  was:
In the append design doc in https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc, it says
* A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file

However, this feature is not yet implemented.


> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Luke Lu (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12664982#action_12664982 ] 

Luke Lu commented on HADOOP-4379:
---------------------------------

@dhruba: unfortunately the patch (applied to the stock 0.19.0) didn't work. The files did show non-zero size after the sync but they're all 7 bytes. Restarting HDFS make the correct content show up.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12677151#action_12677151 ] 

dhruba borthakur commented on HADOOP-4379:
------------------------------------------

What is the value of leaseRecoveryPeriod in the above piece of code?

The fact that the lease recovery took 1 hour is very interesting. The namenode has two limits: the soft limit of 1 minute and the hard limit of 1 hour. When there is no ping from a writer for the soft limit period, then a new writer can recover the lease and append to the file. When there is no ping from a writer for the hard limit period, the namenode itself itself initiates lease recovery.

Is it possible for you to give me access to the namenode log along with the relevant filename in question? 

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2, 0.20.0
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Jim Kellerman (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12678401#action_12678401 ] 

Jim Kellerman commented on HADOOP-4379:
---------------------------------------

We too would like to be able to recover the lease immediately as we know when we go to read the file that the
writer is dead.

Is there anything that we can do to speed this up? We can't contribute to Hadoop, but we are certainly willing to
test and stress test any patch that is available.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2, 0.20.0
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Doug Judd (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12667904#action_12667904 ] 

Doug Judd commented on HADOOP-4379:
-----------------------------------

Hi Dhruba,

When I drop the patch for 5027, the NullPointerExceptions go away.  However, I'm still having troubles.  It appears for some of the files, your trick for obtaining the real length does not work.  The waitForLease spins indefinitely.  For example, here's the log line for our HdfsBroker process where it is hanging:

Jan 27, 2009 5:08:53 PM org.hypertable.DfsBroker.hadoop.HdfsBroker Length
INFO: Getting length of file '/hypertable/servers/10.0.30.106_38060/log/metadata/0

Then in the namenode log file, I see things like this (I've attached the compressed namenode log):

2009-01-27 17:08:53,173 WARN org.apache.hadoop.hdfs.StateChange: DIR* NameSystem.startFile: failed to create file /hypertable/servers/10.0.30.106_38060/log/metadat
a/0 for DFSClient_-1860021252 on client 10.0.30.106 because current leaseholder is trying to recreate file.
2009-01-27 17:08:53,174 INFO org.apache.hadoop.ipc.Server: IPC Server handler 4 on 9000, call append(/hypertable/servers/10.0.30.106_38060/log/metadata/0, DFSClien
t_-1860021252) from 10.0.30.106:43507: error: org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException: failed to create file /hypertable/servers/10.0.30.106_3
8060/log/metadata/0 for DFSClient_-1860021252 on client 10.0.30.106 because current leaseholder is trying to recreate file.
org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException: failed to create file /hypertable/servers/10.0.30.106_38060/log/metadata/0 for DFSClient_-1860021252 
on client 10.0.30.106 because current leaseholder is trying to recreate file.
	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startFileInternal(FSNamesystem.java:1065)
	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.appendFile(FSNamesystem.java:1177)
	at org.apache.hadoop.hdfs.server.namenode.NameNode.append(NameNode.java:321)
	at sun.reflect.GeneratedMethodAccessor14.invoke(Unknown Source)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
	at java.lang.reflect.Method.invoke(Method.java:597)
	at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
	at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)




> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, Reader.java, Reader.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Luke Lu (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12664752#action_12664752 ] 

Luke Lu commented on HADOOP-4379:
---------------------------------

@dhruba: this is the current (0.19.0) behavior: if a client writes a half block to a log in hdfs and calls fsync and then got killed. When the client restarts (on the same node) to read the log, the half block is not there, until you restart the entire hadoop cluster and wait for a few minutes.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Jim Kellerman (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12667152#action_12667152 ] 

Jim Kellerman commented on HADOOP-4379:
---------------------------------------

@Dhruba

I tried applying fsyncConcurrentReaders3.patch to 0.19.0, 0.19 branch (latest), 0.20 branch (latest) and trunk. It did not
apply cleanly to any, but seemed to have less problems with trunk. Building it, and running the Writer and Reader
programs I attached above, they behaved the same way, i.e., the Reader could not see any sync'd data until the first
complete block had been written, and then it could only see the data in the first complete block and none of the data
in the second partial block.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, Reader.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Doug Judd (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12667602#action_12667602 ] 

Doug Judd commented on HADOOP-4379:
-----------------------------------

Hi Dhruba,

I tried your suggestion, but got the following exception when trying to open the file with the 'append' method:

SEVERE: I/O exception while getting length of file '/hypertable/servers/10.0.30.102_38060/log/range_txn/0.log' - org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException: failed to create file /hypertable/servers/10.0.30.102_38060/log/range_txn/0.log for DFSClient_2003773208 on client 10.0.30.102, because this file is already being created by DFSClient_423127459 on 10.0.30.102
	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startFileInternal(FSNamesystem.java:1088)
	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.appendFile(FSNamesystem.java:1177)
	at org.apache.hadoop.hdfs.server.namenode.NameNode.append(NameNode.java:321)
	at sun.reflect.GeneratedMethodAccessor50.invoke(Unknown Source)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
	at java.lang.reflect.Method.invoke(Method.java:597)
	at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
	at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)

I could re-write the whole thing to not be dependent on knowing log length, however, it seems like it ought to be possible to obtain the actual file length in this situation.  The semantics of getFileStatus() seem a little odd.  Sometimes it returns the actual length of the file and sometimes it returns a stale version of the length.  I suppose this is ok as long as it is well documented.  But it should be possible to obtain the actual length of a file.  Would it be possible to add a FileSystem::length(Path path) method that returns the accurate file length by fetching the size of the last block from the primary datanode?

- Doug


> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, Reader.java, Reader.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Doug Judd (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12668186#action_12668186 ] 

Doug Judd commented on HADOOP-4379:
-----------------------------------

Ok, I've uploaded a reproducible test case (see attachments ReopenProblem.java and reopen_test.sh).  This test consistently hangs on both my local mac HDFS installation as well as our 10 node Linux HDFS cluster.  It's not as simple as I suggested in my previous comment as you can see by the code.  Here's the output I see when I run it:

[doug@motherlode000 0.9.2.1]$ ./reopen_test.sh 
Deleted hdfs://motherlode000:9000/hypertable/servers
rmr: cannot remove /hypertable/tables: No such file or directory.
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
length("/hypertable/servers/10.0.30.114_38060/log/root/0") = 163, obtained in 65258 milliseconds
length("/hypertable/servers/10.0.30.114_38060/log/root/0") = 163, obtained in 113 milliseconds
Read 163 bytes from root fragment 0
Read 0 bytes from root fragment 0
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
Waiting for lease recovery ...
[...]


> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Doug Judd (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12677494#action_12677494 ] 

Doug Judd commented on HADOOP-4379:
-----------------------------------

We would like to see the soft lease limit configurable and settable to 0.  Our application ensures that there is only one writer to each file.  We need to be able to recover immediately.


> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2, 0.20.0
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Jim Kellerman (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12676816#action_12676816 ] 

Jim Kellerman commented on HADOOP-4379:
---------------------------------------

Applying this patch to hadoop-0.19.0, it does work, but slowly:

It takes almost an hour to recover the lease of a file when the hdfs cluster is under load:

{code}
2009-02-25 21:39:16,843 DEBUG org.apache.hadoop.hbase.regionserver.HLog: Splitting 3 of 3: hdfs://aa0-000-12.u.powerset.com:8100/hbase/log_208.76.44.139_1235597506284_8020/hlog.dat.1235597820662
2009-02-25 21:39:16,847 DEBUG org.apache.hadoop.hbase.regionserver.HLog: Triggering lease recovery.
...
2009-02-25 22:37:12,755 INFO org.apache.hadoop.hbase.regionserver.HLog: log file splitting completed for hdfs://aa0-000-12.u.powerset.com:8100/hbase/log_208.76.44.139_1235597506284_8020
{code}


> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2, 0.20.0
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Doug Judd (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Doug Judd updated HADOOP-4379:
------------------------------

    Attachment: reopen_test.sh

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Doug Judd (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Doug Judd updated HADOOP-4379:
------------------------------

    Attachment: ReopenProblem.java

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12667356#action_12667356 ] 

Doug Cutting commented on HADOOP-4379:
--------------------------------------

> how do I get the FSDataOutputStream? 

I think it would be better to add a SequenceFile.Writer method that flushes and calls FSDataOutputStream.sync.  The problem is that SequenceFile.Writer#sync() should probably be called flush(), not sync(), but that'd be hard to change.  So perhaps we should add a SequenceFile.Writer#syncFs() method?


> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, Reader.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Jim Kellerman (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Jim Kellerman updated HADOOP-4379:
----------------------------------

    Attachment: namenode.log

Namenode log. Note that this run was done using the waitForLeaseRecovery() from ReopenProblem.java

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2, 0.20.0
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12667535#action_12667535 ] 

dhruba borthakur commented on HADOOP-4379:
------------------------------------------

Hi Doug,

The file length (as returned by getFileStatus) will not change at every write from the client to the datanode. Similarly, every fsync call from the client does not reach the namenode (only the first one per block reaches the namenode). That means the namenode has no good way to know the size of a block when the block is being written to by a writer.

In your case, the writer has died. The namenode has a timeout of 1 hour before it starts lease recovery for this file. The lease recovery process will set the correct file size on the namenode metadata. If you do not want to wait for one hour, then you can manually trigger lease recovery from your application by trying to reopen the file for append(please use FileSystem.append(pathname)). Lease recovery will update the true length of the file in the namenode metadata.



> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, Reader.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

dhruba borthakur updated HADOOP-4379:
-------------------------------------

    Priority: Blocker  (was: Major)

The Hbase folks have requested that this be fixed in the next release form the 0.19 branch.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Tsz Wo (Nicholas), SZE (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Tsz Wo (Nicholas), SZE updated HADOOP-4379:
-------------------------------------------

    Attachment: 4379_20081010TC3.java

4379_20081010TC3.java: a unit test implementing TC3 described in [my previous comment|https://issues.apache.org/jira/browse/HADOOP-4379?focusedCommentId=12638165#action_12638165].

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>         Attachments: 4379_20081010TC3.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12667952#action_12667952 ] 

dhruba borthakur commented on HADOOP-4379:
------------------------------------------

@Doug: the waiting for lease recovery to reclaim the length of the jar file will work only if the original writer has dies and now a new reader wants to read all the data that was written by the writer before it died. This was the use-case for Jim Kellerman.

In your case, you have a FileSystem object that was used to write to the file. Then your application tried to reopen the same file by invoking fs.apend() on the same FileSystem object. The namenode logs clearly shows that the same dfsclient (i.e. filesystem object) was used to try appending to the file. In this case, the namenode rejects the append call because it knows that the original writer is alive and is still writing to it. If your aim is to have a concurrent reader along with a concurrent writer, then the best that this patch can do is to allow the concurent reader to see the new file length only when the block is full. On the other hand, if you can make your application ot depend on the file length, then you can see all the data in the file. Another alternative would be to implement a new call FileSystem.length() than can retrieve the latest length from the datanode, but that can be done as part of a separate JIRA. Please let us know how difficult is it for you to change your app to not depend on the file length and just read till end-of-file?

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, Reader.java, Reader.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

dhruba borthakur updated HADOOP-4379:
-------------------------------------

    Attachment: fsyncConcurrentReaders4.patch

This patch should merge smoothly with trunk.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, Reader.java, Reader.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Jim Kellerman (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12677085#action_12677085 ] 

Jim Kellerman commented on HADOOP-4379:
---------------------------------------

Is there something we can change to make lease recovery faster?
config parameter?
block size?
max file size?

Thanks.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2, 0.20.0
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12678469#action_12678469 ] 

dhruba borthakur commented on HADOOP-4379:
------------------------------------------

Hi Jim,

In the namenode log, I see the followign statement:

NameSystem.startFile: failed to create file /hbase/log_208.76.44.139_1235681989771_8020/hlog.dat.1235682444379 for DFSClient_1908447348 on client 208.76.44.139 because current leaseholder is trying to recreate file.

This means that the same client that originally created the file is trying to re-open the file. Is this possible? It started at time 21:14 and continued all the way to 22:07. An attempt was made to recreate thias file every 10 seconds (which matches the periodicity that you set)

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2, 0.20.0
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Jim Kellerman (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12665257#action_12665257 ] 

Jim Kellerman commented on HADOOP-4379:
---------------------------------------

HBase really needs visibility to partial blocks and restarting HDFS is not an option for a 100+ node cluster.

We don't sync after every record, but sync after a configurable number of writes or after a configurable amount of time has passed and 
there is unsync'd data.

This really needs to be addressed in 0.19.1, 0.20.1 and trunk.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Issue Comment Edited: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Jim Kellerman (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12676816#action_12676816 ] 

jimk edited comment on HADOOP-4379 at 2/25/09 3:33 PM:
----------------------------------------------------------------

Applying this patch to hadoop-0.19.0, it does work, but slowly:

It takes almost an hour to recover the lease of a file when the hdfs cluster is under load:

{code}
2009-02-25 21:39:16,843 DEBUG org.apache.hadoop.hbase.regionserver.HLog: Splitting 3 of 3: hdfs:/x.y.com:8100/hbase/log_10.76.44.139_1235597506284_8020/hlog.dat.1235597820662
2009-02-25 21:39:16,847 DEBUG org.apache.hadoop.hbase.regionserver.HLog: Triggering lease recovery.
...
2009-02-25 22:37:12,755 INFO org.apache.hadoop.hbase.regionserver.HLog: log file splitting completed for hdfs://x.y.com:8100/hbase/log_10.76.44.139_1235597506284_8020
{code}


      was (Author: jimk):
    Applying this patch to hadoop-0.19.0, it does work, but slowly:

It takes almost an hour to recover the lease of a file when the hdfs cluster is under load:

{code}
2009-02-25 21:39:16,843 DEBUG org.apache.hadoop.hbase.regionserver.HLog: Splitting 3 of 3: hdfs://aa0-000-12.u.powerset.com:8100/hbase/log_208.76.44.139_1235597506284_8020/hlog.dat.1235597820662
2009-02-25 21:39:16,847 DEBUG org.apache.hadoop.hbase.regionserver.HLog: Triggering lease recovery.
...
2009-02-25 22:37:12,755 INFO org.apache.hadoop.hbase.regionserver.HLog: log file splitting completed for hdfs://aa0-000-12.u.powerset.com:8100/hbase/log_208.76.44.139_1235597506284_8020
{code}

  
> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2, 0.20.0
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12677118#action_12677118 ] 

dhruba borthakur commented on HADOOP-4379:
------------------------------------------

There is ReopenProblem.java attched to this JIRA.

@Doug: Please use ReopenProblem.java:waitForLeaseRecovery(). This method triggers near-immediate lease recovery and can be used by any application code to take ownership of the file. Does this work for you?

@Jim: Were u using ReopenProblem.java:waitForLeaseRecovery() and still seeing that it takes upto an hour for the waitForLeaseRecovery() method to return success? This can happen if the original client that was writing to the file was alive and was in communication with the namenode. In this case, the new append-er will not be able to recover the original lease.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2, 0.20.0
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12665366#action_12665366 ] 

dhruba borthakur commented on HADOOP-4379:
------------------------------------------

Thanks Doug and Luke for the comments and Jim for providing a good test case. I will incorporate your comments and post a new patch.  (Jim, you need not test the existing patch associated with this JIRA)

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, Reader.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Tsz Wo (Nicholas), SZE (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12667909#action_12667909 ] 

Tsz Wo (Nicholas), SZE commented on HADOOP-4379:
------------------------------------------------

> I think these exceptions are related to the patch in HADOOP-5027.

Please do not use my patch in HADOOP-5027.  It is not clear that whether the fix is good.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, Reader.java, Reader.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12670292#action_12670292 ] 

dhruba borthakur commented on HADOOP-4379:
------------------------------------------

Hi Doug,

ReopenProblem.java is guaranteed to hang. At line 189, you create an file and get an handle on this file. Then , at line 197, you use the *same* file system object and invoke waitForLeaseRecovery(). waitForLeaseRecovery tries to open the same file again for appending to it. It fails as expected.  The namenode log correctly shows that the "same leaseholder is trying to reopen the file for appends"!

does it make sense? Thanks, dhruba

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Doug Judd (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12667881#action_12667881 ] 

Doug Judd commented on HADOOP-4379:
-----------------------------------

Now when I apply your latest patch and the one from 5027 to the 0.19.0 source, the datanodes seem to be going into an infinite loop of NullPointerExceptions.  At the top of the hadoop-zvents-datanode-motherlode007.admin.zvents.com.log I'm seeing this:

[doug@motherlode007 logs]$ more hadoop-zvents-datanode-motherlode007.admin.zvents.com.log 
2009-01-27 15:32:55,828 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: STARTUP_MSG: 
/************************************************************
STARTUP_MSG: Starting DataNode
STARTUP_MSG:   host = motherlode007.admin.zvents.com/10.0.30.114
STARTUP_MSG:   args = []
STARTUP_MSG:   version = 0.19.1-dev
STARTUP_MSG:   build =  -r ; compiled by 'doug' on Tue Jan 27 15:04:06 PST 2009
************************************************************/
2009-01-27 15:32:57,041 INFO org.apache.hadoop.ipc.Client: Retrying connect to server: motherlode000/10.0.30.100:9000. Already tried 0 time(s).
2009-01-27 15:33:00,505 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: Registered FSDatasetStatusMBean
2009-01-27 15:33:00,507 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: Opened info server at 50010
2009-01-27 15:33:00,510 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: Balancing bandwith is 1048576 bytes/s
2009-01-27 15:33:00,783 INFO org.mortbay.http.HttpServer: Version Jetty/5.1.4
2009-01-27 15:33:00,792 INFO org.mortbay.util.Credential: Checking Resource aliases
2009-01-27 15:33:01,839 INFO org.mortbay.util.Container: Started org.mortbay.jetty.servlet.WebApplicationHandler@319c0bd6
2009-01-27 15:33:01,878 INFO org.mortbay.util.Container: Started WebApplicationContext[/static,/static]
2009-01-27 15:33:02,048 INFO org.mortbay.util.Container: Started org.mortbay.jetty.servlet.WebApplicationHandler@5a943dc4
2009-01-27 15:33:02,049 INFO org.mortbay.util.Container: Started WebApplicationContext[/logs,/logs]
2009-01-27 15:33:02,754 INFO org.mortbay.util.Container: Started org.mortbay.jetty.servlet.WebApplicationHandler@6d581e80
2009-01-27 15:33:02,760 INFO org.mortbay.util.Container: Started WebApplicationContext[/,/]
2009-01-27 15:33:02,763 INFO org.mortbay.http.SocketListener: Started SocketListener on 0.0.0.0:50075
2009-01-27 15:33:02,764 INFO org.mortbay.util.Container: Started org.mortbay.jetty.Server@5c435a3a
2009-01-27 15:33:02,769 INFO org.apache.hadoop.metrics.jvm.JvmMetrics: Initializing JVM Metrics with processName=DataNode, sessionId=null
2009-01-27 15:33:02,825 INFO org.apache.hadoop.ipc.metrics.RpcMetrics: Initializing RPC Metrics with hostName=DataNode, port=50020
2009-01-27 15:33:02,831 INFO org.apache.hadoop.ipc.Server: IPC Server Responder: starting
2009-01-27 15:33:02,834 INFO org.apache.hadoop.ipc.Server: IPC Server handler 0 on 50020: starting
2009-01-27 15:33:02,834 INFO org.apache.hadoop.ipc.Server: IPC Server listener on 50020: starting
2009-01-27 15:33:02,836 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: dnRegistration = DatanodeRegistration(motherlode007.admin.zvents.com:50010, storageID
=DS-745224472-10.0.30.114-50010-1230665635246, infoPort=50075, ipcPort=50020)
2009-01-27 15:33:02,837 INFO org.apache.hadoop.ipc.Server: IPC Server handler 1 on 50020: starting
2009-01-27 15:33:02,837 INFO org.apache.hadoop.ipc.Server: IPC Server handler 2 on 50020: starting
2009-01-27 15:33:02,839 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: DatanodeRegistration(10.0.30.114:50010, storageID=DS-745224472-10.0.30.114-50010-1230
665635246, infoPort=50075, ipcPort=50020)In DataNode.run, data = FSDataset{dirpath='/data1/hadoop/dfs/data/current,/data2/hadoop/dfs/data/current,/data3/hadoop/dfs
/data/current'}
2009-01-27 15:33:02,840 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: using BLOCKREPORT_INTERVAL of 3600000msec Initial delay: 0msec
2009-01-27 15:33:02,932 WARN org.apache.hadoop.hdfs.server.datanode.DataNode: org.apache.hadoop.ipc.RemoteException: java.io.IOException: java.lang.NullPointerException
	at org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor.reportDiff(DatanodeDescriptor.java:396)
	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.processReport(FSNamesystem.java:2803)
	at org.apache.hadoop.hdfs.server.namenode.NameNode.blockReport(NameNode.java:636)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
	at java.lang.reflect.Method.invoke(Method.java:597)
	at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
	at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)

	at org.apache.hadoop.ipc.Client.call(Client.java:696)
	at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:216)
	at $Proxy4.blockReport(Unknown Source)
	at org.apache.hadoop.hdfs.server.datanode.DataNode.offerService(DataNode.java:723)
	at org.apache.hadoop.hdfs.server.datanode.DataNode.run(DataNode.java:1100)
	at java.lang.Thread.run(Thread.java:619)

2009-01-27 15:33:02,953 WARN org.apache.hadoop.hdfs.server.datanode.DataNode: org.apache.hadoop.ipc.RemoteException: java.io.IOException: java.lang.NullPointerExce
ption
	at org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor.reportDiff(DatanodeDescriptor.java:396)
[...]


And the file is growing rapidly with the following exceptions tacked on to the end:

2009-01-27 16:10:55,973 WARN org.apache.hadoop.hdfs.server.datanode.DataNode: org.apache.hadoop.ipc.RemoteException: java.io.IOException: java.lang.NullPointerException

	at org.apache.hadoop.ipc.Client.call(Client.java:696)
	at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:216)
	at $Proxy4.blockReport(Unknown Source)
	at org.apache.hadoop.hdfs.server.datanode.DataNode.offerService(DataNode.java:723)
	at org.apache.hadoop.hdfs.server.datanode.DataNode.run(DataNode.java:1100)
	at java.lang.Thread.run(Thread.java:619)


It appears that these exceptions started happening within about 5 seconds after startup, so it doesn't look like it has anything to do with Hypertable.  Is it ok to apply these patches to the 0.19.0 source?  Or should I be applying them to the trunk?

- Doug


> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, Reader.java, Reader.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Jim Kellerman (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12678542#action_12678542 ] 

Jim Kellerman commented on HADOOP-4379:
---------------------------------------

No, the original writer was killed at 21:08 and never restarted.

Lease recovery started at 21:14 by another process on the same machine.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2, 0.20.0
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

dhruba borthakur updated HADOOP-4379:
-------------------------------------

    Attachment: Reader.java

This read waits for lease recovery to complete before reading data.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, Reader.java, Reader.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12701554#action_12701554 ] 

dhruba borthakur commented on HADOOP-4379:
------------------------------------------

Hi Stack, I will post a new merged patch for this one. I am hoping that I get some file system folks to review it once I post the patch.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2, 0.20.0
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12678609#action_12678609 ] 

dhruba borthakur commented on HADOOP-4379:
------------------------------------------

Hi Jim, I think the namenode log that you posted is a partial one. It does not have calles to commitBlockSync (that should have occured when the new writer started to append to the file). if so, can you pl post the entire namenode log?

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2, 0.20.0
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12667212#action_12667212 ] 

dhruba borthakur commented on HADOOP-4379:
------------------------------------------

@Jim: The SequenceFile.Writer.sync() does not actually invoke the FSDataOutputStream.sync. 

The FSDataOutputStream.sync() is a heavy weight operation and the SequenceFile.Writer.sync()  is typically invoked with small chunks of data, that's the reason why the SequenceFile class does not automatically invoke the fs sync call.

I changed your test case to invoke the FSDaOutputStream.sync call but still see some problems. Will post an update as soon as I get to the bottom of this.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, Reader.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Doug Judd (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Doug Judd updated HADOOP-4379:
------------------------------

    Attachment: hypertable-namenode.log.gz

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, Reader.java, Reader.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Jim Kellerman (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12665309#action_12665309 ] 

Jim Kellerman commented on HADOOP-4379:
---------------------------------------

I have attached two test programs that demonstrate the problem.

Writer.java writes 100 records of approx 1KB each per sync. Its command line argument is the number of sync's to perform before aborting.
Reader.java is run after Writer and tries to read records from the file created by Writer.

If you run Writer with 4096 syncs:

bin/hadoop Writer 4096

and then run Reader, it gets an EOFException without having read any records (note that exception occurs in constructor:
{code}
Exception in thread "main" java.io.EOFException
        at java.io.DataInputStream.readFully(Unknown Source)
        at java.io.DataInputStream.readFully(Unknown Source)
        at org.apache.hadoop.io.SequenceFile$Reader.init(SequenceFile.java:1450)
        at org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile.java:1428)
        at org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile.java:1417)
        at org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile.java:1412)
        at Reader.main(Reader.java:34)
{code}

If you run Writer with 8192 syncs:

bin/hadoop Writer 8192

and then run Reader, it reads 684783 records before hitting an EOFException (in next())
{code}
Read 684783 lines
java.io.EOFException
        at java.io.DataInputStream.readFully(Unknown Source)
        at org.apache.hadoop.io.DataOutputBuffer$Buffer.write(DataOutputBuffer.java:63)
        at org.apache.hadoop.io.DataOutputBuffer.write(DataOutputBuffer.java:101)
        at org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:1930)
        at org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:1830)
        at org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:1876)
        at Reader.main(Reader.java:41)
{code}
684783 records that are approximately 1KB in length is approximately 64MB (a hadoop block).

In this test the partial block containing 134417 records is lost (even though a considerable 
number of syncs took place between filling the first block and the end of Writer).

So you can read from an improperly closed file but only up to the last full block, confirming
Luke's observation.

I will look into the patch shortly.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, Reader.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Doug Judd (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12677097#action_12677097 ] 

Doug Judd commented on HADOOP-4379:
-----------------------------------

There should be a way to forcibly take over a lease.  In our application, ownership of the various files is controlled at a higher level (i.e. Chubby).  A server will not come up and attempt to recover a file (or check its length) unless it is the sole owner of the file (i.e. it will have obtained a Chubby lock).

Can we add some sort of API that allows a client to forcibly take over a lease immediately?



> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2, 0.20.0
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Jim Kellerman (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12677159#action_12677159 ] 

Jim Kellerman commented on HADOOP-4379:
---------------------------------------

leaseRecoveryPeriod is 10000ms

file name is  hdfs://aa0-000-12.u.powerset.com:8100/hbase/log_208.76.44.139_1235681989771_8020/hlog.dat.1235682444379

I will attach the namenode log.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2, 0.20.0
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Doug Judd (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12664986#action_12664986 ] 

Doug Judd commented on HADOOP-4379:
-----------------------------------

Hi Dhruba,

I've been working with Luke a little on this.  Here are more details.  The log that gets written in the test is very small.  The first thing the software does when it creates the log is it writes a 7-byte header.  Then later as the test proceeds, the system will append a small entry and the do a sync.  We use the FSDataOutputStream class.  The sequence of operations looks something like this:

out_stream.write(data, 0, amount);
out_stream.sync()
[...]

When the test completes, all of the logs are exactly 7 bytes long.  It remains this way even if I wait 10 minutes or kill the Hypertable java process and wait several minutes as well.  Here is the listing:

[doug@motherlode000 aol-basic]$ hadoop fs -ls /hypertable/servers/10.0.30.1*_38060/log/range_txn
-rw-r--r--   3 doug supergroup          7 2009-01-17 19:52 /hypertable/servers/10.0.30.102_38060/log/range_txn/0.log
-rw-r--r--   3 doug supergroup          7 2009-01-17 19:52 /hypertable/servers/10.0.30.104_38060/log/range_txn/0.log
-rw-r--r--   3 doug supergroup          7 2009-01-17 19:52 /hypertable/servers/10.0.30.106_38060/log/range_txn/0.log
-rw-r--r--   3 doug supergroup          7 2009-01-17 19:52 /hypertable/servers/10.0.30.108_38060/log/range_txn/0.log
-rw-r--r--   3 doug supergroup          7 2009-01-17 19:52 /hypertable/servers/10.0.30.110_38060/log/range_txn/0.log
-rw-r--r--   3 doug supergroup          7 2009-01-17 19:52 /hypertable/servers/10.0.30.112_38060/log/range_txn/0.log
-rw-r--r--   3 doug supergroup          7 2009-01-17 19:52 /hypertable/servers/10.0.30.114_38060/log/range_txn/0.log
-rw-r--r--   3 doug supergroup          7 2009-01-17 19:52 /hypertable/servers/10.0.30.116_38060/log/range_txn/0.log

After shutting down HDFS and restarting it again, the listing looks like this:

[doug@motherlode000 aol-basic]$ hadoop fs -ls /hypertable/servers/10.0.30.1*_38060/log/range_txn
-rw-r--r--   3 doug supergroup        564 2009-01-17 19:52 /hypertable/servers/10.0.30.102_38060/log/range_txn/0.log
-rw-r--r--   3 doug supergroup         84 2009-01-17 19:52 /hypertable/servers/10.0.30.104_38060/log/range_txn/0.log
-rw-r--r--   3 doug supergroup       1063 2009-01-17 19:52 /hypertable/servers/10.0.30.106_38060/log/range_txn/0.log
-rw-r--r--   3 doug supergroup        634 2009-01-17 19:52 /hypertable/servers/10.0.30.108_38060/log/range_txn/0.log
-rw-r--r--   3 doug supergroup        217 2009-01-17 19:52 /hypertable/servers/10.0.30.110_38060/log/range_txn/0.log
-rw-r--r--   3 doug supergroup       1943 2009-01-17 19:52 /hypertable/servers/10.0.30.112_38060/log/range_txn/0.log
-rw-r--r--   3 doug supergroup       1072 2009-01-17 19:52 /hypertable/servers/10.0.30.114_38060/log/range_txn/0.log
-rw-r--r--   3 doug supergroup        525 2009-01-17 19:52 /hypertable/servers/10.0.30.116_38060/log/range_txn/0.log

The last time I ran this test I encountered a problem where it appeared that some of our commits were lost.  Here's what I did:

1. ran tests (which create a table with 75274825 cells)
2. kill Hypertable
3. shutdown HDFS
4. restart HDFS
5. restart Hypertable (which re-plays the commit logs)
6. dumped the table

The table dump in #6 came up short (e.g. 72M entries).  It appears that some of the commit logs (different log than the range_txn log) came back up incomplete.

Let us know if you want us to run an instrumented version or anything.  We can send you the Hadoop log files if that helps.  Thanks!

- Doug


> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Jim Kellerman (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Jim Kellerman updated HADOOP-4379:
----------------------------------

    Attachment: Reader.java

Second of two test programs that demonstrate the problem.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, Reader.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Konstantin Shvachko (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12668138#action_12668138 ] 

Konstantin Shvachko commented on HADOOP-4379:
---------------------------------------------

[See related comment here|https://issues.apache.org/jira/browse/HADOOP-5027#action_12668136]

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, Reader.java, Reader.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12667559#action_12667559 ] 

dhruba borthakur commented on HADOOP-4379:
------------------------------------------

@Jim: I updated your test cases Reader.java and Writer.java and have successfully tested them at my site. Please let me know if you still see problems.

@Doug: Please use the latest patch that I uploaded. This has the SequenceFile.Writer.syncFS() method. 

(Also, I suggest that you should pull in the patch available at HADOOP-5027)

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, Reader.java, Reader.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Jim Kellerman (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Jim Kellerman updated HADOOP-4379:
----------------------------------

    Attachment: Writer.java

First of two test files demonstrating the problem

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, Reader.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

dhruba borthakur updated HADOOP-4379:
-------------------------------------

    Attachment: fsyncConcurrentReaders.txt

When a fsync occurs, the client informs the namenode the amount of data that it has written to the last block. The NameNode updates its meatadata with tis value.This means that a new getBlockLocation call will automatically retrieve the last good size of the last block.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Luke Lu (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12664739#action_12664739 ] 

Luke Lu commented on HADOOP-4379:
---------------------------------

If TC3 is too hard in general. A TC2.5, like be able to open the file on the same machine to see the half block (after the fsync) would be nice. It would go a long way to make the transaction log work and make things restartable.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "stack (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12721409#action_12721409 ] 

stack commented on HADOOP-4379:
-------------------------------

I can manufacture a situation where we're stuck trying to open-for-append and we never seem to succeed (It ran all night last night cycling trying to open-for-append).

4 servers.  All running datanodes and hbase regionservers.  Running head of the hadoop 0.20.0 branch with fsyncConcurrentReaders11_20.txt applied.  In hbase, I have dfs.append set to true.

Start up an upload.  After a few minutes, kill datanode on *.*.44.142.  I leave the hbase regionserver on *.*.44.142 running.

After a little while, the regionserver on *.*.44.139 fails because of the following DFSClient exception:

{code}
2009-06-18 18:24:28,514 [regionserver/0:0:0:0:0:0:0:0:60021.cacheFlusher] FATAL org.apache.hadoop.hbase.regionserver.MemcacheFlusher: Replay of hlog required. Forcing server shutdown
org.apache.hadoop.hbase.DroppedSnapshotException: region: TestTable,0452187774,1245349374706
    at org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(HRegion.java:936)
    at org.apache.hadoop.hbase.regionserver.HRegion.flushcache(HRegion.java:829)
    at org.apache.hadoop.hbase.regionserver.MemcacheFlusher.flushRegion(MemcacheFlusher.java:268)
    at org.apache.hadoop.hbase.regionserver.MemcacheFlusher.run(MemcacheFlusher.java:149)
Caused by: java.io.IOException: Bad connect ack with firstBadLink *.*.44.142:51010
    at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.createBlockOutputStream(DFSClient.java:2871)
    at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.nextBlockOutputStream(DFSClient.java:2794)
    at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.access$2000(DFSClient.java:2078)
    at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:2264)
{code}

(Why does this fail?  Why does one datanode missing cause this fail?)

The hbase master, also on *.*.44.139, then goes to recover the files of the crashed regionserver process.  It gets stuck looping trying to open-for-append:

{code}
2009-06-18 18:25:04,587 [HMaster] INFO org.apache.hadoop.hbase.regionserver.HLog: Failed open for append, waiting on lease recovery: hdfs://aa0-000-12.u.powerset.com:9002/hbase/.logs/aa0-000-12.u.powerset.com,60021,1245348995684/hlog.dat.1245349434556
org.apache.hadoop.ipc.RemoteException: org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException: failed to create file /hbase/.logs/aa0-000-12.u.powerset.com,60021,1245348995684/hlog.dat.1245349434556 for DFSClient_2060803895 on client *.*.44.139, because this file is already being created by DFSClient_-937484673 on *.*.44.139
    at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startFileInternal(FSNamesystem.java:1058)
    at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.appendFile(FSNamesystem.java:1146)
    at org.apache.hadoop.hdfs.server.namenode.NameNode.append(NameNode.java:392)
    at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
    at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
    at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
    at java.lang.reflect.Method.invoke(Method.java:597)
    at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:508)
    at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:959)
    at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:955)
    at java.security.AccessController.doPrivileged(Native Method)
    at javax.security.auth.Subject.doAs(Subject.java:396)
    at org.apache.hadoop.ipc.Server$Handler.run(Server.java:953)

    at org.apache.hadoop.ipc.Client.call(Client.java:739)
    at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:220)
    at $Proxy0.append(Unknown Source)
    at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
    at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
    at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
    at java.lang.reflect.Method.invoke(Method.java:597)
    at org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:82)
    at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:59)
    at $Proxy0.append(Unknown Source)
    at org.apache.hadoop.hdfs.DFSClient.append(DFSClient.java:487)
    at org.apache.hadoop.hdfs.DistributedFileSystem.append(DistributedFileSystem.java:186)
    at org.apache.hadoop.fs.FileSystem.append(FileSystem.java:525)
    at org.apache.hadoop.hbase.regionserver.HLog.recoverLog(HLog.java:1037)
    at org.apache.hadoop.hbase.regionserver.HLog.splitLog(HLog.java:816)
    at org.apache.hadoop.hbase.regionserver.HLog.splitLog(HLog.java:756)
    at org.apache.hadoop.hbase.master.ProcessServerShutdown.process(ProcessServerShutdown.java:274)
    at org.apache.hadoop.hbase.master.HMaster.processToDoQueue(HMaster.java:467)
    at org.apache.hadoop.hbase.master.HMaster.run(HMaster.java:401)
{code}

Let me attach the namenode log.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders11_20.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Hairong Kuang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12713669#action_12713669 ] 

Hairong Kuang commented on HADOOP-4379:
---------------------------------------

It seems to me that the current implementation of sync does not guarantee that synced data become visible to any new reader.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "stack (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

stack updated HADOOP-4379:
--------------------------

    Attachment: hadoop-stack-namenode-aa0-000-12.u.powerset.com.log.gz

Namenode log that covers the forever cycle on open-for-append

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders11_20.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hadoop-stack-namenode-aa0-000-12.u.powerset.com.log.gz, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Hairong Kuang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12710555#action_12710555 ] 

Hairong Kuang commented on HADOOP-4379:
---------------------------------------

If one of the datanodes in the pipeline has a very very slow disk, it is possible that when sync returns, not all the syned data get flushed to the disk yet. If this datanode dies and restarts before the lease recovery starts, I think the lease recovery will remove the synced data.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "stack (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12713329#action_12713329 ] 

stack commented on HADOOP-4379:
-------------------------------

... and the read finds a zero-length file (nothing read from the file).

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12667886#action_12667886 ] 

dhruba borthakur commented on HADOOP-4379:
------------------------------------------

I think these exceptions are related to the patch in HADOOP-5027. let me check.


> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, Reader.java, Reader.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Doug Judd (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12667514#action_12667514 ] 

Doug Judd commented on HADOOP-4379:
-----------------------------------

It looks like our code depends on the length, as returned by getFileStatus:

length = mFilesystem.getFileStatus(new Path(fileName)).getLen();

It appears that the length returned by this API call is not correct.  Can you take a look at it and maybe patch this one as well?


> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, Reader.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "stack (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12713316#action_12713316 ] 

stack commented on HADOOP-4379:
-------------------------------

After a chat with Hairong and Dhruba, it was thought that the wait on a successful append, close, then open to read was not necessary in this patch; that a new reader should be able to read up to the dead writers last sync.  I tried it and all I get are EOFs when the new reader tries to read the dead writers file.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12667760#action_12667760 ] 

dhruba borthakur commented on HADOOP-4379:
------------------------------------------

Hi Doug,

the exception is expected. Please look at the test file Reader.java that I attached to this JIRA. It shows how the reader waits for the lease recovery to end (to ensure that correct file size is updated on namenode). Please let me know if this approach is suitable for your application.

We could enhance FileSystem.getFileStatus() to contact the datanode and retrieve and return the most-current file length (only for files that have a concurrent writer). This will not have have performance impact for most map-reduce applications.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, Reader.java, Reader.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12667877#action_12667877 ] 

dhruba borthakur commented on HADOOP-4379:
------------------------------------------

Thanks Jim. I will wait to hear from Doug Judd on whether it solves his problem. 

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, Reader.java, Reader.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

dhruba borthakur updated HADOOP-4379:
-------------------------------------

    Fix Version/s: 0.19.1

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12667359#action_12667359 ] 

dhruba borthakur commented on HADOOP-4379:
------------------------------------------

I will add a new method SequenceFile.Writer.syncFS(), test it, and upload a new patch. Thanks Doug and Jim.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, Reader.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "stack (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12713779#action_12713779 ] 

stack commented on HADOOP-4379:
-------------------------------

I tested some more and new reader sees 0 data and gets an EOF each time.

I went back to testing the wait-on-successful-append, close, and new open for read and ran into the following issue again (I'd seen this in earlier patch) where we're spinnning waiting on the append to succeed:

{code}
2009-05-27 22:16:53,186 [HMaster] INFO org.apache.hadoop.hbase.regionserver.HLog: Failed open for append, waiting on lease recovery: hdfs://Y/hbase/.logs/X,1243460581044/hlog.dat.1243462406950
org.apache.hadoop.ipc.RemoteException: org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException: failed to create file /hbase/.logs/X,1243460581044/hlog.dat.1243462406950 for DFSClient_-843497834 on client X.X.X.X, because this file is already being created by DFSClient_-181950545 on Y.Y.Y.Y
    at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startFileInternal(FSNamesystem.java:1058)
    at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.appendFile(FSNamesystem.java:1146)
    at org.apache.hadoop.hdfs.server.namenode.NameNode.append(NameNode.java:392)
    at sun.reflect.GeneratedMethodAccessor18.invoke(Unknown Source)
    at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
    at java.lang.reflect.Method.invoke(Method.java:597)
    at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:508)
    at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:959)
    at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:955)
    at java.security.AccessController.doPrivileged(Native Method)
    at javax.security.auth.Subject.doAs(Subject.java:396)
    at org.apache.hadoop.ipc.Server$Handler.run(Server.java:953)

    at org.apache.hadoop.ipc.Client.call(Client.java:739)
    at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:220)
    at $Proxy0.append(Unknown Source)
    at sun.reflect.GeneratedMethodAccessor11.invoke(Unknown Source)
    at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
    at java.lang.reflect.Method.invoke(Method.java:597)
    at org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:82)
    at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:59)
    at $Proxy0.append(Unknown Source)
    at org.apache.hadoop.hdfs.DFSClient.append(DFSClient.java:487)
    at org.apache.hadoop.hdfs.DistributedFileSystem.append(DistributedFileSystem.java:186)
    at org.apache.hadoop.fs.FileSystem.append(FileSystem.java:525)
    at org.apache.hadoop.hbase.regionserver.HLog.recoverLog(HLog.java:956)
    at org.apache.hadoop.hbase.regionserver.HLog.splitLog(HLog.java:800)
    at org.apache.hadoop.hbase.regionserver.HLog.splitLog(HLog.java:753)
    at org.apache.hadoop.hbase.master.ProcessServerShutdown.process(ProcessServerShutdown.java:248)
    at org.apache.hadoop.hbase.master.HMaster.processToDoQueue(HMaster.java:459)
    at org.apache.hadoop.hbase.master.HMaster.run(HMaster.java:395)
{code}

... and then this started up and won't stop:

{code}
2009-05-27 22:16:59,753 [HMaster] INFO org.apache.hadoop.hdfs.DFSClient: Could not complete file /hbase/.logs/X,60021,1243460581044/hlog.dat.1243462406950 retrying...
2009-05-27 22:17:00,163 [HMaster] INFO org.apache.hadoop.hdfs.DFSClient: Could not complete file /hbase/.logs/X,1243460581044/hlog.dat.1243462406950 retrying...
2009-05-27 22:17:00,573 [HMaster] INFO org.apache.hadoop.hdfs.DFSClient: Could not complete file /hbase/.logs/X,1243460581044/hlog.dat.1243462406950 retrying...
.....
{code}

Namenode log is here:  http://www.duboce.net/~stack/could_not_complete_file_nn.log.gz



> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Jim Kellerman (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Jim Kellerman updated HADOOP-4379:
----------------------------------

    Attachment: namenode.log

@Dhruba

You were correct. I had INFO logging turned off. Here is a new name node log file with INFO enabled.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2, 0.20.0
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "stack (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12701392#action_12701392 ] 

stack commented on HADOOP-4379:
-------------------------------

@Dhruba Any luck testing at scale? (Looks like HADOOP-5332 was committed).  Thanks.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2, 0.20.0
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Jim Kellerman (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12680246#action_12680246 ] 

Jim Kellerman commented on HADOOP-4379:
---------------------------------------

Dhruba,

What branch/revision did you test this patch against?

I've tried several in 0.19 branch (0.19.0, and revisions just before 0.19.1 (and before HADOOP-5225 was committed, but none are able to
recover the lease.

It appears that 5225 was not committed to 0.20.0 or to trunk. Is that correct? If so should I try a revision of Hadoop off of 0.20 or trunk?

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2, 0.20.0
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12680281#action_12680281 ] 

dhruba borthakur commented on HADOOP-4379:
------------------------------------------

HADOOP-5225 has been applied only to 0.19.

I have tested it only on 0.19 branch (not on trunk). Also with very light load. I will be testing it in scale on 0.19 very veru shortly (I am waiting for HADOOP-5332 to get committed.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2, 0.20.0
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

dhruba borthakur updated HADOOP-4379:
-------------------------------------

    Attachment: fsyncConcurrentReaders3.patch

This patch implements the following (as specified in the design doc for Appends, HADOOP-1700):
A reader checks to see if the file is being written to by another writer. if so, it fetches the size of the last block from the primary datanode. This does not have any performance impact for non-concurrent readers not does it have any impact on sync().

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, Reader.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Jim Kellerman (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12677135#action_12677135 ] 

Jim Kellerman commented on HADOOP-4379:
---------------------------------------

> dhruba borthakur - 26/Feb/09 11:35 AM
> @Jim: Were u using ReopenProblem.java:waitForLeaseRecovery() and still seeing that it takes upto an hour for the
> waitForLeaseRecovery() method to return success?

Well, sort of. my code is similar but inline in the module that uses it:
{code}
        // Recover the files lease if necessary
boolean recovered = false;
while (!recovered) {
  try {
    FSDataOutputStream out = fs.append(logfiles[i].getPath());
    out.close();
    recovered = true;
  } catch (IOException e) {
    if (LOG.isDebugEnabled()) {
      LOG.debug("Triggering lease recovery.");
    }
    try {
      Thread.sleep(leaseRecoveryPeriod);
    } catch (InterruptedException ex) {
      // ignore it and try again
    }
  }
}
{code}
  
> This can happen if the original client that was writing to the file was alive and was in communication with the namenode.
> In this case, the new append-er will not be able to recover the original lease.

This was definitely not the case. The original writing process was kill'ed -9

Like Hypertable, HBase has a mechanism to detect when the original writing process is dead.


> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2, 0.20.0
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Doug Judd (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12667499#action_12667499 ] 

Doug Judd commented on HADOOP-4379:
-----------------------------------

This may be a problem on our end.  When I shut down hypertable and then copy each range_txn log file to the local filesystem, the files seem to contain more than just the 7-byte header.

[doug@motherlode000 aol-basic]$ ls -l rsml/
total 32
-rw-rw-r-- 1 doug doug  564 Jan 26 15:35 102.log
-rw-rw-r-- 1 doug doug   84 Jan 26 15:35 104.log
-rw-rw-r-- 1 doug doug 1206 Jan 26 15:35 106.log
-rw-rw-r-- 1 doug doug  634 Jan 26 15:35 108.log
-rw-rw-r-- 1 doug doug   84 Jan 26 15:35 110.log
-rw-rw-r-- 1 doug doug 1942 Jan 26 15:36 112.log
-rw-rw-r-- 1 doug doug 1074 Jan 26 15:36 114.log
-rw-rw-r-- 1 doug doug  525 Jan 26 15:36 116.log


> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, Reader.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Assigned: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

dhruba borthakur reassigned HADOOP-4379:
----------------------------------------

    Assignee: dhruba borthakur

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "stack (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12714168#action_12714168 ] 

stack commented on HADOOP-4379:
-------------------------------

In this test run, the append never succeeds.... or at least, after 25 minutes it still has not successfully done the append open.  We try the append, fail with an AlreadyBeingCreatedException, sleep a second, and then cycle.  Usually it takes well under a minute to successfully open-to-append.  Namenode log is here: www.duboce.net:~stack/wontstop_namenode.log.gz.  In this case, I killed datanode and the hbase regionserver simulating a machine falling off the cluster (Previous, I was mostly just killing the server process and not the datanode).

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Doug Judd (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12677158#action_12677158 ] 

Doug Judd commented on HADOOP-4379:
-----------------------------------

Hi Dhruba,

When you say "near-immediate" what does that mean?  Is it the 1 minute soft limit?  If so, that's way too long.  If a node bounces in our cluster, we can't have the database locked up for an entire minute waiting for the lease to expire when we know for a 100% fact that the original leaseholder is dead.  A one minute database lock-up would make Hypertable unsuitable for serving live data.

- Doug


> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2, 0.20.0
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Doug Judd (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12667972#action_12667972 ] 

Doug Judd commented on HADOOP-4379:
-----------------------------------

Dhruba,

The application is not trying to reopen a file that it already has open.  It appears that HDFS is getting confused and thinks that this is the case.  One thing that is slightly different in this case is that nothing gets written to the file by the original writer.  Here is the sequence of relevant operations:

File gets created
[nothing gets appended to it]
process gets killed
process starts up again
File is reopened with append to obtain length

Can you verify that this particular usage pattern is handled properly?  I'll try to come up with a stripped down test case tomorrow.

- Doug


> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, Reader.java, Reader.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Jim Kellerman (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12667874#action_12667874 ] 

Jim Kellerman commented on HADOOP-4379:
---------------------------------------

Tested this patch in conjunction with HADOOP-5027, Works!!!!!!!!!!!!!!!

I would like to see this in 0.19.1, 0.20.0 and trunk.

+++++1


> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, Reader.java, Reader.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12664758#action_12664758 ] 

dhruba borthakur commented on HADOOP-4379:
------------------------------------------

@Luke: the attached patch should solve your problem. If you have the time, would it be possible for you to apply this patch and then retry your test case?

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Doug Judd (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12670349#action_12670349 ] 

Doug Judd commented on HADOOP-4379:
-----------------------------------

Hi Dhruba,

The file that is opened on line 189 ( /hypertable/servers/10.0.30.114_38060/log/root/1 ) is different than the one passed into waitForLeaseRecovery on line 197.  On line 194, the file object gets reset to a different path ( /hypertable/servers/10.0.30.114_38060/log/metadata/0 ).  So, this program should not hang.

- Doug


> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Luke Lu (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12665196#action_12665196 ] 

Luke Lu commented on HADOOP-4379:
---------------------------------

On the design of the TC3: I think that client informing namenode on each sync is too expensive and not scalable. Hypertable can easily generate 1M transactions/s from a small cluster, which the namenode cannot possibly handle. Since restarting HDFS incurs no data loss, I think the basic logic is probably already fine. For most cases, I think that making new readers a bit more expensive to create is probably the right trade-off:

1. We could append and sync a file and only inform the namenode on the first sync, hinting that we need to do something special for the new readers of the file.
2. Upon creating a new reader for the file, namenode would request a block report (an incremental one, if we try to be clever) from the involved data nodes and return the right info the reader.

I think that this would cover most of the use cases for append and sync reasonably.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Jim Kellerman (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12667350#action_12667350 ] 

Jim Kellerman commented on HADOOP-4379:
---------------------------------------

@Dhruba

If I have a SequenceFile.Writer, how do I get the FSDataOutputStream? It is a protected member of SequenceFile.Writer.
Are you planning to add an accessor?

Thanks.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>             Fix For: 0.19.1
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, Reader.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "stack (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12710546#action_12710546 ] 

stack commented on HADOOP-4379:
-------------------------------

v9 works well in basic testing.  If I crash the writer application, a subsequent reader can pick up edits from the crashed writer if the reader process does append, close and then reopen.

I've been trying to break the patch but it seems robust (v7 and v8 had issues that Dhruba fixed).

The append/close process can take from 10-60 seconds in my basic testing.  Usually its about 20 seconds.  It would be better if there were no pause.  Can this be addressed?





> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

dhruba borthakur updated HADOOP-4379:
-------------------------------------

    Attachment: fsyncConcurrentReaders11_20.txt

Patch for 0.20 branch. Fixes a concurrentmodificationexception.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders11_20.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Jim Kellerman (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Jim Kellerman updated HADOOP-4379:
----------------------------------

    Fix Version/s: 0.20.0

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2, 0.20.0
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

dhruba borthakur updated HADOOP-4379:
-------------------------------------

    Attachment: fsyncConcurrentReaders9.patch

Fixed block reports and blockReceived to handle change in generation stamp of a block.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Jim Kellerman (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12678820#action_12678820 ] 

Jim Kellerman commented on HADOOP-4379:
---------------------------------------

More info. When I generated the new namenode.log, the writer was killed at 16:54. The first attempt to grab the lease was at 16:56
and it was finally granted at 17:54.


> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2, 0.20.0
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (HADOOP-4379) In HDFS, sync() not yet guarantees data available to the new readers

Posted by "Nigel Daley (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-4379?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Nigel Daley updated HADOOP-4379:
--------------------------------

    Fix Version/s:     (was: 0.19.1)
                   0.19.2

As discussed on core-dev@ (http://www.nabble.com/Hadoop-0.19.1-td21739202.html) we will disable append in 0.19.1.  Moving these append related issues to 0.19.2.

> In HDFS, sync() not yet guarantees data available to the new readers
> --------------------------------------------------------------------
>
>                 Key: HADOOP-4379
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4379
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: dfs
>            Reporter: Tsz Wo (Nicholas), SZE
>            Assignee: dhruba borthakur
>            Priority: Blocker
>             Fix For: 0.19.2
>
>         Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, hypertable-namenode.log.gz, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java
>
>
> In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says
> * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file
> However, this feature is not yet implemented.  Note that the operation 'flushed' is now called "sync".

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.