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.CopyRecoveredEditsTask;
41 import org.apache.hadoop.hbase.snapshot.ReferenceRegionHFilesTask;
42 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
43 import org.apache.hadoop.hbase.snapshot.TableInfoCopyTask;
44 import org.apache.hadoop.hbase.snapshot.TakeSnapshotUtils;
45 import org.apache.hadoop.hbase.util.FSUtils;
46 import org.apache.hadoop.hbase.util.Pair;
47 import org.apache.zookeeper.KeeperException;
48
49
50
51
52
53
54 @InterfaceAudience.Private
55 @InterfaceStability.Evolving
56 public class DisabledTableSnapshotHandler extends TakeSnapshotHandler {
57 private static final Log LOG = LogFactory.getLog(DisabledTableSnapshotHandler.class);
58 private final TimeoutExceptionInjector timeoutInjector;
59
60
61
62
63
64
65 public DisabledTableSnapshotHandler(SnapshotDescription snapshot,
66 final MasterServices masterServices, final MasterMetrics metricsMaster) {
67 super(snapshot, masterServices, metricsMaster);
68
69
70 timeoutInjector = TakeSnapshotUtils.getMasterTimerAndBindToMonitor(snapshot, conf, monitor);
71 }
72
73
74
75 @Override
76 public void snapshotRegions(List<Pair<HRegionInfo, ServerName>> regionsAndLocations)
77 throws IOException, KeeperException {
78 try {
79 timeoutInjector.start();
80
81
82
83
84 Set<String> serverNames = new HashSet<String>();
85 Set<HRegionInfo> regions = new HashSet<HRegionInfo>();
86 for (Pair<HRegionInfo, ServerName> p : regionsAndLocations) {
87 regions.add(p.getFirst());
88 serverNames.add(p.getSecond().toString());
89 }
90
91
92 LOG.info("Starting to write region info and WALs for regions for offline snapshot:"
93 + SnapshotDescriptionUtils.toString(snapshot));
94 for (HRegionInfo regionInfo : regions) {
95
96 Path snapshotRegionDir = TakeSnapshotUtils.getRegionSnapshotDirectory(snapshot, rootDir,
97 regionInfo.getEncodedName());
98 HRegion.writeRegioninfoOnFilesystem(regionInfo, snapshotRegionDir, fs, conf);
99
100 monitor.rethrowException();
101
102
103 Path regionDir = HRegion.getRegionDir(rootDir, regionInfo);
104 new CopyRecoveredEditsTask(snapshot, monitor, fs, regionDir, snapshotRegionDir).call();
105 monitor.rethrowException();
106 status.setStatus("Completed copying recovered edits for offline snapshot of table: "
107 + snapshot.getTable());
108
109
110 new ReferenceRegionHFilesTask(snapshot, monitor, regionDir, fs, snapshotRegionDir).call();
111 monitor.rethrowException();
112 status.setStatus("Completed referencing HFiles for offline snapshot of table: " +
113 snapshot.getTable());
114 }
115
116
117 LOG.info("Starting to copy tableinfo for offline snapshot: " +
118 SnapshotDescriptionUtils.toString(snapshot));
119 TableInfoCopyTask tableInfoCopyTask = new TableInfoCopyTask(this.monitor, snapshot, fs,
120 FSUtils.getRootDir(conf));
121 tableInfoCopyTask.call();
122 monitor.rethrowException();
123 status.setStatus("Finished copying tableinfo for snapshot of table: " + snapshot.getTable());
124 } catch (Exception e) {
125
126 String reason = "Failed snapshot " + SnapshotDescriptionUtils.toString(snapshot)
127 + " due to exception:" + e.getMessage();
128 ForeignException ee = new ForeignException(reason, e);
129 monitor.receive(ee);
130 status.abort("Snapshot of table: "+ snapshot.getTable() +" failed because " + e.getMessage());
131 } finally {
132 LOG.debug("Marking snapshot" + SnapshotDescriptionUtils.toString(snapshot)
133 + " as finished.");
134
135
136
137 timeoutInjector.complete();
138 }
139 }
140 }