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