zhuobin zheng created HBASE-26482:
-------------------------------------

             Summary: HMaster may clean replication wals in rare cases
                 Key: HBASE-26482
                 URL: https://issues.apache.org/jira/browse/HBASE-26482
             Project: HBase
          Issue Type: Bug
          Components: Replication
            Reporter: zhuobin zheng


In our cluster, i can found some FileNotFoundException when 
ReplicationSourceWALReader running for replication recovery queue.

I guss the wal most likely removed by hmaste. And i found something to support 
it.

The method getAllWALs: 
[https://github.com/apache/hbase/blob/master/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java#L509
   
|https://github.com/apache/hbase/blob/master/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java#L509]Use
 zk cversion of /hbase/replication/rs as an optimistic lock to control 
concurrent ops.

But, zk cversion *only can only reflect the changes of child nodes, but not the 
changes of grandchildren.*

So, HMaster may loss some wal from this method in follow situation.
 # HMaster do log clean , and invoke getAllWALs to filter log which should not 
be deleted.
 # HMaster cache current cversion of /hbase/replication/rs  as *v0*
 # HMaster cache all RS server name, and traverse them, get the WAL in each 
Queue
 # *RS2* dead after HMaster traverse {*}RS1{*}, and before traverse *RS2*
 # *RS1* claim one queue of *RS2,* which named *peerid-RS2* now
 # By the way , the cversion of /hbase/replication/rs not changed before all of 
*RS2* queue is removed, because the children of /hbase/replication/rs not 
change.
 # So, Hmaster will lost the wals in *peerid-RS2,* because we have already 
traversed *RS1 ,* and ** this queue not exists in *RS2*

The above expression is currently only speculation, not confirmed

Flie Not Found Log.

 
{code:java}
// code placeholder
2021-11-22 15:18:39,593 ERROR 
[ReplicationExecutor-0.replicationSource,peer_id-hostname,60020,1636802867348.replicationSource.wal-reader.hostname%2C60020%2C1636802867348,peer_id-hostname,60020,1636802867348]
 regionserver.WALEntryStream: Couldn't locate log: 
hdfs://namenode/hbase/oldWALs/hostname%2C60020%2C1636802867348.1636944748704
2021-11-22 15:18:39,593 ERROR 
[ReplicationExecutor-0.replicationSource,peer_id-hostname,60020,1636802867348.replicationSource.wal-reader.hostname%2C60020%2C1636802867348,peer_id-hostname,60020,1636802867348]
 regionserver.ReplicationSourceWALReader: Failed to read stream of replication 
entries
java.io.FileNotFoundException: File does not exist: 
hdfs://namenode/hbase/oldWALs/hostname%2C60020%2C1636802867348.1636944748704
        at 
org.apache.hadoop.hdfs.DistributedFileSystem$29.doCall(DistributedFileSystem.java:1612)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem$29.doCall(DistributedFileSystem.java:1605)
        at 
org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1620)
        at 
org.apache.hadoop.hbase.regionserver.wal.ReaderBase.init(ReaderBase.java:64)
        at 
org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.init(ProtobufLogReader.java:168)
        at 
org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:321)
        at 
org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:303)
        at 
org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:291)
        at 
org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:427)
        at 
org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.openReader(WALEntryStream.java:355)
        at 
org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.openNextLog(WALEntryStream.java:303)
        at 
org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.checkReader(WALEntryStream.java:294)
        at 
org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.tryAdvanceEntry(WALEntryStream.java:175)
        at 
org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.hasNext(WALEntryStream.java:101)
        at 
org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.readWALEntries(ReplicationSourceWALReader.java:192)
        at 
org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.run(ReplicationSourceWALReader.java:138)
 {code}
 

 



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

Reply via email to