You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "Duo Zhang (JIRA)" <ji...@apache.org> on 2018/03/24 08:08:00 UTC

[jira] [Commented] (HBASE-20271) ReplicationSourceWALReader.switched should use the file name instead of the path object directly

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

Duo Zhang commented on HBASE-20271:
-----------------------------------

When hitting FNFE in WALEntryStream we will try to open the wal in the archive directory. Used to think this will not effect the current path in WALEntryStream since it is always peeked from the logQueue, but after re-reading the code I found that it is wrong. We will set current path in openReader and in handleFileNotFound we will pass the archived path into openReader so the current path will be changed.

Let me prepare a patch to fix it.

> ReplicationSourceWALReader.switched should use the file name instead of the path object directly
> ------------------------------------------------------------------------------------------------
>
>                 Key: HBASE-20271
>                 URL: https://issues.apache.org/jira/browse/HBASE-20271
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Duo Zhang
>            Priority: Major
>
> {noformat}
> 2018-03-24 08:29:29,965 ERROR [RS_REFRESH_PEER-regionserver/ubuntu:0-0.replicationSource,2.replicationSource.shipperubuntu%2C35197%2C1521851267085,2] helpers.MarkerIgnoringBase(159): ***** ABORTING region server ubuntu,35197,1521851267085: Failed to operate on replication queue *****
> org.apache.hadoop.hbase.replication.ReplicationException: Failed to set log position (serverName=ubuntu,35197,1521851267085, queueId=2, fileName=ubuntu%2C35197%2C1521851267085.1521851344947, position=2533)
> 	at org.apache.hadoop.hbase.replication.ZKReplicationQueueStorage.setWALPosition(ZKReplicationQueueStorage.java:237)
> 	at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.lambda$9(ReplicationSourceManager.java:488)
> 	at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.abortWhenFail(ReplicationSourceManager.java:455)
> 	at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.logPositionAndCleanOldLogs(ReplicationSourceManager.java:488)
> 	at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceShipper.updateLogPosition(ReplicationSourceShipper.java:232)
> 	at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceShipper.shipEdits(ReplicationSourceShipper.java:134)
> 	at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceShipper.run(ReplicationSourceShipper.java:104)
> Caused by: org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode
> 	at org.apache.zookeeper.KeeperException.create(KeeperException.java:111)
> 	at org.apache.zookeeper.ZooKeeper.multiInternal(ZooKeeper.java:1006)
> 	at org.apache.zookeeper.ZooKeeper.multi(ZooKeeper.java:910)
> 	at org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.multi(RecoverableZooKeeper.java:663)
> 	at org.apache.hadoop.hbase.zookeeper.ZKUtil.multiOrSequential(ZKUtil.java:1670)
> 	at org.apache.hadoop.hbase.replication.ZKReplicationQueueStorage.setWALPosition(ZKReplicationQueueStorage.java:235)
> 	... 6 more
> 2018-03-24 08:29:30,025 ERROR [RpcServer.default.FPBQ.Fifo.handler=2,queue=0,port=37509] master.MasterRpcServices(508): Region server ubuntu,35197,1521851267085 reported a fatal error:
> ***** ABORTING region server ubuntu,35197,1521851267085: Failed to operate on replication queue *****
> Cause:
> org.apache.hadoop.hbase.replication.ReplicationException: Failed to set log position (serverName=ubuntu,35197,1521851267085, queueId=2, fileName=ubuntu%2C35197%2C1521851267085.1521851344947, position=2533)
> 	at org.apache.hadoop.hbase.replication.ZKReplicationQueueStorage.setWALPosition(ZKReplicationQueueStorage.java:237)
> 	at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.lambda$9(ReplicationSourceManager.java:488)
> 	at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.abortWhenFail(ReplicationSourceManager.java:455)
> 	at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.logPositionAndCleanOldLogs(ReplicationSourceManager.java:488)
> 	at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceShipper.updateLogPosition(ReplicationSourceShipper.java:232)
> 	at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceShipper.shipEdits(ReplicationSourceShipper.java:134)
> 	at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceShipper.run(ReplicationSourceShipper.java:104)
> Caused by: org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode
> 	at org.apache.zookeeper.KeeperException.create(KeeperException.java:111)
> 	at org.apache.zookeeper.ZooKeeper.multiInternal(ZooKeeper.java:1006)
> 	at org.apache.zookeeper.ZooKeeper.multi(ZooKeeper.java:910)
> 	at org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.multi(RecoverableZooKeeper.java:663)
> 	at org.apache.hadoop.hbase.zookeeper.ZKUtil.multiOrSequential(ZKUtil.java:1670)
> 	at org.apache.hadoop.hbase.replication.ZKReplicationQueueStorage.setWALPosition(ZKReplicationQueueStorage.java:235)
> 	... 6 more
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)