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

Reply via email to