[ 
https://issues.apache.org/jira/browse/HBASE-22620?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16870944#comment-16870944
 ] 

leizhang commented on HBASE-22620:
----------------------------------

  I view the code that create the entryBatch,just as follows :
{code:java}
ReplicationSourceWALReaderThread.class -> run() -> entryStream.hasNext() -> 
tryAdvanceEntry() ->checkReader()->openNextLog()->readNextEntryAndSetPosition()
{code}
  when reaching the end of a wal file ,it will switch to the next hlog, so the 
current log position update logic is correct,and all the hlog are available by 
replication source endpoint and switch correctly no matter whether we have 
entry to replicate,  but the zk refs cleaning up job may be blocked due to the  
blocking queue's take()  method. 

  Imagine a cluster configured replication(set hbase.replication to true) but 
all tables don't open the replication property, all the log of the cluster will 
be keep.  

> 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)

Reply via email to