[
https://issues.apache.org/jira/browse/HBASE-22784?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16949119#comment-16949119
]
Karthick edited comment on HBASE-22784 at 10/11/19 4:05 AM:
------------------------------------------------------------
[~wchevreuil] we applied the patch in hbase-1.4.10 and we noticed random
region server aborts because of ReplicationQueuesZKImpl#setLogPosition() in
ReplicationSourceShipperThread.
{code:java}
2019-10-05 08:17:28,132 FATAL
[regionserver//172.20.20.20:16020.replicationSource.172.20.20.20%2C16020%2C1570193969775,2]
regionserver.HRegionServer: ABORTING region server
172.20.20.20,16020,1570193969775: Failed to write replication wal position
(filename=172.20.20.20%2C16020%2C1570193969775.1570288637045,
position=127494739)2019-10-05 08:17:28,132 FATAL
[regionserver//172.20.20.20:16020.replicationSource.172.20.20.20%2C16020%2C1570193969775,2]
regionserver.HRegionServer: ABORTING region server
172.20.20.20,16020,1570193969775: Failed to write replication wal position
(filename=172.20.20.20%2C16020%2C1570193969775.1570288637045,
position=127494739)org.apache.zookeeper.KeeperException$NoNodeException:
KeeperErrorCode = NoNode for
/hbase/replication/rs/172.20.20.20,16020,1570193969775/2/172.20.20.20%2C16020%2C1570193969775.1570288637045
at org.apache.zookeeper.KeeperException.create(KeeperException.java:111) at
org.apache.zookeeper.KeeperException.create(KeeperException.java:51) at
org.apache.zookeeper.ZooKeeper.setData(ZooKeeper.java:1327) at
org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.setData(RecoverableZooKeeper.java:422)
at org.apache.hadoop.hbase.zookeeper.ZKUtil.setData(ZKUtil.java:824) at
org.apache.hadoop.hbase.zookeeper.ZKUtil.setData(ZKUtil.java:874) at
org.apache.hadoop.hbase.zookeeper.ZKUtil.setData(ZKUtil.java:868) at
org.apache.hadoop.hbase.replication.ReplicationQueuesZKImpl.setLogPosition(ReplicationQueuesZKImpl.java:155)
at
org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.logPositionAndCleanOldLogs(ReplicationSourceManager.java:194)
at
org.apache.hadoop.hbase.replication.regionserver.ReplicationSource$ReplicationSourceShipperThread.updateLogPosition(ReplicationSource.java:727)
at
org.apache.hadoop.hbase.replication.regionserver.ReplicationSource$ReplicationSourceShipperThread.shipEdits(ReplicationSource.java:698)
at
org.apache.hadoop.hbase.replication.regionserver.ReplicationSource$ReplicationSourceShipperThread.run(ReplicationSource.java:551)2019-10-05
08:17:28,133 FATAL
[regionserver//172.20.20.20:16020.replicationSource.172.20.20.20%2C16020%2C1570193969775,2]
regionserver.HRegionServer: RegionServer abort: loaded coprocessors are:
[org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint{code}
Please provide us a solution and let us know if you need more logs regarding
this.
was (Author: karthickram):
[~wchevreuil] we applied the patch in hbase-1.4.10 and we noticed random
region server aborts because of ReplicationQueuesZKImpl#setLogPosition() in
ReplicationSourceShipperThread.
{quote}2019-10-05 08:17:28,132 FATAL
[regionserver//172.20.20.20:16020.replicationSource.172.20.20.20%2C16020%2C1570193969775,2]
regionserver.HRegionServer: ABORTING region server
172.20.20.20,16020,1570193969775: Failed to write replication wal position
(filename=172.20.20.20%2C16020%2C1570193969775.1570288637045,
position=127494739)2019-10-05 08:17:28,132 FATAL
[regionserver//172.20.20.20:16020.replicationSource.172.20.20.20%2C16020%2C1570193969775,2]
regionserver.HRegionServer: ABORTING region server
172.20.20.20,16020,1570193969775: Failed to write replication wal position
(filename=172.20.20.20%2C16020%2C1570193969775.1570288637045,
position=127494739)org.apache.zookeeper.KeeperException$NoNodeException:
KeeperErrorCode = NoNode for
/hbase/replication/rs/172.20.20.20,16020,1570193969775/2/172.20.20.20%2C16020%2C1570193969775.1570288637045
at org.apache.zookeeper.KeeperException.create(KeeperException.java:111) at
org.apache.zookeeper.KeeperException.create(KeeperException.java:51) at
org.apache.zookeeper.ZooKeeper.setData(ZooKeeper.java:1327) at
org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.setData(RecoverableZooKeeper.java:422)
at org.apache.hadoop.hbase.zookeeper.ZKUtil.setData(ZKUtil.java:824) at
org.apache.hadoop.hbase.zookeeper.ZKUtil.setData(ZKUtil.java:874) at
org.apache.hadoop.hbase.zookeeper.ZKUtil.setData(ZKUtil.java:868) at
org.apache.hadoop.hbase.replication.ReplicationQueuesZKImpl.setLogPosition(ReplicationQueuesZKImpl.java:155)
at
org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.logPositionAndCleanOldLogs(ReplicationSourceManager.java:194)
at
org.apache.hadoop.hbase.replication.regionserver.ReplicationSource$ReplicationSourceShipperThread.updateLogPosition(ReplicationSource.java:727)
at
org.apache.hadoop.hbase.replication.regionserver.ReplicationSource$ReplicationSourceShipperThread.shipEdits(ReplicationSource.java:698)
at
org.apache.hadoop.hbase.replication.regionserver.ReplicationSource$ReplicationSourceShipperThread.run(ReplicationSource.java:551)2019-10-05
08:17:28,133 FATAL
[regionserver//172.20.20.20:16020.replicationSource.172.20.20.20%2C16020%2C1570193969775,2]
regionserver.HRegionServer: RegionServer abort: loaded coprocessors are:
[org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint
{quote}
Please provide us a solution for this and let us know if you need more logs
regarding this.
> OldWALs not cleared in a replication slave cluster (cyclic replication bw 2
> clusters)
> -------------------------------------------------------------------------------------
>
> Key: HBASE-22784
> URL: https://issues.apache.org/jira/browse/HBASE-22784
> Project: HBase
> Issue Type: Bug
> Components: regionserver, Replication
> Affects Versions: 1.4.9, 1.4.10
> Reporter: Solvannan R M
> Assignee: Wellington Chevreuil
> Priority: Blocker
> Fix For: 1.5.0, 1.4.11
>
> Attachments: HBASE-22784.branch-1.001.patch,
> HBASE-22784.branch-1.002.patch, HBASE-22784.branch-1.003.patch,
> HBASE-22784.branch-1.004.patch
>
>
> When a cluster is passive (receiving edits only via replication) in a cyclic
> replication setup of 2 clusters, OldWALs size keeps on growing. On analysing,
> we observed the following behaviour.
> # New entry is added to WAL (Edit replicated from other cluster).
> # ReplicationSourceWALReaderThread(RSWALRT) reads and applies the configured
> filters (due to cyclic replication setup, ClusterMarkingEntryFilter discards
> new entry from other cluster).
> # Entry is null, RSWALRT neither updates the batch stats
> (WALEntryBatch.lastWalPosition) nor puts it in the entryBatchQueue.
> # ReplicationSource thread is blocked in entryBachQueue.take().
> # So ReplicationSource#updateLogPosition has never invoked and WAL file is
> never cleared from ReplicationQueue.
> # Hence LogCleaner on the master, doesn't deletes the oldWAL files from
> hadoop.
> NOTE: When a new edit is added via hbase-client, ReplicationSource thread
> process and clears the oldWAL files from replication queues and hence master
> cleans up the WALs
> Please provide us a solution
>
--
This message was sent by Atlassian Jira
(v8.3.4#803005)