[
https://issues.apache.org/jira/browse/HBASE-22620?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
leizhang updated HBASE-22620:
-----------------------------
Description: When I open the replication feature on my hbase cluster (20
regionserver nodes) and added a peer cluster, for example, I create a table
with 3 regions with REPLICATION_SCOPE set to 1, which opened on 3 regionservers
of 20. Due to no data(entryBatch) 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) (was: 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))
> 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: Bug
> 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) and added a peer cluster, for example, I create a table with 3 regions
> with REPLICATION_SCOPE set to 1, which opened on 3 regionservers of 20. Due
> to no data(entryBatch) 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)