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