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.master.MasterFileSystem;
38 import org.apache.hadoop.hbase.master.MasterServices;
39 import org.apache.hadoop.hbase.master.SnapshotSentinel;
40 import org.apache.hadoop.hbase.master.handler.TableEventHandler;
41 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
42 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
43 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
44 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
45 import org.apache.hadoop.hbase.util.Bytes;
46
47
48
49
50
51
52
53 @InterfaceAudience.Private
54 public class RestoreSnapshotHandler extends TableEventHandler implements SnapshotSentinel {
55 private static final Log LOG = LogFactory.getLog(RestoreSnapshotHandler.class);
56
57 private final HTableDescriptor hTableDescriptor;
58 private final SnapshotDescription snapshot;
59
60 private final ForeignExceptionDispatcher monitor;
61 private volatile boolean stopped = false;
62
63 public RestoreSnapshotHandler(final MasterServices masterServices,
64 final SnapshotDescription snapshot, final HTableDescriptor htd)
65 throws IOException {
66 super(EventType.C_M_RESTORE_SNAPSHOT, htd.getName(), masterServices, masterServices);
67
68
69 this.snapshot = snapshot;
70
71
72 this.monitor = new ForeignExceptionDispatcher();
73
74
75 getTableDescriptor();
76
77
78 this.hTableDescriptor = htd;
79 }
80
81
82
83
84
85
86
87
88
89 @Override
90 protected void handleTableOperation(List<HRegionInfo> hris) throws IOException {
91 MasterFileSystem fileSystemManager = masterServices.getMasterFileSystem();
92 CatalogTracker catalogTracker = masterServices.getCatalogTracker();
93 FileSystem fs = fileSystemManager.getFileSystem();
94 Path rootDir = fileSystemManager.getRootDir();
95 byte[] tableName = hTableDescriptor.getName();
96 Path tableDir = HTableDescriptor.getTableDir(rootDir, tableName);
97
98 try {
99
100 this.masterServices.getTableDescriptors().add(hTableDescriptor);
101
102
103 LOG.debug("Starting restore snapshot=" + SnapshotDescriptionUtils.toString(snapshot));
104 Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir);
105 RestoreSnapshotHelper restoreHelper = new RestoreSnapshotHelper(
106 masterServices.getConfiguration(), fs,
107 snapshot, snapshotDir, hTableDescriptor, tableDir, monitor);
108 RestoreSnapshotHelper.RestoreMetaChanges metaChanges = restoreHelper.restoreHdfsRegions();
109
110
111 hris.clear();
112 if (metaChanges.hasRegionsToAdd()) hris.addAll(metaChanges.getRegionsToAdd());
113 if (metaChanges.hasRegionsToRestore()) hris.addAll(metaChanges.getRegionsToRestore());
114 List<HRegionInfo> hrisToRemove = metaChanges.getRegionsToRemove();
115 MetaEditor.mutateRegions(catalogTracker, hrisToRemove, hris);
116
117
118 LOG.info("Restore snapshot=" + SnapshotDescriptionUtils.toString(snapshot) + " on table=" +
119 Bytes.toString(tableName) + " completed!");
120 } catch (IOException e) {
121 String msg = "restore snapshot=" + SnapshotDescriptionUtils.toString(snapshot)
122 + " failed. Try re-running the restore command.";
123 LOG.error(msg, e);
124 monitor.receive(new ForeignException(masterServices.getServerName().toString(), e));
125 throw new RestoreSnapshotException(msg, e);
126 } finally {
127 this.stopped = true;
128 }
129 }
130
131 @Override
132 public boolean isFinished() {
133 return this.stopped;
134 }
135
136 @Override
137 public SnapshotDescription getSnapshot() {
138 return snapshot;
139 }
140
141 @Override
142 public void cancel(String why) {
143 if (this.stopped) return;
144 this.stopped = true;
145 String msg = "Stopping restore snapshot=" + SnapshotDescriptionUtils.toString(snapshot)
146 + " because: " + why;
147 LOG.info(msg);
148 CancellationException ce = new CancellationException(why);
149 this.monitor.receive(new ForeignException(masterServices.getServerName().toString(), ce));
150 }
151
152 public ForeignException getExceptionIfFailed() {
153 return this.monitor.getException();
154 }
155 }