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