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 "stack (JIRA)" <ji...@apache.org> on 2009/06/18 20:50:07 UTC

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

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