1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21 package org.apache.hadoop.hbase.master;
22
23 import org.apache.commons.logging.Log;
24 import org.apache.commons.logging.LogFactory;
25 import org.apache.hadoop.conf.Configuration;
26 import org.apache.hadoop.hbase.*;
27 import org.apache.hadoop.hbase.coprocessor.*;
28 import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
29 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
30
31 import java.io.IOException;
32
33
34
35
36
37
38 public class MasterCoprocessorHost
39 extends CoprocessorHost<MasterCoprocessorHost.MasterEnvironment> {
40
41 private static final Log LOG = LogFactory.getLog(MasterCoprocessorHost.class);
42
43
44
45
46
47 static class MasterEnvironment extends CoprocessorHost.Environment
48 implements MasterCoprocessorEnvironment {
49 private MasterServices masterServices;
50
51 public MasterEnvironment(final Class<?> implClass, final Coprocessor impl,
52 final int priority, final int seq, final Configuration conf,
53 final MasterServices services) {
54 super(impl, priority, seq, conf);
55 this.masterServices = services;
56 }
57
58 public MasterServices getMasterServices() {
59 return masterServices;
60 }
61 }
62
63 private MasterServices masterServices;
64
65 MasterCoprocessorHost(final MasterServices services, final Configuration conf) {
66 this.conf = conf;
67 this.masterServices = services;
68 loadSystemCoprocessors(conf, MASTER_COPROCESSOR_CONF_KEY);
69 }
70
71 @Override
72 public MasterEnvironment createEnvironment(final Class<?> implClass,
73 final Coprocessor instance, final int priority, final int seq,
74 final Configuration conf) {
75 for (Class c : implClass.getInterfaces()) {
76 if (CoprocessorProtocol.class.isAssignableFrom(c)) {
77 masterServices.registerProtocol(c, (CoprocessorProtocol)instance);
78 break;
79 }
80 }
81 return new MasterEnvironment(implClass, instance, priority, seq, conf,
82 masterServices);
83 }
84
85 @Override
86 protected void abortServer(final CoprocessorEnvironment env, final Throwable e) {
87 abortServer("master", masterServices, env, e);
88 }
89
90
91 void preCreateTable(HTableDescriptor htd, HRegionInfo[] regions)
92 throws IOException {
93 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
94 for (MasterEnvironment env: coprocessors) {
95 if (env.getInstance() instanceof MasterObserver) {
96 ctx = ObserverContext.createAndPrepare(env, ctx);
97 try {
98 ((MasterObserver)env.getInstance()).preCreateTable(ctx, htd, regions);
99 } catch (Throwable e) {
100 handleCoprocessorThrowable(env, e);
101 }
102 if (ctx.shouldComplete()) {
103 break;
104 }
105 }
106 }
107 }
108
109 void postCreateTable(HTableDescriptor htd, HRegionInfo[] regions)
110 throws IOException {
111 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
112 for (MasterEnvironment env: coprocessors) {
113 if (env.getInstance() instanceof MasterObserver) {
114 ctx = ObserverContext.createAndPrepare(env, ctx);
115 try {
116 ((MasterObserver)env.getInstance()).postCreateTable(ctx, htd, regions);
117 } catch (Throwable e) {
118 handleCoprocessorThrowable(env, e);
119 }
120 if (ctx.shouldComplete()) {
121 break;
122 }
123 }
124 }
125 }
126
127 void preDeleteTable(byte[] tableName) throws IOException {
128 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
129 for (MasterEnvironment env: coprocessors) {
130 if (env.getInstance() instanceof MasterObserver) {
131 ctx = ObserverContext.createAndPrepare(env, ctx);
132 try {
133 ((MasterObserver)env.getInstance()).preDeleteTable(ctx, tableName);
134 } catch (Throwable e) {
135 handleCoprocessorThrowable(env, e);
136 }
137 if (ctx.shouldComplete()) {
138 break;
139 }
140 }
141 }
142 }
143
144 void postDeleteTable(byte[] tableName) throws IOException {
145 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
146 for (MasterEnvironment env: coprocessors) {
147 if (env.getInstance() instanceof MasterObserver) {
148 ctx = ObserverContext.createAndPrepare(env, ctx);
149 try {
150 ((MasterObserver)env.getInstance()).postDeleteTable(ctx, tableName);
151 } catch (Throwable e) {
152 handleCoprocessorThrowable(env, e);
153 }
154 if (ctx.shouldComplete()) {
155 break;
156 }
157 }
158 }
159 }
160
161 void preModifyTable(final byte[] tableName, HTableDescriptor htd)
162 throws IOException {
163 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
164 for (MasterEnvironment env: coprocessors) {
165 if (env.getInstance() instanceof MasterObserver) {
166 ctx = ObserverContext.createAndPrepare(env, ctx);
167 try {
168 ((MasterObserver)env.getInstance()).preModifyTable(ctx, tableName,
169 htd);
170 } catch (Throwable e) {
171 handleCoprocessorThrowable(env, e);
172 }
173 if (ctx.shouldComplete()) {
174 break;
175 }
176 }
177 }
178 }
179
180 void postModifyTable(final byte[] tableName, HTableDescriptor htd)
181 throws IOException {
182 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
183 for (MasterEnvironment env: coprocessors) {
184 if (env.getInstance() instanceof MasterObserver) {
185 ctx = ObserverContext.createAndPrepare(env, ctx);
186 try {
187 ((MasterObserver)env.getInstance()).postModifyTable(ctx, tableName,
188 htd);
189 } catch (Throwable e) {
190 handleCoprocessorThrowable(env, e);
191 }
192 if (ctx.shouldComplete()) {
193 break;
194 }
195 }
196 }
197 }
198
199 boolean preAddColumn(byte [] tableName, HColumnDescriptor column)
200 throws IOException {
201 boolean bypass = false;
202 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
203 for (MasterEnvironment env: coprocessors) {
204 if (env.getInstance() instanceof MasterObserver) {
205 ctx = ObserverContext.createAndPrepare(env, ctx);
206 try {
207 ((MasterObserver)env.getInstance()).preAddColumn(ctx, tableName, column);
208 } catch (Throwable e) {
209 handleCoprocessorThrowable(env, e);
210 }
211 bypass |= ctx.shouldBypass();
212 if (ctx.shouldComplete()) {
213 break;
214 }
215 }
216 }
217 return bypass;
218 }
219
220 void postAddColumn(byte [] tableName, HColumnDescriptor column)
221 throws IOException {
222 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
223 for (MasterEnvironment env: coprocessors) {
224 if (env.getInstance() instanceof MasterObserver) {
225 ctx = ObserverContext.createAndPrepare(env, ctx);
226 try {
227 ((MasterObserver)env.getInstance()).postAddColumn(ctx, tableName,
228 column);
229 } catch (Throwable e) {
230 handleCoprocessorThrowable(env, e);
231 }
232 if (ctx.shouldComplete()) {
233 break;
234 }
235 }
236 }
237 }
238
239 boolean preModifyColumn(byte [] tableName, HColumnDescriptor descriptor)
240 throws IOException {
241 boolean bypass = false;
242 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
243 for (MasterEnvironment env: coprocessors) {
244 if (env.getInstance() instanceof MasterObserver) {
245 ctx = ObserverContext.createAndPrepare(env, ctx);
246 try {
247 ((MasterObserver)env.getInstance()).preModifyColumn(
248 ctx, tableName, descriptor);
249 } catch (Throwable e) {
250 handleCoprocessorThrowable(env, e);
251 }
252 bypass |= ctx.shouldBypass();
253 if (ctx.shouldComplete()) {
254 break;
255 }
256 }
257 }
258 return bypass;
259 }
260
261 void postModifyColumn(byte [] tableName, HColumnDescriptor descriptor)
262 throws IOException {
263 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
264 for (MasterEnvironment env: coprocessors) {
265 if (env.getInstance() instanceof MasterObserver) {
266 ctx = ObserverContext.createAndPrepare(env, ctx);
267 try {
268 ((MasterObserver)env.getInstance()).postModifyColumn(
269 ctx, tableName, descriptor);
270 } catch (Throwable e) {
271 handleCoprocessorThrowable(env, e);
272 }
273 if (ctx.shouldComplete()) {
274 break;
275 }
276 }
277 }
278 }
279
280 boolean preDeleteColumn(final byte [] tableName, final byte [] c)
281 throws IOException {
282 boolean bypass = false;
283 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
284 for (MasterEnvironment env: coprocessors) {
285 if (env.getInstance() instanceof MasterObserver) {
286 ctx = ObserverContext.createAndPrepare(env, ctx);
287 try {
288 ((MasterObserver)env.getInstance()).preDeleteColumn(ctx, tableName, c);
289 } catch (Throwable e) {
290 handleCoprocessorThrowable(env, e);
291 }
292 bypass |= ctx.shouldBypass();
293 if (ctx.shouldComplete()) {
294 break;
295 }
296 }
297 }
298 return bypass;
299 }
300
301 void postDeleteColumn(final byte [] tableName, final byte [] c)
302 throws IOException {
303 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
304 for (MasterEnvironment env: coprocessors) {
305 if (env.getInstance() instanceof MasterObserver) {
306 ctx = ObserverContext.createAndPrepare(env, ctx);
307 try {
308 ((MasterObserver)env.getInstance()).postDeleteColumn(ctx, tableName,
309 c);
310 } catch (Throwable e) {
311 handleCoprocessorThrowable(env, e);
312 }
313 if (ctx.shouldComplete()) {
314 break;
315 }
316 }
317 }
318 }
319
320 void preEnableTable(final byte [] tableName) throws IOException {
321 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
322 for (MasterEnvironment env: coprocessors) {
323 if (env.getInstance() instanceof MasterObserver) {
324 ctx = ObserverContext.createAndPrepare(env, ctx);
325 try {
326 ((MasterObserver)env.getInstance()).preEnableTable(ctx, tableName);
327 } catch (Throwable e) {
328 handleCoprocessorThrowable(env, e);
329 }
330 if (ctx.shouldComplete()) {
331 break;
332 }
333 }
334 }
335 }
336
337 void postEnableTable(final byte [] tableName) throws IOException {
338 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
339 for (MasterEnvironment env: coprocessors) {
340 if (env.getInstance() instanceof MasterObserver) {
341 ctx = ObserverContext.createAndPrepare(env, ctx);
342 try {
343 ((MasterObserver)env.getInstance()).postEnableTable(ctx, tableName);
344 } catch (Throwable e) {
345 handleCoprocessorThrowable(env, e);
346 }
347 if (ctx.shouldComplete()) {
348 break;
349 }
350 }
351 }
352 }
353
354 void preDisableTable(final byte [] tableName) throws IOException {
355 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
356 for (MasterEnvironment env: coprocessors) {
357 if (env.getInstance() instanceof MasterObserver) {
358 ctx = ObserverContext.createAndPrepare(env, ctx);
359 try {
360 ((MasterObserver)env.getInstance()).preDisableTable(ctx, tableName);
361 } catch (Throwable e) {
362 handleCoprocessorThrowable(env, e);
363 }
364 if (ctx.shouldComplete()) {
365 break;
366 }
367 }
368 }
369 }
370
371 void postDisableTable(final byte [] tableName) throws IOException {
372 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
373 for (MasterEnvironment env: coprocessors) {
374 if (env.getInstance() instanceof MasterObserver) {
375 ctx = ObserverContext.createAndPrepare(env, ctx);
376 try {
377 ((MasterObserver)env.getInstance()).postDisableTable(ctx, tableName);
378 } catch (Throwable e) {
379 handleCoprocessorThrowable(env, e);
380 }
381 if (ctx.shouldComplete()) {
382 break;
383 }
384 }
385 }
386 }
387
388 boolean preMove(final HRegionInfo region, final ServerName srcServer, final ServerName destServer)
389 throws IOException {
390 boolean bypass = false;
391 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
392 for (MasterEnvironment env: coprocessors) {
393 if (env.getInstance() instanceof MasterObserver) {
394 ctx = ObserverContext.createAndPrepare(env, ctx);
395 try {
396 ((MasterObserver)env.getInstance()).preMove(
397 ctx, region, srcServer, destServer);
398 } catch (Throwable e) {
399 handleCoprocessorThrowable(env, e);
400 }
401 bypass |= ctx.shouldBypass();
402 if (ctx.shouldComplete()) {
403 break;
404 }
405 }
406 }
407 return bypass;
408 }
409
410 void postMove(final HRegionInfo region, final ServerName srcServer, final ServerName destServer)
411 throws IOException {
412 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
413 for (MasterEnvironment env: coprocessors) {
414 if (env.getInstance() instanceof MasterObserver) {
415 ctx = ObserverContext.createAndPrepare(env, ctx);
416 try {
417 ((MasterObserver)env.getInstance()).postMove(
418 ctx, region, srcServer, destServer);
419 } catch (Throwable e) {
420 handleCoprocessorThrowable(env, e);
421 }
422 if (ctx.shouldComplete()) {
423 break;
424 }
425 }
426 }
427 }
428
429 boolean preAssign(final HRegionInfo regionInfo) throws IOException {
430 boolean bypass = false;
431 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
432 for (MasterEnvironment env: coprocessors) {
433 if (env.getInstance() instanceof MasterObserver) {
434 ctx = ObserverContext.createAndPrepare(env, ctx);
435 try {
436 ((MasterObserver) env.getInstance()).preAssign(ctx, regionInfo);
437 } catch (Throwable e) {
438 handleCoprocessorThrowable(env, e);
439 }
440 bypass |= ctx.shouldBypass();
441 if (ctx.shouldComplete()) {
442 break;
443 }
444 }
445 }
446 return bypass;
447 }
448
449 void postAssign(final HRegionInfo regionInfo) throws IOException {
450 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
451 for (MasterEnvironment env: coprocessors) {
452 if (env.getInstance() instanceof MasterObserver) {
453 ctx = ObserverContext.createAndPrepare(env, ctx);
454 try {
455 ((MasterObserver)env.getInstance()).postAssign(ctx, regionInfo);
456 } catch (Throwable e) {
457 handleCoprocessorThrowable(env, e);
458 }
459 if (ctx.shouldComplete()) {
460 break;
461 }
462 }
463 }
464 }
465
466 boolean preUnassign(final HRegionInfo regionInfo, final boolean force)
467 throws IOException {
468 boolean bypass = false;
469 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
470 for (MasterEnvironment env: coprocessors) {
471 if (env.getInstance() instanceof MasterObserver) {
472 ctx = ObserverContext.createAndPrepare(env, ctx);
473 try {
474 ((MasterObserver)env.getInstance()).preUnassign(
475 ctx, regionInfo, force);
476 } catch (Throwable e) {
477 handleCoprocessorThrowable(env, e);
478 }
479 bypass |= ctx.shouldBypass();
480 if (ctx.shouldComplete()) {
481 break;
482 }
483 }
484 }
485 return bypass;
486 }
487
488 void postUnassign(final HRegionInfo regionInfo, final boolean force)
489 throws IOException {
490 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
491 for (MasterEnvironment env: coprocessors) {
492 if (env.getInstance() instanceof MasterObserver) {
493 ctx = ObserverContext.createAndPrepare(env, ctx);
494 try {
495 ((MasterObserver)env.getInstance()).postUnassign(
496 ctx, regionInfo, force);
497 } catch (Throwable e) {
498 handleCoprocessorThrowable(env, e);
499 }
500 if (ctx.shouldComplete()) {
501 break;
502 }
503 }
504 }
505 }
506
507 boolean preBalance() throws IOException {
508 boolean bypass = false;
509 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
510 for (MasterEnvironment env: coprocessors) {
511 if (env.getInstance() instanceof MasterObserver) {
512 ctx = ObserverContext.createAndPrepare(env, ctx);
513 try {
514 ((MasterObserver)env.getInstance()).preBalance(ctx);
515 } catch (Throwable e) {
516 handleCoprocessorThrowable(env, e);
517 }
518 bypass |= ctx.shouldBypass();
519 if (ctx.shouldComplete()) {
520 break;
521 }
522 }
523 }
524 return bypass;
525 }
526
527 void postBalance() throws IOException {
528 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
529 for (MasterEnvironment env: coprocessors) {
530 if (env.getInstance() instanceof MasterObserver) {
531 ctx = ObserverContext.createAndPrepare(env, ctx);
532 try {
533 ((MasterObserver)env.getInstance()).postBalance(ctx);
534 } catch (Throwable e) {
535 handleCoprocessorThrowable(env, e);
536 }
537 if (ctx.shouldComplete()) {
538 break;
539 }
540 }
541 }
542 }
543
544 boolean preBalanceSwitch(final boolean b) throws IOException {
545 boolean balance = b;
546 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
547 for (MasterEnvironment env: coprocessors) {
548 if (env.getInstance() instanceof MasterObserver) {
549 ctx = ObserverContext.createAndPrepare(env, ctx);
550 try {
551 balance = ((MasterObserver)env.getInstance()).preBalanceSwitch(
552 ctx, balance);
553 } catch (Throwable e) {
554 handleCoprocessorThrowable(env, e);
555 }
556 if (ctx.shouldComplete()) {
557 break;
558 }
559 }
560 }
561 return balance;
562 }
563
564 void postBalanceSwitch(final boolean oldValue, final boolean newValue)
565 throws IOException {
566 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
567 for (MasterEnvironment env: coprocessors) {
568 if (env.getInstance() instanceof MasterObserver) {
569 ctx = ObserverContext.createAndPrepare(env, ctx);
570 try {
571 ((MasterObserver)env.getInstance()).postBalanceSwitch(
572 ctx, oldValue, newValue);
573 } catch (Throwable e) {
574 handleCoprocessorThrowable(env, e);
575 }
576 if (ctx.shouldComplete()) {
577 break;
578 }
579 }
580 }
581 }
582
583 void preShutdown() throws IOException {
584 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
585 for (MasterEnvironment env: coprocessors) {
586 if (env.getInstance() instanceof MasterObserver) {
587 ctx = ObserverContext.createAndPrepare(env, ctx);
588 try {
589 ((MasterObserver)env.getInstance()).preShutdown(ctx);
590 } catch (Throwable e) {
591 handleCoprocessorThrowable(env, e);
592 }
593 if (ctx.shouldComplete()) {
594 break;
595 }
596 }
597 }
598 }
599
600 void preStopMaster() throws IOException {
601 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
602 for (MasterEnvironment env: coprocessors) {
603 if (env.getInstance() instanceof MasterObserver) {
604 ctx = ObserverContext.createAndPrepare(env, ctx);
605 try {
606 ((MasterObserver)env.getInstance()).preStopMaster(ctx);
607 } catch (Throwable e) {
608 handleCoprocessorThrowable(env, e);
609 }
610 if (ctx.shouldComplete()) {
611 break;
612 }
613 }
614 }
615 }
616
617 void postStartMaster() throws IOException {
618 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
619 for (MasterEnvironment env: coprocessors) {
620 if (env.getInstance() instanceof MasterObserver) {
621 ctx = ObserverContext.createAndPrepare(env, ctx);
622 try {
623 ((MasterObserver)env.getInstance()).postStartMaster(ctx);
624 } catch (Throwable e) {
625 handleCoprocessorThrowable(env, e);
626 }
627 if (ctx.shouldComplete()) {
628 break;
629 }
630 }
631 }
632 }
633
634 public void preSnapshot(final SnapshotDescription snapshot,
635 final HTableDescriptor hTableDescriptor) throws IOException {
636 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
637 for (MasterEnvironment env: coprocessors) {
638 if (env.getInstance() instanceof MasterObserver) {
639 ctx = ObserverContext.createAndPrepare(env, ctx);
640 try {
641 ((MasterObserver)env.getInstance()).preSnapshot(ctx, snapshot, hTableDescriptor);
642 } catch (Throwable e) {
643 handleCoprocessorThrowable(env, e);
644 }
645 if (ctx.shouldComplete()) {
646 break;
647 }
648 }
649 }
650 }
651
652 public void postSnapshot(final SnapshotDescription snapshot,
653 final HTableDescriptor hTableDescriptor) throws IOException {
654 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
655 for (MasterEnvironment env: coprocessors) {
656 if (env.getInstance() instanceof MasterObserver) {
657 ctx = ObserverContext.createAndPrepare(env, ctx);
658 try {
659 ((MasterObserver)env.getInstance()).postSnapshot(ctx, snapshot, hTableDescriptor);
660 } catch (Throwable e) {
661 handleCoprocessorThrowable(env, e);
662 }
663 if (ctx.shouldComplete()) {
664 break;
665 }
666 }
667 }
668 }
669
670 public void preCloneSnapshot(final SnapshotDescription snapshot,
671 final HTableDescriptor hTableDescriptor) throws IOException {
672 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
673 for (MasterEnvironment env: coprocessors) {
674 if (env.getInstance() instanceof MasterObserver) {
675 ctx = ObserverContext.createAndPrepare(env, ctx);
676 try {
677 ((MasterObserver)env.getInstance()).preCloneSnapshot(ctx, snapshot, hTableDescriptor);
678 } catch (Throwable e) {
679 handleCoprocessorThrowable(env, e);
680 }
681 if (ctx.shouldComplete()) {
682 break;
683 }
684 }
685 }
686 }
687
688 public void postCloneSnapshot(final SnapshotDescription snapshot,
689 final HTableDescriptor hTableDescriptor) throws IOException {
690 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
691 for (MasterEnvironment env: coprocessors) {
692 if (env.getInstance() instanceof MasterObserver) {
693 ctx = ObserverContext.createAndPrepare(env, ctx);
694 try {
695 ((MasterObserver)env.getInstance()).postCloneSnapshot(ctx, snapshot, hTableDescriptor);
696 } catch (Throwable e) {
697 handleCoprocessorThrowable(env, e);
698 }
699 if (ctx.shouldComplete()) {
700 break;
701 }
702 }
703 }
704 }
705
706 public void preRestoreSnapshot(final SnapshotDescription snapshot,
707 final HTableDescriptor hTableDescriptor) throws IOException {
708 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
709 for (MasterEnvironment env: coprocessors) {
710 if (env.getInstance() instanceof MasterObserver) {
711 ctx = ObserverContext.createAndPrepare(env, ctx);
712 try {
713 ((MasterObserver)env.getInstance()).preRestoreSnapshot(ctx, snapshot, hTableDescriptor);
714 } catch (Throwable e) {
715 handleCoprocessorThrowable(env, e);
716 }
717 if (ctx.shouldComplete()) {
718 break;
719 }
720 }
721 }
722 }
723
724 public void postRestoreSnapshot(final SnapshotDescription snapshot,
725 final HTableDescriptor hTableDescriptor) throws IOException {
726 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
727 for (MasterEnvironment env: coprocessors) {
728 if (env.getInstance() instanceof MasterObserver) {
729 ctx = ObserverContext.createAndPrepare(env, ctx);
730 try {
731 ((MasterObserver)env.getInstance()).postRestoreSnapshot(ctx, snapshot, hTableDescriptor);
732 } catch (Throwable e) {
733 handleCoprocessorThrowable(env, e);
734 }
735 if (ctx.shouldComplete()) {
736 break;
737 }
738 }
739 }
740 }
741
742 public void preDeleteSnapshot(final SnapshotDescription snapshot) throws IOException {
743 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
744 for (MasterEnvironment env: coprocessors) {
745 if (env.getInstance() instanceof MasterObserver) {
746 ctx = ObserverContext.createAndPrepare(env, ctx);
747 try {
748 ((MasterObserver)env.getInstance()).preDeleteSnapshot(ctx, snapshot);
749 } catch (Throwable e) {
750 handleCoprocessorThrowable(env, e);
751 }
752 if (ctx.shouldComplete()) {
753 break;
754 }
755 }
756 }
757 }
758
759 public void postDeleteSnapshot(final SnapshotDescription snapshot) throws IOException {
760 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
761 for (MasterEnvironment env: coprocessors) {
762 if (env.getInstance() instanceof MasterObserver) {
763 ctx = ObserverContext.createAndPrepare(env, ctx);
764 try {
765 ((MasterObserver)env.getInstance()).postDeleteSnapshot(ctx, snapshot);
766 } catch (Throwable e) {
767 handleCoprocessorThrowable(env, e);
768 }
769 if (ctx.shouldComplete()) {
770 break;
771 }
772 }
773 }
774 }
775 }