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