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.master.snapshot;
19  
20  import java.io.FileNotFoundException;
21  import java.io.IOException;
22  import java.util.ArrayList;
23  import java.util.Collections;
24  import java.util.HashMap;
25  import java.util.HashSet;
26  import java.util.Iterator;
27  import java.util.List;
28  import java.util.Map;
29  import java.util.Set;
30  import java.util.concurrent.ThreadPoolExecutor;
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.fs.FSDataInputStream;
38  import org.apache.hadoop.fs.FileStatus;
39  import org.apache.hadoop.fs.FileSystem;
40  import org.apache.hadoop.fs.Path;
41  import org.apache.hadoop.hbase.HConstants;
42  import org.apache.hadoop.hbase.HTableDescriptor;
43  import org.apache.hadoop.hbase.Stoppable;
44  import org.apache.hadoop.hbase.catalog.MetaReader;
45  import org.apache.hadoop.hbase.errorhandling.ForeignException;
46  import org.apache.hadoop.hbase.executor.ExecutorService;
47  import org.apache.hadoop.hbase.master.AssignmentManager;
48  import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
49  import org.apache.hadoop.hbase.master.MasterFileSystem;
50  import org.apache.hadoop.hbase.master.MasterServices;
51  import org.apache.hadoop.hbase.master.SnapshotSentinel;
52  import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
53  import org.apache.hadoop.hbase.master.cleaner.HFileLinkCleaner;
54  import org.apache.hadoop.hbase.master.metrics.MasterMetrics;
55  import org.apache.hadoop.hbase.procedure.Procedure;
56  import org.apache.hadoop.hbase.procedure.ProcedureCoordinator;
57  import org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs;
58  import org.apache.hadoop.hbase.procedure.ZKProcedureCoordinatorRpcs;
59  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
60  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type;
61  import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
62  import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
63  import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
64  import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
65  import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
66  import org.apache.hadoop.hbase.snapshot.SnapshotDoesNotExistException;
67  import org.apache.hadoop.hbase.snapshot.SnapshotExistsException;
68  import org.apache.hadoop.hbase.snapshot.TablePartiallyOpenException;
69  import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
70  import org.apache.hadoop.hbase.util.Bytes;
71  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
72  import org.apache.hadoop.hbase.util.FSTableDescriptors;
73  import org.apache.hadoop.hbase.util.FSUtils;
74  import org.apache.zookeeper.KeeperException;
75  
76  /**
77   * This class manages the procedure of taking and restoring snapshots. There is only one
78   * SnapshotManager for the master.
79   * <p>
80   * The class provides methods for monitoring in-progress snapshot actions.
81   * <p>
82   * Note: Currently there can only be one snapshot being taken at a time over the cluster. This is a
83   * simplification in the current implementation.
84   */
85  @InterfaceAudience.Private
86  @InterfaceStability.Unstable
87  public class SnapshotManager implements Stoppable {
88    private static final Log LOG = LogFactory.getLog(SnapshotManager.class);
89  
90    /** By default, check to see if the snapshot is complete every WAKE MILLIS (ms) */
91    private static final int SNAPSHOT_WAKE_MILLIS_DEFAULT = 500;
92  
93    /**
94     * Wait time before removing a finished sentinel from the in-progress map
95     *
96     * NOTE: This is used as a safety auto cleanup.
97     * The snapshot and restore handlers map entries are removed when a user asks if a snapshot or
98     * restore is completed. This operation is part of the HBaseAdmin snapshot/restore API flow.
99     * In case something fails on the client side and the snapshot/restore state is not reclaimed
100    * after a default timeout, the entry is removed from the in-progress map.
101    * At this point, if the user asks for the snapshot/restore status, the result will be
102    * snapshot done if exists or failed if it doesn't exists.
103    */
104   private static final int SNAPSHOT_SENTINELS_CLEANUP_TIMEOUT = 60 * 1000;
105 
106   /** Enable or disable snapshot support */
107   public static final String HBASE_SNAPSHOT_ENABLED = "hbase.snapshot.enabled";
108 
109   /**
110    * Conf key for # of ms elapsed between checks for snapshot errors while waiting for
111    * completion.
112    */
113   private static final String SNAPSHOT_WAKE_MILLIS_KEY = "hbase.snapshot.master.wakeMillis";
114 
115   /** By default, check to see if the snapshot is complete (ms) */
116   private static final int SNAPSHOT_TIMEOUT_MILLIS_DEFAULT = 60000;
117 
118   /**
119    * Conf key for # of ms elapsed before injecting a snapshot timeout error when waiting for
120    * completion.
121    */
122   private static final String SNAPSHOT_TIMEOUT_MILLIS_KEY = "hbase.snapshot.master.timeoutMillis";
123 
124   /** Name of the operation to use in the controller */
125   public static final String ONLINE_SNAPSHOT_CONTROLLER_DESCRIPTION = "online-snapshot";
126 
127   /** Conf key for # of threads used by the SnapshotManager thread pool */
128   private static final String SNAPSHOT_POOL_THREADS_KEY = "hbase.snapshot.master.threads";
129 
130   /** number of current operations running on the master */
131   private static final int SNAPSHOT_POOL_THREADS_DEFAULT = 1;
132 
133   private boolean stopped;
134   private final MasterServices master;  // Needed by TableEventHandlers
135   private final MasterMetrics metricsMaster;
136   private final ProcedureCoordinator coordinator;
137 
138   // Is snapshot feature enabled?
139   private boolean isSnapshotSupported = false;
140 
141   // Snapshot handlers map, with table name as key.
142   // The map is always accessed and modified under the object lock using synchronized.
143   // snapshotTable() will insert an Handler in the table.
144   // isSnapshotDone() will remove the handler requested if the operation is finished.
145   private Map<String, SnapshotSentinel> snapshotHandlers = new HashMap<String, SnapshotSentinel>();
146 
147   // Restore Sentinels map, with table name as key.
148   // The map is always accessed and modified under the object lock using synchronized.
149   // restoreSnapshot()/cloneSnapshot() will insert an Handler in the table.
150   // isRestoreDone() will remove the handler requested if the operation is finished.
151   private Map<String, SnapshotSentinel> restoreHandlers = new HashMap<String, SnapshotSentinel>();
152 
153   private final Path rootDir;
154   private final ExecutorService executorService;
155 
156   /**
157    * Construct a snapshot manager.
158    * @param master
159    */
160   public SnapshotManager(final MasterServices master, final MasterMetrics metricsMaster)
161       throws KeeperException, IOException, UnsupportedOperationException {
162     this.master = master;
163     this.metricsMaster = metricsMaster;
164 
165     this.rootDir = master.getMasterFileSystem().getRootDir();
166     checkSnapshotSupport(master.getConfiguration(), master.getMasterFileSystem());
167 
168     // get the configuration for the coordinator
169     Configuration conf = master.getConfiguration();
170     long wakeFrequency = conf.getInt(SNAPSHOT_WAKE_MILLIS_KEY, SNAPSHOT_WAKE_MILLIS_DEFAULT);
171     long timeoutMillis = conf.getLong(SNAPSHOT_TIMEOUT_MILLIS_KEY, SNAPSHOT_TIMEOUT_MILLIS_DEFAULT);
172     int opThreads = conf.getInt(SNAPSHOT_POOL_THREADS_KEY, SNAPSHOT_POOL_THREADS_DEFAULT);
173 
174     // setup the default procedure coordinator
175     String name = master.getServerName().toString();
176     ThreadPoolExecutor tpool = ProcedureCoordinator.defaultPool(name, opThreads);
177     ProcedureCoordinatorRpcs comms = new ZKProcedureCoordinatorRpcs(
178         master.getZooKeeper(), SnapshotManager.ONLINE_SNAPSHOT_CONTROLLER_DESCRIPTION, name);
179     this.coordinator = new ProcedureCoordinator(comms, tpool, timeoutMillis, wakeFrequency);
180     this.executorService = master.getExecutorService();
181     resetTempDir();
182   }
183 
184   /**
185    * Fully specify all necessary components of a snapshot manager. Exposed for testing.
186    * @param master services for the master where the manager is running
187    * @param coordinator procedure coordinator instance.  exposed for testing.
188    * @param pool HBase ExecutorServcie instance, exposed for testing.
189    */
190   public SnapshotManager(final MasterServices master, final MasterMetrics metricsMaster,
191       ProcedureCoordinator coordinator, ExecutorService pool)
192       throws IOException, UnsupportedOperationException {
193     this.master = master;
194     this.metricsMaster = metricsMaster;
195 
196     this.rootDir = master.getMasterFileSystem().getRootDir();
197     checkSnapshotSupport(master.getConfiguration(), master.getMasterFileSystem());
198 
199     this.coordinator = coordinator;
200     this.executorService = pool;
201     resetTempDir();
202   }
203 
204   /**
205    * Gets the list of all completed snapshots.
206    * @return list of SnapshotDescriptions
207    * @throws IOException File system exception
208    */
209   public List<SnapshotDescription> getCompletedSnapshots() throws IOException {
210     return getCompletedSnapshots(SnapshotDescriptionUtils.getSnapshotsDir(rootDir));
211   }
212 
213   /**
214    * Gets the list of all completed snapshots.
215    * @param snapshotDir snapshot directory
216    * @return list of SnapshotDescriptions
217    * @throws IOException File system exception
218    */
219   private List<SnapshotDescription> getCompletedSnapshots(Path snapshotDir) throws IOException {
220     List<SnapshotDescription> snapshotDescs = new ArrayList<SnapshotDescription>();
221     // first create the snapshot root path and check to see if it exists
222     FileSystem fs = master.getMasterFileSystem().getFileSystem();
223     if (snapshotDir == null) snapshotDir = SnapshotDescriptionUtils.getSnapshotsDir(rootDir);
224 
225     // if there are no snapshots, return an empty list
226     if (!fs.exists(snapshotDir)) {
227       return snapshotDescs;
228     }
229 
230     // ignore all the snapshots in progress
231     FileStatus[] snapshots = fs.listStatus(snapshotDir,
232       new SnapshotDescriptionUtils.CompletedSnaphotDirectoriesFilter(fs));
233     // loop through all the completed snapshots
234     for (FileStatus snapshot : snapshots) {
235       Path info = new Path(snapshot.getPath(), SnapshotDescriptionUtils.SNAPSHOTINFO_FILE);
236       // if the snapshot is bad
237       if (!fs.exists(info)) {
238         LOG.error("Snapshot information for " + snapshot.getPath() + " doesn't exist");
239         continue;
240       }
241       FSDataInputStream in = null;
242       try {
243         in = fs.open(info);
244         SnapshotDescription desc = SnapshotDescription.parseFrom(in);
245         snapshotDescs.add(desc);
246       } catch (IOException e) {
247         LOG.warn("Found a corrupted snapshot " + snapshot.getPath(), e);
248       } finally {
249         if (in != null) {
250           in.close();
251         }
252       }
253     }
254     return snapshotDescs;
255   }
256 
257   /**
258    * Cleans up any snapshots in the snapshot/.tmp directory that were left from failed
259    * snapshot attempts.
260    *
261    * @throws IOException if we can't reach the filesystem
262    */
263   void resetTempDir() throws IOException {
264     // cleanup any existing snapshots.
265     Path tmpdir = SnapshotDescriptionUtils.getWorkingSnapshotDir(rootDir);
266     if (!master.getMasterFileSystem().getFileSystem().delete(tmpdir, true)) {
267       LOG.warn("Couldn't delete working snapshot directory: " + tmpdir);
268     }
269   }
270 
271   /**
272    * Delete the specified snapshot
273    * @param snapshot
274    * @throws SnapshotDoesNotExistException If the specified snapshot does not exist.
275    * @throws IOException For filesystem IOExceptions
276    */
277   public void deleteSnapshot(SnapshotDescription snapshot) throws SnapshotDoesNotExistException, IOException {
278 
279     // call coproc pre hook
280     MasterCoprocessorHost cpHost = master.getCoprocessorHost();
281     if (cpHost != null) {
282       cpHost.preDeleteSnapshot(snapshot);
283     }
284 
285     // check to see if it is completed
286     if (!isSnapshotCompleted(snapshot)) {
287       throw new SnapshotDoesNotExistException(snapshot);
288     }
289 
290     String snapshotName = snapshot.getName();
291     LOG.debug("Deleting snapshot: " + snapshotName);
292     // first create the snapshot description and check to see if it exists
293     MasterFileSystem fs = master.getMasterFileSystem();
294     Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, rootDir);
295 
296     // delete the existing snapshot
297     if (!fs.getFileSystem().delete(snapshotDir, true)) {
298       throw new HBaseSnapshotException("Failed to delete snapshot directory: " + snapshotDir);
299     }
300 
301     // call coproc post hook
302     if (cpHost != null) {
303       cpHost.postDeleteSnapshot(snapshot);
304     }
305 
306   }
307 
308   /**
309    * Check if the specified snapshot is done
310    *
311    * @param expected
312    * @return true if snapshot is ready to be restored, false if it is still being taken.
313    * @throws IOException IOException if error from HDFS or RPC
314    * @throws UnknownSnapshotException if snapshot is invalid or does not exist.
315    */
316   public boolean isSnapshotDone(SnapshotDescription expected) throws IOException {
317     // check the request to make sure it has a snapshot
318     if (expected == null) {
319       throw new UnknownSnapshotException(
320          "No snapshot name passed in request, can't figure out which snapshot you want to check.");
321     }
322 
323     String ssString = SnapshotDescriptionUtils.toString(expected);
324 
325     // check to see if the sentinel exists,
326     // and if the task is complete removes it from the in-progress snapshots map.
327     SnapshotSentinel handler = removeSentinelIfFinished(this.snapshotHandlers, expected);
328 
329     // stop tracking "abandoned" handlers
330     cleanupSentinels();
331 
332     if (handler == null) {
333       // If there's no handler in the in-progress map, it means one of the following:
334       //   - someone has already requested the snapshot state
335       //   - the requested snapshot was completed long time ago (cleanupSentinels() timeout)
336       //   - the snapshot was never requested
337       // In those cases returns to the user the "done state" if the snapshots exists on disk,
338       // otherwise raise an exception saying that the snapshot is not running and doesn't exist.
339       if (!isSnapshotCompleted(expected)) {
340         throw new UnknownSnapshotException("Snapshot " + ssString
341             + " is not currently running or one of the known completed snapshots.");
342       }
343       // was done, return true;
344       return true;
345     }
346 
347     // pass on any failure we find in the sentinel
348     try {
349       handler.rethrowExceptionIfFailed();
350     } catch (ForeignException e) {
351       // Give some procedure info on an exception.
352       String status;
353       Procedure p = coordinator.getProcedure(expected.getName());
354       if (p != null) {
355         status = p.getStatus();
356       } else {
357         status = expected.getName() + " not found in proclist " + coordinator.getProcedureNames();
358       }
359       throw new HBaseSnapshotException("Snapshot " + ssString +  " had an error.  " + status, e,
360           expected);
361     }
362 
363     // check to see if we are done
364     if (handler.isFinished()) {
365       LOG.debug("Snapshot '" + ssString + "' has completed, notifying client.");
366       return true;
367     } else if (LOG.isDebugEnabled()) {
368       LOG.debug("Snapshoting '" + ssString + "' is still in progress!");
369     }
370     return false;
371   }
372 
373   /**
374    * Check to see if the specified table has a snapshot in progress.  Currently we have a
375    * limitation only allowing a single snapshot per table at a time.
376    * @param tableName name of the table being snapshotted.
377    * @return <tt>true</tt> if there is a snapshot in progress on the specified table.
378    */
379   synchronized boolean isTakingSnapshot(final String tableName) {
380     SnapshotSentinel handler = this.snapshotHandlers.get(tableName);
381     return handler != null && !handler.isFinished();
382   }
383 
384   /**
385    * Check to make sure that we are OK to run the passed snapshot. Checks to make sure that we
386    * aren't already running a snapshot or restore on the requested table.
387    * @param snapshot description of the snapshot we want to start
388    * @throws HBaseSnapshotException if the filesystem could not be prepared to start the snapshot
389    */
390   private synchronized void prepareToTakeSnapshot(SnapshotDescription snapshot)
391       throws HBaseSnapshotException {
392     FileSystem fs = master.getMasterFileSystem().getFileSystem();
393     Path workingDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshot, rootDir);
394 
395     // make sure we aren't already running a snapshot
396     if (isTakingSnapshot(snapshot.getTable())) {
397       SnapshotSentinel handler = this.snapshotHandlers.get(snapshot.getTable());
398       throw new SnapshotCreationException("Rejected taking "
399           + SnapshotDescriptionUtils.toString(snapshot)
400           + " because we are already running another snapshot "
401           + SnapshotDescriptionUtils.toString(handler.getSnapshot()), snapshot);
402     }
403 
404     // make sure we aren't running a restore on the same table
405     if (isRestoringTable(snapshot.getTable())) {
406       SnapshotSentinel handler = restoreHandlers.get(snapshot.getTable());
407       throw new SnapshotCreationException("Rejected taking "
408           + SnapshotDescriptionUtils.toString(snapshot)
409           + " because we are already have a restore in progress on the same snapshot "
410           + SnapshotDescriptionUtils.toString(handler.getSnapshot()), snapshot);
411     }
412 
413     try {
414       // delete the working directory, since we aren't running the snapshot. Likely leftovers
415       // from a failed attempt.
416       fs.delete(workingDir, true);
417 
418       // recreate the working directory for the snapshot
419       if (!fs.mkdirs(workingDir)) {
420         throw new SnapshotCreationException("Couldn't create working directory (" + workingDir
421             + ") for snapshot" , snapshot);
422       }
423     } catch (HBaseSnapshotException e) {
424       throw e;
425     } catch (IOException e) {
426       throw new SnapshotCreationException(
427           "Exception while checking to see if snapshot could be started.", e, snapshot);
428     }
429   }
430 
431   /**
432    * Take a snapshot of a disabled table.
433    * @param snapshot description of the snapshot to take. Modified to be {@link Type#DISABLED}.
434    * @throws HBaseSnapshotException if the snapshot could not be started
435    */
436   private synchronized void snapshotDisabledTable(SnapshotDescription snapshot)
437       throws HBaseSnapshotException {
438     // setup the snapshot
439     prepareToTakeSnapshot(snapshot);
440 
441     // set the snapshot to be a disabled snapshot, since the client doesn't know about that
442     snapshot = snapshot.toBuilder().setType(Type.DISABLED).build();
443 
444     // Take the snapshot of the disabled table
445     DisabledTableSnapshotHandler handler =
446         new DisabledTableSnapshotHandler(snapshot, master, metricsMaster);
447     snapshotTable(snapshot, handler);
448   }
449 
450   /**
451    * Take a snapshot of an enabled table.
452    * @param snapshot description of the snapshot to take.
453    * @throws HBaseSnapshotException if the snapshot could not be started
454    */
455   private synchronized void snapshotEnabledTable(SnapshotDescription snapshot)
456       throws HBaseSnapshotException {
457     // setup the snapshot
458     prepareToTakeSnapshot(snapshot);
459 
460     // Take the snapshot of the enabled table
461     EnabledTableSnapshotHandler handler =
462         new EnabledTableSnapshotHandler(snapshot, master, this, metricsMaster);
463     snapshotTable(snapshot, handler);
464   }
465 
466   /**
467    * Take a snapshot using the specified handler.
468    * On failure the snapshot temporary working directory is removed.
469    * NOTE: prepareToTakeSnapshot() called before this one takes care of the rejecting the
470    *       snapshot request if the table is busy with another snapshot/restore operation.
471    * @param snapshot the snapshot description
472    * @param handler the snapshot handler
473    */
474   private synchronized void snapshotTable(SnapshotDescription snapshot,
475       final TakeSnapshotHandler handler) throws HBaseSnapshotException {
476     try {
477       handler.prepare();
478       this.executorService.submit(handler);
479       this.snapshotHandlers.put(snapshot.getTable(), handler);
480     } catch (Exception e) {
481       // cleanup the working directory by trying to delete it from the fs.
482       Path workingDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshot, rootDir);
483       try {
484         if (!this.master.getMasterFileSystem().getFileSystem().delete(workingDir, true)) {
485           LOG.error("Couldn't delete working directory (" + workingDir + " for snapshot:" +
486               SnapshotDescriptionUtils.toString(snapshot));
487         }
488       } catch (IOException e1) {
489         LOG.error("Couldn't delete working directory (" + workingDir + " for snapshot:" +
490             SnapshotDescriptionUtils.toString(snapshot));
491       }
492       // fail the snapshot
493       throw new SnapshotCreationException("Could not build snapshot handler", e, snapshot);
494     }
495   }
496 
497   /**
498    * Take a snapshot based on the enabled/disabled state of the table.
499    *
500    * @param snapshot
501    * @throws HBaseSnapshotException when a snapshot specific exception occurs.
502    * @throws IOException when some sort of generic IO exception occurs.
503    */
504   public void takeSnapshot(SnapshotDescription snapshot) throws IOException {
505     // check to see if we already completed the snapshot
506     if (isSnapshotCompleted(snapshot)) {
507       throw new SnapshotExistsException("Snapshot '" + snapshot.getName()
508           + "' already stored on the filesystem.", snapshot);
509     }
510 
511     LOG.debug("No existing snapshot, attempting snapshot...");
512 
513     // stop tracking "abandoned" handlers
514     cleanupSentinels();
515 
516     // check to see if the table exists
517     HTableDescriptor desc = null;
518     try {
519       desc = master.getTableDescriptors().get(snapshot.getTable());
520     } catch (FileNotFoundException e) {
521       String msg = "Table:" + snapshot.getTable() + " info doesn't exist!";
522       LOG.error(msg);
523       throw new SnapshotCreationException(msg, e, snapshot);
524     } catch (IOException e) {
525       throw new SnapshotCreationException("Error while geting table description for table "
526           + snapshot.getTable(), e, snapshot);
527     }
528     if (desc == null) {
529       throw new SnapshotCreationException("Table '" + snapshot.getTable()
530           + "' doesn't exist, can't take snapshot.", snapshot);
531     }
532 
533     // set the snapshot version, now that we are ready to take it
534     snapshot = snapshot.toBuilder().setVersion(SnapshotDescriptionUtils.SNAPSHOT_LAYOUT_VERSION)
535         .build();
536 
537     // call pre coproc hook
538     MasterCoprocessorHost cpHost = master.getCoprocessorHost();
539     if (cpHost != null) {
540       cpHost.preSnapshot(snapshot, desc);
541     }
542 
543     // if the table is enabled, then have the RS run actually the snapshot work
544     AssignmentManager assignmentMgr = master.getAssignmentManager();
545     if (assignmentMgr.getZKTable().isEnabledTable(snapshot.getTable())) {
546       LOG.debug("Table enabled, starting distributed snapshot.");
547       snapshotEnabledTable(snapshot);
548       LOG.debug("Started snapshot: " + SnapshotDescriptionUtils.toString(snapshot));
549     }
550     // For disabled table, snapshot is created by the master
551     else if (assignmentMgr.getZKTable().isDisabledTable(snapshot.getTable())) {
552       LOG.debug("Table is disabled, running snapshot entirely on master.");
553       snapshotDisabledTable(snapshot);
554       LOG.debug("Started snapshot: " + SnapshotDescriptionUtils.toString(snapshot));
555     } else {
556       LOG.error("Can't snapshot table '" + snapshot.getTable()
557           + "', isn't open or closed, we don't know what to do!");
558       TablePartiallyOpenException tpoe = new TablePartiallyOpenException(snapshot.getTable()
559           + " isn't fully open.");
560       throw new SnapshotCreationException("Table is not entirely open or closed", tpoe, snapshot);
561     }
562 
563     // call post coproc hook
564     if (cpHost != null) {
565       cpHost.postSnapshot(snapshot, desc);
566     }
567   }
568 
569   /**
570    * Set the handler for the current snapshot
571    * <p>
572    * Exposed for TESTING
573    * @param tableName
574    * @param handler handler the master should use
575    *
576    * TODO get rid of this if possible, repackaging, modify tests.
577    */
578   public synchronized void setSnapshotHandlerForTesting(final String tableName,
579       final SnapshotSentinel handler) {
580     if (handler != null) {
581       this.snapshotHandlers.put(tableName, handler);
582     } else {
583       this.snapshotHandlers.remove(tableName);
584     }
585   }
586 
587   /**
588    * @return distributed commit coordinator for all running snapshots
589    */
590   ProcedureCoordinator getCoordinator() {
591     return coordinator;
592   }
593 
594   /**
595    * Check to see if the snapshot is one of the currently completed snapshots
596    * Returns true if the snapshot exists in the "completed snapshots folder".
597    *
598    * @param snapshot expected snapshot to check
599    * @return <tt>true</tt> if the snapshot is stored on the {@link FileSystem}, <tt>false</tt> if is
600    *         not stored
601    * @throws IOException if the filesystem throws an unexpected exception,
602    * @throws IllegalArgumentException if snapshot name is invalid.
603    */
604   private boolean isSnapshotCompleted(SnapshotDescription snapshot) throws IOException {
605     try {
606       final Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir);
607       FileSystem fs = master.getMasterFileSystem().getFileSystem();
608 
609       // check to see if the snapshot already exists
610       return fs.exists(snapshotDir);
611     } catch (IllegalArgumentException iae) {
612       throw new UnknownSnapshotException("Unexpected exception thrown", iae);
613     }
614   }
615 
616   /**
617    * Clone the specified snapshot into a new table.
618    * The operation will fail if the destination table has a snapshot or restore in progress.
619    *
620    * @param snapshot Snapshot Descriptor
621    * @param hTableDescriptor Table Descriptor of the table to create
622    */
623   synchronized void cloneSnapshot(final SnapshotDescription snapshot,
624       final HTableDescriptor hTableDescriptor) throws HBaseSnapshotException {
625     String tableName = hTableDescriptor.getNameAsString();
626 
627     // make sure we aren't running a snapshot on the same table
628     if (isTakingSnapshot(tableName)) {
629       throw new RestoreSnapshotException("Snapshot in progress on the restore table=" + tableName);
630     }
631 
632     // make sure we aren't running a restore on the same table
633     if (isRestoringTable(tableName)) {
634       throw new RestoreSnapshotException("Restore already in progress on the table=" + tableName);
635     }
636 
637     try {
638       CloneSnapshotHandler handler =
639         new CloneSnapshotHandler(master, snapshot, hTableDescriptor, metricsMaster);
640       this.executorService.submit(handler);
641       this.restoreHandlers.put(tableName, handler);
642     } catch (Exception e) {
643       String msg = "Couldn't clone the snapshot=" + SnapshotDescriptionUtils.toString(snapshot) +
644         " on table=" + tableName;
645       LOG.error(msg, e);
646       throw new RestoreSnapshotException(msg, e);
647     }
648   }
649 
650   /**
651    * Restore the specified snapshot
652    * @param reqSnapshot
653    * @throws IOException
654    */
655   public void restoreSnapshot(SnapshotDescription reqSnapshot) throws IOException {
656     FileSystem fs = master.getMasterFileSystem().getFileSystem();
657     Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(reqSnapshot, rootDir);
658     MasterCoprocessorHost cpHost = master.getCoprocessorHost();
659 
660     // check if the snapshot exists
661     if (!fs.exists(snapshotDir)) {
662       LOG.error("A Snapshot named '" + reqSnapshot.getName() + "' does not exist.");
663       throw new SnapshotDoesNotExistException(reqSnapshot);
664     }
665 
666     // read snapshot information
667     SnapshotDescription fsSnapshot = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir);
668     HTableDescriptor snapshotTableDesc = FSTableDescriptors.getTableDescriptor(fs, snapshotDir);
669     String tableName = reqSnapshot.getTable();
670 
671     // stop tracking "abandoned" handlers
672     cleanupSentinels();
673 
674     // Execute the restore/clone operation
675     if (MetaReader.tableExists(master.getCatalogTracker(), tableName)) {
676       if (master.getAssignmentManager().getZKTable().isEnabledTable(fsSnapshot.getTable())) {
677         throw new UnsupportedOperationException("Table '" +
678           fsSnapshot.getTable() + "' must be disabled in order to perform a restore operation.");
679       }
680 
681       // call coproc pre hook
682       if (cpHost != null) {
683         cpHost.preRestoreSnapshot(reqSnapshot, snapshotTableDesc);
684       }
685       restoreSnapshot(fsSnapshot, snapshotTableDesc);
686       LOG.info("Restore snapshot=" + fsSnapshot.getName() + " as table=" + tableName);
687 
688       if (cpHost != null) {
689         cpHost.postRestoreSnapshot(reqSnapshot, snapshotTableDesc);
690       }
691     } else {
692       HTableDescriptor htd = RestoreSnapshotHelper.cloneTableSchema(snapshotTableDesc,
693                                                          Bytes.toBytes(tableName));
694       if (cpHost != null) {
695         cpHost.preCloneSnapshot(reqSnapshot, htd);
696       }
697       cloneSnapshot(fsSnapshot, htd);
698       LOG.info("Clone snapshot=" + fsSnapshot.getName() + " as table=" + tableName);
699 
700       if (cpHost != null) {
701         cpHost.postCloneSnapshot(reqSnapshot, htd);
702       }
703     }
704   }
705 
706   /**
707    * Restore the specified snapshot.
708    * The restore will fail if the destination table has a snapshot or restore in progress.
709    *
710    * @param snapshot Snapshot Descriptor
711    * @param hTableDescriptor Table Descriptor
712    */
713   private synchronized void restoreSnapshot(final SnapshotDescription snapshot,
714       final HTableDescriptor hTableDescriptor) throws HBaseSnapshotException {
715     String tableName = hTableDescriptor.getNameAsString();
716 
717     // make sure we aren't running a snapshot on the same table
718     if (isTakingSnapshot(tableName)) {
719       throw new RestoreSnapshotException("Snapshot in progress on the restore table=" + tableName);
720     }
721 
722     // make sure we aren't running a restore on the same table
723     if (isRestoringTable(tableName)) {
724       throw new RestoreSnapshotException("Restore already in progress on the table=" + tableName);
725     }
726 
727     try {
728       RestoreSnapshotHandler handler =
729         new RestoreSnapshotHandler(master, snapshot, hTableDescriptor, metricsMaster);
730       this.executorService.submit(handler);
731       restoreHandlers.put(hTableDescriptor.getNameAsString(), handler);
732     } catch (Exception e) {
733       String msg = "Couldn't restore the snapshot=" + SnapshotDescriptionUtils.toString(
734           snapshot)  +
735           " on table=" + tableName;
736       LOG.error(msg, e);
737       throw new RestoreSnapshotException(msg, e);
738     }
739   }
740 
741   /**
742    * Verify if the restore of the specified table is in progress.
743    *
744    * @param tableName table under restore
745    * @return <tt>true</tt> if there is a restore in progress of the specified table.
746    */
747   private synchronized boolean isRestoringTable(final String tableName) {
748     SnapshotSentinel sentinel = this.restoreHandlers.get(tableName);
749     return(sentinel != null && !sentinel.isFinished());
750   }
751 
752   /**
753    * Returns the status of a restore operation.
754    * If the in-progress restore is failed throws the exception that caused the failure.
755    *
756    * @param snapshot
757    * @return false if in progress, true if restore is completed or not requested.
758    * @throws IOException if there was a failure during the restore
759    */
760   public boolean isRestoreDone(final SnapshotDescription snapshot) throws IOException {
761     // check to see if the sentinel exists,
762     // and if the task is complete removes it from the in-progress restore map.
763     SnapshotSentinel sentinel = removeSentinelIfFinished(this.restoreHandlers, snapshot);
764 
765     // stop tracking "abandoned" handlers
766     cleanupSentinels();
767 
768     if (sentinel == null) {
769       // there is no sentinel so restore is not in progress.
770       return true;
771     }
772 
773     LOG.debug("Verify snapshot=" + snapshot.getName() + " against="
774         + sentinel.getSnapshot().getName() + " table=" + snapshot.getTable());
775 
776     // If the restore is failed, rethrow the exception
777     sentinel.rethrowExceptionIfFailed();
778 
779     // check to see if we are done
780     if (sentinel.isFinished()) {
781       LOG.debug("Restore snapshot=" + SnapshotDescriptionUtils.toString(snapshot) +
782           " has completed. Notifying the client.");
783       return true;
784     }
785 
786     if (LOG.isDebugEnabled()) {
787       LOG.debug("Sentinel is not yet finished with restoring snapshot=" +
788           SnapshotDescriptionUtils.toString(snapshot));
789     }
790     return false;
791   }
792 
793   /**
794    * Return the handler if it is currently live and has the same snapshot target name.
795    * The handler is removed from the sentinels map if completed.
796    * @param sentinels live handlers
797    * @param snapshot snapshot description
798    * @return null if doesn't match, else a live handler.
799    */
800   private synchronized SnapshotSentinel removeSentinelIfFinished(
801       final Map<String, SnapshotSentinel> sentinels, final SnapshotDescription snapshot) {
802     SnapshotSentinel h = sentinels.get(snapshot.getTable());
803     if (h == null) {
804       return null;
805     }
806 
807     if (!h.getSnapshot().getName().equals(snapshot.getName())) {
808       // specified snapshot is to the one currently running
809       return null;
810     }
811 
812     // Remove from the "in-progress" list once completed
813     if (h.isFinished()) {
814       sentinels.remove(snapshot.getTable());
815     }
816 
817     return h;
818   }
819 
820   /**
821    * Removes "abandoned" snapshot/restore requests.
822    * As part of the HBaseAdmin snapshot/restore API the operation status is checked until completed,
823    * and the in-progress maps are cleaned up when the status of a completed task is requested.
824    * To avoid having sentinels staying around for long time if something client side is failed,
825    * each operation tries to clean up the in-progress maps sentinels finished from a long time.
826    */
827   private void cleanupSentinels() {
828     cleanupSentinels(this.snapshotHandlers);
829     cleanupSentinels(this.restoreHandlers);
830   }
831 
832   /**
833    * Remove the sentinels that are marked as finished and the completion time
834    * has exceeded the removal timeout.
835    * @param sentinels map of sentinels to clean
836    */
837   private synchronized void cleanupSentinels(final Map<String, SnapshotSentinel> sentinels) {
838     long currentTime = EnvironmentEdgeManager.currentTimeMillis();
839     Iterator<Map.Entry<String, SnapshotSentinel>> it = sentinels.entrySet().iterator();
840     while (it.hasNext()) {
841       Map.Entry<String, SnapshotSentinel> entry = it.next();
842       SnapshotSentinel sentinel = entry.getValue();
843       if (sentinel.isFinished() &&
844           (currentTime - sentinel.getCompletionTimestamp()) > SNAPSHOT_SENTINELS_CLEANUP_TIMEOUT)
845       {
846         it.remove();
847       }
848     }
849   }
850 
851   //
852   // Implementing Stoppable interface
853   //
854 
855   @Override
856   public void stop(String why) {
857     // short circuit
858     if (this.stopped) return;
859     // make sure we get stop
860     this.stopped = true;
861     // pass the stop onto take snapshot handlers
862     for (SnapshotSentinel snapshotHandler: this.snapshotHandlers.values()) {
863       snapshotHandler.cancel(why);
864     }
865 
866     // pass the stop onto all the restore handlers
867     for (SnapshotSentinel restoreHandler: this.restoreHandlers.values()) {
868       restoreHandler.cancel(why);
869     }
870 
871     try {
872       coordinator.close();
873     } catch (IOException e) {
874       LOG.error("stop ProcedureCoordinator error", e);
875     }
876   }
877 
878   @Override
879   public boolean isStopped() {
880     return this.stopped;
881   }
882 
883   /**
884    * Throws an exception if snapshot operations (take a snapshot, restore, clone) are not supported.
885    * Called at the beginning of snapshot() and restoreSnapshot() methods.
886    * @throws UnsupportedOperationException if snapshot are not supported
887    */
888   public void checkSnapshotSupport() throws UnsupportedOperationException {
889     if (!this.isSnapshotSupported) {
890       throw new UnsupportedOperationException(
891         "To use snapshots, You must add to the hbase-site.xml of the HBase Master: '" +
892           HBASE_SNAPSHOT_ENABLED + "' property with value 'true'.");
893     }
894   }
895 
896   /**
897    * Called at startup, to verify if snapshot operation is supported, and to avoid
898    * starting the master if there're snapshots present but the cleaners needed are missing.
899    * Otherwise we can end up with snapshot data loss.
900    * @param conf The {@link Configuration} object to use
901    * @param mfs The MasterFileSystem to use
902    * @throws IOException in case of file-system operation failure
903    * @throws UnsupportedOperationException in case cleaners are missing and
904    *         there're snapshot in the system
905    */
906   private void checkSnapshotSupport(final Configuration conf, final MasterFileSystem mfs)
907       throws IOException, UnsupportedOperationException {
908     // Verify if snapshot is disabled by the user
909     String enabled = conf.get(HBASE_SNAPSHOT_ENABLED);
910     boolean snapshotEnabled = conf.getBoolean(HBASE_SNAPSHOT_ENABLED, false);
911     boolean userDisabled = (enabled != null && enabled.trim().length() > 0 && !snapshotEnabled);
912 
913     // Extract cleaners from conf
914     Set<String> hfileCleaners = new HashSet<String>();
915     String[] cleaners = conf.getStrings(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS);
916     if (cleaners != null) Collections.addAll(hfileCleaners, cleaners);
917 
918     Set<String> logCleaners = new HashSet<String>();
919     cleaners = conf.getStrings(HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS);
920     if (cleaners != null) Collections.addAll(logCleaners, cleaners);
921 
922     // check if an older version of snapshot directory was present
923     Path oldSnapshotDir = new Path(mfs.getRootDir(), HConstants.OLD_SNAPSHOT_DIR_NAME);
924     FileSystem fs = mfs.getFileSystem();
925     List<SnapshotDescription> ss = getCompletedSnapshots(new Path(rootDir, oldSnapshotDir));
926     if (ss != null && !ss.isEmpty()) {
927       LOG.error("Snapshots from an earlier release were found under: " + oldSnapshotDir);
928       LOG.error("Please rename the directory as " + HConstants.SNAPSHOT_DIR_NAME);
929     }
930 
931     // If the user has enabled the snapshot, we force the cleaners to be present
932     // otherwise we still need to check if cleaners are enabled or not and verify
933     // that there're no snapshot in the .snapshot folder.
934     if (snapshotEnabled) {
935       // Inject snapshot cleaners, if snapshot.enable is true
936       hfileCleaners.add(SnapshotHFileCleaner.class.getName());
937       hfileCleaners.add(HFileLinkCleaner.class.getName());
938       logCleaners.add(SnapshotLogCleaner.class.getName());
939 
940       // Set cleaners conf
941       conf.setStrings(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS,
942         hfileCleaners.toArray(new String[hfileCleaners.size()]));
943       conf.setStrings(HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS,
944         logCleaners.toArray(new String[logCleaners.size()]));
945     } else {
946       // Verify if cleaners are present
947       snapshotEnabled = logCleaners.contains(SnapshotLogCleaner.class.getName()) &&
948         hfileCleaners.contains(SnapshotHFileCleaner.class.getName()) &&
949         hfileCleaners.contains(HFileLinkCleaner.class.getName());
950 
951       // Warn if the cleaners are enabled but the snapshot.enabled property is false/not set.
952       if (snapshotEnabled) {
953         LOG.warn("Snapshot log and hfile cleaners are present in the configuration, " +
954           "but the '" + HBASE_SNAPSHOT_ENABLED + "' property " +
955           (userDisabled ? "is set to 'false'." : "is not set."));
956       }
957     }
958 
959     // Mark snapshot feature as enabled if cleaners are present and user has not disabled it.
960     this.isSnapshotSupported = snapshotEnabled && !userDisabled;
961 
962     // If cleaners are not enabled, verify that there're no snapshot in the .snapshot folder
963     // otherwise we end up with snapshot data loss.
964     if (!snapshotEnabled) {
965       LOG.info("Snapshot feature is not enabled, missing log and hfile cleaners.");
966       Path snapshotDir = SnapshotDescriptionUtils.getSnapshotsDir(mfs.getRootDir());
967       if (fs.exists(snapshotDir)) {
968         FileStatus[] snapshots = FSUtils.listStatus(fs, snapshotDir,
969           new SnapshotDescriptionUtils.CompletedSnaphotDirectoriesFilter(fs));
970         if (snapshots != null) {
971           LOG.error("Snapshots are present, but cleaners are not enabled.");
972           checkSnapshotSupport();
973         }
974       }
975     }
976   }
977 }