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 import java.util.concurrent.ThreadPoolExecutor;
25
26 import org.apache.commons.logging.Log;
27 import org.apache.commons.logging.LogFactory;
28 import org.apache.hadoop.conf.Configuration;
29 import org.apache.hadoop.hbase.classification.InterfaceAudience;
30 import org.apache.hadoop.hbase.classification.InterfaceStability;
31 import org.apache.hadoop.hbase.HRegionInfo;
32 import org.apache.hadoop.hbase.ServerName;
33 import org.apache.hadoop.hbase.errorhandling.ForeignException;
34 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionListener;
35 import org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector;
36 import org.apache.hadoop.hbase.master.MasterServices;
37 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
38 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
39 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
40 import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
41 import org.apache.hadoop.hbase.util.FSUtils;
42 import org.apache.hadoop.hbase.util.ModifyRegionUtils;
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 public DisabledTableSnapshotHandler(SnapshotDescription snapshot,
62 final MasterServices masterServices) {
63 super(snapshot, masterServices);
64
65
66 timeoutInjector = getMasterTimerAndBindToMonitor(snapshot, conf, monitor);
67 }
68
69 @Override
70 public DisabledTableSnapshotHandler prepare() throws Exception {
71 return (DisabledTableSnapshotHandler) super.prepare();
72 }
73
74
75
76 @Override
77 public void snapshotRegions(List<Pair<HRegionInfo, ServerName>> regionsAndLocations)
78 throws IOException, KeeperException {
79 try {
80 timeoutInjector.start();
81
82
83
84
85 Set<HRegionInfo> regions = new HashSet<HRegionInfo>();
86 for (Pair<HRegionInfo, ServerName> p : regionsAndLocations) {
87 regions.add(p.getFirst());
88 }
89
90
91 String msg = "Starting to write region info and WALs for regions for offline snapshot:"
92 + ClientSnapshotDescriptionUtils.toString(snapshot);
93 LOG.info(msg);
94 status.setStatus(msg);
95
96 ThreadPoolExecutor exec = SnapshotManifest.createExecutor(conf, "DisabledTableSnapshot");
97 try {
98 ModifyRegionUtils.editRegions(exec, regions, new ModifyRegionUtils.RegionEditTask() {
99 @Override
100 public void editRegion(final HRegionInfo regionInfo) throws IOException {
101 snapshotManifest.addRegion(FSUtils.getTableDir(rootDir, snapshotTable), regionInfo);
102 }
103 });
104 } finally {
105 exec.shutdown();
106 }
107 } catch (Exception e) {
108
109 String reason = "Failed snapshot " + ClientSnapshotDescriptionUtils.toString(snapshot)
110 + " due to exception:" + e.getMessage();
111 ForeignException ee = new ForeignException(reason, e);
112 monitor.receive(ee);
113 status.abort("Snapshot of table: "+ snapshotTable + " failed because " + e.getMessage());
114 } finally {
115 LOG.debug("Marking snapshot" + ClientSnapshotDescriptionUtils.toString(snapshot)
116 + " as finished.");
117
118
119
120 timeoutInjector.complete();
121 }
122 }
123
124
125
126
127
128
129
130
131
132 private TimeoutExceptionInjector getMasterTimerAndBindToMonitor(SnapshotDescription snapshot,
133 Configuration conf, ForeignExceptionListener monitor) {
134 long maxTime = SnapshotDescriptionUtils.getMaxMasterTimeout(conf, snapshot.getType(),
135 SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME);
136 return new TimeoutExceptionInjector(monitor, maxTime);
137 }
138 }