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