[
https://issues.apache.org/jira/browse/HBASE-22620?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16871929#comment-16871929
]
leizhang commented on HBASE-22620:
----------------------------------
I print the stack info of my regionserver, and find the thread blocking here:
{code:java}
"regionserver/hostxxxxxxx/ipxxxxxxxxxx:16020.replicationSource,1-EventThread"
#421 daemon prio=5 os_prio=0 tid=0x00007f084412d800 nid=0x6b01c waiting on
condition [0x00007ef574423000]
The network connection was aborted by the local system.
at sun.misc.Unsafe.park(Native Method)
- parking to wait for <0x00007eff45024a28> (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}
> 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)