[
https://issues.apache.org/jira/browse/HBASE-22620?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16873305#comment-16873305
]
leizhang commented on HBASE-22620:
----------------------------------
did you expect one just like this ? Hbase version 1.4.9
{code:java}
main-EventThread.replicationSource,1.replicationSource.xxxxxxxxxx.hbase.lq2%2C16020%2C1561379323483,1"
#153306 daemon prio=5 os_prio=0 tid=0x00007f0844681800 nid=0xe49 waiting on
condition [0x00007ef573a1d000]
java.lang.Thread.State: WAITING (parking)
at sun.misc.Unsafe.park(Native Method)
- parking to wait for <0x00007f05ba84c3d8> (a
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
at
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
at
org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReaderThread.take(ReplicationSourceWALReaderThread.java:227)
at
org.apache.hadoop.hbase.replication.regionserver.ReplicationSource$ReplicationSourceShipperThread.run(ReplicationSource.java:550)
"main.replicationSource,1-EventThread" #153305 daemon prio=5 os_prio=0
tid=0x00007f0844765800 nid=0xe48 waiting on condition [0x00007ef57391c000]
java.lang.Thread.State: WAITING (parking)
at sun.misc.Unsafe.park(Native Method)
- parking to wait for <0x00007f05ba847f50> (a
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
at
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:501)
{code}
once the peer was added, if no entry to replicate, the hlog refs will
accumulate on zk, and hlog accumulate in /oldWALs
I also found that due to the hug data amout (about 30T hlog file under
/hbase/oldWALs ), when I execute the command "remove_peer 'peer1' " on my
cluster, the master show the log bellow, and all regionservers abort
{code:java}
//代码占位符
ERROR [B.defaultRpcServer.handler=172,queue=22,port=16000]
master.MasterRpcServices: Region server xxxxxxxxxxxxxxxx,16020,1503477315622
reported a fatal error:
ABORTING region server xxxxxxxxxxxxxxx,16020,1503477315622: Failed to delete
queue (queueId=peer1)
Cause:
org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode =
ConnectionLoss
at org.apache.zookeeper.KeeperException.create(KeeperException.java:99)
at org.apache.zookeeper.ZooKeeper.multiInternal(ZooKeeper.java:935)
at org.apache.zookeeper.ZooKeeper.multi(ZooKeeper.java:915)
at
org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.multi(RecoverableZooKeeper.java:672)
at org.apache.hadoop.hbase.zookeeper.ZKUtil.multiOrSequential(ZKUtil.java:1671)
at
org.apache.hadoop.hbase.zookeeper.ZKUtil.deleteNodeRecursivelyMultiOrSequential(ZKUtil.java:1413)
at
org.apache.hadoop.hbase.zookeeper.ZKUtil.deleteNodeRecursively(ZKUtil.java:1280)
at
org.apache.hadoop.hbase.replication.ReplicationQueuesZKImpl.removeQueue(ReplicationQueuesZKImpl.java:93)
at
org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.deleteSource(ReplicationSourceManager.java:298)
at
org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.removePeer(ReplicationSourceManager.java:579)
at
org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.peerRemoved(ReplicationSourceManager.java:590)
at
org.apache.hadoop.hbase.replication.ReplicationTrackerZKImpl$PeersWatcher.nodeDeleted(ReplicationTrackerZKImpl.java:171)
at
org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.process(ZooKeeperWatcher.java:628)
at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:522)
at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:498)
{code}
So i have to remove the hlog refs on zk manually and let regionserver clean
hlogs nomally.
> When a cluster open replication,regionserver will not clean up the walLog
> references on zk due to no wal entry need to be replicated
> ------------------------------------------------------------------------------------------------------------------------------------
>
> Key: HBASE-22620
> URL: https://issues.apache.org/jira/browse/HBASE-22620
> Project: HBase
> Issue Type: Improvement
> Components: Replication
> Affects Versions: 2.0.3, 1.4.9
> Reporter: leizhang
> Priority: Major
>
> When I open the replication feature on my hbase cluster (20 regionserver
> nodes), for example, I create a table with 3 regions, which opened on 3
> regionservers of 20. Due to no data to replicate ,the left 17 nodes
> accumulate lots of wal references on the zk node
> "/hbase/replication/rs/\{resionserver}/\{peerId}/" and will not be cleaned
> up, which cause lots of wal file on hdfs will not be cleaned up either. When
> I check my test cluster after about four months, it accumulates about 5w wal
> files in the oldWal directory on hdfs. The source code shows that only there
> are data to be replicated, and after some data is replicated in the source
> endpoint, then it will executed the useless wal file check, and clean their
> references on zk, and the hdfs useless wal files will be cleaned up normally.
> So I think do we need other method to trigger the useless wal cleaning job in
> a replication cluster? May be in the replication progress report schedule
> task (just like ReplicationStatisticsTask.class)
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)