[ 
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.

Reply via email to