001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.master.snapshot;
019
020import java.io.FileNotFoundException;
021import java.io.IOException;
022import java.util.ArrayList;
023import java.util.Collections;
024import java.util.HashMap;
025import java.util.HashSet;
026import java.util.Iterator;
027import java.util.List;
028import java.util.Map;
029import java.util.Set;
030import java.util.concurrent.ConcurrentHashMap;
031import java.util.concurrent.Executors;
032import java.util.concurrent.ScheduledExecutorService;
033import java.util.concurrent.ScheduledFuture;
034import java.util.concurrent.ThreadPoolExecutor;
035import java.util.concurrent.TimeUnit;
036import java.util.concurrent.locks.ReadWriteLock;
037import java.util.concurrent.locks.ReentrantReadWriteLock;
038
039import org.apache.hadoop.conf.Configuration;
040import org.apache.hadoop.fs.FSDataInputStream;
041import org.apache.hadoop.fs.FileStatus;
042import org.apache.hadoop.fs.FileSystem;
043import org.apache.hadoop.fs.Path;
044import org.apache.hadoop.hbase.HBaseInterfaceAudience;
045import org.apache.hadoop.hbase.HConstants;
046import org.apache.hadoop.hbase.MetaTableAccessor;
047import org.apache.hadoop.hbase.Stoppable;
048import org.apache.hadoop.hbase.TableName;
049import org.apache.hadoop.hbase.client.TableDescriptor;
050import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
051import org.apache.hadoop.hbase.client.TableState;
052import org.apache.hadoop.hbase.errorhandling.ForeignException;
053import org.apache.hadoop.hbase.executor.ExecutorService;
054import org.apache.hadoop.hbase.ipc.RpcServer;
055import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
056import org.apache.hadoop.hbase.master.MasterFileSystem;
057import org.apache.hadoop.hbase.master.MasterServices;
058import org.apache.hadoop.hbase.master.MetricsMaster;
059import org.apache.hadoop.hbase.master.SnapshotSentinel;
060import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
061import org.apache.hadoop.hbase.master.cleaner.HFileLinkCleaner;
062import org.apache.hadoop.hbase.master.procedure.CloneSnapshotProcedure;
063import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
064import org.apache.hadoop.hbase.master.procedure.RestoreSnapshotProcedure;
065import org.apache.hadoop.hbase.procedure.MasterProcedureManager;
066import org.apache.hadoop.hbase.procedure.Procedure;
067import org.apache.hadoop.hbase.procedure.ProcedureCoordinator;
068import org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs;
069import org.apache.hadoop.hbase.procedure.ZKProcedureCoordinator;
070import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
071import org.apache.hadoop.hbase.security.AccessDeniedException;
072import org.apache.hadoop.hbase.security.User;
073import org.apache.hadoop.hbase.security.access.AccessChecker;
074import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
075import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
076import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
077import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
078import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
079import org.apache.hadoop.hbase.snapshot.SnapshotDoesNotExistException;
080import org.apache.hadoop.hbase.snapshot.SnapshotExistsException;
081import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
082import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil;
083import org.apache.hadoop.hbase.snapshot.TablePartiallyOpenException;
084import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
085import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
086import org.apache.hadoop.hbase.util.FSUtils;
087import org.apache.hadoop.hbase.util.NonceKey;
088import org.apache.yetus.audience.InterfaceAudience;
089import org.apache.yetus.audience.InterfaceStability;
090import org.apache.zookeeper.KeeperException;
091import org.slf4j.Logger;
092import org.slf4j.LoggerFactory;
093import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
094import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair;
095import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription;
096import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
097import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription.Type;
098import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
099import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
100
101/**
102 * This class manages the procedure of taking and restoring snapshots. There is only one
103 * SnapshotManager for the master.
104 * <p>
105 * The class provides methods for monitoring in-progress snapshot actions.
106 * <p>
107 * Note: Currently there can only be one snapshot being taken at a time over the cluster. This is a
108 * simplification in the current implementation.
109 */
110@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
111@InterfaceStability.Unstable
112public class SnapshotManager extends MasterProcedureManager implements Stoppable {
113  private static final Logger LOG = LoggerFactory.getLogger(SnapshotManager.class);
114
115  /** By default, check to see if the snapshot is complete every WAKE MILLIS (ms) */
116  private static final int SNAPSHOT_WAKE_MILLIS_DEFAULT = 500;
117
118  /**
119   * Wait time before removing a finished sentinel from the in-progress map
120   *
121   * NOTE: This is used as a safety auto cleanup.
122   * The snapshot and restore handlers map entries are removed when a user asks if a snapshot or
123   * restore is completed. This operation is part of the HBaseAdmin snapshot/restore API flow.
124   * In case something fails on the client side and the snapshot/restore state is not reclaimed
125   * after a default timeout, the entry is removed from the in-progress map.
126   * At this point, if the user asks for the snapshot/restore status, the result will be
127   * snapshot done if exists or failed if it doesn't exists.
128   */
129  public static final String HBASE_SNAPSHOT_SENTINELS_CLEANUP_TIMEOUT_MILLIS =
130      "hbase.snapshot.sentinels.cleanup.timeoutMillis";
131  public static final long SNAPSHOT_SENTINELS_CLEANUP_TIMEOUT_MILLS_DEFAULT = 60 * 1000L;
132
133  /** Enable or disable snapshot support */
134  public static final String HBASE_SNAPSHOT_ENABLED = "hbase.snapshot.enabled";
135
136  /**
137   * Conf key for # of ms elapsed between checks for snapshot errors while waiting for
138   * completion.
139   */
140  private static final String SNAPSHOT_WAKE_MILLIS_KEY = "hbase.snapshot.master.wakeMillis";
141
142  /** Name of the operation to use in the controller */
143  public static final String ONLINE_SNAPSHOT_CONTROLLER_DESCRIPTION = "online-snapshot";
144
145  /** Conf key for # of threads used by the SnapshotManager thread pool */
146  public static final String SNAPSHOT_POOL_THREADS_KEY = "hbase.snapshot.master.threads";
147
148  /** number of current operations running on the master */
149  public static final int SNAPSHOT_POOL_THREADS_DEFAULT = 1;
150
151  private boolean stopped;
152  private MasterServices master;  // Needed by TableEventHandlers
153  private ProcedureCoordinator coordinator;
154
155  // Is snapshot feature enabled?
156  private boolean isSnapshotSupported = false;
157
158  // Snapshot handlers map, with table name as key.
159  // The map is always accessed and modified under the object lock using synchronized.
160  // snapshotTable() will insert an Handler in the table.
161  // isSnapshotDone() will remove the handler requested if the operation is finished.
162  private final Map<TableName, SnapshotSentinel> snapshotHandlers = new ConcurrentHashMap<>();
163  private final ScheduledExecutorService scheduleThreadPool =
164      Executors.newScheduledThreadPool(1, new ThreadFactoryBuilder()
165          .setNameFormat("SnapshotHandlerChoreCleaner").setDaemon(true).build());
166  private ScheduledFuture<?> snapshotHandlerChoreCleanerTask;
167
168  // Restore map, with table name as key, procedure ID as value.
169  // The map is always accessed and modified under the object lock using synchronized.
170  // restoreSnapshot()/cloneSnapshot() will insert a procedure ID in the map.
171  //
172  // TODO: just as the Apache HBase 1.x implementation, this map would not survive master
173  // restart/failover. This is just a stopgap implementation until implementation of taking
174  // snapshot using Procedure-V2.
175  private Map<TableName, Long> restoreTableToProcIdMap = new HashMap<>();
176
177  private Path rootDir;
178  private ExecutorService executorService;
179
180  /**
181   * Read write lock between taking snapshot and snapshot HFile cleaner. The cleaner should skip to
182   * check the HFiles if any snapshot is in progress, otherwise it may clean a HFile which would
183   * belongs to the newly creating snapshot. So we should grab the write lock first when cleaner
184   * start to work. (See HBASE-21387)
185   */
186  private ReentrantReadWriteLock takingSnapshotLock = new ReentrantReadWriteLock(true);
187
188  public SnapshotManager() {}
189
190  /**
191   * Fully specify all necessary components of a snapshot manager. Exposed for testing.
192   * @param master services for the master where the manager is running
193   * @param coordinator procedure coordinator instance.  exposed for testing.
194   * @param pool HBase ExecutorServcie instance, exposed for testing.
195   */
196  @VisibleForTesting
197  SnapshotManager(final MasterServices master, ProcedureCoordinator coordinator,
198      ExecutorService pool, int sentinelCleanInterval)
199      throws IOException, UnsupportedOperationException {
200    this.master = master;
201
202    this.rootDir = master.getMasterFileSystem().getRootDir();
203    Configuration conf = master.getConfiguration();
204    checkSnapshotSupport(conf, master.getMasterFileSystem());
205
206    this.coordinator = coordinator;
207    this.executorService = pool;
208    resetTempDir();
209    snapshotHandlerChoreCleanerTask = this.scheduleThreadPool.scheduleAtFixedRate(
210      this::cleanupSentinels, sentinelCleanInterval, sentinelCleanInterval, TimeUnit.SECONDS);
211  }
212
213  /**
214   * Gets the list of all completed snapshots.
215   * @return list of SnapshotDescriptions
216   * @throws IOException File system exception
217   */
218  public List<SnapshotDescription> getCompletedSnapshots() throws IOException {
219    return getCompletedSnapshots(SnapshotDescriptionUtils.getSnapshotsDir(rootDir), true);
220  }
221
222  /**
223   * Gets the list of all completed snapshots.
224   * @param snapshotDir snapshot directory
225   * @param withCpCall Whether to call CP hooks
226   * @return list of SnapshotDescriptions
227   * @throws IOException File system exception
228   */
229  private List<SnapshotDescription> getCompletedSnapshots(Path snapshotDir, boolean withCpCall)
230      throws IOException {
231    List<SnapshotDescription> snapshotDescs = new ArrayList<>();
232    // first create the snapshot root path and check to see if it exists
233    FileSystem fs = master.getMasterFileSystem().getFileSystem();
234    if (snapshotDir == null) snapshotDir = SnapshotDescriptionUtils.getSnapshotsDir(rootDir);
235
236    // if there are no snapshots, return an empty list
237    if (!fs.exists(snapshotDir)) {
238      return snapshotDescs;
239    }
240
241    // ignore all the snapshots in progress
242    FileStatus[] snapshots = fs.listStatus(snapshotDir,
243      new SnapshotDescriptionUtils.CompletedSnaphotDirectoriesFilter(fs));
244    MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost();
245    withCpCall = withCpCall && cpHost != null;
246    // loop through all the completed snapshots
247    for (FileStatus snapshot : snapshots) {
248      Path info = new Path(snapshot.getPath(), SnapshotDescriptionUtils.SNAPSHOTINFO_FILE);
249      // if the snapshot is bad
250      if (!fs.exists(info)) {
251        LOG.error("Snapshot information for " + snapshot.getPath() + " doesn't exist");
252        continue;
253      }
254      FSDataInputStream in = null;
255      try {
256        in = fs.open(info);
257        SnapshotDescription desc = SnapshotDescription.parseFrom(in);
258        org.apache.hadoop.hbase.client.SnapshotDescription descPOJO = (withCpCall)
259            ? ProtobufUtil.createSnapshotDesc(desc) : null;
260        if (withCpCall) {
261          try {
262            cpHost.preListSnapshot(descPOJO);
263          } catch (AccessDeniedException e) {
264            LOG.warn("Current user does not have access to " + desc.getName() + " snapshot. "
265                + "Either you should be owner of this snapshot or admin user.");
266            // Skip this and try for next snapshot
267            continue;
268          }
269        }
270        snapshotDescs.add(desc);
271
272        // call coproc post hook
273        if (withCpCall) {
274          cpHost.postListSnapshot(descPOJO);
275        }
276      } catch (IOException e) {
277        LOG.warn("Found a corrupted snapshot " + snapshot.getPath(), e);
278      } finally {
279        if (in != null) {
280          in.close();
281        }
282      }
283    }
284    return snapshotDescs;
285  }
286
287  /**
288   * Cleans up any snapshots in the snapshot/.tmp directory that were left from failed
289   * snapshot attempts.
290   *
291   * @throws IOException if we can't reach the filesystem
292   */
293  private void resetTempDir() throws IOException {
294    // cleanup any existing snapshots.
295    Path tmpdir = SnapshotDescriptionUtils.getWorkingSnapshotDir(rootDir);
296    if (master.getMasterFileSystem().getFileSystem().exists(tmpdir)) {
297      if (!master.getMasterFileSystem().getFileSystem().delete(tmpdir, true)) {
298        LOG.warn("Couldn't delete working snapshot directory: " + tmpdir);
299      }
300    }
301  }
302
303  /**
304   * Delete the specified snapshot
305   * @param snapshot
306   * @throws SnapshotDoesNotExistException If the specified snapshot does not exist.
307   * @throws IOException For filesystem IOExceptions
308   */
309  public void deleteSnapshot(SnapshotDescription snapshot) throws IOException {
310    // check to see if it is completed
311    if (!isSnapshotCompleted(snapshot)) {
312      throw new SnapshotDoesNotExistException(ProtobufUtil.createSnapshotDesc(snapshot));
313    }
314
315    String snapshotName = snapshot.getName();
316    // first create the snapshot description and check to see if it exists
317    FileSystem fs = master.getMasterFileSystem().getFileSystem();
318    Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, rootDir);
319    // Get snapshot info from file system. The one passed as parameter is a "fake" snapshotInfo with
320    // just the "name" and it does not contains the "real" snapshot information
321    snapshot = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir);
322
323    // call coproc pre hook
324    MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost();
325    org.apache.hadoop.hbase.client.SnapshotDescription snapshotPOJO = null;
326    if (cpHost != null) {
327      snapshotPOJO = ProtobufUtil.createSnapshotDesc(snapshot);
328      cpHost.preDeleteSnapshot(snapshotPOJO);
329    }
330
331    LOG.debug("Deleting snapshot: " + snapshotName);
332    // delete the existing snapshot
333    if (!fs.delete(snapshotDir, true)) {
334      throw new HBaseSnapshotException("Failed to delete snapshot directory: " + snapshotDir);
335    }
336
337    // call coproc post hook
338    if (cpHost != null) {
339      cpHost.postDeleteSnapshot(snapshotPOJO);
340    }
341
342  }
343
344  /**
345   * Check if the specified snapshot is done
346   *
347   * @param expected
348   * @return true if snapshot is ready to be restored, false if it is still being taken.
349   * @throws IOException IOException if error from HDFS or RPC
350   * @throws UnknownSnapshotException if snapshot is invalid or does not exist.
351   */
352  public boolean isSnapshotDone(SnapshotDescription expected) throws IOException {
353    // check the request to make sure it has a snapshot
354    if (expected == null) {
355      throw new UnknownSnapshotException(
356         "No snapshot name passed in request, can't figure out which snapshot you want to check.");
357    }
358
359    String ssString = ClientSnapshotDescriptionUtils.toString(expected);
360
361    // check to see if the sentinel exists,
362    // and if the task is complete removes it from the in-progress snapshots map.
363    SnapshotSentinel handler = removeSentinelIfFinished(this.snapshotHandlers, expected);
364
365    // stop tracking "abandoned" handlers
366    cleanupSentinels();
367
368    if (handler == null) {
369      // If there's no handler in the in-progress map, it means one of the following:
370      //   - someone has already requested the snapshot state
371      //   - the requested snapshot was completed long time ago (cleanupSentinels() timeout)
372      //   - the snapshot was never requested
373      // In those cases returns to the user the "done state" if the snapshots exists on disk,
374      // otherwise raise an exception saying that the snapshot is not running and doesn't exist.
375      if (!isSnapshotCompleted(expected)) {
376        throw new UnknownSnapshotException("Snapshot " + ssString
377            + " is not currently running or one of the known completed snapshots.");
378      }
379      // was done, return true;
380      return true;
381    }
382
383    // pass on any failure we find in the sentinel
384    try {
385      handler.rethrowExceptionIfFailed();
386    } catch (ForeignException e) {
387      // Give some procedure info on an exception.
388      String status;
389      Procedure p = coordinator.getProcedure(expected.getName());
390      if (p != null) {
391        status = p.getStatus();
392      } else {
393        status = expected.getName() + " not found in proclist " + coordinator.getProcedureNames();
394      }
395      throw new HBaseSnapshotException("Snapshot " + ssString +  " had an error.  " + status, e,
396        ProtobufUtil.createSnapshotDesc(expected));
397    }
398
399    // check to see if we are done
400    if (handler.isFinished()) {
401      LOG.debug("Snapshot '" + ssString + "' has completed, notifying client.");
402      return true;
403    } else if (LOG.isDebugEnabled()) {
404      LOG.debug("Snapshoting '" + ssString + "' is still in progress!");
405    }
406    return false;
407  }
408
409  /**
410   * Check to see if there is a snapshot in progress with the same name or on the same table.
411   * Currently we have a limitation only allowing a single snapshot per table at a time. Also we
412   * don't allow snapshot with the same name.
413   * @param snapshot description of the snapshot being checked.
414   * @return <tt>true</tt> if there is a snapshot in progress with the same name or on the same
415   *         table.
416   */
417  synchronized boolean isTakingSnapshot(final SnapshotDescription snapshot) {
418    TableName snapshotTable = TableName.valueOf(snapshot.getTable());
419    if (isTakingSnapshot(snapshotTable)) {
420      return true;
421    }
422    Iterator<Map.Entry<TableName, SnapshotSentinel>> it = this.snapshotHandlers.entrySet().iterator();
423    while (it.hasNext()) {
424      Map.Entry<TableName, SnapshotSentinel> entry = it.next();
425      SnapshotSentinel sentinel = entry.getValue();
426      if (snapshot.getName().equals(sentinel.getSnapshot().getName()) && !sentinel.isFinished()) {
427        return true;
428      }
429    }
430    return false;
431  }
432
433  /**
434   * Check to see if the specified table has a snapshot in progress.  Currently we have a
435   * limitation only allowing a single snapshot per table at a time.
436   * @param tableName name of the table being snapshotted.
437   * @return <tt>true</tt> if there is a snapshot in progress on the specified table.
438   */
439  public boolean isTakingSnapshot(final TableName tableName) {
440    SnapshotSentinel handler = this.snapshotHandlers.get(tableName);
441    return handler != null && !handler.isFinished();
442  }
443
444  /**
445   * Check to make sure that we are OK to run the passed snapshot. Checks to make sure that we
446   * aren't already running a snapshot or restore on the requested table.
447   * @param snapshot description of the snapshot we want to start
448   * @throws HBaseSnapshotException if the filesystem could not be prepared to start the snapshot
449   */
450  private synchronized void prepareToTakeSnapshot(SnapshotDescription snapshot)
451      throws HBaseSnapshotException {
452    FileSystem fs = master.getMasterFileSystem().getFileSystem();
453    Path workingDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshot, rootDir);
454    TableName snapshotTable =
455        TableName.valueOf(snapshot.getTable());
456
457    // make sure we aren't already running a snapshot
458    if (isTakingSnapshot(snapshot)) {
459      SnapshotSentinel handler = this.snapshotHandlers.get(snapshotTable);
460      throw new SnapshotCreationException("Rejected taking "
461          + ClientSnapshotDescriptionUtils.toString(snapshot)
462          + " because we are already running another snapshot "
463          + (handler != null ? ("on the same table " +
464              ClientSnapshotDescriptionUtils.toString(handler.getSnapshot()))
465              : "with the same name"), ProtobufUtil.createSnapshotDesc(snapshot));
466    }
467
468    // make sure we aren't running a restore on the same table
469    if (isRestoringTable(snapshotTable)) {
470      throw new SnapshotCreationException("Rejected taking "
471          + ClientSnapshotDescriptionUtils.toString(snapshot)
472          + " because we are already have a restore in progress on the same snapshot.");
473    }
474
475    try {
476      // delete the working directory, since we aren't running the snapshot. Likely leftovers
477      // from a failed attempt.
478      fs.delete(workingDir, true);
479
480      // recreate the working directory for the snapshot
481      if (!fs.mkdirs(workingDir)) {
482        throw new SnapshotCreationException(
483            "Couldn't create working directory (" + workingDir + ") for snapshot",
484            ProtobufUtil.createSnapshotDesc(snapshot));
485      }
486    } catch (HBaseSnapshotException e) {
487      throw e;
488    } catch (IOException e) {
489      throw new SnapshotCreationException(
490          "Exception while checking to see if snapshot could be started.", e,
491          ProtobufUtil.createSnapshotDesc(snapshot));
492    }
493  }
494
495  /**
496   * Take a snapshot of a disabled table.
497   * @param snapshot description of the snapshot to take. Modified to be {@link Type#DISABLED}.
498   * @throws HBaseSnapshotException if the snapshot could not be started
499   */
500  private synchronized void snapshotDisabledTable(SnapshotDescription snapshot)
501      throws HBaseSnapshotException {
502    // setup the snapshot
503    prepareToTakeSnapshot(snapshot);
504
505    // set the snapshot to be a disabled snapshot, since the client doesn't know about that
506    snapshot = snapshot.toBuilder().setType(Type.DISABLED).build();
507
508    // Take the snapshot of the disabled table
509    DisabledTableSnapshotHandler handler =
510        new DisabledTableSnapshotHandler(snapshot, master, this);
511    snapshotTable(snapshot, handler);
512  }
513
514  /**
515   * Take a snapshot of an enabled table.
516   * @param snapshot description of the snapshot to take.
517   * @throws HBaseSnapshotException if the snapshot could not be started
518   */
519  private synchronized void snapshotEnabledTable(SnapshotDescription snapshot)
520      throws HBaseSnapshotException {
521    // setup the snapshot
522    prepareToTakeSnapshot(snapshot);
523
524    // Take the snapshot of the enabled table
525    EnabledTableSnapshotHandler handler =
526        new EnabledTableSnapshotHandler(snapshot, master, this);
527    snapshotTable(snapshot, handler);
528  }
529
530  /**
531   * Take a snapshot using the specified handler.
532   * On failure the snapshot temporary working directory is removed.
533   * NOTE: prepareToTakeSnapshot() called before this one takes care of the rejecting the
534   *       snapshot request if the table is busy with another snapshot/restore operation.
535   * @param snapshot the snapshot description
536   * @param handler the snapshot handler
537   */
538  private synchronized void snapshotTable(SnapshotDescription snapshot,
539      final TakeSnapshotHandler handler) throws HBaseSnapshotException {
540    try {
541      handler.prepare();
542      this.executorService.submit(handler);
543      this.snapshotHandlers.put(TableName.valueOf(snapshot.getTable()), handler);
544    } catch (Exception e) {
545      // cleanup the working directory by trying to delete it from the fs.
546      Path workingDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshot, rootDir);
547      try {
548        if (!this.master.getMasterFileSystem().getFileSystem().delete(workingDir, true)) {
549          LOG.error("Couldn't delete working directory (" + workingDir + " for snapshot:" +
550              ClientSnapshotDescriptionUtils.toString(snapshot));
551        }
552      } catch (IOException e1) {
553        LOG.error("Couldn't delete working directory (" + workingDir + " for snapshot:" +
554            ClientSnapshotDescriptionUtils.toString(snapshot));
555      }
556      // fail the snapshot
557      throw new SnapshotCreationException("Could not build snapshot handler", e,
558        ProtobufUtil.createSnapshotDesc(snapshot));
559    }
560  }
561
562  public ReadWriteLock getTakingSnapshotLock() {
563    return this.takingSnapshotLock;
564  }
565
566  /**
567   * The snapshot operation processing as following: <br>
568   * 1. Create a Snapshot Handler, and do some initialization; <br>
569   * 2. Put the handler into snapshotHandlers <br>
570   * So when we consider if any snapshot is taking, we should consider both the takingSnapshotLock
571   * and snapshotHandlers;
572   * @return true to indicate that there're some running snapshots.
573   */
574  public synchronized boolean isTakingAnySnapshot() {
575    return this.takingSnapshotLock.getReadHoldCount() > 0 || this.snapshotHandlers.size() > 0;
576  }
577
578  /**
579   * Take a snapshot based on the enabled/disabled state of the table.
580   * @param snapshot
581   * @throws HBaseSnapshotException when a snapshot specific exception occurs.
582   * @throws IOException when some sort of generic IO exception occurs.
583   */
584  public void takeSnapshot(SnapshotDescription snapshot) throws IOException {
585    this.takingSnapshotLock.readLock().lock();
586    try {
587      takeSnapshotInternal(snapshot);
588    } finally {
589      this.takingSnapshotLock.readLock().unlock();
590    }
591  }
592
593  private void takeSnapshotInternal(SnapshotDescription snapshot) throws IOException {
594    // check to see if we already completed the snapshot
595    if (isSnapshotCompleted(snapshot)) {
596      throw new SnapshotExistsException(
597          "Snapshot '" + snapshot.getName() + "' already stored on the filesystem.",
598          ProtobufUtil.createSnapshotDesc(snapshot));
599    }
600
601    LOG.debug("No existing snapshot, attempting snapshot...");
602
603    // stop tracking "abandoned" handlers
604    cleanupSentinels();
605
606    // check to see if the table exists
607    TableDescriptor desc = null;
608    try {
609      desc = master.getTableDescriptors().get(
610          TableName.valueOf(snapshot.getTable()));
611    } catch (FileNotFoundException e) {
612      String msg = "Table:" + snapshot.getTable() + " info doesn't exist!";
613      LOG.error(msg);
614      throw new SnapshotCreationException(msg, e, ProtobufUtil.createSnapshotDesc(snapshot));
615    } catch (IOException e) {
616      throw new SnapshotCreationException(
617          "Error while geting table description for table " + snapshot.getTable(), e,
618          ProtobufUtil.createSnapshotDesc(snapshot));
619    }
620    if (desc == null) {
621      throw new SnapshotCreationException(
622          "Table '" + snapshot.getTable() + "' doesn't exist, can't take snapshot.",
623          ProtobufUtil.createSnapshotDesc(snapshot));
624    }
625    SnapshotDescription.Builder builder = snapshot.toBuilder();
626    // if not specified, set the snapshot format
627    if (!snapshot.hasVersion()) {
628      builder.setVersion(SnapshotDescriptionUtils.SNAPSHOT_LAYOUT_VERSION);
629    }
630    RpcServer.getRequestUser().ifPresent(user -> {
631      if (User.isHBaseSecurityEnabled(master.getConfiguration())) {
632        builder.setOwner(user.getShortName());
633      }
634    });
635    snapshot = builder.build();
636
637    // call pre coproc hook
638    MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost();
639    org.apache.hadoop.hbase.client.SnapshotDescription snapshotPOJO = null;
640    if (cpHost != null) {
641      snapshotPOJO = ProtobufUtil.createSnapshotDesc(snapshot);
642      cpHost.preSnapshot(snapshotPOJO, desc);
643    }
644
645    // if the table is enabled, then have the RS run actually the snapshot work
646    TableName snapshotTable = TableName.valueOf(snapshot.getTable());
647    if (master.getTableStateManager().isTableState(snapshotTable,
648        TableState.State.ENABLED)) {
649      LOG.debug("Table enabled, starting distributed snapshot.");
650      snapshotEnabledTable(snapshot);
651      LOG.debug("Started snapshot: " + ClientSnapshotDescriptionUtils.toString(snapshot));
652    }
653    // For disabled table, snapshot is created by the master
654    else if (master.getTableStateManager().isTableState(snapshotTable,
655        TableState.State.DISABLED)) {
656      LOG.debug("Table is disabled, running snapshot entirely on master.");
657      snapshotDisabledTable(snapshot);
658      LOG.debug("Started snapshot: " + ClientSnapshotDescriptionUtils.toString(snapshot));
659    } else {
660      LOG.error("Can't snapshot table '" + snapshot.getTable()
661          + "', isn't open or closed, we don't know what to do!");
662      TablePartiallyOpenException tpoe = new TablePartiallyOpenException(snapshot.getTable()
663          + " isn't fully open.");
664      throw new SnapshotCreationException("Table is not entirely open or closed", tpoe,
665        ProtobufUtil.createSnapshotDesc(snapshot));
666    }
667
668    // call post coproc hook
669    if (cpHost != null) {
670      cpHost.postSnapshot(snapshotPOJO, desc);
671    }
672  }
673
674  /**
675   * Set the handler for the current snapshot
676   * <p>
677   * Exposed for TESTING
678   * @param tableName
679   * @param handler handler the master should use
680   *
681   * TODO get rid of this if possible, repackaging, modify tests.
682   */
683  public synchronized void setSnapshotHandlerForTesting(
684      final TableName tableName,
685      final SnapshotSentinel handler) {
686    if (handler != null) {
687      this.snapshotHandlers.put(tableName, handler);
688    } else {
689      this.snapshotHandlers.remove(tableName);
690    }
691  }
692
693  /**
694   * @return distributed commit coordinator for all running snapshots
695   */
696  ProcedureCoordinator getCoordinator() {
697    return coordinator;
698  }
699
700  /**
701   * Check to see if the snapshot is one of the currently completed snapshots
702   * Returns true if the snapshot exists in the "completed snapshots folder".
703   *
704   * @param snapshot expected snapshot to check
705   * @return <tt>true</tt> if the snapshot is stored on the {@link FileSystem}, <tt>false</tt> if is
706   *         not stored
707   * @throws IOException if the filesystem throws an unexpected exception,
708   * @throws IllegalArgumentException if snapshot name is invalid.
709   */
710  private boolean isSnapshotCompleted(SnapshotDescription snapshot) throws IOException {
711    try {
712      final Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir);
713      FileSystem fs = master.getMasterFileSystem().getFileSystem();
714      // check to see if the snapshot already exists
715      return fs.exists(snapshotDir);
716    } catch (IllegalArgumentException iae) {
717      throw new UnknownSnapshotException("Unexpected exception thrown", iae);
718    }
719  }
720
721  /**
722   * Clone the specified snapshot.
723   * The clone will fail if the destination table has a snapshot or restore in progress.
724   *
725   * @param reqSnapshot Snapshot Descriptor from request
726   * @param tableName table to clone
727   * @param snapshot Snapshot Descriptor
728   * @param snapshotTableDesc Table Descriptor
729   * @param nonceKey unique identifier to prevent duplicated RPC
730   * @return procId the ID of the clone snapshot procedure
731   * @throws IOException
732   */
733  private long cloneSnapshot(final SnapshotDescription reqSnapshot, final TableName tableName,
734      final SnapshotDescription snapshot, final TableDescriptor snapshotTableDesc,
735      final NonceKey nonceKey, final boolean restoreAcl) throws IOException {
736    MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost();
737    TableDescriptor htd = TableDescriptorBuilder.copy(tableName, snapshotTableDesc);
738    org.apache.hadoop.hbase.client.SnapshotDescription snapshotPOJO = null;
739    if (cpHost != null) {
740      snapshotPOJO = ProtobufUtil.createSnapshotDesc(snapshot);
741      cpHost.preCloneSnapshot(snapshotPOJO, htd);
742    }
743    long procId;
744    try {
745      procId = cloneSnapshot(snapshot, htd, nonceKey, restoreAcl);
746    } catch (IOException e) {
747      LOG.error("Exception occurred while cloning the snapshot " + snapshot.getName()
748        + " as table " + tableName.getNameAsString(), e);
749      throw e;
750    }
751    LOG.info("Clone snapshot=" + snapshot.getName() + " as table=" + tableName);
752
753    if (cpHost != null) {
754      cpHost.postCloneSnapshot(snapshotPOJO, htd);
755    }
756    return procId;
757  }
758
759  /**
760   * Clone the specified snapshot into a new table.
761   * The operation will fail if the destination table has a snapshot or restore in progress.
762   *
763   * @param snapshot Snapshot Descriptor
764   * @param tableDescriptor Table Descriptor of the table to create
765   * @param nonceKey unique identifier to prevent duplicated RPC
766   * @return procId the ID of the clone snapshot procedure
767   */
768  synchronized long cloneSnapshot(final SnapshotDescription snapshot,
769      final TableDescriptor tableDescriptor, final NonceKey nonceKey, final boolean restoreAcl)
770      throws HBaseSnapshotException {
771    TableName tableName = tableDescriptor.getTableName();
772
773    // make sure we aren't running a snapshot on the same table
774    if (isTakingSnapshot(tableName)) {
775      throw new RestoreSnapshotException("Snapshot in progress on the restore table=" + tableName);
776    }
777
778    // make sure we aren't running a restore on the same table
779    if (isRestoringTable(tableName)) {
780      throw new RestoreSnapshotException("Restore already in progress on the table=" + tableName);
781    }
782
783    try {
784      long procId = master.getMasterProcedureExecutor().submitProcedure(
785        new CloneSnapshotProcedure(master.getMasterProcedureExecutor().getEnvironment(),
786                tableDescriptor, snapshot, restoreAcl),
787        nonceKey);
788      this.restoreTableToProcIdMap.put(tableName, procId);
789      return procId;
790    } catch (Exception e) {
791      String msg = "Couldn't clone the snapshot="
792        + ClientSnapshotDescriptionUtils.toString(snapshot) + " on table=" + tableName;
793      LOG.error(msg, e);
794      throw new RestoreSnapshotException(msg, e);
795    }
796  }
797
798  /**
799   * Restore or Clone the specified snapshot
800   * @param reqSnapshot
801   * @param nonceKey unique identifier to prevent duplicated RPC
802   * @throws IOException
803   */
804  public long restoreOrCloneSnapshot(final SnapshotDescription reqSnapshot, final NonceKey nonceKey,
805      final boolean restoreAcl) throws IOException {
806    FileSystem fs = master.getMasterFileSystem().getFileSystem();
807    Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(reqSnapshot, rootDir);
808
809    // check if the snapshot exists
810    if (!fs.exists(snapshotDir)) {
811      LOG.error("A Snapshot named '" + reqSnapshot.getName() + "' does not exist.");
812      throw new SnapshotDoesNotExistException(
813        ProtobufUtil.createSnapshotDesc(reqSnapshot));
814    }
815
816    // Get snapshot info from file system. The reqSnapshot is a "fake" snapshotInfo with
817    // just the snapshot "name" and table name to restore. It does not contains the "real" snapshot
818    // information.
819    SnapshotDescription snapshot = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir);
820    SnapshotManifest manifest = SnapshotManifest.open(master.getConfiguration(), fs,
821        snapshotDir, snapshot);
822    TableDescriptor snapshotTableDesc = manifest.getTableDescriptor();
823    TableName tableName = TableName.valueOf(reqSnapshot.getTable());
824
825    // stop tracking "abandoned" handlers
826    cleanupSentinels();
827
828    // Verify snapshot validity
829    SnapshotReferenceUtil.verifySnapshot(master.getConfiguration(), fs, manifest);
830
831    // Execute the restore/clone operation
832    long procId;
833    if (MetaTableAccessor.tableExists(master.getConnection(), tableName)) {
834      procId = restoreSnapshot(reqSnapshot, tableName, snapshot, snapshotTableDesc, nonceKey,
835        restoreAcl);
836    } else {
837      procId =
838          cloneSnapshot(reqSnapshot, tableName, snapshot, snapshotTableDesc, nonceKey, restoreAcl);
839    }
840    return procId;
841  }
842
843  /**
844   * Restore the specified snapshot. The restore will fail if the destination table has a snapshot
845   * or restore in progress.
846   * @param reqSnapshot Snapshot Descriptor from request
847   * @param tableName table to restore
848   * @param snapshot Snapshot Descriptor
849   * @param snapshotTableDesc Table Descriptor
850   * @param nonceKey unique identifier to prevent duplicated RPC
851   * @param restoreAcl true to restore acl of snapshot
852   * @return procId the ID of the restore snapshot procedure
853   * @throws IOException
854   */
855  private long restoreSnapshot(final SnapshotDescription reqSnapshot, final TableName tableName,
856      final SnapshotDescription snapshot, final TableDescriptor snapshotTableDesc,
857      final NonceKey nonceKey, final boolean restoreAcl) throws IOException {
858    MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost();
859
860    if (master.getTableStateManager().isTableState(
861      TableName.valueOf(snapshot.getTable()), TableState.State.ENABLED)) {
862      throw new UnsupportedOperationException("Table '" +
863        TableName.valueOf(snapshot.getTable()) + "' must be disabled in order to " +
864        "perform a restore operation.");
865    }
866
867    // call Coprocessor pre hook
868    org.apache.hadoop.hbase.client.SnapshotDescription snapshotPOJO = null;
869    if (cpHost != null) {
870      snapshotPOJO = ProtobufUtil.createSnapshotDesc(snapshot);
871      cpHost.preRestoreSnapshot(snapshotPOJO, snapshotTableDesc);
872    }
873
874    long procId;
875    try {
876      procId = restoreSnapshot(snapshot, snapshotTableDesc, nonceKey, restoreAcl);
877    } catch (IOException e) {
878      LOG.error("Exception occurred while restoring the snapshot " + snapshot.getName()
879        + " as table " + tableName.getNameAsString(), e);
880      throw e;
881    }
882    LOG.info("Restore snapshot=" + snapshot.getName() + " as table=" + tableName);
883
884    if (cpHost != null) {
885      cpHost.postRestoreSnapshot(snapshotPOJO, snapshotTableDesc);
886    }
887
888    return procId;
889  }
890
891  /**
892   * Restore the specified snapshot. The restore will fail if the destination table has a snapshot
893   * or restore in progress.
894   * @param snapshot Snapshot Descriptor
895   * @param tableDescriptor Table Descriptor
896   * @param nonceKey unique identifier to prevent duplicated RPC
897   * @param restoreAcl true to restore acl of snapshot
898   * @return procId the ID of the restore snapshot procedure
899   */
900  private synchronized long restoreSnapshot(final SnapshotDescription snapshot,
901      final TableDescriptor tableDescriptor, final NonceKey nonceKey, final boolean restoreAcl)
902      throws HBaseSnapshotException {
903    final TableName tableName = tableDescriptor.getTableName();
904
905    // make sure we aren't running a snapshot on the same table
906    if (isTakingSnapshot(tableName)) {
907      throw new RestoreSnapshotException("Snapshot in progress on the restore table=" + tableName);
908    }
909
910    // make sure we aren't running a restore on the same table
911    if (isRestoringTable(tableName)) {
912      throw new RestoreSnapshotException("Restore already in progress on the table=" + tableName);
913    }
914
915    try {
916      long procId = master.getMasterProcedureExecutor().submitProcedure(
917        new RestoreSnapshotProcedure(master.getMasterProcedureExecutor().getEnvironment(),
918                tableDescriptor, snapshot, restoreAcl),
919        nonceKey);
920      this.restoreTableToProcIdMap.put(tableName, procId);
921      return procId;
922    } catch (Exception e) {
923      String msg = "Couldn't restore the snapshot=" + ClientSnapshotDescriptionUtils.toString(
924          snapshot)  +
925          " on table=" + tableName;
926      LOG.error(msg, e);
927      throw new RestoreSnapshotException(msg, e);
928    }
929  }
930
931  /**
932   * Verify if the restore of the specified table is in progress.
933   *
934   * @param tableName table under restore
935   * @return <tt>true</tt> if there is a restore in progress of the specified table.
936   */
937  private synchronized boolean isRestoringTable(final TableName tableName) {
938    Long procId = this.restoreTableToProcIdMap.get(tableName);
939    if (procId == null) {
940      return false;
941    }
942    ProcedureExecutor<MasterProcedureEnv> procExec = master.getMasterProcedureExecutor();
943    if (procExec.isRunning() && !procExec.isFinished(procId)) {
944      return true;
945    } else {
946      this.restoreTableToProcIdMap.remove(tableName);
947      return false;
948    }
949  }
950
951  /**
952   * Return the handler if it is currently live and has the same snapshot target name.
953   * The handler is removed from the sentinels map if completed.
954   * @param sentinels live handlers
955   * @param snapshot snapshot description
956   * @return null if doesn't match, else a live handler.
957   */
958  private synchronized SnapshotSentinel removeSentinelIfFinished(
959      final Map<TableName, SnapshotSentinel> sentinels,
960      final SnapshotDescription snapshot) {
961    if (!snapshot.hasTable()) {
962      return null;
963    }
964
965    TableName snapshotTable = TableName.valueOf(snapshot.getTable());
966    SnapshotSentinel h = sentinels.get(snapshotTable);
967    if (h == null) {
968      return null;
969    }
970
971    if (!h.getSnapshot().getName().equals(snapshot.getName())) {
972      // specified snapshot is to the one currently running
973      return null;
974    }
975
976    // Remove from the "in-progress" list once completed
977    if (h.isFinished()) {
978      sentinels.remove(snapshotTable);
979    }
980
981    return h;
982  }
983
984  /**
985   * Removes "abandoned" snapshot/restore requests.
986   * As part of the HBaseAdmin snapshot/restore API the operation status is checked until completed,
987   * and the in-progress maps are cleaned up when the status of a completed task is requested.
988   * To avoid having sentinels staying around for long time if something client side is failed,
989   * each operation tries to clean up the in-progress maps sentinels finished from a long time.
990   */
991  private void cleanupSentinels() {
992    cleanupSentinels(this.snapshotHandlers);
993    cleanupCompletedRestoreInMap();
994  }
995
996  /**
997   * Remove the sentinels that are marked as finished and the completion time
998   * has exceeded the removal timeout.
999   * @param sentinels map of sentinels to clean
1000   */
1001  private synchronized void cleanupSentinels(final Map<TableName, SnapshotSentinel> sentinels) {
1002    long currentTime = EnvironmentEdgeManager.currentTime();
1003    long sentinelsCleanupTimeoutMillis =
1004        master.getConfiguration().getLong(HBASE_SNAPSHOT_SENTINELS_CLEANUP_TIMEOUT_MILLIS,
1005          SNAPSHOT_SENTINELS_CLEANUP_TIMEOUT_MILLS_DEFAULT);
1006    Iterator<Map.Entry<TableName, SnapshotSentinel>> it = sentinels.entrySet().iterator();
1007    while (it.hasNext()) {
1008      Map.Entry<TableName, SnapshotSentinel> entry = it.next();
1009      SnapshotSentinel sentinel = entry.getValue();
1010      if (sentinel.isFinished()
1011          && (currentTime - sentinel.getCompletionTimestamp()) > sentinelsCleanupTimeoutMillis) {
1012        it.remove();
1013      }
1014    }
1015  }
1016
1017  /**
1018   * Remove the procedures that are marked as finished
1019   */
1020  private synchronized void cleanupCompletedRestoreInMap() {
1021    ProcedureExecutor<MasterProcedureEnv> procExec = master.getMasterProcedureExecutor();
1022    Iterator<Map.Entry<TableName, Long>> it = restoreTableToProcIdMap.entrySet().iterator();
1023    while (it.hasNext()) {
1024      Map.Entry<TableName, Long> entry = it.next();
1025      Long procId = entry.getValue();
1026      if (procExec.isRunning() && procExec.isFinished(procId)) {
1027        it.remove();
1028      }
1029    }
1030  }
1031
1032  //
1033  // Implementing Stoppable interface
1034  //
1035
1036  @Override
1037  public void stop(String why) {
1038    // short circuit
1039    if (this.stopped) return;
1040    // make sure we get stop
1041    this.stopped = true;
1042    // pass the stop onto take snapshot handlers
1043    for (SnapshotSentinel snapshotHandler: this.snapshotHandlers.values()) {
1044      snapshotHandler.cancel(why);
1045    }
1046    if (snapshotHandlerChoreCleanerTask != null) {
1047      snapshotHandlerChoreCleanerTask.cancel(true);
1048    }
1049    try {
1050      if (coordinator != null) {
1051        coordinator.close();
1052      }
1053    } catch (IOException e) {
1054      LOG.error("stop ProcedureCoordinator error", e);
1055    }
1056  }
1057
1058  @Override
1059  public boolean isStopped() {
1060    return this.stopped;
1061  }
1062
1063  /**
1064   * Throws an exception if snapshot operations (take a snapshot, restore, clone) are not supported.
1065   * Called at the beginning of snapshot() and restoreSnapshot() methods.
1066   * @throws UnsupportedOperationException if snapshot are not supported
1067   */
1068  public void checkSnapshotSupport() throws UnsupportedOperationException {
1069    if (!this.isSnapshotSupported) {
1070      throw new UnsupportedOperationException(
1071        "To use snapshots, You must add to the hbase-site.xml of the HBase Master: '" +
1072          HBASE_SNAPSHOT_ENABLED + "' property with value 'true'.");
1073    }
1074  }
1075
1076  /**
1077   * Called at startup, to verify if snapshot operation is supported, and to avoid
1078   * starting the master if there're snapshots present but the cleaners needed are missing.
1079   * Otherwise we can end up with snapshot data loss.
1080   * @param conf The {@link Configuration} object to use
1081   * @param mfs The MasterFileSystem to use
1082   * @throws IOException in case of file-system operation failure
1083   * @throws UnsupportedOperationException in case cleaners are missing and
1084   *         there're snapshot in the system
1085   */
1086  private void checkSnapshotSupport(final Configuration conf, final MasterFileSystem mfs)
1087      throws IOException, UnsupportedOperationException {
1088    // Verify if snapshot is disabled by the user
1089    String enabled = conf.get(HBASE_SNAPSHOT_ENABLED);
1090    boolean snapshotEnabled = conf.getBoolean(HBASE_SNAPSHOT_ENABLED, false);
1091    boolean userDisabled = (enabled != null && enabled.trim().length() > 0 && !snapshotEnabled);
1092
1093    // Extract cleaners from conf
1094    Set<String> hfileCleaners = new HashSet<>();
1095    String[] cleaners = conf.getStrings(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS);
1096    if (cleaners != null) Collections.addAll(hfileCleaners, cleaners);
1097
1098    Set<String> logCleaners = new HashSet<>();
1099    cleaners = conf.getStrings(HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS);
1100    if (cleaners != null) Collections.addAll(logCleaners, cleaners);
1101
1102    // check if an older version of snapshot directory was present
1103    Path oldSnapshotDir = new Path(mfs.getRootDir(), HConstants.OLD_SNAPSHOT_DIR_NAME);
1104    FileSystem fs = mfs.getFileSystem();
1105    List<SnapshotDescription> ss = getCompletedSnapshots(new Path(rootDir, oldSnapshotDir), false);
1106    if (ss != null && !ss.isEmpty()) {
1107      LOG.error("Snapshots from an earlier release were found under: " + oldSnapshotDir);
1108      LOG.error("Please rename the directory as " + HConstants.SNAPSHOT_DIR_NAME);
1109    }
1110
1111    // If the user has enabled the snapshot, we force the cleaners to be present
1112    // otherwise we still need to check if cleaners are enabled or not and verify
1113    // that there're no snapshot in the .snapshot folder.
1114    if (snapshotEnabled) {
1115      // Inject snapshot cleaners, if snapshot.enable is true
1116      hfileCleaners.add(SnapshotHFileCleaner.class.getName());
1117      hfileCleaners.add(HFileLinkCleaner.class.getName());
1118
1119      // Set cleaners conf
1120      conf.setStrings(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS,
1121        hfileCleaners.toArray(new String[hfileCleaners.size()]));
1122      conf.setStrings(HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS,
1123        logCleaners.toArray(new String[logCleaners.size()]));
1124    } else {
1125      // Verify if cleaners are present
1126      snapshotEnabled =
1127        hfileCleaners.contains(SnapshotHFileCleaner.class.getName()) &&
1128        hfileCleaners.contains(HFileLinkCleaner.class.getName());
1129
1130      // Warn if the cleaners are enabled but the snapshot.enabled property is false/not set.
1131      if (snapshotEnabled) {
1132        LOG.warn("Snapshot log and hfile cleaners are present in the configuration, " +
1133          "but the '" + HBASE_SNAPSHOT_ENABLED + "' property " +
1134          (userDisabled ? "is set to 'false'." : "is not set."));
1135      }
1136    }
1137
1138    // Mark snapshot feature as enabled if cleaners are present and user has not disabled it.
1139    this.isSnapshotSupported = snapshotEnabled && !userDisabled;
1140
1141    // If cleaners are not enabled, verify that there're no snapshot in the .snapshot folder
1142    // otherwise we end up with snapshot data loss.
1143    if (!snapshotEnabled) {
1144      LOG.info("Snapshot feature is not enabled, missing log and hfile cleaners.");
1145      Path snapshotDir = SnapshotDescriptionUtils.getSnapshotsDir(mfs.getRootDir());
1146      if (fs.exists(snapshotDir)) {
1147        FileStatus[] snapshots = FSUtils.listStatus(fs, snapshotDir,
1148          new SnapshotDescriptionUtils.CompletedSnaphotDirectoriesFilter(fs));
1149        if (snapshots != null) {
1150          LOG.error("Snapshots are present, but cleaners are not enabled.");
1151          checkSnapshotSupport();
1152        }
1153      }
1154    }
1155  }
1156
1157  @Override
1158  public void initialize(MasterServices master, MetricsMaster metricsMaster) throws KeeperException,
1159      IOException, UnsupportedOperationException {
1160    this.master = master;
1161
1162    this.rootDir = master.getMasterFileSystem().getRootDir();
1163    checkSnapshotSupport(master.getConfiguration(), master.getMasterFileSystem());
1164
1165    // get the configuration for the coordinator
1166    Configuration conf = master.getConfiguration();
1167    long wakeFrequency = conf.getInt(SNAPSHOT_WAKE_MILLIS_KEY, SNAPSHOT_WAKE_MILLIS_DEFAULT);
1168    long timeoutMillis = Math.max(conf.getLong(SnapshotDescriptionUtils.SNAPSHOT_TIMEOUT_MILLIS_KEY,
1169                    SnapshotDescriptionUtils.SNAPSHOT_TIMEOUT_MILLIS_DEFAULT),
1170            conf.getLong(SnapshotDescriptionUtils.MASTER_SNAPSHOT_TIMEOUT_MILLIS,
1171                    SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME));
1172    int opThreads = conf.getInt(SNAPSHOT_POOL_THREADS_KEY, SNAPSHOT_POOL_THREADS_DEFAULT);
1173
1174    // setup the default procedure coordinator
1175    String name = master.getServerName().toString();
1176    ThreadPoolExecutor tpool = ProcedureCoordinator.defaultPool(name, opThreads);
1177    ProcedureCoordinatorRpcs comms = new ZKProcedureCoordinator(
1178        master.getZooKeeper(), SnapshotManager.ONLINE_SNAPSHOT_CONTROLLER_DESCRIPTION, name);
1179
1180    this.coordinator = new ProcedureCoordinator(comms, tpool, timeoutMillis, wakeFrequency);
1181    this.executorService = master.getExecutorService();
1182    resetTempDir();
1183    snapshotHandlerChoreCleanerTask =
1184        scheduleThreadPool.scheduleAtFixedRate(this::cleanupSentinels, 10, 10, TimeUnit.SECONDS);
1185  }
1186
1187  @Override
1188  public String getProcedureSignature() {
1189    return ONLINE_SNAPSHOT_CONTROLLER_DESCRIPTION;
1190  }
1191
1192  @Override
1193  public void execProcedure(ProcedureDescription desc) throws IOException {
1194    takeSnapshot(toSnapshotDescription(desc));
1195  }
1196
1197  @Override
1198  public void checkPermissions(ProcedureDescription desc, AccessChecker accessChecker, User user)
1199      throws IOException {
1200    // Done by AccessController as part of preSnapshot coprocessor hook (legacy code path).
1201    // In future, when we AC is removed for good, that check should be moved here.
1202  }
1203
1204  @Override
1205  public boolean isProcedureDone(ProcedureDescription desc) throws IOException {
1206    return isSnapshotDone(toSnapshotDescription(desc));
1207  }
1208
1209  private SnapshotDescription toSnapshotDescription(ProcedureDescription desc)
1210      throws IOException {
1211    SnapshotDescription.Builder builder = SnapshotDescription.newBuilder();
1212    if (!desc.hasInstance()) {
1213      throw new IOException("Snapshot name is not defined: " + desc.toString());
1214    }
1215    String snapshotName = desc.getInstance();
1216    List<NameStringPair> props = desc.getConfigurationList();
1217    String table = null;
1218    for (NameStringPair prop : props) {
1219      if ("table".equalsIgnoreCase(prop.getName())) {
1220        table = prop.getValue();
1221      }
1222    }
1223    if (table == null) {
1224      throw new IOException("Snapshot table is not defined: " + desc.toString());
1225    }
1226    TableName tableName = TableName.valueOf(table);
1227    builder.setTable(tableName.getNameAsString());
1228    builder.setName(snapshotName);
1229    builder.setType(SnapshotDescription.Type.FLUSH);
1230    return builder.build();
1231  }
1232}