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 static org.apache.hadoop.hbase.HConstants.DEFAULT_HBASE_RPC_TIMEOUT;
021import static org.apache.hadoop.hbase.HConstants.HBASE_RPC_TIMEOUT_KEY;
022
023import java.io.IOException;
024import java.util.List;
025import java.util.concurrent.CancellationException;
026import org.apache.hadoop.conf.Configuration;
027import org.apache.hadoop.fs.FileSystem;
028import org.apache.hadoop.fs.Path;
029import org.apache.hadoop.hbase.DoNotRetryIOException;
030import org.apache.hadoop.hbase.ServerName;
031import org.apache.hadoop.hbase.TableName;
032import org.apache.hadoop.hbase.client.RegionInfo;
033import org.apache.hadoop.hbase.client.TableDescriptor;
034import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
035import org.apache.hadoop.hbase.errorhandling.ForeignException;
036import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
037import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
038import org.apache.hadoop.hbase.executor.EventHandler;
039import org.apache.hadoop.hbase.executor.EventType;
040import org.apache.hadoop.hbase.master.MasterServices;
041import org.apache.hadoop.hbase.master.MetricsSnapshot;
042import org.apache.hadoop.hbase.master.SnapshotSentinel;
043import org.apache.hadoop.hbase.master.locking.LockManager;
044import org.apache.hadoop.hbase.master.locking.LockManager.MasterLock;
045import org.apache.hadoop.hbase.monitoring.MonitoredTask;
046import org.apache.hadoop.hbase.monitoring.TaskMonitor;
047import org.apache.hadoop.hbase.procedure2.LockType;
048import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
049import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
050import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
051import org.apache.hadoop.hbase.util.CommonFSUtils;
052import org.apache.hadoop.hbase.util.Pair;
053import org.apache.yetus.audience.InterfaceAudience;
054import org.apache.zookeeper.KeeperException;
055import org.slf4j.Logger;
056import org.slf4j.LoggerFactory;
057
058import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
059
060import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
061import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
062
063/**
064 * A handler for taking snapshots from the master. This is not a subclass of TableEventHandler
065 * because using that would incur an extra hbase:meta scan. The {@link #snapshotRegions(List)} call
066 * should get implemented for each snapshot flavor.
067 */
068@InterfaceAudience.Private
069public abstract class TakeSnapshotHandler extends EventHandler
070  implements SnapshotSentinel, ForeignExceptionSnare {
071  private static final Logger LOG = LoggerFactory.getLogger(TakeSnapshotHandler.class);
072  public static final String HBASE_SNAPSHOT_MASTER_LOCK_ACQUIRE_TIMEOUT =
073    "hbase.snapshot.master.lock.acquire.timeout";
074
075  private volatile boolean finished;
076
077  // none of these should ever be null
078  protected final MasterServices master;
079  protected final MetricsSnapshot metricsSnapshot = new MetricsSnapshot();
080  protected final SnapshotDescription snapshot;
081  protected final Configuration conf;
082  protected final FileSystem rootFs;
083  protected final FileSystem workingDirFs;
084  protected final Path rootDir;
085  private final Path snapshotDir;
086  protected final Path workingDir;
087  private final MasterSnapshotVerifier verifier;
088  protected final ForeignExceptionDispatcher monitor;
089  private final LockManager.MasterLock tableLock;
090  protected final MonitoredTask status;
091  protected final TableName snapshotTable;
092  protected final SnapshotManifest snapshotManifest;
093  protected final SnapshotManager snapshotManager;
094  /**
095   * Snapshot creation requires table lock. If any region of the table is in transition, table lock
096   * cannot be acquired by LockProcedure and hence snapshot creation could hang for potentially very
097   * long time. This timeout will ensure snapshot creation fails-fast by waiting for only given
098   * timeout.
099   */
100  private final long lockAcquireTimeoutMs;
101
102  protected TableDescriptor htd;
103
104  /**
105   * @param snapshot       descriptor of the snapshot to take
106   * @param masterServices master services provider
107   * @throws IllegalArgumentException if the working snapshot directory set from the configuration
108   *                                  is the same as the completed snapshot directory
109   * @throws IOException              if the file system of the working snapshot directory cannot be
110   *                                  determined
111   */
112  public TakeSnapshotHandler(SnapshotDescription snapshot, final MasterServices masterServices,
113    final SnapshotManager snapshotManager) throws IOException {
114    super(masterServices, EventType.C_M_SNAPSHOT_TABLE);
115    assert snapshot != null : "SnapshotDescription must not be nul1";
116    assert masterServices != null : "MasterServices must not be nul1";
117    this.master = masterServices;
118    this.conf = this.master.getConfiguration();
119    this.rootDir = this.master.getMasterFileSystem().getRootDir();
120    this.workingDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshot, rootDir, conf);
121    Preconditions.checkArgument(
122      !SnapshotDescriptionUtils.isSubDirectoryOf(workingDir, rootDir)
123        || SnapshotDescriptionUtils.isWithinDefaultWorkingDir(workingDir, conf),
124      "The working directory " + workingDir + " cannot be in the root directory unless it is "
125        + "within the default working directory");
126
127    this.snapshot = snapshot;
128    this.snapshotManager = snapshotManager;
129    this.snapshotTable = TableName.valueOf(snapshot.getTable());
130    this.rootFs = this.master.getMasterFileSystem().getFileSystem();
131    this.snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir);
132    this.workingDirFs = this.workingDir.getFileSystem(this.conf);
133    this.monitor = new ForeignExceptionDispatcher(snapshot.getName());
134
135    this.tableLock = master.getLockManager().createMasterLock(snapshotTable, LockType.EXCLUSIVE,
136      this.getClass().getName() + ": take snapshot " + snapshot.getName());
137
138    // prepare the verify
139    this.verifier = new MasterSnapshotVerifier(masterServices, snapshot, workingDirFs);
140    // update the running tasks
141    this.status = TaskMonitor.get().createStatus(
142      "Taking " + snapshot.getType() + " snapshot on table: " + snapshotTable, false, true);
143    this.snapshotManifest =
144      SnapshotManifest.create(conf, rootFs, workingDir, snapshot, monitor, status);
145    this.lockAcquireTimeoutMs = conf.getLong(HBASE_SNAPSHOT_MASTER_LOCK_ACQUIRE_TIMEOUT,
146      conf.getLong(HBASE_RPC_TIMEOUT_KEY, DEFAULT_HBASE_RPC_TIMEOUT));
147  }
148
149  private TableDescriptor loadTableDescriptor() throws IOException {
150    TableDescriptor htd = this.master.getTableDescriptors().get(snapshotTable);
151    if (htd == null) {
152      throw new IOException("TableDescriptor missing for " + snapshotTable);
153    }
154    if (htd.getMaxFileSize() == -1 && this.snapshot.getMaxFileSize() > 0) {
155      htd = TableDescriptorBuilder.newBuilder(htd).setValue(TableDescriptorBuilder.MAX_FILESIZE,
156        Long.toString(this.snapshot.getMaxFileSize())).build();
157    }
158    return htd;
159  }
160
161  @Override
162  public TakeSnapshotHandler prepare() throws Exception {
163    super.prepare();
164    // after this, you should ensure to release this lock in case of exceptions
165    if (this.tableLock.tryAcquire(this.lockAcquireTimeoutMs)) {
166      try {
167        this.htd = loadTableDescriptor(); // check that .tableinfo is present
168      } catch (Exception e) {
169        this.tableLock.release();
170        throw e;
171      }
172    } else {
173      LOG.error("Master lock could not be acquired in {} ms", lockAcquireTimeoutMs);
174      throw new DoNotRetryIOException("Master lock could not be acquired");
175    }
176    return this;
177  }
178
179  /**
180   * Execute the core common portions of taking a snapshot. The {@link #snapshotRegions(List)} call
181   * should get implemented for each snapshot flavor.
182   */
183  @Override
184  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "REC_CATCH_EXCEPTION",
185      justification = "Intentional")
186  public void process() {
187    String msg = "Running " + snapshot.getType() + " table snapshot " + snapshot.getName() + " "
188      + eventType + " on table " + snapshotTable;
189    LOG.info(msg);
190    MasterLock tableLockToRelease = this.tableLock;
191    status.setStatus(msg);
192    try {
193      if (downgradeToSharedTableLock()) {
194        // release the exclusive lock and hold the shared lock instead
195        tableLockToRelease = master.getLockManager().createMasterLock(snapshotTable,
196          LockType.SHARED, this.getClass().getName() + ": take snapshot " + snapshot.getName());
197        tableLock.release();
198        boolean isTableLockAcquired = tableLockToRelease.tryAcquire(this.lockAcquireTimeoutMs);
199        if (!isTableLockAcquired) {
200          LOG.error("Could not acquire shared lock on table {} in {} ms", snapshotTable,
201            lockAcquireTimeoutMs);
202          throw new IOException("Could not acquire shared lock on table " + snapshotTable);
203        }
204      }
205      // If regions move after this meta scan, the region specific snapshot should fail, triggering
206      // an external exception that gets captured here.
207
208      // write down the snapshot info in the working directory
209      SnapshotDescriptionUtils.writeSnapshotInfo(snapshot, workingDir, workingDirFs);
210      snapshotManifest.addTableDescriptor(this.htd);
211      monitor.rethrowException();
212
213      List<Pair<RegionInfo, ServerName>> regionsAndLocations =
214        master.getAssignmentManager().getTableRegionsAndLocations(snapshotTable, false);
215
216      // run the snapshot
217      snapshotRegions(regionsAndLocations);
218      monitor.rethrowException();
219
220      // flush the in-memory state, and write the single manifest
221      status.setStatus("Consolidate snapshot: " + snapshot.getName());
222      snapshotManifest.consolidate();
223
224      // verify the snapshot is valid
225      status.setStatus("Verifying snapshot: " + snapshot.getName());
226      verifier.verifySnapshot(workingDir, true);
227
228      // complete the snapshot, atomically moving from tmp to .snapshot dir.
229      SnapshotDescriptionUtils.completeSnapshot(this.snapshotDir, this.workingDir, this.rootFs,
230        this.workingDirFs, this.conf);
231      finished = true;
232      msg = "Snapshot " + snapshot.getName() + " of table " + snapshotTable + " completed";
233      status.markComplete(msg);
234      LOG.info(msg);
235      metricsSnapshot.addSnapshot(status.getCompletionTimestamp() - status.getStartTime());
236      if (master.getMasterCoprocessorHost() != null) {
237        master.getMasterCoprocessorHost()
238          .postCompletedSnapshotAction(ProtobufUtil.createSnapshotDesc(snapshot), this.htd);
239      }
240    } catch (Exception e) { // FindBugs: REC_CATCH_EXCEPTION
241      status.abort("Failed to complete snapshot " + snapshot.getName() + " on table "
242        + snapshotTable + " because " + e.getMessage());
243      String reason = "Failed taking snapshot " + ClientSnapshotDescriptionUtils.toString(snapshot)
244        + " due to exception:" + e.getMessage();
245      LOG.error(reason, e);
246      ForeignException ee = new ForeignException(reason, e);
247      monitor.receive(ee);
248      // need to mark this completed to close off and allow cleanup to happen.
249      cancel(reason);
250    } finally {
251      LOG.debug("Launching cleanup of working dir:" + workingDir);
252      try {
253        // if the working dir is still present, the snapshot has failed. it is present we delete
254        // it.
255        if (workingDirFs.exists(workingDir) && !workingDirFs.delete(workingDir, true)) {
256          LOG.error("Couldn't delete snapshot working directory: {}", workingDir);
257        }
258      } catch (IOException e) {
259        LOG.error("Couldn't get or delete snapshot working directory: {}", workingDir, e);
260      }
261      if (LOG.isDebugEnabled()) {
262        LOG.debug("Table snapshot journal : \n" + status.prettyPrintJournal());
263      }
264      tableLockToRelease.release();
265    }
266  }
267
268  /**
269   * When taking snapshot, first we must acquire the exclusive table lock to confirm that there are
270   * no ongoing merge/split procedures. But later, we should try our best to release the exclusive
271   * lock as this may hurt the availability, because we need to hold the shared lock when assigning
272   * regions.
273   * <p/>
274   * See HBASE-21480 for more details.
275   */
276  protected abstract boolean downgradeToSharedTableLock();
277
278  /**
279   * Snapshot the specified regions
280   */
281  protected abstract void snapshotRegions(List<Pair<RegionInfo, ServerName>> regions)
282    throws IOException, KeeperException;
283
284  /**
285   * Take a snapshot of the specified disabled region
286   */
287  protected void snapshotDisabledRegion(final RegionInfo regionInfo) throws IOException {
288    snapshotManifest.addRegion(CommonFSUtils.getTableDir(rootDir, snapshotTable), regionInfo);
289    monitor.rethrowException();
290    status.setStatus("Completed referencing HFiles for offline region " + regionInfo.toString()
291      + " of table: " + snapshotTable);
292  }
293
294  @Override
295  public void cancel(String why) {
296    if (finished) return;
297
298    this.finished = true;
299    LOG.info("Stop taking snapshot=" + ClientSnapshotDescriptionUtils.toString(snapshot)
300      + " because: " + why);
301    CancellationException ce = new CancellationException(why);
302    monitor.receive(new ForeignException(master.getServerName().toString(), ce));
303  }
304
305  @Override
306  public boolean isFinished() {
307    return finished;
308  }
309
310  @Override
311  public long getCompletionTimestamp() {
312    return this.status.getCompletionTimestamp();
313  }
314
315  @Override
316  public SnapshotDescription getSnapshot() {
317    return snapshot;
318  }
319
320  @Override
321  public ForeignException getExceptionIfFailed() {
322    return monitor.getException();
323  }
324
325  @Override
326  public void rethrowExceptionIfFailed() throws ForeignException {
327    monitor.rethrowException();
328  }
329
330  @Override
331  public void rethrowException() throws ForeignException {
332    monitor.rethrowException();
333  }
334
335  @Override
336  public boolean hasException() {
337    return monitor.hasException();
338  }
339
340  @Override
341  public ForeignException getException() {
342    return monitor.getException();
343  }
344}