[ 
https://issues.apache.org/jira/browse/HBASE-497?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12579702#action_12579702
 ] 

Bryan Duxbury commented on HBASE-497:
-------------------------------------

If an IOException gets thrown up the stack all the way back to 
HRegionServer#batchUpdate, then the HRS will call checkFileSystem. If there is 
actually an FS error (which there would have to be in order for two 
IOExceptions to occur in a row inside rollWriter - I think), then 
checkFileSystem sets abortRequested and stopRequested, which should kill the 
main HRS thread.

In this issue, the reason we are not seeing the HRS go down is that there 
actually isn't an FS problem - it's totally our fault for not trying to reopen 
the log writer. I suspect that we will be able to recover from this kind of 
error with the code previously posted.

I will put up a new patch with the logging improvements requested.

> RegionServer needs to recover if datanode goes down
> ---------------------------------------------------
>
>                 Key: HBASE-497
>                 URL: https://issues.apache.org/jira/browse/HBASE-497
>             Project: Hadoop HBase
>          Issue Type: Bug
>    Affects Versions: 0.16.0
>            Reporter: Michael Bieniosek
>            Priority: Blocker
>             Fix For: 0.1.0, 0.2.0
>
>         Attachments: 497_0.1.patch
>
>
> If I take down a datanode, the regionserver will repeatedly return this error:
> java.io.IOException: Stream closed.
>         at 
> org.apache.hadoop.dfs.DFSClient$DFSOutputStream.isClosed(DFSClient.java:1875)
>         at 
> org.apache.hadoop.dfs.DFSClient$DFSOutputStream.writeChunk(DFSClient.java:2096)
>         at 
> org.apache.hadoop.fs.FSOutputSummer.writeChecksumChunk(FSOutputSummer.java:141)
>         at 
> org.apache.hadoop.fs.FSOutputSummer.flushBuffer(FSOutputSummer.java:124)
>         at org.apache.hadoop.fs.FSOutputSummer.write1(FSOutputSummer.java:112)
>         at org.apache.hadoop.fs.FSOutputSummer.write(FSOutputSummer.java:86)
>         at 
> org.apache.hadoop.fs.FSDataOutputStream$PositionCache.write(FSDataOutputStream.java:41)
>         at java.io.DataOutputStream.write(Unknown Source)
>         at 
> org.apache.hadoop.io.SequenceFile$Writer.append(SequenceFile.java:977)
>         at org.apache.hadoop.hbase.HLog.append(HLog.java:377)
>         at org.apache.hadoop.hbase.HRegion.update(HRegion.java:1455)
>         at org.apache.hadoop.hbase.HRegion.batchUpdate(HRegion.java:1259)
>         at 
> org.apache.hadoop.hbase.HRegionServer.batchUpdate(HRegionServer.java:1433)
>         at sun.reflect.GeneratedMethodAccessor9.invoke(Unknown Source)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(Unknown Source)
>         at java.lang.reflect.Method.invoke(Unknown Source)
>         at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:413)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:910)
> It appears that hbase/dfsclient does not attempt to reopen the stream.

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