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