[
https://issues.apache.org/jira/browse/HBASE-22620?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16874710#comment-16874710
]
leizhang commented on HBASE-22620:
----------------------------------
not only the pressure on zk.consider when there is a large amount of data
write in, which is writen by non-replication tables , the hlog is not empty, it
will also accumulate hlogs on hdfs . this is the reason which causes hlogs
under hdfs directory /oldWALs reached about 30TB.
the -HBASE-20206- may be not helpful for this issue. I view the code of
HBASE2.1.0 and find the logic is :
{code:java}
WALEntryBatch entryBatch = entryReader.poll(getEntriesTimeout);
if (entryBatch == null) {
// since there is no logs need to replicate, we refresh the ageOfLastShippedOp
source.getSourceMetrics().setAgeOfLastShippedOp(EnvironmentEdgeManager.currentTime(),
walGroupId);
continue;
}
// the NO_MORE_DATA instance has no path so do not call shipEdits
if (entryBatch == WALEntryBatch.NO_MORE_DATA) {
noMoreData();
} else {
shipEdits(entryBatch);
}
{code}
the entryReader.take() in hbase 1.4.9 has been replaced by
entryReader.poll(getEntriesTimeout) , indeed, the thread will not be blocked
any more . but if entryBatch is null , it only update the age of the metric ,
and then the loop continue , the shipEdit() method is still not called . could
you show me when entry is null , where is the logic to handle the old hlogs ?
Thank you .
> 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: 1.2.4, 1.4.9
> Reporter: leizhang
> Priority: Major
> Fix For: 2.1.0
>
>
> 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)