1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.master.snapshot;
21
22 import java.io.IOException;
23 import java.util.LinkedList;
24 import java.util.List;
25 import java.util.concurrent.CancellationException;
26
27 import org.apache.commons.logging.Log;
28 import org.apache.commons.logging.LogFactory;
29 import org.apache.hadoop.classification.InterfaceAudience;
30 import org.apache.hadoop.fs.FileSystem;
31 import org.apache.hadoop.fs.Path;
32 import org.apache.hadoop.hbase.HRegionInfo;
33 import org.apache.hadoop.hbase.HTableDescriptor;
34 import org.apache.hadoop.hbase.catalog.CatalogTracker;
35 import org.apache.hadoop.hbase.catalog.MetaEditor;
36 import org.apache.hadoop.hbase.errorhandling.ForeignException;
37 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
38 import org.apache.hadoop.hbase.master.AssignmentManager;
39 import org.apache.hadoop.hbase.master.MasterFileSystem;
40 import org.apache.hadoop.hbase.master.MasterServices;
41 import org.apache.hadoop.hbase.master.SnapshotSentinel;
42 import org.apache.hadoop.hbase.master.handler.TableEventHandler;
43 import org.apache.hadoop.hbase.master.metrics.MasterMetrics;
44 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
45 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
46 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
47 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
48 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
49 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
50 import org.apache.hadoop.hbase.util.Bytes;
51
52
53
54
55
56
57
58 @InterfaceAudience.Private
59 public class RestoreSnapshotHandler extends TableEventHandler implements SnapshotSentinel {
60 private static final Log LOG = LogFactory.getLog(RestoreSnapshotHandler.class);
61
62 private final HTableDescriptor hTableDescriptor;
63 private final SnapshotDescription snapshot;
64
65 private final ForeignExceptionDispatcher monitor;
66 private final MasterMetrics metricsMaster;
67 private final MonitoredTask status;
68
69 private volatile boolean stopped = false;
70
71 public RestoreSnapshotHandler(final MasterServices masterServices,
72 final SnapshotDescription snapshot, final HTableDescriptor htd,
73 final MasterMetrics metricsMaster) throws IOException {
74 super(EventType.C_M_RESTORE_SNAPSHOT, htd.getName(), masterServices, masterServices);
75 this.metricsMaster = metricsMaster;
76
77
78 this.snapshot = snapshot;
79
80
81 this.monitor = new ForeignExceptionDispatcher();
82
83
84 getTableDescriptor();
85
86
87 this.hTableDescriptor = htd;
88
89 this.status = TaskMonitor.get().createStatus(
90 "Restoring snapshot '" + snapshot.getName() + "' to table "
91 + hTableDescriptor.getNameAsString());
92 }
93
94
95
96
97
98
99
100
101
102 @Override
103 protected void handleTableOperation(List<HRegionInfo> hris) throws IOException {
104 MasterFileSystem fileSystemManager = masterServices.getMasterFileSystem();
105 CatalogTracker catalogTracker = masterServices.getCatalogTracker();
106 FileSystem fs = fileSystemManager.getFileSystem();
107 Path rootDir = fileSystemManager.getRootDir();
108 byte[] tableName = hTableDescriptor.getName();
109 Path tableDir = HTableDescriptor.getTableDir(rootDir, tableName);
110
111 try {
112
113 this.masterServices.getTableDescriptors().add(hTableDescriptor);
114
115
116 LOG.debug("Starting restore snapshot=" + SnapshotDescriptionUtils.toString(snapshot));
117 Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir);
118 RestoreSnapshotHelper restoreHelper = new RestoreSnapshotHelper(
119 masterServices.getConfiguration(), fs,
120 snapshot, snapshotDir, hTableDescriptor, tableDir, monitor, status);
121 RestoreSnapshotHelper.RestoreMetaChanges metaChanges = restoreHelper.restoreHdfsRegions();
122
123
124
125
126
127
128
129
130 forceRegionsOffline(metaChanges);
131 forceRegionsOffline(metaChanges);
132
133
134
135
136
137
138
139
140
141
142 List<HRegionInfo> hrisToRemove = new LinkedList<HRegionInfo>();
143 if (metaChanges.hasRegionsToRemove()) hrisToRemove.addAll(metaChanges.getRegionsToRemove());
144 if (metaChanges.hasRegionsToRestore()) hrisToRemove.addAll(metaChanges.getRegionsToRestore());
145 MetaEditor.deleteRegions(catalogTracker, hrisToRemove);
146
147
148
149
150
151
152
153 hris.clear();
154 if (metaChanges.hasRegionsToAdd()) hris.addAll(metaChanges.getRegionsToAdd());
155 if (metaChanges.hasRegionsToRestore()) hris.addAll(metaChanges.getRegionsToRestore());
156 MetaEditor.addRegionsToMeta(catalogTracker, hris);
157 metaChanges.updateMetaParentRegions(catalogTracker, hris);
158
159
160 LOG.info("Restore snapshot=" + SnapshotDescriptionUtils.toString(snapshot) + " on table=" +
161 Bytes.toString(tableName) + " completed!");
162 } catch (IOException e) {
163 String msg = "restore snapshot=" + SnapshotDescriptionUtils.toString(snapshot)
164 + " failed. Try re-running the restore command.";
165 LOG.error(msg, e);
166 monitor.receive(new ForeignException(masterServices.getServerName().toString(), e));
167 throw new RestoreSnapshotException(msg, e);
168 } finally {
169 this.stopped = true;
170 }
171 }
172
173 private void forceRegionsOffline(final RestoreSnapshotHelper.RestoreMetaChanges metaChanges) {
174 forceRegionsOffline(metaChanges.getRegionsToAdd());
175 forceRegionsOffline(metaChanges.getRegionsToRestore());
176 forceRegionsOffline(metaChanges.getRegionsToRemove());
177 }
178
179 private void forceRegionsOffline(final List<HRegionInfo> hris) {
180 AssignmentManager am = this.masterServices.getAssignmentManager();
181 if (hris != null) {
182 for (HRegionInfo hri: hris) {
183 am.regionOffline(hri);
184 }
185 }
186 }
187
188 @Override
189 protected void completed(final Throwable exception) {
190 this.stopped = true;
191 if (exception != null) {
192 status.abort("Restore snapshot '" + snapshot.getName() + "' failed because " +
193 exception.getMessage());
194 } else {
195 status.markComplete("Restore snapshot '"+ snapshot.getName() +"' completed!");
196 }
197 metricsMaster.addSnapshotRestore(status.getCompletionTimestamp() - status.getStartTime());
198 super.completed(exception);
199 }
200
201 @Override
202 public boolean isFinished() {
203 return this.stopped;
204 }
205
206 @Override
207 public long getCompletionTimestamp() {
208 return this.status.getCompletionTimestamp();
209 }
210
211 @Override
212 public SnapshotDescription getSnapshot() {
213 return snapshot;
214 }
215
216 @Override
217 public void cancel(String why) {
218 if (this.stopped) return;
219 this.stopped = true;
220 String msg = "Stopping restore snapshot=" + SnapshotDescriptionUtils.toString(snapshot)
221 + " because: " + why;
222 LOG.info(msg);
223 CancellationException ce = new CancellationException(why);
224 this.monitor.receive(new ForeignException(masterServices.getServerName().toString(), ce));
225 }
226
227 @Override
228 public ForeignException getExceptionIfFailed() {
229 return this.monitor.getException();
230 }
231
232 @Override
233 public void rethrowExceptionIfFailed() throws ForeignException {
234 monitor.rethrowException();
235 }
236 }