1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.regionserver.snapshot;
19
20 import java.io.IOException;
21 import java.util.ArrayList;
22 import java.util.Collection;
23 import java.util.List;
24 import java.util.concurrent.Callable;
25 import java.util.concurrent.ExecutionException;
26 import java.util.concurrent.ExecutorCompletionService;
27 import java.util.concurrent.Future;
28 import java.util.concurrent.LinkedBlockingQueue;
29 import java.util.concurrent.ThreadPoolExecutor;
30 import java.util.concurrent.TimeUnit;
31
32 import org.apache.commons.logging.Log;
33 import org.apache.commons.logging.LogFactory;
34 import org.apache.hadoop.classification.InterfaceAudience;
35 import org.apache.hadoop.classification.InterfaceStability;
36 import org.apache.hadoop.conf.Configuration;
37 import org.apache.hadoop.hbase.DaemonThreadFactory;
38 import org.apache.hadoop.hbase.TableName;
39 import org.apache.hadoop.hbase.errorhandling.ForeignException;
40 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
41 import org.apache.hadoop.hbase.master.snapshot.MasterSnapshotVerifier;
42 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
43 import org.apache.hadoop.hbase.procedure.ProcedureMember;
44 import org.apache.hadoop.hbase.procedure.ProcedureMemberRpcs;
45 import org.apache.hadoop.hbase.procedure.Subprocedure;
46 import org.apache.hadoop.hbase.procedure.SubprocedureFactory;
47 import org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs;
48 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
49 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
50 import org.apache.hadoop.hbase.regionserver.HRegion;
51 import org.apache.hadoop.hbase.regionserver.HRegionServer;
52 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
53 import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
54 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
55 import org.apache.zookeeper.KeeperException;
56
57 import com.google.protobuf.InvalidProtocolBufferException;
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72 @InterfaceAudience.Private
73 @InterfaceStability.Unstable
74 public class RegionServerSnapshotManager {
75 private static final Log LOG = LogFactory.getLog(RegionServerSnapshotManager.class);
76
77
78 private static final String CONCURENT_SNAPSHOT_TASKS_KEY = "hbase.snapshot.region.concurrentTasks";
79 private static final int DEFAULT_CONCURRENT_SNAPSHOT_TASKS = 3;
80
81
82 public static final String SNAPSHOT_REQUEST_THREADS_KEY = "hbase.snapshot.region.pool.threads";
83
84 public static final int SNAPSHOT_REQUEST_THREADS_DEFAULT = 10;
85
86
87 public static final String SNAPSHOT_TIMEOUT_MILLIS_KEY = "hbase.snapshot.region.timeout";
88
89 public static final long SNAPSHOT_TIMEOUT_MILLIS_DEFAULT = 60000;
90
91
92 public static final String SNAPSHOT_REQUEST_WAKE_MILLIS_KEY = "hbase.snapshot.region.wakefrequency";
93
94 private static final long SNAPSHOT_REQUEST_WAKE_MILLIS_DEFAULT = 500;
95
96 private final RegionServerServices rss;
97 private final ProcedureMemberRpcs memberRpcs;
98 private final ProcedureMember member;
99
100
101
102
103
104
105
106
107 RegionServerSnapshotManager(Configuration conf, HRegionServer parent,
108 ProcedureMemberRpcs memberRpc, ProcedureMember procMember) {
109 this.rss = parent;
110 this.memberRpcs = memberRpc;
111 this.member = procMember;
112 }
113
114
115
116
117
118
119 public RegionServerSnapshotManager(RegionServerServices rss)
120 throws KeeperException {
121 this.rss = rss;
122 ZooKeeperWatcher zkw = rss.getZooKeeper();
123 this.memberRpcs = new ZKProcedureMemberRpcs(zkw,
124 SnapshotManager.ONLINE_SNAPSHOT_CONTROLLER_DESCRIPTION);
125
126
127 Configuration conf = rss.getConfiguration();
128 long keepAlive = conf.getLong(SNAPSHOT_TIMEOUT_MILLIS_KEY, SNAPSHOT_TIMEOUT_MILLIS_DEFAULT);
129 int opThreads = conf.getInt(SNAPSHOT_REQUEST_THREADS_KEY, SNAPSHOT_REQUEST_THREADS_DEFAULT);
130
131
132 ThreadPoolExecutor pool = ProcedureMember.defaultPool(rss.getServerName().toString(),
133 opThreads, keepAlive);
134 this.member = new ProcedureMember(memberRpcs, pool, new SnapshotSubprocedureBuilder());
135 }
136
137
138
139
140 public void start() {
141 LOG.debug("Start Snapshot Manager " + rss.getServerName().toString());
142 this.memberRpcs.start(rss.getServerName().toString(), member);
143 }
144
145
146
147
148
149
150 public void stop(boolean force) throws IOException {
151 String mode = force ? "abruptly" : "gracefully";
152 LOG.info("Stopping RegionServerSnapshotManager " + mode + ".");
153
154 try {
155 this.member.close();
156 } finally {
157 this.memberRpcs.close();
158 }
159 }
160
161
162
163
164
165
166
167
168
169
170
171 public Subprocedure buildSubprocedure(SnapshotDescription snapshot) {
172
173
174 if (rss.isStopping() || rss.isStopped()) {
175 throw new IllegalStateException("Can't start snapshot on RS: " + rss.getServerName()
176 + ", because stopping/stopped!");
177 }
178
179
180
181 List<HRegion> involvedRegions;
182 try {
183 involvedRegions = getRegionsToSnapshot(snapshot);
184 } catch (IOException e1) {
185 throw new IllegalStateException("Failed to figure out if we should handle a snapshot - "
186 + "something has gone awry with the online regions.", e1);
187 }
188
189
190
191
192
193 LOG.debug("Launching subprocedure for snapshot " + snapshot.getName() + " from table "
194 + snapshot.getTable());
195 ForeignExceptionDispatcher exnDispatcher = new ForeignExceptionDispatcher(snapshot.getName());
196 Configuration conf = rss.getConfiguration();
197 long timeoutMillis = conf.getLong(SNAPSHOT_TIMEOUT_MILLIS_KEY,
198 SNAPSHOT_TIMEOUT_MILLIS_DEFAULT);
199 long wakeMillis = conf.getLong(SNAPSHOT_REQUEST_WAKE_MILLIS_KEY,
200 SNAPSHOT_REQUEST_WAKE_MILLIS_DEFAULT);
201
202 switch (snapshot.getType()) {
203 case FLUSH:
204 SnapshotSubprocedurePool taskManager =
205 new SnapshotSubprocedurePool(rss.getServerName().toString(), conf);
206 return new FlushSnapshotSubprocedure(member, exnDispatcher, wakeMillis,
207 timeoutMillis, involvedRegions, snapshot, taskManager);
208 default:
209 throw new UnsupportedOperationException("Unrecognized snapshot type:" + snapshot.getType());
210 }
211 }
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228 private List<HRegion> getRegionsToSnapshot(SnapshotDescription snapshot) throws IOException {
229 return rss.getOnlineRegions(TableName.valueOf(snapshot.getTable()));
230 }
231
232
233
234
235 public class SnapshotSubprocedureBuilder implements SubprocedureFactory {
236
237 @Override
238 public Subprocedure buildSubprocedure(String name, byte[] data) {
239 try {
240
241 SnapshotDescription snapshot = SnapshotDescription.parseFrom(data);
242 return RegionServerSnapshotManager.this.buildSubprocedure(snapshot);
243 } catch (InvalidProtocolBufferException e) {
244 throw new IllegalArgumentException("Could not read snapshot information from request.");
245 }
246 }
247
248 }
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263 static class SnapshotSubprocedurePool {
264 private final ExecutorCompletionService<Void> taskPool;
265 private final ThreadPoolExecutor executor;
266 private volatile boolean stopped;
267 private final List<Future<Void>> futures = new ArrayList<Future<Void>>();
268 private final String name;
269
270 SnapshotSubprocedurePool(String name, Configuration conf) {
271
272 long keepAlive = conf.getLong(
273 RegionServerSnapshotManager.SNAPSHOT_TIMEOUT_MILLIS_KEY,
274 RegionServerSnapshotManager.SNAPSHOT_TIMEOUT_MILLIS_DEFAULT);
275 int threads = conf.getInt(CONCURENT_SNAPSHOT_TASKS_KEY, DEFAULT_CONCURRENT_SNAPSHOT_TASKS);
276 this.name = name;
277 executor = new ThreadPoolExecutor(1, threads, keepAlive, TimeUnit.MILLISECONDS,
278 new LinkedBlockingQueue<Runnable>(), new DaemonThreadFactory("rs("
279 + name + ")-snapshot-pool"));
280 taskPool = new ExecutorCompletionService<Void>(executor);
281 }
282
283 boolean hasTasks() {
284 return futures.size() != 0;
285 }
286
287
288
289
290
291
292
293 void submitTask(final Callable<Void> task) {
294 Future<Void> f = this.taskPool.submit(task);
295 futures.add(f);
296 }
297
298
299
300
301
302
303
304
305
306 boolean waitForOutstandingTasks() throws ForeignException, InterruptedException {
307 LOG.debug("Waiting for local region snapshots to finish.");
308
309 int sz = futures.size();
310 try {
311
312 for (int i = 0; i < sz; i++) {
313 Future<Void> f = taskPool.take();
314 f.get();
315 if (!futures.remove(f)) {
316 LOG.warn("unexpected future" + f);
317 }
318 LOG.debug("Completed " + (i+1) + "/" + sz + " local region snapshots.");
319 }
320 LOG.debug("Completed " + sz + " local region snapshots.");
321 return true;
322 } catch (InterruptedException e) {
323 LOG.warn("Got InterruptedException in SnapshotSubprocedurePool", e);
324 if (!stopped) {
325 Thread.currentThread().interrupt();
326 throw new ForeignException("SnapshotSubprocedurePool", e);
327 }
328
329 } catch (ExecutionException e) {
330 if (e.getCause() instanceof ForeignException) {
331 LOG.warn("Rethrowing ForeignException from SnapshotSubprocedurePool", e);
332 throw (ForeignException)e.getCause();
333 }
334 LOG.warn("Got Exception in SnapshotSubprocedurePool", e);
335 throw new ForeignException(name, e.getCause());
336 } finally {
337 cancelTasks();
338 }
339 return false;
340 }
341
342
343
344
345
346 void cancelTasks() throws InterruptedException {
347 Collection<Future<Void>> tasks = futures;
348 LOG.debug("cancelling " + tasks.size() + " tasks for snapshot " + name);
349 for (Future<Void> f: tasks) {
350
351
352
353
354 f.cancel(false);
355 }
356
357
358 while (!futures.isEmpty()) {
359
360 LOG.warn("Removing cancelled elements from taskPool");
361 futures.remove(taskPool.take());
362 }
363 stop();
364 }
365
366
367
368
369 void stop() {
370 if (this.stopped) return;
371
372 this.stopped = true;
373 this.executor.shutdownNow();
374 }
375 }
376 }