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.List;
24 import java.util.concurrent.CancellationException;
25
26 import org.apache.commons.logging.Log;
27 import org.apache.commons.logging.LogFactory;
28 import org.apache.hadoop.classification.InterfaceAudience;
29 import org.apache.hadoop.fs.FileSystem;
30 import org.apache.hadoop.fs.Path;
31 import org.apache.hadoop.hbase.HRegionInfo;
32 import org.apache.hadoop.hbase.HTableDescriptor;
33 import org.apache.hadoop.hbase.catalog.CatalogTracker;
34 import org.apache.hadoop.hbase.catalog.MetaEditor;
35 import org.apache.hadoop.hbase.errorhandling.ForeignException;
36 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
37 import org.apache.hadoop.hbase.exceptions.RestoreSnapshotException;
38 import org.apache.hadoop.hbase.executor.EventType;
39 import org.apache.hadoop.hbase.master.MasterFileSystem;
40 import org.apache.hadoop.hbase.master.MasterServices;
41 import org.apache.hadoop.hbase.master.MetricsMaster;
42 import org.apache.hadoop.hbase.master.SnapshotSentinel;
43 import org.apache.hadoop.hbase.master.handler.TableEventHandler;
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.ClientSnapshotDescriptionUtils;
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 MetricsMaster 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 MetricsMaster 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 public RestoreSnapshotHandler prepare() throws IOException {
95 return (RestoreSnapshotHandler) super.prepare();
96 }
97
98
99
100
101
102
103
104
105
106 @Override
107 protected void handleTableOperation(List<HRegionInfo> hris) throws IOException {
108 MasterFileSystem fileSystemManager = masterServices.getMasterFileSystem();
109 CatalogTracker catalogTracker = masterServices.getCatalogTracker();
110 FileSystem fs = fileSystemManager.getFileSystem();
111 Path rootDir = fileSystemManager.getRootDir();
112 byte[] tableName = hTableDescriptor.getName();
113 Path tableDir = HTableDescriptor.getTableDir(rootDir, tableName);
114
115 try {
116
117 this.masterServices.getTableDescriptors().add(hTableDescriptor);
118
119
120 LOG.debug("Starting restore snapshot=" + ClientSnapshotDescriptionUtils.toString(snapshot));
121 Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir);
122 RestoreSnapshotHelper restoreHelper = new RestoreSnapshotHelper(
123 masterServices.getConfiguration(), fs,
124 snapshot, snapshotDir, hTableDescriptor, tableDir, monitor, status);
125 RestoreSnapshotHelper.RestoreMetaChanges metaChanges = restoreHelper.restoreHdfsRegions();
126
127
128 hris.clear();
129 status.setStatus("Preparing to restore each region");
130 if (metaChanges.hasRegionsToAdd()) hris.addAll(metaChanges.getRegionsToAdd());
131 if (metaChanges.hasRegionsToRestore()) hris.addAll(metaChanges.getRegionsToRestore());
132 List<HRegionInfo> hrisToRemove = metaChanges.getRegionsToRemove();
133 MetaEditor.mutateRegions(catalogTracker, hrisToRemove, hris);
134
135
136 LOG.info("Restore snapshot=" + ClientSnapshotDescriptionUtils.toString(snapshot) +
137 " on table=" + Bytes.toString(tableName) + " completed!");
138 } catch (IOException e) {
139 String msg = "restore snapshot=" + ClientSnapshotDescriptionUtils.toString(snapshot)
140 + " failed. Try re-running the restore command.";
141 LOG.error(msg, e);
142 monitor.receive(new ForeignException(masterServices.getServerName().toString(), e));
143 throw new RestoreSnapshotException(msg, e);
144 }
145 }
146
147 @Override
148 protected void completed(final Throwable exception) {
149 this.stopped = true;
150 if (exception != null) {
151 status.abort("Restore snapshot '" + snapshot.getName() + "' failed because " +
152 exception.getMessage());
153 } else {
154 status.markComplete("Restore snapshot '"+ snapshot.getName() +"'!");
155 }
156 metricsMaster.addSnapshotRestore(status.getCompletionTimestamp() - status.getStartTime());
157 super.completed(exception);
158 }
159
160 @Override
161 public boolean isFinished() {
162 return this.stopped;
163 }
164
165 @Override
166 public long getCompletionTimestamp() {
167 return this.status.getCompletionTimestamp();
168 }
169
170 @Override
171 public SnapshotDescription getSnapshot() {
172 return snapshot;
173 }
174
175 @Override
176 public void cancel(String why) {
177 if (this.stopped) return;
178 this.stopped = true;
179 String msg = "Stopping restore snapshot=" + ClientSnapshotDescriptionUtils.toString(snapshot)
180 + " because: " + why;
181 LOG.info(msg);
182 CancellationException ce = new CancellationException(why);
183 this.monitor.receive(new ForeignException(masterServices.getServerName().toString(), ce));
184 }
185
186 @Override
187 public ForeignException getExceptionIfFailed() {
188 return this.monitor.getException();
189 }
190
191 @Override
192 public void rethrowExceptionIfFailed() throws ForeignException {
193 monitor.rethrowException();
194 }
195 }