View Javadoc

1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
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   * Thread that flushes cache on request
46   *
47   * NOTE: This class extends Thread rather than Chore because the sleep time
48   * can be interrupted when there is something to do, rather than the Chore
49   * sleep time which is invariant.
50   *
51   * @see FlushRequester
52   */
53  class MemStoreFlusher extends Thread implements FlushRequester {
54    static final Log LOG = LogFactory.getLog(MemStoreFlusher.class);
55    // These two data members go together.  Any entry in the one must have
56    // a corresponding entry in the other.
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     * @param conf
80     * @param server
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    * Calculate size using passed <code>key</code> for configured
115    * percentage of <code>max</code>.
116    * @param max
117    * @param defaultLimit
118    * @param key
119    * @param c
120    * @return Limit.
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         // This entry has no delay so it will be added at the top of the flush
171         // queue.  It'll come out near immediately.
172         FlushQueueEntry fqe = new FlushQueueEntry(r);
173         this.regionsInQueue.put(r, fqe);
174         this.flushQueue.add(fqe);
175       }
176     }
177   }
178 
179   /**
180    * Only interrupt once it's done with a run through the work loop.
181    */
182   void interruptIfNecessary() {
183     lock.lock();
184     try {
185       this.interrupt();
186     } finally {
187       lock.unlock();
188     }
189   }
190 
191   /*
192    * A flushRegion that checks store file count.  If too many, puts the flush
193    * on delay queue to retry later.
194    * @param fqe
195    * @return true if the region was successfully flushed, false otherwise. If 
196    * false, there will be accompanying log messages explaining why the log was
197    * not flushed.
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         // If this is first time we've been put off, then emit a log message.
210         if (fqe.getRequeueCount() <= 0) {
211           // Note: We don't impose blockingStoreFiles constraint on meta regions
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         // Put back on the queue.  Have it come back out of the queue
217         // after a delay of this.blockingWaitTime / 100 ms.
218         this.flushQueue.add(fqe.requeue(this.blockingWaitTime / 100));
219         // Tell a lie, it's not flushed but it's ok
220         return true;
221       }
222     }
223     return flushRegion(region, false);
224   }
225 
226   /*
227    * Flush a region.
228    * @param region Region to flush.
229    * @param emergencyFlush Set if we are being force flushed. If true the region
230    * needs to be removed from the flush queue. If false, when we were called
231    * from the main flusher run loop and we got the entry to flush by calling
232    * poll on the flush queue (which removed it).
233    *
234    * @return true if the region was successfully flushed, false otherwise. If
235    * false, there will be accompanying log messages explaining why the log was
236    * not flushed.
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         // Need to remove from region from delay queue.  When NOT an
243         // emergencyFlush, then item was removed via a flushQueue.poll.
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       // Cache flush can fail in a few places. If it fails in a critical
254       // section, we get a DroppedSnapshotException and a replay of hlog
255       // is required. Currently the only way to do this is a restart of
256       // the server. Abort because hdfs is probably bad (HBASE-644 is a case
257       // where hdfs was bad but passed the hdfs check).
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    * Check if the regionserver's memstore memory usage is greater than the
284    * limit. If so, flush regions with the biggest memstores until we're down
285    * to the lower limit. This method blocks callers until we're down to a safe
286    * amount of memstore consumption.
287    */
288   public synchronized void reclaimMemStoreMemory() {
289     if (server.getGlobalMemStoreSize() >= globalMemStoreLimit) {
290       flushSomeRegions();
291     }
292   }
293 
294   /*
295    * Emergency!  Need to flush memory.
296    */
297   private synchronized void flushSomeRegions() {
298     // keep flushing until we hit the low water mark
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       // flush the region with the biggest memstore
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    * Datastructure used in the flush queue.  Holds region and retry count.
334    * Keeps tabs on how old this object is.  Implements {@link Delayed}.  On
335    * construction, the delay is zero. When added to a delay queue, we'll come
336    * out near immediately.  Call {@link #requeue(long)} passing delay in
337    * milliseconds before readding to delay queue if you want it to stay there
338    * a while.
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      * @param maximumWait
354      * @return True if we have been delayed > <code>maximumWait</code> milliseconds.
355      */
356     public boolean isMaximumWait(final long maximumWait) {
357       return (System.currentTimeMillis() - this.createTime) > maximumWait;
358     }
359 
360     /**
361      * @return Count of times {@link #resetDelay()} was called; i.e this is
362      * number of times we've been requeued.
363      */
364     public int getRequeueCount() {
365       return this.requeueCount;
366     }
367  
368     /**
369      * @param when When to expire, when to come up out of the queue.
370      * Specify in milliseconds.  This method adds System.currentTimeMillis()
371      * to whatever you pass.
372      * @return This.
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 }