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.HRegionInfo;
34  import org.apache.hadoop.hbase.HTableDescriptor;
35  import org.apache.hadoop.hbase.ServerName;
36  import org.apache.hadoop.hbase.catalog.MetaReader;
37  import org.apache.hadoop.hbase.errorhandling.ForeignException;
38  import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
39  import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
40  import org.apache.hadoop.hbase.executor.EventHandler;
41  import org.apache.hadoop.hbase.master.MasterServices;
42  import org.apache.hadoop.hbase.master.SnapshotSentinel;
43  import org.apache.hadoop.hbase.master.metrics.MasterMetrics;
44  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
45  import org.apache.hadoop.hbase.monitoring.TaskMonitor;
46  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
47  import org.apache.hadoop.hbase.regionserver.HRegion;
48  import org.apache.hadoop.hbase.snapshot.CopyRecoveredEditsTask;
49  import org.apache.hadoop.hbase.snapshot.ReferenceRegionHFilesTask;
50  import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
51  import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
52  import org.apache.hadoop.hbase.snapshot.TableInfoCopyTask;
53  import org.apache.hadoop.hbase.snapshot.TakeSnapshotUtils;
54  import org.apache.hadoop.hbase.util.Bytes;
55  import org.apache.hadoop.hbase.util.Pair;
56  import org.apache.zookeeper.KeeperException;
57  
58  /**
59   * A handler for taking snapshots from the master.
60   *
61   * This is not a subclass of TableEventHandler because using that would incur an extra META scan.
62   *
63   * The {@link #snapshotRegions(List)} call should get implemented for each snapshot flavor.
64   */
65  @InterfaceAudience.Private
66  public abstract class TakeSnapshotHandler extends EventHandler implements SnapshotSentinel,
67      ForeignExceptionSnare {
68    private static final Log LOG = LogFactory.getLog(TakeSnapshotHandler.class);
69  
70    private volatile boolean finished;
71  
72    // none of these should ever be null
73    protected final MasterServices master;
74    protected final MasterMetrics metricsMaster;
75    protected final SnapshotDescription snapshot;
76    protected final Configuration conf;
77    protected final FileSystem fs;
78    protected final Path rootDir;
79    private final Path snapshotDir;
80    protected final Path workingDir;
81    private final MasterSnapshotVerifier verifier;
82    protected final ForeignExceptionDispatcher monitor;
83    protected final MonitoredTask status;
84  
85    /**
86     * @param snapshot descriptor of the snapshot to take
87     * @param masterServices master services provider
88     * @throws IOException on unexpected error
89     */
90    public TakeSnapshotHandler(SnapshotDescription snapshot, final MasterServices masterServices,
91        final MasterMetrics metricsMaster) {
92      super(masterServices, EventType.C_M_SNAPSHOT_TABLE);
93      assert snapshot != null : "SnapshotDescription must not be nul1";
94      assert masterServices != null : "MasterServices must not be nul1";
95  
96      this.master = masterServices;
97      this.metricsMaster = metricsMaster;
98      this.snapshot = snapshot;
99      this.conf = this.master.getConfiguration();
100     this.fs = this.master.getMasterFileSystem().getFileSystem();
101     this.rootDir = this.master.getMasterFileSystem().getRootDir();
102     this.snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir);
103     this.workingDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshot, rootDir);
104     this.monitor = new ForeignExceptionDispatcher(snapshot.getName());
105 
106     // prepare the verify
107     this.verifier = new MasterSnapshotVerifier(masterServices, snapshot, rootDir);
108     // update the running tasks
109     this.status = TaskMonitor.get().createStatus(
110       "Taking " + snapshot.getType() + " snapshot on table: " + snapshot.getTable());
111   }
112 
113   private HTableDescriptor loadTableDescriptor()
114       throws FileNotFoundException, IOException {
115     final String name = snapshot.getTable();
116     HTableDescriptor htd =
117       this.master.getTableDescriptors().get(name);
118     if (htd == null) {
119       throw new IOException("HTableDescriptor missing for " + name);
120     }
121     return htd;
122   }
123 
124   public TakeSnapshotHandler prepare() throws Exception {
125     loadTableDescriptor(); // check that .tableinfo is present
126     return this;
127   }
128 
129   /**
130    * Execute the core common portions of taking a snapshot. The {@link #snapshotRegions(List)}
131    * call should get implemented for each snapshot flavor.
132    */
133   @Override
134   public void process() {
135     String msg = "Running " + snapshot.getType() + " table snapshot " + snapshot.getName() + " "
136         + eventType + " on table " + snapshot.getTable();
137     LOG.info(msg);
138     status.setStatus(msg);
139     try {
140       // If regions move after this meta scan, the region specific snapshot should fail, triggering
141       // an external exception that gets captured here.
142 
143       // write down the snapshot info in the working directory
144       SnapshotDescriptionUtils.writeSnapshotInfo(snapshot, workingDir, this.fs);
145       new TableInfoCopyTask(monitor, snapshot, fs, rootDir).call();
146       monitor.rethrowException();
147 
148       List<Pair<HRegionInfo, ServerName>> regionsAndLocations =
149           MetaReader.getTableRegionsAndLocations(this.server.getCatalogTracker(),
150             Bytes.toBytes(snapshot.getTable()), false);
151 
152       // run the snapshot
153       snapshotRegions(regionsAndLocations);
154       monitor.rethrowException();
155 
156       // extract each pair to separate lists
157       Set<String> serverNames = new HashSet<String>();
158       for (Pair<HRegionInfo, ServerName> p : regionsAndLocations) {
159         if (p != null && p.getFirst() != null && p.getSecond() != null) {
160           HRegionInfo hri = p.getFirst();
161           if (hri.isOffline() && (hri.isSplit() || hri.isSplitParent())) continue;
162           serverNames.add(p.getSecond().toString());
163         }
164       }
165 
166       // verify the snapshot is valid
167       status.setStatus("Verifying snapshot: " + snapshot.getName());
168       verifier.verifySnapshot(this.workingDir, serverNames);
169 
170       // complete the snapshot, atomically moving from tmp to .snapshot dir.
171       completeSnapshot(this.snapshotDir, this.workingDir, this.fs);
172       status.markComplete("Snapshot " + snapshot.getName() + " of table " + snapshot.getTable()
173           + " completed");
174       metricsMaster.addSnapshot(status.getCompletionTimestamp() - status.getStartTime());
175     } catch (Exception e) {
176       status.abort("Failed to complete snapshot " + snapshot.getName() + " on table " +
177           snapshot.getTable() + " because " + e.getMessage());
178       String reason = "Failed taking snapshot " + SnapshotDescriptionUtils.toString(snapshot)
179           + " due to exception:" + e.getMessage();
180       LOG.error(reason, e);
181       ForeignException ee = new ForeignException(reason, e);
182       monitor.receive(ee);
183       // need to mark this completed to close off and allow cleanup to happen.
184       cancel("Failed to take snapshot '" + SnapshotDescriptionUtils.toString(snapshot)
185           + "' due to exception");
186     } finally {
187       LOG.debug("Launching cleanup of working dir:" + workingDir);
188       try {
189         // if the working dir is still present, the snapshot has failed.  it is present we delete
190         // it.
191         if (fs.exists(workingDir) && !this.fs.delete(workingDir, true)) {
192           LOG.error("Couldn't delete snapshot working directory:" + workingDir);
193         }
194       } catch (IOException e) {
195         LOG.error("Couldn't delete snapshot working directory:" + workingDir);
196       }
197     }
198   }
199 
200   /**
201    * Reset the manager to allow another snapshot to proceed
202    *
203    * @param snapshotDir final path of the snapshot
204    * @param workingDir directory where the in progress snapshot was built
205    * @param fs {@link FileSystem} where the snapshot was built
206    * @throws SnapshotCreationException if the snapshot could not be moved
207    * @throws IOException the filesystem could not be reached
208    */
209   public void completeSnapshot(Path snapshotDir, Path workingDir, FileSystem fs)
210       throws SnapshotCreationException, IOException {
211     LOG.debug("Sentinel is done, just moving the snapshot from " + workingDir + " to "
212         + snapshotDir);
213     if (!fs.rename(workingDir, snapshotDir)) {
214       throw new SnapshotCreationException("Failed to move working directory(" + workingDir
215           + ") to completed directory(" + snapshotDir + ").");
216     }
217     finished = true;
218   }
219 
220   /**
221    * Take a snapshot of the specified disabled region
222    */
223   protected void snapshotDisabledRegion(final HRegionInfo regionInfo)
224       throws IOException {
225     // 1 copy the regionInfo files to the snapshot
226     Path snapshotRegionDir = TakeSnapshotUtils.getRegionSnapshotDirectory(snapshot, rootDir,
227       regionInfo.getEncodedName());
228     HRegion.writeRegioninfoOnFilesystem(regionInfo, snapshotRegionDir, fs, conf);
229     // check for error for each region
230     monitor.rethrowException();
231 
232     // 2 for each region, copy over its recovered.edits directory
233     Path regionDir = HRegion.getRegionDir(rootDir, regionInfo);
234     new CopyRecoveredEditsTask(snapshot, monitor, fs, regionDir, snapshotRegionDir).call();
235     monitor.rethrowException();
236     status.setStatus("Completed copying recovered edits for offline snapshot of table: "
237         + snapshot.getTable());
238 
239     // 3 reference all the files in the region
240     new ReferenceRegionHFilesTask(snapshot, monitor, regionDir, fs, snapshotRegionDir).call();
241     monitor.rethrowException();
242     status.setStatus("Completed referencing HFiles for offline snapshot of table: " +
243       snapshot.getTable());
244   }
245 
246   /**
247    * Snapshot the specified regions
248    */
249   protected abstract void snapshotRegions(List<Pair<HRegionInfo, ServerName>> regions)
250       throws IOException, KeeperException;
251 
252   @Override
253   public void cancel(String why) {
254     if (finished) return;
255 
256     this.finished = true;
257     LOG.info("Stop taking snapshot=" + SnapshotDescriptionUtils.toString(snapshot) + " because: "
258         + why);
259     CancellationException ce = new CancellationException(why);
260     monitor.receive(new ForeignException(master.getServerName().toString(), ce));
261   }
262 
263   @Override
264   public boolean isFinished() {
265     return finished;
266   }
267 
268   @Override
269   public long getCompletionTimestamp() {
270     return this.status.getCompletionTimestamp();
271   }
272 
273   @Override
274   public SnapshotDescription getSnapshot() {
275     return snapshot;
276   }
277 
278   @Override
279   public ForeignException getExceptionIfFailed() {
280     return monitor.getException();
281   }
282 
283   @Override
284   public void rethrowExceptionIfFailed() throws ForeignException {
285     monitor.rethrowException();
286   }
287 
288   @Override
289   public void rethrowException() throws ForeignException {
290     monitor.rethrowException();
291   }
292 
293   @Override
294   public boolean hasException() {
295     return monitor.hasException();
296   }
297 
298   @Override
299   public ForeignException getException() {
300     return monitor.getException();
301   }
302 
303 }