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;
21
22 import org.apache.commons.logging.Log;
23 import org.apache.commons.logging.LogFactory;
24 import org.apache.hadoop.classification.InterfaceAudience;
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.protobuf.generated.HBaseProtos.SnapshotDescription;
29
30 import java.io.IOException;
31 import java.util.List;
32
33
34
35
36
37
38 @InterfaceAudience.Private
39 public class MasterCoprocessorHost
40 extends CoprocessorHost<MasterCoprocessorHost.MasterEnvironment> {
41
42 private static final Log LOG = LogFactory.getLog(MasterCoprocessorHost.class);
43
44
45
46
47
48 static class MasterEnvironment extends CoprocessorHost.Environment
49 implements MasterCoprocessorEnvironment {
50 private MasterServices masterServices;
51
52 public MasterEnvironment(final Class<?> implClass, final Coprocessor impl,
53 final int priority, final int seq, final Configuration conf,
54 final MasterServices services) {
55 super(impl, priority, seq, conf);
56 this.masterServices = services;
57 }
58
59 public MasterServices getMasterServices() {
60 return masterServices;
61 }
62 }
63
64 private MasterServices masterServices;
65
66 MasterCoprocessorHost(final MasterServices services, final Configuration conf) {
67 this.conf = conf;
68 this.masterServices = services;
69 loadSystemCoprocessors(conf, MASTER_COPROCESSOR_CONF_KEY);
70 }
71
72 @Override
73 public MasterEnvironment createEnvironment(final Class<?> implClass,
74 final Coprocessor instance, final int priority, final int seq,
75 final Configuration conf) {
76 for (Class c : implClass.getInterfaces()) {
77 if (CoprocessorService.class.isAssignableFrom(c)) {
78 masterServices.registerService(((CoprocessorService)instance).getService());
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 public 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 public 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 public void preCreateTableHandler(HTableDescriptor htd, HRegionInfo[] regions)
128 throws IOException {
129 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
130 for (MasterEnvironment env : coprocessors) {
131 if (env.getInstance() instanceof MasterObserver) {
132 ctx = ObserverContext.createAndPrepare(env, ctx);
133 try {
134 ((MasterObserver) env.getInstance()).preCreateTableHandler(ctx, htd,
135 regions);
136 } catch (Throwable e) {
137 handleCoprocessorThrowable(env, e);
138 }
139 if (ctx.shouldComplete()) {
140 break;
141 }
142 }
143 }
144 }
145
146 public void postCreateTableHandler(HTableDescriptor htd, HRegionInfo[] regions)
147 throws IOException {
148 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
149 for (MasterEnvironment env : coprocessors) {
150 if (env.getInstance() instanceof MasterObserver) {
151 ctx = ObserverContext.createAndPrepare(env, ctx);
152 try {
153 ((MasterObserver) env.getInstance()).postCreateTableHandler(ctx, htd,
154 regions);
155 } catch (Throwable e) {
156 handleCoprocessorThrowable(env, e);
157 }
158 if (ctx.shouldComplete()) {
159 break;
160 }
161 }
162 }
163 }
164
165 public void preDeleteTable(byte[] tableName) throws IOException {
166 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
167 for (MasterEnvironment env: coprocessors) {
168 if (env.getInstance() instanceof MasterObserver) {
169 ctx = ObserverContext.createAndPrepare(env, ctx);
170 try {
171 ((MasterObserver)env.getInstance()).preDeleteTable(ctx, tableName);
172 } catch (Throwable e) {
173 handleCoprocessorThrowable(env, e);
174 }
175 if (ctx.shouldComplete()) {
176 break;
177 }
178 }
179 }
180 }
181
182 public void postDeleteTable(byte[] tableName) throws IOException {
183 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
184 for (MasterEnvironment env: coprocessors) {
185 if (env.getInstance() instanceof MasterObserver) {
186 ctx = ObserverContext.createAndPrepare(env, ctx);
187 try {
188 ((MasterObserver)env.getInstance()).postDeleteTable(ctx, tableName);
189 } catch (Throwable e) {
190 handleCoprocessorThrowable(env, e);
191 }
192 if (ctx.shouldComplete()) {
193 break;
194 }
195 }
196 }
197 }
198
199 public void preDeleteTableHandler(byte[] tableName) throws IOException {
200 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
201 for (MasterEnvironment env : coprocessors) {
202 if (env.getInstance() instanceof MasterObserver) {
203 ctx = ObserverContext.createAndPrepare(env, ctx);
204 try {
205 ((MasterObserver) env.getInstance()).preDeleteTableHandler(ctx,
206 tableName);
207 } catch (Throwable e) {
208 handleCoprocessorThrowable(env, e);
209 }
210 if (ctx.shouldComplete()) {
211 break;
212 }
213 }
214 }
215 }
216
217 public void postDeleteTableHandler(byte[] tableName) throws IOException {
218 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
219 for (MasterEnvironment env : coprocessors) {
220 if (env.getInstance() instanceof MasterObserver) {
221 ctx = ObserverContext.createAndPrepare(env, ctx);
222 try {
223 ((MasterObserver) env.getInstance()).postDeleteTableHandler(ctx,
224 tableName);
225 } catch (Throwable e) {
226 handleCoprocessorThrowable(env, e);
227 }
228 if (ctx.shouldComplete()) {
229 break;
230 }
231 }
232 }
233 }
234 public void preModifyTable(final byte[] tableName, HTableDescriptor htd)
235 throws IOException {
236 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
237 for (MasterEnvironment env: coprocessors) {
238 if (env.getInstance() instanceof MasterObserver) {
239 ctx = ObserverContext.createAndPrepare(env, ctx);
240 try {
241 ((MasterObserver)env.getInstance()).preModifyTable(ctx, tableName,
242 htd);
243 } catch (Throwable e) {
244 handleCoprocessorThrowable(env, e);
245 }
246 if (ctx.shouldComplete()) {
247 break;
248 }
249 }
250 }
251 }
252
253 public void postModifyTable(final byte[] tableName, HTableDescriptor htd)
254 throws IOException {
255 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
256 for (MasterEnvironment env: coprocessors) {
257 if (env.getInstance() instanceof MasterObserver) {
258 ctx = ObserverContext.createAndPrepare(env, ctx);
259 try {
260 ((MasterObserver)env.getInstance()).postModifyTable(ctx, tableName,
261 htd);
262 } catch (Throwable e) {
263 handleCoprocessorThrowable(env, e);
264 }
265 if (ctx.shouldComplete()) {
266 break;
267 }
268 }
269 }
270 }
271
272 public void preModifyTableHandler(final byte[] tableName, HTableDescriptor htd)
273 throws IOException {
274 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
275 for (MasterEnvironment env : coprocessors) {
276 if (env.getInstance() instanceof MasterObserver) {
277 ctx = ObserverContext.createAndPrepare(env, ctx);
278 try {
279 ((MasterObserver) env.getInstance()).preModifyTableHandler(ctx,
280 tableName, htd);
281 } catch (Throwable e) {
282 handleCoprocessorThrowable(env, e);
283 }
284 if (ctx.shouldComplete()) {
285 break;
286 }
287 }
288 }
289 }
290
291 public void postModifyTableHandler(final byte[] tableName,
292 HTableDescriptor htd) throws IOException {
293 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
294 for (MasterEnvironment env : coprocessors) {
295 if (env.getInstance() instanceof MasterObserver) {
296 ctx = ObserverContext.createAndPrepare(env, ctx);
297 try {
298 ((MasterObserver) env.getInstance()).postModifyTableHandler(ctx,
299 tableName, htd);
300 } catch (Throwable e) {
301 handleCoprocessorThrowable(env, e);
302 }
303 if (ctx.shouldComplete()) {
304 break;
305 }
306 }
307 }
308 }
309
310 public boolean preAddColumn(byte [] tableName, HColumnDescriptor column)
311 throws IOException {
312 boolean bypass = false;
313 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
314 for (MasterEnvironment env: coprocessors) {
315 if (env.getInstance() instanceof MasterObserver) {
316 ctx = ObserverContext.createAndPrepare(env, ctx);
317 try {
318 ((MasterObserver)env.getInstance()).preAddColumn(ctx, tableName, column);
319 } catch (Throwable e) {
320 handleCoprocessorThrowable(env, e);
321 }
322 bypass |= ctx.shouldBypass();
323 if (ctx.shouldComplete()) {
324 break;
325 }
326 }
327 }
328 return bypass;
329 }
330
331 public void postAddColumn(byte [] tableName, HColumnDescriptor column)
332 throws IOException {
333 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
334 for (MasterEnvironment env: coprocessors) {
335 if (env.getInstance() instanceof MasterObserver) {
336 ctx = ObserverContext.createAndPrepare(env, ctx);
337 try {
338 ((MasterObserver)env.getInstance()).postAddColumn(ctx, tableName,
339 column);
340 } catch (Throwable e) {
341 handleCoprocessorThrowable(env, e);
342 }
343 if (ctx.shouldComplete()) {
344 break;
345 }
346 }
347 }
348 }
349
350 public boolean preAddColumnHandler(byte[] tableName, HColumnDescriptor column)
351 throws IOException {
352 boolean bypass = false;
353 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
354 for (MasterEnvironment env : coprocessors) {
355 if (env.getInstance() instanceof MasterObserver) {
356 ctx = ObserverContext.createAndPrepare(env, ctx);
357 try {
358 ((MasterObserver) env.getInstance()).preAddColumnHandler(ctx,
359 tableName, column);
360 } catch (Throwable e) {
361 handleCoprocessorThrowable(env, e);
362 }
363 bypass |= ctx.shouldBypass();
364 if (ctx.shouldComplete()) {
365 break;
366 }
367 }
368 }
369 return bypass;
370 }
371
372 public void postAddColumnHandler(byte[] tableName, HColumnDescriptor column)
373 throws IOException {
374 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
375 for (MasterEnvironment env : coprocessors) {
376 if (env.getInstance() instanceof MasterObserver) {
377 ctx = ObserverContext.createAndPrepare(env, ctx);
378 try {
379 ((MasterObserver) env.getInstance()).postAddColumnHandler(ctx,
380 tableName, column);
381 } catch (Throwable e) {
382 handleCoprocessorThrowable(env, e);
383 }
384 if (ctx.shouldComplete()) {
385 break;
386 }
387 }
388 }
389 }
390
391 public boolean preModifyColumn(byte [] tableName, HColumnDescriptor descriptor)
392 throws IOException {
393 boolean bypass = false;
394 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
395 for (MasterEnvironment env: coprocessors) {
396 if (env.getInstance() instanceof MasterObserver) {
397 ctx = ObserverContext.createAndPrepare(env, ctx);
398 try {
399 ((MasterObserver)env.getInstance()).preModifyColumn(
400 ctx, tableName, descriptor);
401 } catch (Throwable e) {
402 handleCoprocessorThrowable(env, e);
403 }
404 bypass |= ctx.shouldBypass();
405 if (ctx.shouldComplete()) {
406 break;
407 }
408 }
409 }
410 return bypass;
411 }
412
413 public void postModifyColumn(byte [] tableName, HColumnDescriptor descriptor)
414 throws IOException {
415 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
416 for (MasterEnvironment env: coprocessors) {
417 if (env.getInstance() instanceof MasterObserver) {
418 ctx = ObserverContext.createAndPrepare(env, ctx);
419 try {
420 ((MasterObserver)env.getInstance()).postModifyColumn(
421 ctx, tableName, descriptor);
422 } catch (Throwable e) {
423 handleCoprocessorThrowable(env, e);
424 }
425 if (ctx.shouldComplete()) {
426 break;
427 }
428 }
429 }
430 }
431
432 public boolean preModifyColumnHandler(byte[] tableName,
433 HColumnDescriptor descriptor) throws IOException {
434 boolean bypass = false;
435 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
436 for (MasterEnvironment env : coprocessors) {
437 if (env.getInstance() instanceof MasterObserver) {
438 ctx = ObserverContext.createAndPrepare(env, ctx);
439 try {
440 ((MasterObserver) env.getInstance()).preModifyColumnHandler(ctx,
441 tableName, descriptor);
442 } catch (Throwable e) {
443 handleCoprocessorThrowable(env, e);
444 }
445 bypass |= ctx.shouldBypass();
446 if (ctx.shouldComplete()) {
447 break;
448 }
449 }
450 }
451 return bypass;
452 }
453
454 public void postModifyColumnHandler(byte[] tableName,
455 HColumnDescriptor descriptor) throws IOException {
456 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
457 for (MasterEnvironment env : coprocessors) {
458 if (env.getInstance() instanceof MasterObserver) {
459 ctx = ObserverContext.createAndPrepare(env, ctx);
460 try {
461 ((MasterObserver) env.getInstance()).postModifyColumnHandler(ctx,
462 tableName, descriptor);
463 } catch (Throwable e) {
464 handleCoprocessorThrowable(env, e);
465 }
466 if (ctx.shouldComplete()) {
467 break;
468 }
469 }
470 }
471 }
472
473 boolean preDeleteColumn(final byte [] tableName, final byte [] c)
474 throws IOException {
475 boolean bypass = false;
476 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
477 for (MasterEnvironment env: coprocessors) {
478 if (env.getInstance() instanceof MasterObserver) {
479 ctx = ObserverContext.createAndPrepare(env, ctx);
480 try {
481 ((MasterObserver)env.getInstance()).preDeleteColumn(ctx, tableName, c);
482 } catch (Throwable e) {
483 handleCoprocessorThrowable(env, e);
484 }
485 bypass |= ctx.shouldBypass();
486 if (ctx.shouldComplete()) {
487 break;
488 }
489 }
490 }
491 return bypass;
492 }
493
494 public void postDeleteColumn(final byte [] tableName, final byte [] c)
495 throws IOException {
496 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
497 for (MasterEnvironment env: coprocessors) {
498 if (env.getInstance() instanceof MasterObserver) {
499 ctx = ObserverContext.createAndPrepare(env, ctx);
500 try {
501 ((MasterObserver)env.getInstance()).postDeleteColumn(ctx, tableName,
502 c);
503 } catch (Throwable e) {
504 handleCoprocessorThrowable(env, e);
505 }
506 if (ctx.shouldComplete()) {
507 break;
508 }
509 }
510 }
511 }
512
513 public boolean preDeleteColumnHandler(final byte[] tableName, final byte[] c)
514 throws IOException {
515 boolean bypass = false;
516 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
517 for (MasterEnvironment env : coprocessors) {
518 if (env.getInstance() instanceof MasterObserver) {
519 ctx = ObserverContext.createAndPrepare(env, ctx);
520 try {
521 ((MasterObserver) env.getInstance()).preDeleteColumnHandler(ctx,
522 tableName, c);
523 } catch (Throwable e) {
524 handleCoprocessorThrowable(env, e);
525 }
526 bypass |= ctx.shouldBypass();
527 if (ctx.shouldComplete()) {
528 break;
529 }
530 }
531 }
532 return bypass;
533 }
534
535 public void postDeleteColumnHandler(final byte[] tableName, final byte[] c)
536 throws IOException {
537 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
538 for (MasterEnvironment env : coprocessors) {
539 if (env.getInstance() instanceof MasterObserver) {
540 ctx = ObserverContext.createAndPrepare(env, ctx);
541 try {
542 ((MasterObserver) env.getInstance()).postDeleteColumnHandler(ctx,
543 tableName, c);
544 } catch (Throwable e) {
545 handleCoprocessorThrowable(env, e);
546 }
547 if (ctx.shouldComplete()) {
548 break;
549 }
550 }
551 }
552 }
553
554 public void preEnableTable(final byte [] tableName) throws IOException {
555 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
556 for (MasterEnvironment env: coprocessors) {
557 if (env.getInstance() instanceof MasterObserver) {
558 ctx = ObserverContext.createAndPrepare(env, ctx);
559 try {
560 ((MasterObserver)env.getInstance()).preEnableTable(ctx, tableName);
561 } catch (Throwable e) {
562 handleCoprocessorThrowable(env, e);
563 }
564 if (ctx.shouldComplete()) {
565 break;
566 }
567 }
568 }
569 }
570
571 public void postEnableTable(final byte [] tableName) throws IOException {
572 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
573 for (MasterEnvironment env: coprocessors) {
574 if (env.getInstance() instanceof MasterObserver) {
575 ctx = ObserverContext.createAndPrepare(env, ctx);
576 try {
577 ((MasterObserver)env.getInstance()).postEnableTable(ctx, tableName);
578 } catch (Throwable e) {
579 handleCoprocessorThrowable(env, e);
580 }
581 if (ctx.shouldComplete()) {
582 break;
583 }
584 }
585 }
586 }
587
588 public void preEnableTableHandler(final byte[] tableName) throws IOException {
589 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
590 for (MasterEnvironment env : coprocessors) {
591 if (env.getInstance() instanceof MasterObserver) {
592 ctx = ObserverContext.createAndPrepare(env, ctx);
593 try {
594 ((MasterObserver) env.getInstance()).preEnableTableHandler(ctx,
595 tableName);
596 } catch (Throwable e) {
597 handleCoprocessorThrowable(env, e);
598 }
599 if (ctx.shouldComplete()) {
600 break;
601 }
602 }
603 }
604 }
605
606 public void postEnableTableHandler(final byte[] tableName) throws IOException {
607 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
608 for (MasterEnvironment env : coprocessors) {
609 if (env.getInstance() instanceof MasterObserver) {
610 ctx = ObserverContext.createAndPrepare(env, ctx);
611 try {
612 ((MasterObserver) env.getInstance()).postEnableTableHandler(ctx,
613 tableName);
614 } catch (Throwable e) {
615 handleCoprocessorThrowable(env, e);
616 }
617 if (ctx.shouldComplete()) {
618 break;
619 }
620 }
621 }
622 }
623
624 public void preDisableTable(final byte [] tableName) throws IOException {
625 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
626 for (MasterEnvironment env: coprocessors) {
627 if (env.getInstance() instanceof MasterObserver) {
628 ctx = ObserverContext.createAndPrepare(env, ctx);
629 try {
630 ((MasterObserver)env.getInstance()).preDisableTable(ctx, tableName);
631 } catch (Throwable e) {
632 handleCoprocessorThrowable(env, e);
633 }
634 if (ctx.shouldComplete()) {
635 break;
636 }
637 }
638 }
639 }
640
641 public void postDisableTable(final byte [] tableName) throws IOException {
642 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
643 for (MasterEnvironment env: coprocessors) {
644 if (env.getInstance() instanceof MasterObserver) {
645 ctx = ObserverContext.createAndPrepare(env, ctx);
646 try {
647 ((MasterObserver)env.getInstance()).postDisableTable(ctx, tableName);
648 } catch (Throwable e) {
649 handleCoprocessorThrowable(env, e);
650 }
651 if (ctx.shouldComplete()) {
652 break;
653 }
654 }
655 }
656 }
657
658 public void preDisableTableHandler(final byte[] tableName) throws IOException {
659 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
660 for (MasterEnvironment env : coprocessors) {
661 if (env.getInstance() instanceof MasterObserver) {
662 ctx = ObserverContext.createAndPrepare(env, ctx);
663 try {
664 ((MasterObserver) env.getInstance()).preDisableTableHandler(ctx,
665 tableName);
666 } catch (Throwable e) {
667 handleCoprocessorThrowable(env, e);
668 }
669 if (ctx.shouldComplete()) {
670 break;
671 }
672 }
673 }
674 }
675
676 public void postDisableTableHandler(final byte[] tableName)
677 throws IOException {
678 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
679 for (MasterEnvironment env : coprocessors) {
680 if (env.getInstance() instanceof MasterObserver) {
681 ctx = ObserverContext.createAndPrepare(env, ctx);
682 try {
683 ((MasterObserver) env.getInstance()).postDisableTableHandler(ctx,
684 tableName);
685 } catch (Throwable e) {
686 handleCoprocessorThrowable(env, e);
687 }
688 if (ctx.shouldComplete()) {
689 break;
690 }
691 }
692 }
693 }
694
695 public boolean preMove(final HRegionInfo region, final ServerName srcServer,
696 final ServerName destServer) throws IOException {
697 boolean bypass = false;
698 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
699 for (MasterEnvironment env: coprocessors) {
700 if (env.getInstance() instanceof MasterObserver) {
701 ctx = ObserverContext.createAndPrepare(env, ctx);
702 try {
703 ((MasterObserver)env.getInstance()).preMove(
704 ctx, region, srcServer, destServer);
705 } catch (Throwable e) {
706 handleCoprocessorThrowable(env, e);
707 }
708 bypass |= ctx.shouldBypass();
709 if (ctx.shouldComplete()) {
710 break;
711 }
712 }
713 }
714 return bypass;
715 }
716
717 public void postMove(final HRegionInfo region, final ServerName srcServer,
718 final ServerName destServer) throws IOException {
719 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
720 for (MasterEnvironment env: coprocessors) {
721 if (env.getInstance() instanceof MasterObserver) {
722 ctx = ObserverContext.createAndPrepare(env, ctx);
723 try {
724 ((MasterObserver)env.getInstance()).postMove(
725 ctx, region, srcServer, destServer);
726 } catch (Throwable e) {
727 handleCoprocessorThrowable(env, e);
728 }
729 if (ctx.shouldComplete()) {
730 break;
731 }
732 }
733 }
734 }
735
736 boolean preAssign(final HRegionInfo regionInfo) throws IOException {
737 boolean bypass = false;
738 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
739 for (MasterEnvironment env: coprocessors) {
740 if (env.getInstance() instanceof MasterObserver) {
741 ctx = ObserverContext.createAndPrepare(env, ctx);
742 try {
743 ((MasterObserver) env.getInstance()).preAssign(ctx, regionInfo);
744 } catch (Throwable e) {
745 handleCoprocessorThrowable(env, e);
746 }
747 bypass |= ctx.shouldBypass();
748 if (ctx.shouldComplete()) {
749 break;
750 }
751 }
752 }
753 return bypass;
754 }
755
756 void postAssign(final HRegionInfo regionInfo) throws IOException {
757 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
758 for (MasterEnvironment env: coprocessors) {
759 if (env.getInstance() instanceof MasterObserver) {
760 ctx = ObserverContext.createAndPrepare(env, ctx);
761 try {
762 ((MasterObserver)env.getInstance()).postAssign(ctx, regionInfo);
763 } catch (Throwable e) {
764 handleCoprocessorThrowable(env, e);
765 }
766 if (ctx.shouldComplete()) {
767 break;
768 }
769 }
770 }
771 }
772
773 boolean preUnassign(final HRegionInfo regionInfo, final boolean force)
774 throws IOException {
775 boolean bypass = false;
776 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
777 for (MasterEnvironment env: coprocessors) {
778 if (env.getInstance() instanceof MasterObserver) {
779 ctx = ObserverContext.createAndPrepare(env, ctx);
780 try {
781 ((MasterObserver)env.getInstance()).preUnassign(
782 ctx, regionInfo, force);
783 } catch (Throwable e) {
784 handleCoprocessorThrowable(env, e);
785 }
786 bypass |= ctx.shouldBypass();
787 if (ctx.shouldComplete()) {
788 break;
789 }
790 }
791 }
792 return bypass;
793 }
794
795 void postUnassign(final HRegionInfo regionInfo, final boolean force)
796 throws IOException {
797 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
798 for (MasterEnvironment env: coprocessors) {
799 if (env.getInstance() instanceof MasterObserver) {
800 ctx = ObserverContext.createAndPrepare(env, ctx);
801 try {
802 ((MasterObserver)env.getInstance()).postUnassign(
803 ctx, regionInfo, force);
804 } catch (Throwable e) {
805 handleCoprocessorThrowable(env, e);
806 }
807 if (ctx.shouldComplete()) {
808 break;
809 }
810 }
811 }
812 }
813
814 void preRegionOffline(final HRegionInfo regionInfo) throws IOException {
815 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
816 for (MasterEnvironment env : coprocessors) {
817 if (env.getInstance() instanceof MasterObserver) {
818 ctx = ObserverContext.createAndPrepare(env, ctx);
819 try {
820 ((MasterObserver) env.getInstance()).preRegionOffline(ctx, regionInfo);
821 } catch (Throwable e) {
822 handleCoprocessorThrowable(env, e);
823 }
824 if (ctx.shouldComplete()) {
825 break;
826 }
827 }
828 }
829 }
830
831 void postRegionOffline(final HRegionInfo regionInfo) throws IOException {
832 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
833 for (MasterEnvironment env : coprocessors) {
834 if (env.getInstance() instanceof MasterObserver) {
835 ctx = ObserverContext.createAndPrepare(env, ctx);
836 try {
837 ((MasterObserver) env.getInstance()).postRegionOffline(ctx, regionInfo);
838 } catch (Throwable e) {
839 handleCoprocessorThrowable(env, e);
840 }
841 if (ctx.shouldComplete()) {
842 break;
843 }
844 }
845 }
846 }
847
848 boolean preBalance() throws IOException {
849 boolean bypass = false;
850 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
851 for (MasterEnvironment env: coprocessors) {
852 if (env.getInstance() instanceof MasterObserver) {
853 ctx = ObserverContext.createAndPrepare(env, ctx);
854 try {
855 ((MasterObserver)env.getInstance()).preBalance(ctx);
856 } catch (Throwable e) {
857 handleCoprocessorThrowable(env, e);
858 }
859 bypass |= ctx.shouldBypass();
860 if (ctx.shouldComplete()) {
861 break;
862 }
863 }
864 }
865 return bypass;
866 }
867
868 void postBalance(List<RegionPlan> plans) throws IOException {
869 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
870 for (MasterEnvironment env: coprocessors) {
871 if (env.getInstance() instanceof MasterObserver) {
872 ctx = ObserverContext.createAndPrepare(env, ctx);
873 try {
874 ((MasterObserver)env.getInstance()).postBalance(ctx, plans);
875 } catch (Throwable e) {
876 handleCoprocessorThrowable(env, e);
877 }
878 if (ctx.shouldComplete()) {
879 break;
880 }
881 }
882 }
883 }
884
885 boolean preBalanceSwitch(final boolean b) throws IOException {
886 boolean balance = b;
887 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
888 for (MasterEnvironment env: coprocessors) {
889 if (env.getInstance() instanceof MasterObserver) {
890 ctx = ObserverContext.createAndPrepare(env, ctx);
891 try {
892 balance = ((MasterObserver)env.getInstance()).preBalanceSwitch(
893 ctx, balance);
894 } catch (Throwable e) {
895 handleCoprocessorThrowable(env, e);
896 }
897 if (ctx.shouldComplete()) {
898 break;
899 }
900 }
901 }
902 return balance;
903 }
904
905 void postBalanceSwitch(final boolean oldValue, final boolean newValue)
906 throws IOException {
907 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
908 for (MasterEnvironment env: coprocessors) {
909 if (env.getInstance() instanceof MasterObserver) {
910 ctx = ObserverContext.createAndPrepare(env, ctx);
911 try {
912 ((MasterObserver)env.getInstance()).postBalanceSwitch(
913 ctx, oldValue, newValue);
914 } catch (Throwable e) {
915 handleCoprocessorThrowable(env, e);
916 }
917 if (ctx.shouldComplete()) {
918 break;
919 }
920 }
921 }
922 }
923
924 void preShutdown() throws IOException {
925 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
926 for (MasterEnvironment env: coprocessors) {
927 if (env.getInstance() instanceof MasterObserver) {
928 ctx = ObserverContext.createAndPrepare(env, ctx);
929 try {
930 ((MasterObserver)env.getInstance()).preShutdown(ctx);
931 } catch (Throwable e) {
932 handleCoprocessorThrowable(env, e);
933 }
934 if (ctx.shouldComplete()) {
935 break;
936 }
937 }
938 }
939 }
940
941 void preStopMaster() throws IOException {
942 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
943 for (MasterEnvironment env: coprocessors) {
944 if (env.getInstance() instanceof MasterObserver) {
945 ctx = ObserverContext.createAndPrepare(env, ctx);
946 try {
947 ((MasterObserver)env.getInstance()).preStopMaster(ctx);
948 } catch (Throwable e) {
949 handleCoprocessorThrowable(env, e);
950 }
951 if (ctx.shouldComplete()) {
952 break;
953 }
954 }
955 }
956 }
957
958 void postStartMaster() throws IOException {
959 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
960 for (MasterEnvironment env: coprocessors) {
961 if (env.getInstance() instanceof MasterObserver) {
962 ctx = ObserverContext.createAndPrepare(env, ctx);
963 try {
964 ((MasterObserver)env.getInstance()).postStartMaster(ctx);
965 } catch (Throwable e) {
966 handleCoprocessorThrowable(env, e);
967 }
968 if (ctx.shouldComplete()) {
969 break;
970 }
971 }
972 }
973 }
974
975 public void preSnapshot(final SnapshotDescription snapshot,
976 final HTableDescriptor hTableDescriptor) throws IOException {
977 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
978 for (MasterEnvironment env: coprocessors) {
979 if (env.getInstance() instanceof MasterObserver) {
980 ctx = ObserverContext.createAndPrepare(env, ctx);
981 try {
982 ((MasterObserver)env.getInstance()).preSnapshot(ctx, snapshot, hTableDescriptor);
983 } catch (Throwable e) {
984 handleCoprocessorThrowable(env, e);
985 }
986 if (ctx.shouldComplete()) {
987 break;
988 }
989 }
990 }
991 }
992
993 public void postSnapshot(final SnapshotDescription snapshot,
994 final HTableDescriptor hTableDescriptor) throws IOException {
995 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
996 for (MasterEnvironment env: coprocessors) {
997 if (env.getInstance() instanceof MasterObserver) {
998 ctx = ObserverContext.createAndPrepare(env, ctx);
999 try {
1000 ((MasterObserver)env.getInstance()).postSnapshot(ctx, snapshot, hTableDescriptor);
1001 } catch (Throwable e) {
1002 handleCoprocessorThrowable(env, e);
1003 }
1004 if (ctx.shouldComplete()) {
1005 break;
1006 }
1007 }
1008 }
1009 }
1010
1011 public void preCloneSnapshot(final SnapshotDescription snapshot,
1012 final HTableDescriptor hTableDescriptor) throws IOException {
1013 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
1014 for (MasterEnvironment env: coprocessors) {
1015 if (env.getInstance() instanceof MasterObserver) {
1016 ctx = ObserverContext.createAndPrepare(env, ctx);
1017 try {
1018 ((MasterObserver)env.getInstance()).preCloneSnapshot(ctx, snapshot, hTableDescriptor);
1019 } catch (Throwable e) {
1020 handleCoprocessorThrowable(env, e);
1021 }
1022 if (ctx.shouldComplete()) {
1023 break;
1024 }
1025 }
1026 }
1027 }
1028
1029 public void postCloneSnapshot(final SnapshotDescription snapshot,
1030 final HTableDescriptor hTableDescriptor) throws IOException {
1031 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
1032 for (MasterEnvironment env: coprocessors) {
1033 if (env.getInstance() instanceof MasterObserver) {
1034 ctx = ObserverContext.createAndPrepare(env, ctx);
1035 try {
1036 ((MasterObserver)env.getInstance()).postCloneSnapshot(ctx, snapshot, hTableDescriptor);
1037 } catch (Throwable e) {
1038 handleCoprocessorThrowable(env, e);
1039 }
1040 if (ctx.shouldComplete()) {
1041 break;
1042 }
1043 }
1044 }
1045 }
1046
1047 public void preRestoreSnapshot(final SnapshotDescription snapshot,
1048 final HTableDescriptor hTableDescriptor) throws IOException {
1049 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
1050 for (MasterEnvironment env: coprocessors) {
1051 if (env.getInstance() instanceof MasterObserver) {
1052 ctx = ObserverContext.createAndPrepare(env, ctx);
1053 try {
1054 ((MasterObserver)env.getInstance()).preRestoreSnapshot(ctx, snapshot, hTableDescriptor);
1055 } catch (Throwable e) {
1056 handleCoprocessorThrowable(env, e);
1057 }
1058 if (ctx.shouldComplete()) {
1059 break;
1060 }
1061 }
1062 }
1063 }
1064
1065 public void postRestoreSnapshot(final SnapshotDescription snapshot,
1066 final HTableDescriptor hTableDescriptor) throws IOException {
1067 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
1068 for (MasterEnvironment env: coprocessors) {
1069 if (env.getInstance() instanceof MasterObserver) {
1070 ctx = ObserverContext.createAndPrepare(env, ctx);
1071 try {
1072 ((MasterObserver)env.getInstance()).postRestoreSnapshot(ctx, snapshot, hTableDescriptor);
1073 } catch (Throwable e) {
1074 handleCoprocessorThrowable(env, e);
1075 }
1076 if (ctx.shouldComplete()) {
1077 break;
1078 }
1079 }
1080 }
1081 }
1082
1083 public void preDeleteSnapshot(final SnapshotDescription snapshot) throws IOException {
1084 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
1085 for (MasterEnvironment env: coprocessors) {
1086 if (env.getInstance() instanceof MasterObserver) {
1087 ctx = ObserverContext.createAndPrepare(env, ctx);
1088 try {
1089 ((MasterObserver)env.getInstance()).preDeleteSnapshot(ctx, snapshot);
1090 } catch (Throwable e) {
1091 handleCoprocessorThrowable(env, e);
1092 }
1093 if (ctx.shouldComplete()) {
1094 break;
1095 }
1096 }
1097 }
1098 }
1099
1100 public void postDeleteSnapshot(final SnapshotDescription snapshot) throws IOException {
1101 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
1102 for (MasterEnvironment env: coprocessors) {
1103 if (env.getInstance() instanceof MasterObserver) {
1104 ctx = ObserverContext.createAndPrepare(env, ctx);
1105 try {
1106 ((MasterObserver)env.getInstance()).postDeleteSnapshot(ctx, snapshot);
1107 } catch (Throwable e) {
1108 handleCoprocessorThrowable(env, e);
1109 }
1110 if (ctx.shouldComplete()) {
1111 break;
1112 }
1113 }
1114 }
1115 }
1116 }