[
https://issues.apache.org/jira/browse/HBASE-20475?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16454297#comment-16454297
]
Zheng Hu commented on HBASE-20475:
----------------------------------
Found the cause why the test to be flaky again, and it happened randomly ...
see the following log:
{code}
2018-04-25 18:08:24,432 INFO [regionserver/asf915:0.logRoller]
wal.AbstractFSWAL(671): Rolled WAL
/user/jenkins/test-data/be2e424e-e2f6-4ac1-91f1-d33621a46da3/WALs/asf915.gq1.ygridcore.net,42474,1524679689830/asf915.gq1.ygridcore.net%2C42
474%2C1524679689830.1524679692092 with entries=13, filesize=3.55 KB; new WAL
/user/jenkins/test-data/be2e424e-e2f6-4ac1-91f1-d33621a46da3/WALs/asf915.gq1.ygridcore.net,42474,1524679689830/asf915.gq1.ygridcore.net%2C42474%2C1524679689830.15
24679704418
......
2018-04-25 18:08:36,957 INFO [ReplicationExecutor-0]
replication.ZKReplicationQueueStorage(387): Atomically moving
asf915.gq1.ygridcore.net,42474,1524679689830/2's WALs to
asf915.gq1.ygridcore.net,34222,1524679706894
2018-04-25 18:08:36,957 DEBUG
[RpcServer.replication.FPBQ.Fifo.handler=2,queue=0,port=36265]
ipc.AbstractRpcClient(200):
Codec=org.apache.hadoop.hbase.codec.KeyValueCodecWithTags@396b2d58,
compressor=null, tcpKeepAlive=true, tcpNoDelay=tru
e, connectTO=10000, readTO=20000, writeTO=60000, minIdleTimeBeforeClose=120000,
maxRetries=0, fallbackAllowed=true, bind address=null
2018-04-25 18:08:36,959 DEBUG [ReplicationExecutor-0]
replication.ZKReplicationQueueStorage(414): Creating
asf915.gq1.ygridcore.net%2C42474%2C1524679689830.1524679704418 with data
PBUF\x08\x9F\x1A
2018-04-25 18:08:36,961 INFO [ReplicationExecutor-0]
replication.ZKReplicationQueueStorage(426): Atomically moved
asf915.gq1.ygridcore.net,42474,1524679689830/2's WALs to
asf915.gq1.ygridcore.net,34222,1524679706894
{code}
Step.1 Atomically moving -> it means that we begin to claimQueue from the
deadRS to destination RS.
Step.2 at 2018-04-25 18:08:24,432 , the rs start to roll the WAL, and at
2018-04-25 18:08:36,959 , it created the WAL.
Step.3 Atomically moved ... -> the NodeFailoverWorker finished to
cliamQueue from deadRS to dest RS, but exclude the WAL
asf915.gq1.ygridcore.net%2C42474%2C1524679689830.1524679704418, because had no
ZNODE yet.
So when our RecoveredReplicationSourceShipper try to ship the edits and
setWALPosition, it found that the znode did not exist, and the RS crashed
finally. (It's strange here: the new rs had been transferring queue, but the
dead RS was still creating the new WAL...)
{code}
2018-04-25 18:08:39,107 DEBUG
[ReplicationExecutor-0.replicationSource,2-asf915.gq1.ygridcore.net,42474,1524679689830.replicationSource.wal-reader.asf915.gq1.ygridcore.net%2C42474%2C1524679689830,2-asf915.gq1.ygridcore.net,42474,1524679689
830] regionserver.WALEntryStream(250): Reached the end of log
hdfs://localhost:43322/user/jenkins/test-data/be2e424e-e2f6-4ac1-91f1-d33621a46da3/oldWALs/asf915.gq1.ygridcore.net%2C42474%2C1524679689830.1524679704418
2018-04-25 18:08:39,109 DEBUG
[ReplicationExecutor-0.replicationSource,2-asf915.gq1.ygridcore.net,42474,1524679689830.replicationSource.shipperasf915.gq1.ygridcore.net%2C42474%2C1524679689830,2-asf915.gq1.ygridcore.net,42474,1524679689830]
replication.ReplicationQueueInfo(110): Found dead
servers:[asf915.gq1.ygridcore.net,42474,1524679689830]
2018-04-25 18:08:39,118 ERROR
[ReplicationExecutor-0.replicationSource,2-asf915.gq1.ygridcore.net,42474,1524679689830.replicationSource.shipperasf915.gq1.ygridcore.net%2C42474%2C1524679689830,2-asf915.gq1.ygridcore.net,42474,1524679689830]
helpers.MarkerIgnoringBase(159): ***** ABORTING region server
asf915.gq1.ygridcore.net,34222,1524679706894: Failed to operate on replication
queue *****
org.apache.hadoop.hbase.replication.ReplicationException: Failed to set log
position (serverName=asf915.gq1.ygridcore.net,34222,1524679706894,
queueId=2-asf915.gq1.ygridcore.net,42474,1524679689830,
fileName=asf915.gq1.ygridcore.net%2C4247
4%2C1524679689830.1524679704418, position=3632)
at
org.apache.hadoop.hbase.replication.ZKReplicationQueueStorage.setWALPosition(ZKReplicationQueueStorage.java:256)
at
org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.lambda$logPositionAndCleanOldLogs$7(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:231)
at
org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceShipper.shipEdits(ReplicationSourceShipper.java:133)
at
org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceShipper.run(ReplicationSourceShipper.java:103)
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:1690)
at
org.apache.hadoop.hbase.replication.ZKReplicationQueueStorage.setWALPosition(ZKReplicationQueueStorage.java:246)
... 6 more
{code}
> Fix the flaky TestReplicationDroppedTables unit test.
> -----------------------------------------------------
>
> Key: HBASE-20475
> URL: https://issues.apache.org/jira/browse/HBASE-20475
> Project: HBase
> Issue Type: Bug
> Affects Versions: 2.1.0
> Reporter: Zheng Hu
> Assignee: Zheng Hu
> Priority: Major
> Fix For: 3.0.0, 2.1.0
>
> Attachments: HBASE-20475-addendum.patch, HBASE-20475.patch
>
>
> See
> https://builds.apache.org/job/HBASE-Find-Flaky-Tests/lastSuccessfulBuild/artifact/dashboard.html
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)