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.IOException;
21 import java.util.HashSet;
22 import java.util.List;
23 import java.util.Set;
24
25 import org.apache.commons.logging.Log;
26 import org.apache.commons.logging.LogFactory;
27 import org.apache.hadoop.classification.InterfaceAudience;
28 import org.apache.hadoop.classification.InterfaceStability;
29 import org.apache.hadoop.fs.Path;
30 import org.apache.hadoop.hbase.HRegionInfo;
31 import org.apache.hadoop.hbase.ServerName;
32 import org.apache.hadoop.hbase.errorhandling.ForeignException;
33 import org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector;
34 import org.apache.hadoop.hbase.master.MasterServices;
35 import org.apache.hadoop.hbase.master.metrics.MasterMetrics;
36 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
37 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
38 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
39 import org.apache.hadoop.hbase.regionserver.HRegion;
40 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
41 import org.apache.hadoop.hbase.snapshot.TableInfoCopyTask;
42 import org.apache.hadoop.hbase.snapshot.TakeSnapshotUtils;
43 import org.apache.hadoop.hbase.util.FSUtils;
44 import org.apache.hadoop.hbase.util.Pair;
45 import org.apache.zookeeper.KeeperException;
46
47
48
49
50
51
52 @InterfaceAudience.Private
53 @InterfaceStability.Evolving
54 public class DisabledTableSnapshotHandler extends TakeSnapshotHandler {
55 private static final Log LOG = LogFactory.getLog(DisabledTableSnapshotHandler.class);
56 private final TimeoutExceptionInjector timeoutInjector;
57
58
59
60
61
62
63 public DisabledTableSnapshotHandler(SnapshotDescription snapshot,
64 final MasterServices masterServices, final MasterMetrics metricsMaster) {
65 super(snapshot, masterServices, metricsMaster);
66
67
68 timeoutInjector = TakeSnapshotUtils.getMasterTimerAndBindToMonitor(snapshot, conf, monitor);
69 }
70
71
72
73 @Override
74 public void snapshotRegions(List<Pair<HRegionInfo, ServerName>> regionsAndLocations)
75 throws IOException, KeeperException {
76 try {
77 timeoutInjector.start();
78
79
80
81
82 Set<HRegionInfo> regions = new HashSet<HRegionInfo>();
83 for (Pair<HRegionInfo, ServerName> p : regionsAndLocations) {
84 regions.add(p.getFirst());
85 }
86
87
88 LOG.info("Starting to write region info and WALs for regions for offline snapshot:"
89 + SnapshotDescriptionUtils.toString(snapshot));
90 for (HRegionInfo regionInfo : regions) {
91 snapshotDisabledRegion(regionInfo);
92 }
93
94
95 LOG.info("Starting to copy tableinfo for offline snapshot: " +
96 SnapshotDescriptionUtils.toString(snapshot));
97 TableInfoCopyTask tableInfoCopyTask = new TableInfoCopyTask(this.monitor, snapshot, fs,
98 FSUtils.getRootDir(conf));
99 tableInfoCopyTask.call();
100 monitor.rethrowException();
101 status.setStatus("Finished copying tableinfo for snapshot of table: " + snapshot.getTable());
102 } catch (Exception e) {
103
104 String reason = "Failed snapshot " + SnapshotDescriptionUtils.toString(snapshot)
105 + " due to exception:" + e.getMessage();
106 ForeignException ee = new ForeignException(reason, e);
107 monitor.receive(ee);
108 status.abort("Snapshot of table: "+ snapshot.getTable() +" failed because " + e.getMessage());
109 } finally {
110 LOG.debug("Marking snapshot" + SnapshotDescriptionUtils.toString(snapshot)
111 + " as finished.");
112
113
114
115 timeoutInjector.complete();
116 }
117 }
118 }