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