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

Ted Yu commented on HBASE-15019:
--------------------------------

lgtm
{code}
821             recoverLease(conf, currentPath);
{code}
Should return value from recoverLease() be checked ?

> Replication stuck when HDFS is restarted
> ----------------------------------------
>
>                 Key: HBASE-15019
>                 URL: https://issues.apache.org/jira/browse/HBASE-15019
>             Project: HBase
>          Issue Type: Bug
>          Components: Replication, wal
>    Affects Versions: 2.0.0, 1.2.0, 1.1.2, 1.0.3, 0.98.16.1
>            Reporter: Matteo Bertozzi
>            Assignee: Matteo Bertozzi
>         Attachments: HBASE-15019-v0_branch-1.2.patch, HBASE-15019-v1.patch, 
> HBASE-15019-v1_0.98.patch, HBASE-15019-v1_branch-1.2.patch
>
>
> RS is normally working and writing on the WAL.
> HDFS is killed and restarted, and the RS try to do a roll.
> The close fail, but the roll succeed (because hdfs is now up) and everything 
> works.
> {noformat}
> 2015-12-11 21:52:28,058 ERROR 
> org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter: Got IOException 
> while writing trailer
> java.io.IOException: All datanodes 10.51.30.152:50010 are bad. Aborting...
>   at 
> org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.setupPipelineForAppendOrRecovery(DFSOutputStream.java:1147)
>   at 
> org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.processDatanodeError(DFSOutputStream.java:945)
>   at 
> org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.run(DFSOutputStream.java:496)
> 2015-12-11 21:52:28,059 ERROR 
> org.apache.hadoop.hbase.regionserver.wal.FSHLog: Failed close of HLog writer
> java.io.IOException: All datanodes 10.51.30.152:50010 are bad. Aborting...
>   at 
> org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.setupPipelineForAppendOrRecovery(DFSOutputStream.java:1147)
>   at 
> org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.processDatanodeError(DFSOutputStream.java:945)
>   at 
> org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.run(DFSOutputStream.java:496)
> 2015-12-11 21:52:28,059 WARN org.apache.hadoop.hbase.regionserver.wal.FSHLog: 
> Riding over HLog close failure! error count=1
> {noformat}
> The problem is on the replication side. that log we rolled and we were not 
> able to close
> is waiting for a lease recovery.
> {noformat}
> 2015-12-11 21:16:31,909 ERROR 
> org.apache.hadoop.hbase.regionserver.wal.HLogFactory: Can't open after 267 
> attempts and 301124ms 
> {noformat}
> the WALFactory notify us about that, but there is nothing on the RS side that 
> perform the WAL recovery.
> {noformat}
> 2015-12-11 21:11:30,921 WARN 
> org.apache.hadoop.hbase.regionserver.wal.HLogFactory: Lease should have 
> recovered. This is not expected. Will retry
> java.io.IOException: Cannot obtain block length for 
> LocatedBlock{BP-1547065147-10.51.30.152-1446756937665:blk_1073801614_61243; 
> getBlockSize()=83; corrupt=false; offset=0; locs=[10.51.30.154:50010, 
> 10.51.30.152:50010, 10.51.30.155:50010]}
>   at 
> org.apache.hadoop.hdfs.DFSInputStream.readBlockLength(DFSInputStream.java:358)
>   at 
> org.apache.hadoop.hdfs.DFSInputStream.fetchLocatedBlocksAndGetLastBlockLength(DFSInputStream.java:300)
>   at org.apache.hadoop.hdfs.DFSInputStream.openInfo(DFSInputStream.java:237)
>   at org.apache.hadoop.hdfs.DFSInputStream.<init>(DFSInputStream.java:230)
>   at org.apache.hadoop.hdfs.DFSClient.open(DFSClient.java:1448)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem$3.doCall(DistributedFileSystem.java:301)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem$3.doCall(DistributedFileSystem.java:297)
>   at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem.open(DistributedFileSystem.java:297)
>   at org.apache.hadoop.fs.FilterFileSystem.open(FilterFileSystem.java:161)
>   at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:766)
>   at 
> org.apache.hadoop.hbase.regionserver.wal.HLogFactory.createReader(HLogFactory.java:116)
>   at 
> org.apache.hadoop.hbase.regionserver.wal.HLogFactory.createReader(HLogFactory.java:89)
>   at 
> org.apache.hadoop.hbase.regionserver.wal.HLogFactory.createReader(HLogFactory.java:77)
>   at 
> org.apache.hadoop.hbase.replication.regionserver.ReplicationHLogReaderManager.openReader(ReplicationHLogReaderManager.java:68)
>   at 
> org.apache.hadoop.hbase.replication.regionserver.ReplicationSource.openReader(ReplicationSource.java:508)
>   at 
> org.apache.hadoop.hbase.replication.regionserver.ReplicationSource.run(ReplicationSource.java:321)
> {noformat}
> the only way to trigger a WAL recovery is to restart and force the master to 
> trigger the lease recovery on WAL split. 
> but there is a case where restarting will not help. If the RS keeps going 
> rolling and flushing the unclosed WAL will be moved in the archive, and at 
> that point the master will never try to do a lease recovery on it. 
> since we know that the RS is still going, should we try to recover the lease 
> on the RS side?
> is it better/safer to trigger an abort on the RS, so we have only the master 
> doing lease recovery?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to