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

Wellington Chevreuil commented on HBASE-22784:
----------------------------------------------

Hi [~solvannan], thanks for the info. Yeah, those show this oldWALs 
accumulation is definitely something abnormal with replication. Some ideas to 
look for:

1) On the same jstack from the thread you showed on your previous comment, do 
you see any running thread with name containing 
_.replicationSource.replicationWALReaderThread._ substring? This is the thread 
responsible for reading the wal, so maybe it faced some unexpected condition 
and halted.

2) Does this issue persists even if this slave cluster RSes are restarted?

3) If so, would it be possible to set TRACE log level for the RSes, then look 
for the following message patters?

 
{noformat}
Didn't read any new entries from WAL{noformat}
Or
{noformat}
Failed to read stream of replication entries{noformat}
Or
{noformat}
Interrupted while sleeping between WAL reads{noformat}
 

On a side note, when you mention:
{quote}When a cluster is passive (receiving edits only via replication) in a 
cyclic replication setup of 2 clusters,
{quote}
It seems you don't really need cyclic replication, as your slave here only 
receives edits via replication and there are only 2 clusters.

> OldWALs not cleared in a replication slave cluster (cyclic replication bw 2 
> clusters)
> -------------------------------------------------------------------------------------
>
>                 Key: HBASE-22784
>                 URL: https://issues.apache.org/jira/browse/HBASE-22784
>             Project: HBase
>          Issue Type: Bug
>          Components: regionserver, Replication
>    Affects Versions: 1.4.9, 1.4.10
>            Reporter: Solvannan R M
>            Priority: Major
>
> When a cluster is passive (receiving edits only via replication) in a cyclic 
> replication setup of 2 clusters, OldWALs size keeps on growing. On analysing, 
> we observed the following behaviour.
>  # New entry is added to WAL (Edit replicated from other cluster).
>  # ReplicationSourceWALReaderThread(RSWALRT) reads and applies the configured 
> filters (due to cyclic replication setup, ClusterMarkingEntryFilter discards 
> new entry from other cluster).
>  # Entry is null, RSWALRT neither updates the batch stats 
> (WALEntryBatch.lastWalPosition) nor puts it in the entryBatchQueue.
>  # ReplicationSource thread is blocked in entryBachQueue.take().
>  # So ReplicationSource#updateLogPosition has never invoked and WAL file is 
> never cleared from ReplicationQueue.
>  # Hence LogCleaner on the master, doesn't deletes the oldWAL files from 
> hadoop.
> NOTE: When a new edit is added via hbase-client, ReplicationSource thread 
> process and clears the oldWAL files from replication queues and hence master 
> cleans up the WALs
> Please provide us a solution
>  



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)

Reply via email to