[
https://issues.apache.org/jira/browse/HBASE-22784?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16903710#comment-16903710
]
Wellington Chevreuil commented on HBASE-22784:
----------------------------------------------
Hi [~solvannan],
Thanks for your comments. Yeah, please consider only the last 3rd patch, first
two still have some issues. Third patch has some checkstyle issues, which I
will address shortly, but is functionally correct.
{quote}If the WAL reader thread does not have any entry batch (after passing
through all the filters) after some configured time threshold, it can queue an
empty batch, with the last read log position, to the entryBatchQueue. Now the
ReplicationSourceShipperThread will read this empty batch and update it's
position and invoke cleanup logic.
{quote}
I think this would also work. I'm not so fond of adding/creating extra objects
for hacking this lack of communication between two threads. There would still
be a need for some similar checks on wether a new _empty_ entry is really
needed, if the shipper thread is stuck to deliver a given entry to target, i.e.
target is down. Because of its sequential nature, reader will keep trying on
this current entry. Meanwhile, if reader thread relies only on timeouts, it may
end up creating and enqueuing unneeded _empty_ entries.
> 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
> Assignee: Wellington Chevreuil
> Priority: Blocker
> Fix For: 1.5.0
>
> Attachments: HBASE-22784.branch-1.001.patch,
> HBASE-22784.branch-1.002.patch, HBASE-22784.branch-1.003.patch
>
>
> 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)