View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.regionserver.snapshot;
19  
20  import java.io.IOException;
21  import java.util.ArrayList;
22  import java.util.Collection;
23  import java.util.List;
24  import java.util.concurrent.Callable;
25  import java.util.concurrent.ExecutionException;
26  import java.util.concurrent.ExecutorCompletionService;
27  import java.util.concurrent.Future;
28  import java.util.concurrent.LinkedBlockingQueue;
29  import java.util.concurrent.ThreadPoolExecutor;
30  import java.util.concurrent.TimeUnit;
31  
32  import org.apache.commons.logging.Log;
33  import org.apache.commons.logging.LogFactory;
34  import org.apache.hadoop.classification.InterfaceAudience;
35  import org.apache.hadoop.classification.InterfaceStability;
36  import org.apache.hadoop.conf.Configuration;
37  import org.apache.hadoop.hbase.DaemonThreadFactory;
38  import org.apache.hadoop.hbase.errorhandling.ForeignException;
39  import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
40  import org.apache.hadoop.hbase.master.snapshot.MasterSnapshotVerifier;
41  import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
42  import org.apache.hadoop.hbase.procedure.ProcedureMember;
43  import org.apache.hadoop.hbase.procedure.ProcedureMemberRpcs;
44  import org.apache.hadoop.hbase.procedure.Subprocedure;
45  import org.apache.hadoop.hbase.procedure.SubprocedureFactory;
46  import org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs;
47  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
48  import org.apache.hadoop.hbase.regionserver.HRegion;
49  import org.apache.hadoop.hbase.regionserver.HRegionServer;
50  import org.apache.hadoop.hbase.regionserver.RegionServerServices;
51  import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
52  import org.apache.hadoop.hbase.util.Bytes;
53  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
54  import org.apache.zookeeper.KeeperException;
55  
56  import com.google.protobuf.InvalidProtocolBufferException;
57  
58  /**
59   * This manager class handles the work dealing with snapshots for a {@link HRegionServer}.
60   * <p>
61   * This provides the mechanism necessary to kick off a online snapshot specific
62   * {@link Subprocedure} that is responsible for the regions being served by this region server.
63   * If any failures occur with the subprocedure, the RegionSeverSnapshotManager's subprocedure
64   * handler, {@link ProcedureMember}, notifies the master's ProcedureCoordinator to abort all
65   * others.
66   * <p>
67   * On startup, requires {@link #start()} to be called.
68   * <p>
69   * On shutdown, requires {@link #stop(boolean)} to be called
70   */
71  @InterfaceAudience.Private
72  @InterfaceStability.Unstable
73  public class RegionServerSnapshotManager {
74    private static final Log LOG = LogFactory.getLog(RegionServerSnapshotManager.class);
75  
76    /** Maximum number of snapshot region tasks that can run concurrently */
77    private static final String CONCURENT_SNAPSHOT_TASKS_KEY = "hbase.snapshot.region.concurrentTasks";
78    private static final int DEFAULT_CONCURRENT_SNAPSHOT_TASKS = 3;
79  
80    /** Conf key for number of request threads to start snapshots on regionservers */
81    public static final String SNAPSHOT_REQUEST_THREADS_KEY = "hbase.snapshot.region.pool.threads";
82    /** # of threads for snapshotting regions on the rs. */
83    public static final int SNAPSHOT_REQUEST_THREADS_DEFAULT = 10;
84  
85    /** Conf key for max time to keep threads in snapshot request pool waiting */
86    public static final String SNAPSHOT_TIMEOUT_MILLIS_KEY = "hbase.snapshot.region.timeout";
87    /** Keep threads alive in request pool for max of 60 seconds */
88    public static final long SNAPSHOT_TIMEOUT_MILLIS_DEFAULT = 60000;
89  
90    /** Conf key for millis between checks to see if snapshot completed or if there are errors*/
91    public static final String SNAPSHOT_REQUEST_WAKE_MILLIS_KEY = "hbase.snapshot.region.wakefrequency";
92    /** Default amount of time to check for errors while regions finish snapshotting */
93    private static final long SNAPSHOT_REQUEST_WAKE_MILLIS_DEFAULT = 500;
94  
95    private final RegionServerServices rss;
96    private final ProcedureMemberRpcs memberRpcs;
97    private final ProcedureMember member;
98  
99    /**
100    * Exposed for testing.
101    * @param conf HBase configuration.
102    * @param parent parent running the snapshot handler
103    * @param memberRpc use specified memberRpc instance
104    * @param procMember use specified ProcedureMember
105    */
106    RegionServerSnapshotManager(Configuration conf, HRegionServer parent,
107       ProcedureMemberRpcs memberRpc, ProcedureMember procMember) {
108     this.rss = parent;
109     this.memberRpcs = memberRpc;
110     this.member = procMember;
111   }
112 
113   /**
114    * Create a default snapshot handler - uses a zookeeper based member controller.
115    * @param rss region server running the handler
116    * @throws KeeperException if the zookeeper cluster cannot be reached
117    */
118   public RegionServerSnapshotManager(RegionServerServices rss)
119       throws KeeperException {
120     this.rss = rss;
121     ZooKeeperWatcher zkw = rss.getZooKeeper();
122     this.memberRpcs = new ZKProcedureMemberRpcs(zkw,
123         SnapshotManager.ONLINE_SNAPSHOT_CONTROLLER_DESCRIPTION);
124 
125     // read in the snapshot request configuration properties
126     Configuration conf = rss.getConfiguration();
127     long keepAlive = conf.getLong(SNAPSHOT_TIMEOUT_MILLIS_KEY, SNAPSHOT_TIMEOUT_MILLIS_DEFAULT);
128     int opThreads = conf.getInt(SNAPSHOT_REQUEST_THREADS_KEY, SNAPSHOT_REQUEST_THREADS_DEFAULT);
129 
130     // create the actual snapshot procedure member
131     ThreadPoolExecutor pool = ProcedureMember.defaultPool(rss.getServerName().toString(),
132       opThreads, keepAlive);
133     this.member = new ProcedureMember(memberRpcs, pool, new SnapshotSubprocedureBuilder());
134   }
135 
136   /**
137    * Start accepting snapshot requests.
138    */
139   public void start() {
140     LOG.debug("Start Snapshot Manager " + rss.getServerName().toString());
141     this.memberRpcs.start(rss.getServerName().toString(), member);
142   }
143 
144   /**
145    * Close <tt>this</tt> and all running snapshot tasks
146    * @param force forcefully stop all running tasks
147    * @throws IOException
148    */
149   public void stop(boolean force) throws IOException {
150     String mode = force ? "abruptly" : "gracefully";
151     LOG.info("Stopping RegionServerSnapshotManager " + mode + ".");
152 
153     try {
154       this.member.close();
155     } finally {
156       this.memberRpcs.close();
157     }
158   }
159 
160   /**
161    * If in a running state, creates the specified subprocedure for handling an online snapshot.
162    *
163    * Because this gets the local list of regions to snapshot and not the set the master had,
164    * there is a possibility of a race where regions may be missed.  This detected by the master in
165    * the snapshot verification step.
166    *
167    * @param snapshot
168    * @return Subprocedure to submit to the ProcedureMemeber.
169    */
170   public Subprocedure buildSubprocedure(SnapshotDescription snapshot) {
171 
172     // don't run a snapshot if the parent is stop(ping)
173     if (rss.isStopping() || rss.isStopped()) {
174       throw new IllegalStateException("Can't start snapshot on RS: " + rss.getServerName()
175           + ", because stopping/stopped!");
176     }
177 
178     // check to see if this server is hosting any regions for the snapshots
179     // check to see if we have regions for the snapshot
180     List<HRegion> involvedRegions;
181     try {
182       involvedRegions = getRegionsToSnapshot(snapshot);
183     } catch (IOException e1) {
184       throw new IllegalStateException("Failed to figure out if we should handle a snapshot - "
185           + "something has gone awry with the online regions.", e1);
186     }
187 
188     // We need to run the subprocedure even if we have no relevant regions.  The coordinator
189     // expects participation in the procedure and without sending message the snapshot attempt
190     // will hang and fail.
191 
192     LOG.debug("Launching subprocedure for snapshot " + snapshot.getName() + " from table "
193         + snapshot.getTable());
194     ForeignExceptionDispatcher exnDispatcher = new ForeignExceptionDispatcher(snapshot.getName());
195     Configuration conf = rss.getConfiguration();
196     long timeoutMillis = conf.getLong(SNAPSHOT_TIMEOUT_MILLIS_KEY,
197         SNAPSHOT_TIMEOUT_MILLIS_DEFAULT);
198     long wakeMillis = conf.getLong(SNAPSHOT_REQUEST_WAKE_MILLIS_KEY,
199         SNAPSHOT_REQUEST_WAKE_MILLIS_DEFAULT);
200 
201     switch (snapshot.getType()) {
202     case FLUSH:
203       SnapshotSubprocedurePool taskManager =
204         new SnapshotSubprocedurePool(rss.getServerName().toString(), conf);
205       return new FlushSnapshotSubprocedure(member, exnDispatcher, wakeMillis,
206           timeoutMillis, involvedRegions, snapshot, taskManager);
207       case SKIPFLUSH:
208         /*
209          * This is to take an online-snapshot without force a coordinated flush to prevent pause
210          * The snapshot type is defined inside the snapshot description. FlushSnapshotSubprocedure
211          * should be renamed to distributedSnapshotSubprocedure, and the flush() behavior can be
212          * turned on/off based on the flush type.
213          * To minimized the code change, class name is not changed.
214          */
215         SnapshotSubprocedurePool taskManager2 =
216             new SnapshotSubprocedurePool(rss.getServerName().toString(), conf);
217         return new FlushSnapshotSubprocedure(member, exnDispatcher, wakeMillis,
218             timeoutMillis, involvedRegions, snapshot, taskManager2);
219     default:
220       throw new UnsupportedOperationException("Unrecognized snapshot type:" + snapshot.getType());
221     }
222   }
223 
224   /**
225    * Determine if the snapshot should be handled on this server
226    *
227    * NOTE: This is racy -- the master expects a list of regionservers.
228    * This means if a region moves somewhere between the calls we'll miss some regions.
229    * For example, a region move during a snapshot could result in a region to be skipped or done
230    * twice.  This is manageable because the {@link MasterSnapshotVerifier} will double check the
231    * region lists after the online portion of the snapshot completes and will explicitly fail the
232    * snapshot.
233    *
234    * @param snapshot
235    * @return the list of online regions. Empty list is returned if no regions are responsible for
236    *         the given snapshot.
237    * @throws IOException
238    */
239   private List<HRegion> getRegionsToSnapshot(SnapshotDescription snapshot) throws IOException {
240     byte[] table = Bytes.toBytes(snapshot.getTable());
241     return rss.getOnlineRegions(table);
242   }
243 
244   /**
245    * Build the actual snapshot runner that will do all the 'hard' work
246    */
247   public class SnapshotSubprocedureBuilder implements SubprocedureFactory {
248 
249     @Override
250     public Subprocedure buildSubprocedure(String name, byte[] data) {
251       try {
252         // unwrap the snapshot information
253         SnapshotDescription snapshot = SnapshotDescription.parseFrom(data);
254         return RegionServerSnapshotManager.this.buildSubprocedure(snapshot);
255       } catch (InvalidProtocolBufferException e) {
256         throw new IllegalArgumentException("Could not read snapshot information from request.");
257       }
258     }
259 
260   }
261 
262   /**
263    * We use the SnapshotSubprocedurePool, a class specific thread pool instead of
264    * {@link org.apache.hadoop.hbase.executor.ExecutorService}.
265    *
266    * It uses a {@link java.util.concurrent.ExecutorCompletionService} which provides queuing of
267    * completed tasks which lets us efficiently cancel pending tasks upon the earliest operation
268    * failures.
269    *
270    * HBase's ExecutorService (different from {@link java.util.concurrent.ExecutorService}) isn't
271    * really built for coordinated tasks where multiple threads as part of one larger task.  In
272    * RS's the HBase Executor services are only used for open and close and not other threadpooled
273    * operations such as compactions and replication  sinks.
274    */
275   static class SnapshotSubprocedurePool {
276     private final ExecutorCompletionService<Void> taskPool;
277     private final ThreadPoolExecutor executor;
278     private volatile boolean stopped;
279     private final List<Future<Void>> futures = new ArrayList<Future<Void>>();
280     private final String name;
281 
282     SnapshotSubprocedurePool(String name, Configuration conf) {
283       // configure the executor service
284       long keepAlive = conf.getLong(
285         RegionServerSnapshotManager.SNAPSHOT_TIMEOUT_MILLIS_KEY,
286         RegionServerSnapshotManager.SNAPSHOT_TIMEOUT_MILLIS_DEFAULT);
287       int threads = conf.getInt(CONCURENT_SNAPSHOT_TASKS_KEY, DEFAULT_CONCURRENT_SNAPSHOT_TASKS);
288       this.name = name;
289       executor = new ThreadPoolExecutor(1, threads, keepAlive, TimeUnit.MILLISECONDS,
290           new LinkedBlockingQueue<Runnable>(), new DaemonThreadFactory("rs("
291               + name + ")-snapshot-pool"));
292       taskPool = new ExecutorCompletionService<Void>(executor);
293     }
294 
295     boolean hasTasks() {
296       return futures.size() != 0;
297     }
298 
299     /**
300      * Submit a task to the pool.
301      *
302      * NOTE: all must be submitted before you can safely {@link #waitForOutstandingTasks()}. This
303      * version does not support issuing tasks from multiple concurrent table snapshots requests.
304      */
305     void submitTask(final Callable<Void> task) {
306       Future<Void> f = this.taskPool.submit(task);
307       futures.add(f);
308     }
309 
310     /**
311      * Wait for all of the currently outstanding tasks submitted via {@link #submitTask(Callable)}.
312      * This *must* be called after all tasks are submitted via submitTask.
313      *
314      * @return <tt>true</tt> on success, <tt>false</tt> otherwise
315      * @throws InterruptedException
316      * @throws SnapshotCreationException if the snapshot failed while we were waiting
317      */
318     boolean waitForOutstandingTasks() throws ForeignException, InterruptedException {
319       LOG.debug("Waiting for local region snapshots to finish.");
320 
321       int sz = futures.size();
322       try {
323         // Using the completion service to process the futures that finish first first.
324         for (int i = 0; i < sz; i++) {
325           Future<Void> f = taskPool.take();
326           f.get();
327           if (!futures.remove(f)) {
328             LOG.warn("unexpected future" + f);
329           }
330           LOG.debug("Completed " + (i+1) + "/" + sz +  " local region snapshots.");
331         }
332         LOG.debug("Completed " + sz +  " local region snapshots.");
333         return true;
334       } catch (InterruptedException e) {
335         LOG.warn("Got InterruptedException in SnapshotSubprocedurePool", e);
336         if (!stopped) {
337           Thread.currentThread().interrupt();
338           throw new ForeignException("SnapshotSubprocedurePool", e);
339         }
340         // we are stopped so we can just exit.
341       } catch (ExecutionException e) {
342         if (e.getCause() instanceof ForeignException) {
343           LOG.warn("Rethrowing ForeignException from SnapshotSubprocedurePool", e);
344           throw (ForeignException)e.getCause();
345         }
346         LOG.warn("Got Exception in SnapshotSubprocedurePool", e);
347         throw new ForeignException(name, e.getCause());
348       } finally {
349         cancelTasks();
350       }
351       return false;
352     }
353 
354     /**
355      * This attempts to cancel out all pending and in progress tasks (interruptions issues)
356      * @throws InterruptedException
357      */
358     void cancelTasks() throws InterruptedException {
359       Collection<Future<Void>> tasks = futures;
360       LOG.debug("cancelling " + tasks.size() + " tasks for snapshot " + name);
361       for (Future<Void> f: tasks) {
362         // TODO Ideally we'd interrupt hbase threads when we cancel.  However it seems that there
363         // are places in the HBase code where row/region locks are taken and not released in a
364         // finally block.  Thus we cancel without interrupting.  Cancellations will be slower to
365         // complete but we won't suffer from unreleased locks due to poor code discipline.
366         f.cancel(false);
367       }
368 
369       // evict remaining tasks and futures from taskPool.
370       while (!futures.isEmpty()) {
371         // block to remove cancelled futures;
372         LOG.warn("Removing cancelled elements from taskPool");
373         futures.remove(taskPool.take());
374       }
375       stop();
376     }
377 
378     /**
379      * Abruptly shutdown the thread pool.  Call when exiting a region server.
380      */
381     void stop() {
382       if (this.stopped) return;
383 
384       this.stopped = true;
385       this.executor.shutdownNow();
386     }
387   }
388 }