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