View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.master.snapshot;
19  
20  import java.io.FileNotFoundException;
21  import java.io.IOException;
22  import java.util.HashSet;
23  import java.util.List;
24  import java.util.Set;
25  import java.util.concurrent.CancellationException;
26  
27  import org.apache.commons.logging.Log;
28  import org.apache.commons.logging.LogFactory;
29  import org.apache.hadoop.classification.InterfaceAudience;
30  import org.apache.hadoop.conf.Configuration;
31  import org.apache.hadoop.fs.FileSystem;
32  import org.apache.hadoop.fs.Path;
33  import org.apache.hadoop.hbase.TableName;
34  import org.apache.hadoop.hbase.HRegionInfo;
35  import org.apache.hadoop.hbase.HTableDescriptor;
36  import org.apache.hadoop.hbase.ServerName;
37  import org.apache.hadoop.hbase.catalog.MetaReader;
38  import org.apache.hadoop.hbase.errorhandling.ForeignException;
39  import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
40  import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
41  import org.apache.hadoop.hbase.executor.EventHandler;
42  import org.apache.hadoop.hbase.executor.EventType;
43  import org.apache.hadoop.hbase.master.MasterServices;
44  import org.apache.hadoop.hbase.master.MetricsSnapshot;
45  import org.apache.hadoop.hbase.master.SnapshotSentinel;
46  import org.apache.hadoop.hbase.master.TableLockManager;
47  import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
48  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
49  import org.apache.hadoop.hbase.monitoring.TaskMonitor;
50  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
51  import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
52  import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
53  import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
54  import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
55  import org.apache.hadoop.hbase.util.FSUtils;
56  import org.apache.hadoop.hbase.util.Pair;
57  import org.apache.zookeeper.KeeperException;
58  
59  /**
60   * A handler for taking snapshots from the master.
61   *
62   * This is not a subclass of TableEventHandler because using that would incur an extra hbase:meta scan.
63   *
64   * The {@link #snapshotRegions(List)} call should get implemented for each snapshot flavor.
65   */
66  @InterfaceAudience.Private
67  public abstract class TakeSnapshotHandler extends EventHandler implements SnapshotSentinel,
68      ForeignExceptionSnare {
69    private static final Log LOG = LogFactory.getLog(TakeSnapshotHandler.class);
70  
71    private volatile boolean finished;
72  
73    // none of these should ever be null
74    protected final MasterServices master;
75    protected final MetricsSnapshot metricsSnapshot = new MetricsSnapshot();
76    protected final SnapshotDescription snapshot;
77    protected final Configuration conf;
78    protected final FileSystem fs;
79    protected final Path rootDir;
80    private final Path snapshotDir;
81    protected final Path workingDir;
82    private final MasterSnapshotVerifier verifier;
83    protected final ForeignExceptionDispatcher monitor;
84    protected final TableLockManager tableLockManager;
85    protected final TableLock tableLock;
86    protected final MonitoredTask status;
87    protected final TableName snapshotTable;
88    protected final SnapshotManifest snapshotManifest;
89  
90    protected HTableDescriptor htd;
91  
92    /**
93     * @param snapshot descriptor of the snapshot to take
94     * @param masterServices master services provider
95     */
96    public TakeSnapshotHandler(SnapshotDescription snapshot, final MasterServices masterServices) {
97      super(masterServices, EventType.C_M_SNAPSHOT_TABLE);
98      assert snapshot != null : "SnapshotDescription must not be nul1";
99      assert masterServices != null : "MasterServices must not be nul1";
100 
101     this.master = masterServices;
102     this.snapshot = snapshot;
103     this.snapshotTable = TableName.valueOf(snapshot.getTable());
104     this.conf = this.master.getConfiguration();
105     this.fs = this.master.getMasterFileSystem().getFileSystem();
106     this.rootDir = this.master.getMasterFileSystem().getRootDir();
107     this.snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir);
108     this.workingDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshot, rootDir);
109     this.monitor = new ForeignExceptionDispatcher(snapshot.getName());
110     this.snapshotManifest = SnapshotManifest.create(conf, fs, workingDir, snapshot, monitor);
111 
112     this.tableLockManager = master.getTableLockManager();
113     this.tableLock = this.tableLockManager.writeLock(
114         snapshotTable,
115         EventType.C_M_SNAPSHOT_TABLE.toString());
116 
117     // prepare the verify
118     this.verifier = new MasterSnapshotVerifier(masterServices, snapshot, rootDir);
119     // update the running tasks
120     this.status = TaskMonitor.get().createStatus(
121       "Taking " + snapshot.getType() + " snapshot on table: " + snapshotTable);
122   }
123 
124   private HTableDescriptor loadTableDescriptor()
125       throws FileNotFoundException, IOException {
126     HTableDescriptor htd =
127       this.master.getTableDescriptors().get(snapshotTable);
128     if (htd == null) {
129       throw new IOException("HTableDescriptor missing for " + snapshotTable);
130     }
131     return htd;
132   }
133 
134   public TakeSnapshotHandler prepare() throws Exception {
135     super.prepare();
136     this.tableLock.acquire(); // after this, you should ensure to release this lock in
137                               // case of exceptions
138     boolean success = false;
139     try {
140       this.htd = loadTableDescriptor(); // check that .tableinfo is present
141       success = true;
142     } finally {
143       if (!success) {
144         releaseTableLock();
145       }
146     }
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   public void process() {
157     String msg = "Running " + snapshot.getType() + " table snapshot " + snapshot.getName() + " "
158         + eventType + " on table " + snapshotTable;
159     LOG.info(msg);
160     status.setStatus(msg);
161     try {
162       // If regions move after this meta scan, the region specific snapshot should fail, triggering
163       // an external exception that gets captured here.
164 
165       // write down the snapshot info in the working directory
166       SnapshotDescriptionUtils.writeSnapshotInfo(snapshot, workingDir, fs);
167       snapshotManifest.addTableDescriptor(this.htd);
168       monitor.rethrowException();
169 
170       List<Pair<HRegionInfo, ServerName>> regionsAndLocations =
171           MetaReader.getTableRegionsAndLocations(this.server.getCatalogTracker(),
172               snapshotTable, false);
173 
174       // run the snapshot
175       snapshotRegions(regionsAndLocations);
176       monitor.rethrowException();
177 
178       // extract each pair to separate lists
179       Set<String> serverNames = new HashSet<String>();
180       for (Pair<HRegionInfo, ServerName> p : regionsAndLocations) {
181         if (p != null && p.getFirst() != null && p.getSecond() != null) {
182           HRegionInfo hri = p.getFirst();
183           if (hri.isOffline() && (hri.isSplit() || hri.isSplitParent())) continue;
184           serverNames.add(p.getSecond().toString());
185         }
186       }
187 
188       // flush the in-memory state, and write the single manifest
189       status.setStatus("Consolidate snapshot: " + snapshot.getName());
190       snapshotManifest.consolidate();
191 
192       // verify the snapshot is valid
193       status.setStatus("Verifying snapshot: " + snapshot.getName());
194       verifier.verifySnapshot(this.workingDir, serverNames);
195 
196       // complete the snapshot, atomically moving from tmp to .snapshot dir.
197       completeSnapshot(this.snapshotDir, this.workingDir, this.fs);
198       msg = "Snapshot " + snapshot.getName() + " of table " + snapshotTable + " completed";
199       status.markComplete(msg);
200       LOG.info(msg);
201       metricsSnapshot.addSnapshot(status.getCompletionTimestamp() - status.getStartTime());
202     } catch (Exception e) {
203       status.abort("Failed to complete snapshot " + snapshot.getName() + " on table " +
204           snapshotTable + " because " + e.getMessage());
205       String reason = "Failed taking snapshot " + ClientSnapshotDescriptionUtils.toString(snapshot)
206           + " due to exception:" + e.getMessage();
207       LOG.error(reason, e);
208       ForeignException ee = new ForeignException(reason, e);
209       monitor.receive(ee);
210       // need to mark this completed to close off and allow cleanup to happen.
211       cancel(reason);
212     } finally {
213       LOG.debug("Launching cleanup of working dir:" + workingDir);
214       try {
215         // if the working dir is still present, the snapshot has failed.  it is present we delete
216         // it.
217         if (fs.exists(workingDir) && !this.fs.delete(workingDir, true)) {
218           LOG.error("Couldn't delete snapshot working directory:" + workingDir);
219         }
220       } catch (IOException e) {
221         LOG.error("Couldn't delete snapshot working directory:" + workingDir);
222       }
223       releaseTableLock();
224     }
225   }
226 
227   protected void releaseTableLock() {
228     if (this.tableLock != null) {
229       try {
230         this.tableLock.release();
231       } catch (IOException ex) {
232         LOG.warn("Could not release the table lock", ex);
233       }
234     }
235   }
236 
237   /**
238    * Reset the manager to allow another snapshot to proceed
239    *
240    * @param snapshotDir final path of the snapshot
241    * @param workingDir directory where the in progress snapshot was built
242    * @param fs {@link FileSystem} where the snapshot was built
243    * @throws SnapshotCreationException if the snapshot could not be moved
244    * @throws IOException the filesystem could not be reached
245    */
246   public void completeSnapshot(Path snapshotDir, Path workingDir, FileSystem fs)
247       throws SnapshotCreationException, IOException {
248     LOG.debug("Sentinel is done, just moving the snapshot from " + workingDir + " to "
249         + snapshotDir);
250     if (!fs.rename(workingDir, snapshotDir)) {
251       throw new SnapshotCreationException("Failed to move working directory(" + workingDir
252           + ") to completed directory(" + snapshotDir + ").");
253     }
254     finished = true;
255   }
256 
257   /**
258    * Snapshot the specified regions
259    */
260   protected abstract void snapshotRegions(List<Pair<HRegionInfo, ServerName>> regions)
261       throws IOException, KeeperException;
262 
263   /**
264    * Take a snapshot of the specified disabled region
265    */
266   protected void snapshotDisabledRegion(final HRegionInfo regionInfo)
267       throws IOException {
268     snapshotManifest.addRegion(FSUtils.getTableDir(rootDir, snapshotTable), regionInfo);
269     monitor.rethrowException();
270     status.setStatus("Completed referencing HFiles for offline region " + regionInfo.toString() +
271         " of table: " + snapshotTable);
272   }
273 
274   @Override
275   public void cancel(String why) {
276     if (finished) return;
277 
278     this.finished = true;
279     LOG.info("Stop taking snapshot=" + ClientSnapshotDescriptionUtils.toString(snapshot) +
280         " because: " + why);
281     CancellationException ce = new CancellationException(why);
282     monitor.receive(new ForeignException(master.getServerName().toString(), ce));
283   }
284 
285   @Override
286   public boolean isFinished() {
287     return finished;
288   }
289 
290   @Override
291   public long getCompletionTimestamp() {
292     return this.status.getCompletionTimestamp();
293   }
294 
295   @Override
296   public SnapshotDescription getSnapshot() {
297     return snapshot;
298   }
299 
300   @Override
301   public ForeignException getExceptionIfFailed() {
302     return monitor.getException();
303   }
304 
305   @Override
306   public void rethrowExceptionIfFailed() throws ForeignException {
307     monitor.rethrowException();
308   }
309 
310   @Override
311   public void rethrowException() throws ForeignException {
312     monitor.rethrowException();
313   }
314 
315   @Override
316   public boolean hasException() {
317     return monitor.hasException();
318   }
319 
320   @Override
321   public ForeignException getException() {
322     return monitor.getException();
323   }
324 
325 }