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 org.apache.commons.logging.Log;
23 import org.apache.commons.logging.LogFactory;
24 import org.apache.hadoop.conf.Configuration;
25 import org.apache.hadoop.hbase.DroppedSnapshotException;
26 import org.apache.hadoop.hbase.HConstants;
27 import org.apache.hadoop.hbase.RemoteExceptionHandler;
28 import org.apache.hadoop.hbase.util.Bytes;
29 import org.apache.hadoop.util.StringUtils;
30
31 import java.io.IOException;
32 import java.lang.management.ManagementFactory;
33 import java.util.ArrayList;
34 import java.util.ConcurrentModificationException;
35 import java.util.HashMap;
36 import java.util.Map;
37 import java.util.SortedMap;
38 import java.util.concurrent.BlockingQueue;
39 import java.util.concurrent.DelayQueue;
40 import java.util.concurrent.Delayed;
41 import java.util.concurrent.TimeUnit;
42 import java.util.concurrent.locks.ReentrantLock;
43
44
45
46
47
48
49
50
51
52
53 class MemStoreFlusher extends Thread implements FlushRequester {
54 static final Log LOG = LogFactory.getLog(MemStoreFlusher.class);
55
56
57 private final BlockingQueue<FlushQueueEntry> flushQueue =
58 new DelayQueue<FlushQueueEntry>();
59 private final Map<HRegion, FlushQueueEntry> regionsInQueue =
60 new HashMap<HRegion, FlushQueueEntry>();
61
62 private final long threadWakeFrequency;
63 private final HRegionServer server;
64 private final ReentrantLock lock = new ReentrantLock();
65
66 protected final long globalMemStoreLimit;
67 protected final long globalMemStoreLimitLowMark;
68
69 private static final float DEFAULT_UPPER = 0.4f;
70 private static final float DEFAULT_LOWER = 0.25f;
71 private static final String UPPER_KEY =
72 "hbase.regionserver.global.memstore.upperLimit";
73 private static final String LOWER_KEY =
74 "hbase.regionserver.global.memstore.lowerLimit";
75 private long blockingStoreFilesNumber;
76 private long blockingWaitTime;
77
78
79
80
81
82 public MemStoreFlusher(final Configuration conf,
83 final HRegionServer server) {
84 super();
85 this.server = server;
86 this.threadWakeFrequency =
87 conf.getLong(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000);
88 long max = ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getMax();
89 this.globalMemStoreLimit = globalMemStoreLimit(max, DEFAULT_UPPER,
90 UPPER_KEY, conf);
91 long lower = globalMemStoreLimit(max, DEFAULT_LOWER, LOWER_KEY, conf);
92 if (lower > this.globalMemStoreLimit) {
93 lower = this.globalMemStoreLimit;
94 LOG.info("Setting globalMemStoreLimitLowMark == globalMemStoreLimit " +
95 "because supplied " + LOWER_KEY + " was > " + UPPER_KEY);
96 }
97 this.globalMemStoreLimitLowMark = lower;
98 this.blockingStoreFilesNumber =
99 conf.getInt("hbase.hstore.blockingStoreFiles", -1);
100 if (this.blockingStoreFilesNumber == -1) {
101 this.blockingStoreFilesNumber = 1 +
102 conf.getInt("hbase.hstore.compactionThreshold", 3);
103 }
104 this.blockingWaitTime = conf.getInt("hbase.hstore.blockingWaitTime",
105 90000);
106 LOG.info("globalMemStoreLimit=" +
107 StringUtils.humanReadableInt(this.globalMemStoreLimit) +
108 ", globalMemStoreLimitLowMark=" +
109 StringUtils.humanReadableInt(this.globalMemStoreLimitLowMark) +
110 ", maxHeap=" + StringUtils.humanReadableInt(max));
111 }
112
113
114
115
116
117
118
119
120
121
122 static long globalMemStoreLimit(final long max,
123 final float defaultLimit, final String key, final Configuration c) {
124 float limit = c.getFloat(key, defaultLimit);
125 return getMemStoreLimit(max, limit, defaultLimit);
126 }
127
128 static long getMemStoreLimit(final long max, final float limit,
129 final float defaultLimit) {
130 if (limit >= 0.9f || limit < 0.1f) {
131 LOG.warn("Setting global memstore limit to default of " + defaultLimit +
132 " because supplied value outside allowed range of 0.1 -> 0.9");
133 }
134 return (long)(max * limit);
135 }
136
137 @Override
138 public void run() {
139 while (!this.server.isStopRequested()) {
140 FlushQueueEntry fqe = null;
141 try {
142 fqe = flushQueue.poll(threadWakeFrequency, TimeUnit.MILLISECONDS);
143 if (fqe == null) {
144 continue;
145 }
146 if (!flushRegion(fqe)) {
147 break;
148 }
149 } catch (InterruptedException ex) {
150 continue;
151 } catch (ConcurrentModificationException ex) {
152 continue;
153 } catch (Exception ex) {
154 LOG.error("Cache flush failed" +
155 (fqe != null ? (" for region " + Bytes.toString(fqe.region.getRegionName())) : ""),
156 ex);
157 if (!server.checkFileSystem()) {
158 break;
159 }
160 }
161 }
162 this.regionsInQueue.clear();
163 this.flushQueue.clear();
164 LOG.info(getName() + " exiting");
165 }
166
167 public void request(HRegion r) {
168 synchronized (regionsInQueue) {
169 if (!regionsInQueue.containsKey(r)) {
170
171
172 FlushQueueEntry fqe = new FlushQueueEntry(r);
173 this.regionsInQueue.put(r, fqe);
174 this.flushQueue.add(fqe);
175 }
176 }
177 }
178
179
180
181
182 void interruptIfNecessary() {
183 lock.lock();
184 try {
185 this.interrupt();
186 } finally {
187 lock.unlock();
188 }
189 }
190
191
192
193
194
195
196
197
198
199 private boolean flushRegion(final FlushQueueEntry fqe) {
200 HRegion region = fqe.region;
201 if (!fqe.region.getRegionInfo().isMetaRegion() &&
202 isTooManyStoreFiles(region)) {
203 if (fqe.isMaximumWait(this.blockingWaitTime)) {
204 LOG.info("Waited " + (System.currentTimeMillis() - fqe.createTime) +
205 "ms on a compaction to clean up 'too many store files'; waited " +
206 "long enough... proceeding with flush of " +
207 region.getRegionNameAsString());
208 } else {
209
210 if (fqe.getRequeueCount() <= 0) {
211
212 LOG.warn("Region " + region.getRegionNameAsString() + " has too many " +
213 "store files; delaying flush up to " + this.blockingWaitTime + "ms");
214 }
215 this.server.compactSplitThread.compactionRequested(region, getName());
216
217
218 this.flushQueue.add(fqe.requeue(this.blockingWaitTime / 100));
219
220 return true;
221 }
222 }
223 return flushRegion(region, false);
224 }
225
226
227
228
229
230
231
232
233
234
235
236
237
238 private boolean flushRegion(final HRegion region, final boolean emergencyFlush) {
239 synchronized (this.regionsInQueue) {
240 FlushQueueEntry fqe = this.regionsInQueue.remove(region);
241 if (fqe != null && emergencyFlush) {
242
243
244 flushQueue.remove(fqe);
245 }
246 lock.lock();
247 }
248 try {
249 if (region.flushcache()) {
250 server.compactSplitThread.compactionRequested(region, getName());
251 }
252 } catch (DroppedSnapshotException ex) {
253
254
255
256
257
258 server.abort("Replay of HLog required. Forcing server shutdown", ex);
259 return false;
260 } catch (IOException ex) {
261 LOG.error("Cache flush failed" +
262 (region != null ? (" for region " + Bytes.toString(region.getRegionName())) : ""),
263 RemoteExceptionHandler.checkIOException(ex));
264 if (!server.checkFileSystem()) {
265 return false;
266 }
267 } finally {
268 lock.unlock();
269 }
270 return true;
271 }
272
273 private boolean isTooManyStoreFiles(HRegion region) {
274 for (Store hstore: region.stores.values()) {
275 if (hstore.getStorefilesCount() > this.blockingStoreFilesNumber) {
276 return true;
277 }
278 }
279 return false;
280 }
281
282
283
284
285
286
287
288 public synchronized void reclaimMemStoreMemory() {
289 if (server.getGlobalMemStoreSize() >= globalMemStoreLimit) {
290 flushSomeRegions();
291 }
292 }
293
294
295
296
297 private synchronized void flushSomeRegions() {
298
299 long globalMemStoreSize = -1;
300 ArrayList<HRegion> regionsToCompact = new ArrayList<HRegion>();
301 for (SortedMap<Long, HRegion> m =
302 this.server.getCopyOfOnlineRegionsSortedBySize();
303 (globalMemStoreSize = server.getGlobalMemStoreSize()) >=
304 this.globalMemStoreLimitLowMark;) {
305
306 if (m.size() <= 0) {
307 LOG.info("No online regions to flush though we've been asked flush " +
308 "some; globalMemStoreSize=" +
309 StringUtils.humanReadableInt(globalMemStoreSize) +
310 ", globalMemStoreLimitLowMark=" +
311 StringUtils.humanReadableInt(this.globalMemStoreLimitLowMark));
312 break;
313 }
314 HRegion biggestMemStoreRegion = m.remove(m.firstKey());
315 LOG.info("Forced flushing of " + biggestMemStoreRegion.toString() +
316 " because global memstore limit of " +
317 StringUtils.humanReadableInt(this.globalMemStoreLimit) +
318 " exceeded; currently " +
319 StringUtils.humanReadableInt(globalMemStoreSize) + " and flushing till " +
320 StringUtils.humanReadableInt(this.globalMemStoreLimitLowMark));
321 if (!flushRegion(biggestMemStoreRegion, true)) {
322 LOG.warn("Flush failed");
323 break;
324 }
325 regionsToCompact.add(biggestMemStoreRegion);
326 }
327 for (HRegion region : regionsToCompact) {
328 server.compactSplitThread.compactionRequested(region, getName());
329 }
330 }
331
332
333
334
335
336
337
338
339
340 static class FlushQueueEntry implements Delayed {
341 private final HRegion region;
342 private final long createTime;
343 private long whenToExpire;
344 private int requeueCount = 0;
345
346 FlushQueueEntry(final HRegion r) {
347 this.region = r;
348 this.createTime = System.currentTimeMillis();
349 this.whenToExpire = this.createTime;
350 }
351
352
353
354
355
356 public boolean isMaximumWait(final long maximumWait) {
357 return (System.currentTimeMillis() - this.createTime) > maximumWait;
358 }
359
360
361
362
363
364 public int getRequeueCount() {
365 return this.requeueCount;
366 }
367
368
369
370
371
372
373
374 public FlushQueueEntry requeue(final long when) {
375 this.whenToExpire = System.currentTimeMillis() + when;
376 this.requeueCount++;
377 return this;
378 }
379
380 @Override
381 public long getDelay(TimeUnit unit) {
382 return unit.convert(this.whenToExpire - System.currentTimeMillis(),
383 TimeUnit.MILLISECONDS);
384 }
385
386 @Override
387 public int compareTo(Delayed other) {
388 return Long.valueOf(getDelay(TimeUnit.MILLISECONDS) -
389 other.getDelay(TimeUnit.MILLISECONDS)).intValue();
390 }
391 }
392 }