[
https://issues.apache.org/jira/browse/HBASE-9736?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Jeffrey Zhong updated HBASE-9736:
---------------------------------
Attachment: hbase-9736.patch
h6. Motivations to allow more splitter per RS:
1) Make it possible to increase max number of WALs per RS. Normally the amount
of data contained in walls should match RS memstore size. Currently by default
we have 32
wals and each has about 128MB, therefore, it only allows to use up to 4GB
memory because a region server forces to flush memstore when number of wals
reaches max. Today a box with 32GB is normal, there should be a demand with
increasing the number of max wals per RS to fully utilize the available memory.
2) For a small cluster(< 30 nodes), allow multiple splitters per RS can address
not enough workers issue
3) For a fancy cluster(such as SSD cluster), allow multiple splitters per RS to
take full advantage of IO(random writes) capacity
Introducing "hbase.regionserver.wal.max.splitters" which allows a user to set a
hard upper bound on the concurrent wals to be processed on one single region
server.
During run time, each SplitLogWorker calculates the actual splitters needed =
MIN( outstanding tasks / available RSs, Max Splitters). Therefore, in a large
cluster(with 100 nodes), each RS will only spawn one splitter even
hbase.regionserver.wal.max.splitters > 1 to spread work items evenly across the
cluster.
The hbase.regionserver.wal.max.splitters is to prevent writes burst when there
are too many walls to be recovered from hurting the overall cluster health.
The patch use region server executor service to concurrently handle multiple
wals. It keeps single SplitLogWorker to maintain a single producer and ZK
listener to pull work items from ZK to easily scale up later. It also refactors
the TestSplitLogWorker by using mocked RS in order to remove the additional
constructor of SplitLogWorker which is only used by test code.
By default, the patch sets MAX splitters per RS = 2.
> Alow more than one log splitter per RS
> --------------------------------------
>
> Key: HBASE-9736
> URL: https://issues.apache.org/jira/browse/HBASE-9736
> Project: HBase
> Issue Type: Improvement
> Components: MTTR
> Reporter: stack
> Assignee: Jeffrey Zhong
> Priority: Critical
> Attachments: hbase-9736.patch
>
>
> IIRC, this is an idea that came from the lads at Xiaomi.
> I have a small cluster of 6 RSs and one went down. It had a few WALs. I see
> this in logs:
> 2013-10-09 05:47:27,890 DEBUG org.apache.hadoop.hbase.master.SplitLogManager:
> total tasks = 25 unassigned = 21
> WAL splitting is held up for want of slots out on the cluster to split WALs.
> We need to be careful we don't overwhelm the foreground regionservers but
> more splitters should help get all back online faster.
--
This message was sent by Atlassian JIRA
(v6.1#6144)