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