1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.regionserver;
21
22 import java.io.IOException;
23 import java.util.HashSet;
24 import java.util.concurrent.BlockingQueue;
25 import java.util.concurrent.LinkedBlockingQueue;
26 import java.util.concurrent.TimeUnit;
27 import java.util.concurrent.locks.ReentrantLock;
28
29 import org.apache.commons.logging.Log;
30 import org.apache.commons.logging.LogFactory;
31 import org.apache.hadoop.conf.Configuration;
32 import org.apache.hadoop.hbase.RemoteExceptionHandler;
33 import org.apache.hadoop.util.StringUtils;
34
35
36
37
38 class CompactSplitThread extends Thread {
39 static final Log LOG = LogFactory.getLog(CompactSplitThread.class);
40
41 private final long frequency;
42 private final ReentrantLock lock = new ReentrantLock();
43
44 private final HRegionServer server;
45 private final Configuration conf;
46
47 private final BlockingQueue<HRegion> compactionQueue =
48 new LinkedBlockingQueue<HRegion>();
49
50 private final HashSet<HRegion> regionsInQueue = new HashSet<HRegion>();
51
52
53
54
55
56
57 private int regionSplitLimit;
58
59
60 public CompactSplitThread(HRegionServer server) {
61 super();
62 this.server = server;
63 this.conf = server.conf;
64 this.regionSplitLimit = conf.getInt("hbase.regionserver.regionSplitLimit",
65 Integer.MAX_VALUE);
66 this.frequency =
67 conf.getLong("hbase.regionserver.thread.splitcompactcheckfrequency",
68 20 * 1000);
69 }
70
71 @Override
72 public void run() {
73 while (!this.server.isStopRequested()) {
74 HRegion r = null;
75 try {
76 r = compactionQueue.poll(this.frequency, TimeUnit.MILLISECONDS);
77 if (r != null && !this.server.isStopRequested()) {
78 synchronized (regionsInQueue) {
79 regionsInQueue.remove(r);
80 }
81 lock.lock();
82 try {
83
84 byte [] midKey = r.compactStores();
85 if (shouldSplitRegion() && midKey != null &&
86 !this.server.isStopRequested()) {
87 split(r, midKey);
88 }
89 } finally {
90 lock.unlock();
91 }
92 }
93 } catch (InterruptedException ex) {
94 continue;
95 } catch (IOException ex) {
96 LOG.error("Compaction/Split failed for region " +
97 r.getRegionNameAsString(),
98 RemoteExceptionHandler.checkIOException(ex));
99 if (!server.checkFileSystem()) {
100 break;
101 }
102 } catch (Exception ex) {
103 LOG.error("Compaction failed" +
104 (r != null ? (" for region " + r.getRegionNameAsString()) : ""),
105 ex);
106 if (!server.checkFileSystem()) {
107 break;
108 }
109 }
110 }
111 regionsInQueue.clear();
112 compactionQueue.clear();
113 LOG.info(getName() + " exiting");
114 }
115
116
117
118
119
120 public synchronized void compactionRequested(final HRegion r,
121 final String why) {
122 compactionRequested(r, false, why);
123 }
124
125
126
127
128
129
130 public synchronized void compactionRequested(final HRegion r,
131 final boolean force, final String why) {
132 if (this.server.stopRequested.get()) {
133 return;
134 }
135 r.setForceMajorCompaction(force);
136 if (LOG.isDebugEnabled()) {
137 LOG.debug("Compaction " + (force? "(major) ": "") +
138 "requested for region " + r.getRegionNameAsString() +
139 (why != null && !why.isEmpty()? " because: " + why: ""));
140 }
141 synchronized (regionsInQueue) {
142 if (!regionsInQueue.contains(r)) {
143 compactionQueue.add(r);
144 regionsInQueue.add(r);
145 }
146 }
147 }
148
149 private void split(final HRegion parent, final byte [] midKey)
150 throws IOException {
151 final long startTime = System.currentTimeMillis();
152 SplitTransaction st = new SplitTransaction(parent, midKey);
153
154
155 if (!st.prepare()) return;
156 try {
157 st.execute(this.server);
158 } catch (IOException ioe) {
159 try {
160 LOG.info("Running rollback of failed split of " +
161 parent.getRegionNameAsString() + "; " + ioe.getMessage());
162 st.rollback(this.server);
163 LOG.info("Successful rollback of failed split of " +
164 parent.getRegionNameAsString());
165 } catch (RuntimeException e) {
166
167 LOG.info("Failed rollback of failed split of " +
168 parent.getRegionNameAsString() + " -- aborting server", e);
169 this.server.abort("Failed split");
170 }
171 return;
172 }
173
174
175
176
177 this.server.reportSplit(parent.getRegionInfo(), st.getFirstDaughter(),
178 st.getSecondDaughter());
179 LOG.info("Region split, META updated, and report to master. Parent=" +
180 parent.getRegionInfo() + ", new regions: " +
181 st.getFirstDaughter() + ", " + st.getSecondDaughter() + ". Split took " +
182 StringUtils.formatTimeDiff(System.currentTimeMillis(), startTime));
183 }
184
185
186
187
188 void interruptIfNecessary() {
189 if (lock.tryLock()) {
190 this.interrupt();
191 }
192 }
193
194
195
196
197
198
199
200 public int getCompactionQueueSize() {
201 return compactionQueue.size();
202 }
203
204 private boolean shouldSplitRegion() {
205 return (regionSplitLimit > server.getNumberOfOnlineRegions());
206 }
207
208
209
210
211 public int getRegionSplitLimit() {
212 return this.regionSplitLimit;
213 }
214 }