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