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 java.util.concurrent.Executors;
23  import java.util.concurrent.PriorityBlockingQueue;
24  import java.util.concurrent.RejectedExecutionException;
25  import java.util.concurrent.ThreadFactory;
26  import java.util.concurrent.ThreadPoolExecutor;
27  import java.util.concurrent.TimeUnit;
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.HConstants;
33  import org.apache.hadoop.hbase.HTableDescriptor;
34  import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
35  
36  import com.google.common.base.Preconditions;
37  
38  /**
39   * Compact region on request and then run split if appropriate
40   */
41  public class CompactSplitThread implements CompactionRequestor {
42    static final Log LOG = LogFactory.getLog(CompactSplitThread.class);
43  
44    private final HRegionServer server;
45    private final Configuration conf;
46  
47    private final ThreadPoolExecutor largeCompactions;
48    private final ThreadPoolExecutor smallCompactions;
49    private final ThreadPoolExecutor splits;
50    private final long throttleSize;
51  
52    /* The default priority for user-specified compaction requests.
53     * The user gets top priority unless we have blocking compactions. (Pri <= 0)
54     */
55    public static final int PRIORITY_USER = 1;
56    public static final int NO_PRIORITY = Integer.MIN_VALUE;
57  
58    /**
59     * Splitting should not take place if the total number of regions exceed this.
60     * This is not a hard limit to the number of regions but it is a guideline to
61     * stop splitting after number of online regions is greater than this.
62     */
63    private int regionSplitLimit;
64    
65    /** @param server */
66    CompactSplitThread(HRegionServer server) {
67      super();
68      this.server = server;
69      this.conf = server.getConfiguration();
70      this.regionSplitLimit = conf.getInt("hbase.regionserver.regionSplitLimit",
71          Integer.MAX_VALUE);
72  
73      int largeThreads = Math.max(1, conf.getInt(
74          "hbase.regionserver.thread.compaction.large", 1));
75      int smallThreads = conf.getInt(
76          "hbase.regionserver.thread.compaction.small", 1);
77      if (conf.get("hbase.regionserver.thread.compaction.throttle") != null) {
78        throttleSize = conf.getLong(
79            "hbase.regionserver.thread.compaction.throttle", 0);
80      } else {
81        // we have a complicated default. see HBASE-3877
82        long flushSize = conf.getLong("hbase.hregion.memstore.flush.size",
83            HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE);
84        long splitSize = conf.getLong("hbase.hregion.max.filesize",
85            HConstants.DEFAULT_MAX_FILE_SIZE);
86        throttleSize = Math.min(flushSize * 2, splitSize / 2);
87      }
88  
89      int splitThreads = conf.getInt("hbase.regionserver.thread.split", 1);
90  
91      // if we have throttle threads, make sure the user also specified size
92      Preconditions.checkArgument(smallThreads == 0 || throttleSize > 0);
93  
94      final String n = Thread.currentThread().getName();
95  
96      this.largeCompactions = new ThreadPoolExecutor(largeThreads, largeThreads,
97          60, TimeUnit.SECONDS, new PriorityBlockingQueue<Runnable>(), 
98          new ThreadFactory() {
99            @Override
100           public Thread newThread(Runnable r) {
101             Thread t = new Thread(r);
102             t.setName(n + "-largeCompactions-" + System.currentTimeMillis());
103             return t;
104           }
105       });
106     this.largeCompactions
107         .setRejectedExecutionHandler(new CompactionRequest.Rejection());
108     if (smallThreads <= 0) {
109       this.smallCompactions = null;
110     } else {
111       this.smallCompactions = new ThreadPoolExecutor(smallThreads, smallThreads,
112           60, TimeUnit.SECONDS, new PriorityBlockingQueue<Runnable>(),
113           new ThreadFactory() {
114             @Override
115             public Thread newThread(Runnable r) {
116               Thread t = new Thread(r);
117               t.setName(n + "-smallCompactions-" + System.currentTimeMillis());
118               return t;
119             }
120         });
121       this.smallCompactions
122           .setRejectedExecutionHandler(new CompactionRequest.Rejection());
123     }
124     this.splits = (ThreadPoolExecutor)
125         Executors.newFixedThreadPool(splitThreads,
126             new ThreadFactory() {
127           @Override
128           public Thread newThread(Runnable r) {
129             Thread t = new Thread(r);
130             t.setName(n + "-splits-" + System.currentTimeMillis());
131             return t;
132           }
133       });
134   }
135 
136   @Override
137   public String toString() {
138     return "compaction_queue="
139         + (smallCompactions != null ? "("
140             + largeCompactions.getQueue().size() + ":"
141             + smallCompactions.getQueue().size() + ")"
142             : largeCompactions.getQueue().size())
143         + ", split_queue=" + splits.getQueue().size();
144   }
145 
146   public synchronized boolean requestSplit(final HRegion r) {
147     // don't split regions that are blocking
148     if (shouldSplitRegion() && r.getCompactPriority() >= PRIORITY_USER) {
149       byte[] midKey = r.checkSplit();
150       if (midKey != null) {
151         requestSplit(r, midKey);
152         return true;
153       }
154     }
155     return false;
156   }
157 
158   public synchronized void requestSplit(final HRegion r, byte[] midKey) {
159     if (midKey == null) {
160       LOG.debug("Region " + r.getRegionNameAsString() +
161         " not splittable because midkey=null");
162       return;
163     }
164     try {
165       this.splits.execute(new SplitRequest(r, midKey, this.server));
166       if (LOG.isDebugEnabled()) {
167         LOG.debug("Split requested for " + r + ".  " + this);
168       }
169     } catch (RejectedExecutionException ree) {
170       LOG.info("Could not execute split for " + r, ree);
171     }
172   }
173 
174   public synchronized void requestCompaction(final HRegion r,
175       final String why) {
176     for(Store s : r.getStores().values()) {
177       requestCompaction(r, s, why, NO_PRIORITY);
178     }
179   }
180 
181   public synchronized void requestCompaction(final HRegion r, final Store s,
182       final String why) {
183     requestCompaction(r, s, why, NO_PRIORITY);
184   }
185 
186   public synchronized void requestCompaction(final HRegion r, final String why,
187       int p) {
188     for(Store s : r.getStores().values()) {
189       requestCompaction(r, s, why, p);
190     }
191   }
192 
193   /**
194    * @param r HRegion store belongs to
195    * @param s Store to request compaction on
196    * @param why Why compaction requested -- used in debug messages
197    * @param priority override the default priority (NO_PRIORITY == decide)
198    */
199   public synchronized void requestCompaction(final HRegion r, final Store s,
200       final String why, int priority) {
201     if (this.server.isStopped()) {
202       return;
203     }
204     CompactionRequest cr = s.requestCompaction();
205     if (cr != null) {
206       cr.setServer(server);
207       if (priority != NO_PRIORITY) {
208         cr.setPriority(priority);
209       }
210       ThreadPoolExecutor pool = largeCompactions;
211       if (smallCompactions != null && throttleSize > cr.getSize()) {
212         // smallCompactions is like the 10 items or less line at Walmart
213         pool = smallCompactions;
214       }
215       pool.execute(cr);
216       if (LOG.isDebugEnabled()) {
217         String type = "";
218         if (smallCompactions != null) {
219           type = (pool == smallCompactions) ? "Small " : "Large ";
220         }
221         LOG.debug(type + "Compaction requested: " + cr
222             + (why != null && !why.isEmpty() ? "; Because: " + why : "")
223             + "; " + this);
224       }
225     }
226   }
227 
228   /**
229    * Only interrupt once it's done with a run through the work loop.
230    */
231   void interruptIfNecessary() {
232     splits.shutdown();
233     largeCompactions.shutdown();
234     if (smallCompactions != null)
235       smallCompactions.shutdown();
236   }
237 
238   private void waitFor(ThreadPoolExecutor t, String name) {
239     boolean done = false;
240     while (!done) {
241       try {
242         done = t.awaitTermination(60, TimeUnit.SECONDS);
243         LOG.debug("Waiting for " + name + " to finish...");
244       } catch (InterruptedException ie) {
245         LOG.debug("Interrupted waiting for " + name + " to finish...");
246       }
247     }
248   }
249 
250   void join() {
251     waitFor(splits, "Split Thread");
252     waitFor(largeCompactions, "Large Compaction Thread");
253     if (smallCompactions != null) {
254       waitFor(smallCompactions, "Small Compaction Thread");
255     }
256   }
257 
258   /**
259    * Returns the current size of the queue containing regions that are
260    * processed.
261    *
262    * @return The current size of the regions queue.
263    */
264   public int getCompactionQueueSize() {
265     int size = largeCompactions.getQueue().size();
266     if (smallCompactions != null)
267       size += smallCompactions.getQueue().size();
268     return size;
269   }
270 
271   private boolean shouldSplitRegion() {
272     return (regionSplitLimit > server.getNumberOfOnlineRegions());
273   }
274 
275   /**
276    * @return the regionSplitLimit
277    */
278   public int getRegionSplitLimit() {
279     return this.regionSplitLimit;
280   }
281 }