1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
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.MetricsMaster;
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.ProtobufUtil;
51 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
52 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
53 import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
54 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
55 import org.apache.hadoop.hbase.snapshot.TableInfoCopyTask;
56 import org.apache.hadoop.hbase.util.Pair;
57 import org.apache.zookeeper.KeeperException;
58
59
60
61
62
63
64
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
74 protected final MasterServices master;
75 protected final MetricsMaster metricsMaster;
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
89
90
91
92
93 public TakeSnapshotHandler(SnapshotDescription snapshot, final MasterServices masterServices,
94 final MetricsMaster metricsMaster) {
95 super(masterServices, EventType.C_M_SNAPSHOT_TABLE);
96 assert snapshot != null : "SnapshotDescription must not be nul1";
97 assert masterServices != null : "MasterServices must not be nul1";
98
99 this.master = masterServices;
100 this.metricsMaster = metricsMaster;
101 this.snapshot = snapshot;
102 this.snapshotTable = TableName.valueOf(snapshot.getTable());
103 this.conf = this.master.getConfiguration();
104 this.fs = this.master.getMasterFileSystem().getFileSystem();
105 this.rootDir = this.master.getMasterFileSystem().getRootDir();
106 this.snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir);
107 this.workingDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshot, rootDir);
108 this.monitor = new ForeignExceptionDispatcher(snapshot.getName());
109
110 this.tableLockManager = master.getTableLockManager();
111 this.tableLock = this.tableLockManager.writeLock(
112 snapshotTable,
113 EventType.C_M_SNAPSHOT_TABLE.toString());
114
115
116 this.verifier = new MasterSnapshotVerifier(masterServices, snapshot, rootDir);
117
118 this.status = TaskMonitor.get().createStatus(
119 "Taking " + snapshot.getType() + " snapshot on table: " + snapshotTable);
120 }
121
122 private HTableDescriptor loadTableDescriptor()
123 throws FileNotFoundException, IOException {
124 HTableDescriptor htd =
125 this.master.getTableDescriptors().get(snapshotTable);
126 if (htd == null) {
127 throw new IOException("HTableDescriptor missing for " + snapshotTable);
128 }
129 return htd;
130 }
131
132 public TakeSnapshotHandler prepare() throws Exception {
133 super.prepare();
134 this.tableLock.acquire();
135
136 boolean success = false;
137 try {
138 loadTableDescriptor();
139 success = true;
140 } finally {
141 if (!success) {
142 releaseTableLock();
143 }
144 }
145
146 return this;
147 }
148
149
150
151
152
153 @Override
154 public void process() {
155 String msg = "Running " + snapshot.getType() + " table snapshot " + snapshot.getName() + " "
156 + eventType + " on table " + snapshotTable;
157 LOG.info(msg);
158 status.setStatus(msg);
159 try {
160
161
162
163
164 SnapshotDescriptionUtils.writeSnapshotInfo(snapshot, workingDir, this.fs);
165 new TableInfoCopyTask(monitor, snapshot, fs, rootDir).call();
166 monitor.rethrowException();
167
168 List<Pair<HRegionInfo, ServerName>> regionsAndLocations =
169 MetaReader.getTableRegionsAndLocations(this.server.getCatalogTracker(),
170 snapshotTable, true);
171
172
173 snapshotRegions(regionsAndLocations);
174 monitor.rethrowException();
175
176
177 Set<String> serverNames = new HashSet<String>();
178 for (Pair<HRegionInfo, ServerName> p : regionsAndLocations) {
179 serverNames.add(p.getSecond().toString());
180 }
181
182
183 status.setStatus("Verifying snapshot: " + snapshot.getName());
184 verifier.verifySnapshot(this.workingDir, serverNames);
185
186
187 completeSnapshot(this.snapshotDir, this.workingDir, this.fs);
188 status.markComplete("Snapshot " + snapshot.getName() + " of table " + snapshotTable
189 + " completed");
190 metricsMaster.addSnapshot(status.getCompletionTimestamp() - status.getStartTime());
191 } catch (Exception e) {
192 status.abort("Failed to complete snapshot " + snapshot.getName() + " on table " +
193 snapshotTable + " because " + e.getMessage());
194 String reason = "Failed taking snapshot " + ClientSnapshotDescriptionUtils.toString(snapshot)
195 + " due to exception:" + e.getMessage();
196 LOG.error(reason, e);
197 ForeignException ee = new ForeignException(reason, e);
198 monitor.receive(ee);
199
200 cancel("Failed to take snapshot '" + ClientSnapshotDescriptionUtils.toString(snapshot)
201 + "' due to exception");
202 } finally {
203 LOG.debug("Launching cleanup of working dir:" + workingDir);
204 try {
205
206
207 if (fs.exists(workingDir) && !this.fs.delete(workingDir, true)) {
208 LOG.error("Couldn't delete snapshot working directory:" + workingDir);
209 }
210 } catch (IOException e) {
211 LOG.error("Couldn't delete snapshot working directory:" + workingDir);
212 }
213 releaseTableLock();
214 }
215 }
216
217 protected void releaseTableLock() {
218 if (this.tableLock != null) {
219 try {
220 this.tableLock.release();
221 } catch (IOException ex) {
222 LOG.warn("Could not release the table lock", ex);
223 }
224 }
225 }
226
227
228
229
230
231
232
233
234
235
236 public void completeSnapshot(Path snapshotDir, Path workingDir, FileSystem fs)
237 throws SnapshotCreationException, IOException {
238 LOG.debug("Sentinel is done, just moving the snapshot from " + workingDir + " to "
239 + snapshotDir);
240 if (!fs.rename(workingDir, snapshotDir)) {
241 throw new SnapshotCreationException("Failed to move working directory(" + workingDir
242 + ") to completed directory(" + snapshotDir + ").");
243 }
244 finished = true;
245 }
246
247
248
249
250 protected abstract void snapshotRegions(List<Pair<HRegionInfo, ServerName>> regions)
251 throws IOException, KeeperException;
252
253 @Override
254 public void cancel(String why) {
255 if (finished) return;
256
257 this.finished = true;
258 LOG.info("Stop taking snapshot=" + ClientSnapshotDescriptionUtils.toString(snapshot) +
259 " because: " + why);
260 CancellationException ce = new CancellationException(why);
261 monitor.receive(new ForeignException(master.getServerName().toString(), ce));
262 }
263
264 @Override
265 public boolean isFinished() {
266 return finished;
267 }
268
269 @Override
270 public long getCompletionTimestamp() {
271 return this.status.getCompletionTimestamp();
272 }
273
274 @Override
275 public SnapshotDescription getSnapshot() {
276 return snapshot;
277 }
278
279 @Override
280 public ForeignException getExceptionIfFailed() {
281 return monitor.getException();
282 }
283
284 @Override
285 public void rethrowExceptionIfFailed() throws ForeignException {
286 monitor.rethrowException();
287 }
288
289 @Override
290 public void rethrowException() throws ForeignException {
291 monitor.rethrowException();
292 }
293
294 @Override
295 public boolean hasException() {
296 return monitor.hasException();
297 }
298
299 @Override
300 public ForeignException getException() {
301 return monitor.getException();
302 }
303
304 }