1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.wal;
20
21 import java.io.IOException;
22 import java.util.List;
23 import java.util.concurrent.atomic.AtomicInteger;
24
25 import org.apache.commons.logging.Log;
26 import org.apache.commons.logging.LogFactory;
27 import org.apache.hadoop.hbase.classification.InterfaceAudience;
28 import org.apache.hadoop.conf.Configuration;
29
30
31 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
32
33
34
35
36
37
38
39 @InterfaceAudience.Private
40 class BoundedRegionGroupingProvider extends RegionGroupingProvider {
41 private static final Log LOG = LogFactory.getLog(BoundedRegionGroupingProvider.class);
42
43 static final String NUM_REGION_GROUPS = "hbase.wal.regiongrouping.numgroups";
44 static final int DEFAULT_NUM_REGION_GROUPS = 2;
45 private WALProvider[] delegates;
46 private AtomicInteger counter = new AtomicInteger(0);
47
48 @Override
49 public void init(final WALFactory factory, final Configuration conf,
50 final List<WALActionsListener> listeners, final String providerId) throws IOException {
51 super.init(factory, conf, listeners, providerId);
52
53 delegates = new WALProvider[Math.max(1, conf.getInt(NUM_REGION_GROUPS,
54 DEFAULT_NUM_REGION_GROUPS))];
55 for (int i = 0; i < delegates.length; i++) {
56 delegates[i] = factory.getProvider(DELEGATE_PROVIDER, DEFAULT_DELEGATE_PROVIDER, listeners,
57 providerId + i);
58 }
59 LOG.info("Configured to run with " + delegates.length + " delegate WAL providers.");
60 }
61
62 @Override
63 WALProvider populateCache(final byte[] group) {
64 final WALProvider temp = delegates[counter.getAndIncrement() % delegates.length];
65 final WALProvider extant = cached.putIfAbsent(group, temp);
66
67
68 return extant == null ? temp : extant;
69 }
70
71 @Override
72 public void shutdown() throws IOException {
73
74 IOException failure = null;
75 for (WALProvider provider : delegates) {
76 try {
77 provider.shutdown();
78 } catch (IOException exception) {
79 LOG.error("Problem shutting down provider '" + provider + "': " + exception.getMessage());
80 LOG.debug("Details of problem shutting down provider '" + provider + "'", exception);
81 failure = exception;
82 }
83 }
84 if (failure != null) {
85 throw failure;
86 }
87 }
88
89 @Override
90 public void close() throws IOException {
91
92 IOException failure = null;
93 for (WALProvider provider : delegates) {
94 try {
95 provider.close();
96 } catch (IOException exception) {
97 LOG.error("Problem closing provider '" + provider + "': " + exception.getMessage());
98 LOG.debug("Details of problem shutting down provider '" + provider + "'", exception);
99 failure = exception;
100 }
101 }
102 if (failure != null) {
103 throw failure;
104 }
105 }
106 }