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 boolean preCreateNamespace(NamespaceDescriptor ns)
91 throws IOException {
92 boolean bypass = false;
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()).preCreateNamespace(
99 ctx, ns);
100 } catch (Throwable e) {
101 handleCoprocessorThrowable(env, e);
102 }
103 bypass |= ctx.shouldBypass();
104 if (ctx.shouldComplete()) {
105 break;
106 }
107 }
108 }
109 return bypass;
110 }
111
112 void postCreateNamespace(NamespaceDescriptor ns)
113 throws IOException {
114 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
115 for (MasterEnvironment env: coprocessors) {
116 if (env.getInstance() instanceof MasterObserver) {
117 ctx = ObserverContext.createAndPrepare(env, ctx);
118 try {
119 ((MasterObserver)env.getInstance()).postCreateNamespace(ctx, ns);
120 } catch (Throwable e) {
121 handleCoprocessorThrowable(env, e);
122 }
123 if (ctx.shouldComplete()) {
124 break;
125 }
126 }
127 }
128 }
129
130 boolean preDeleteNamespace(String namespaceName) throws IOException {
131 boolean bypass = false;
132 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
133 for (MasterEnvironment env: coprocessors) {
134 if (env.getInstance() instanceof MasterObserver) {
135 ctx = ObserverContext.createAndPrepare(env, ctx);
136 try {
137 ((MasterObserver)env.getInstance()).preDeleteNamespace(
138 ctx, namespaceName);
139 } catch (Throwable e) {
140 handleCoprocessorThrowable(env, e);
141 }
142 bypass |= ctx.shouldBypass();
143 if (ctx.shouldComplete()) {
144 break;
145 }
146 }
147 }
148 return bypass;
149 }
150
151 void postDeleteNamespace(String namespaceName) throws IOException {
152 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
153 for (MasterEnvironment env: coprocessors) {
154 if (env.getInstance() instanceof MasterObserver) {
155 ctx = ObserverContext.createAndPrepare(env, ctx);
156 try {
157 ((MasterObserver)env.getInstance()).postDeleteNamespace(ctx, namespaceName);
158 } catch (Throwable e) {
159 handleCoprocessorThrowable(env, e);
160 }
161 if (ctx.shouldComplete()) {
162 break;
163 }
164 }
165 }
166 }
167
168 boolean preModifyNamespace(NamespaceDescriptor ns)
169 throws IOException {
170 boolean bypass = false;
171 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
172 for (MasterEnvironment env: coprocessors) {
173 if (env.getInstance() instanceof MasterObserver) {
174 ctx = ObserverContext.createAndPrepare(env, ctx);
175 try {
176 ((MasterObserver)env.getInstance()).preModifyNamespace(
177 ctx, ns);
178 } catch (Throwable e) {
179 handleCoprocessorThrowable(env, e);
180 }
181 bypass |= ctx.shouldBypass();
182 if (ctx.shouldComplete()) {
183 break;
184 }
185 }
186 }
187 return bypass;
188 }
189
190 void postModifyNamespace(NamespaceDescriptor ns)
191 throws IOException {
192 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
193 for (MasterEnvironment env: coprocessors) {
194 if (env.getInstance() instanceof MasterObserver) {
195 ctx = ObserverContext.createAndPrepare(env, ctx);
196 try {
197 ((MasterObserver)env.getInstance()).postModifyNamespace(ctx, ns);
198 } catch (Throwable e) {
199 handleCoprocessorThrowable(env, e);
200 }
201 if (ctx.shouldComplete()) {
202 break;
203 }
204 }
205 }
206 }
207
208
209 public void preCreateTable(HTableDescriptor htd, HRegionInfo[] regions)
210 throws IOException {
211 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
212 for (MasterEnvironment env: coprocessors) {
213 if (env.getInstance() instanceof MasterObserver) {
214 ctx = ObserverContext.createAndPrepare(env, ctx);
215 try {
216 ((MasterObserver)env.getInstance()).preCreateTable(ctx, htd, regions);
217 } catch (Throwable e) {
218 handleCoprocessorThrowable(env, e);
219 }
220 if (ctx.shouldComplete()) {
221 break;
222 }
223 }
224 }
225 }
226
227 public void postCreateTable(HTableDescriptor htd, HRegionInfo[] regions)
228 throws IOException {
229 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
230 for (MasterEnvironment env: coprocessors) {
231 if (env.getInstance() instanceof MasterObserver) {
232 ctx = ObserverContext.createAndPrepare(env, ctx);
233 try {
234 ((MasterObserver)env.getInstance()).postCreateTable(ctx, htd, regions);
235 } catch (Throwable e) {
236 handleCoprocessorThrowable(env, e);
237 }
238 if (ctx.shouldComplete()) {
239 break;
240 }
241 }
242 }
243 }
244
245 public void preCreateTableHandler(HTableDescriptor htd, HRegionInfo[] regions)
246 throws IOException {
247 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
248 for (MasterEnvironment env : coprocessors) {
249 if (env.getInstance() instanceof MasterObserver) {
250 ctx = ObserverContext.createAndPrepare(env, ctx);
251 try {
252 ((MasterObserver) env.getInstance()).preCreateTableHandler(ctx, htd,
253 regions);
254 } catch (Throwable e) {
255 handleCoprocessorThrowable(env, e);
256 }
257 if (ctx.shouldComplete()) {
258 break;
259 }
260 }
261 }
262 }
263
264 public void postCreateTableHandler(HTableDescriptor htd, HRegionInfo[] regions)
265 throws IOException {
266 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
267 for (MasterEnvironment env : coprocessors) {
268 if (env.getInstance() instanceof MasterObserver) {
269 ctx = ObserverContext.createAndPrepare(env, ctx);
270 try {
271 ((MasterObserver) env.getInstance()).postCreateTableHandler(ctx, htd,
272 regions);
273 } catch (Throwable e) {
274 handleCoprocessorThrowable(env, e);
275 }
276 if (ctx.shouldComplete()) {
277 break;
278 }
279 }
280 }
281 }
282
283 public void preDeleteTable(TableName tableName) throws IOException {
284 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
285 for (MasterEnvironment env: coprocessors) {
286 if (env.getInstance() instanceof MasterObserver) {
287 ctx = ObserverContext.createAndPrepare(env, ctx);
288 try {
289 ((MasterObserver)env.getInstance()).preDeleteTable(ctx, tableName);
290 } catch (Throwable e) {
291 handleCoprocessorThrowable(env, e);
292 }
293 if (ctx.shouldComplete()) {
294 break;
295 }
296 }
297 }
298 }
299
300 public void postDeleteTable(TableName tableName) throws IOException {
301 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
302 for (MasterEnvironment env: coprocessors) {
303 if (env.getInstance() instanceof MasterObserver) {
304 ctx = ObserverContext.createAndPrepare(env, ctx);
305 try {
306 ((MasterObserver)env.getInstance()).postDeleteTable(ctx, tableName);
307 } catch (Throwable e) {
308 handleCoprocessorThrowable(env, e);
309 }
310 if (ctx.shouldComplete()) {
311 break;
312 }
313 }
314 }
315 }
316
317 public void preDeleteTableHandler(TableName tableName) throws IOException {
318 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
319 for (MasterEnvironment env : coprocessors) {
320 if (env.getInstance() instanceof MasterObserver) {
321 ctx = ObserverContext.createAndPrepare(env, ctx);
322 try {
323 ((MasterObserver) env.getInstance()).preDeleteTableHandler(ctx,
324 tableName);
325 } catch (Throwable e) {
326 handleCoprocessorThrowable(env, e);
327 }
328 if (ctx.shouldComplete()) {
329 break;
330 }
331 }
332 }
333 }
334
335 public void postDeleteTableHandler(TableName tableName) throws IOException {
336 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
337 for (MasterEnvironment env : coprocessors) {
338 if (env.getInstance() instanceof MasterObserver) {
339 ctx = ObserverContext.createAndPrepare(env, ctx);
340 try {
341 ((MasterObserver) env.getInstance()).postDeleteTableHandler(ctx,
342 tableName);
343 } catch (Throwable e) {
344 handleCoprocessorThrowable(env, e);
345 }
346 if (ctx.shouldComplete()) {
347 break;
348 }
349 }
350 }
351 }
352 public void preModifyTable(final TableName tableName, HTableDescriptor htd)
353 throws IOException {
354 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
355 for (MasterEnvironment env: coprocessors) {
356 if (env.getInstance() instanceof MasterObserver) {
357 ctx = ObserverContext.createAndPrepare(env, ctx);
358 try {
359 ((MasterObserver)env.getInstance()).preModifyTable(ctx, tableName,
360 htd);
361 } catch (Throwable e) {
362 handleCoprocessorThrowable(env, e);
363 }
364 if (ctx.shouldComplete()) {
365 break;
366 }
367 }
368 }
369 }
370
371 public void postModifyTable(final TableName tableName, HTableDescriptor htd)
372 throws IOException {
373 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
374 for (MasterEnvironment env: coprocessors) {
375 if (env.getInstance() instanceof MasterObserver) {
376 ctx = ObserverContext.createAndPrepare(env, ctx);
377 try {
378 ((MasterObserver)env.getInstance()).postModifyTable(ctx, tableName,
379 htd);
380 } catch (Throwable e) {
381 handleCoprocessorThrowable(env, e);
382 }
383 if (ctx.shouldComplete()) {
384 break;
385 }
386 }
387 }
388 }
389
390 public void preModifyTableHandler(final TableName tableName, HTableDescriptor htd)
391 throws IOException {
392 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
393 for (MasterEnvironment env : coprocessors) {
394 if (env.getInstance() instanceof MasterObserver) {
395 ctx = ObserverContext.createAndPrepare(env, ctx);
396 try {
397 ((MasterObserver) env.getInstance()).preModifyTableHandler(ctx,
398 tableName, htd);
399 } catch (Throwable e) {
400 handleCoprocessorThrowable(env, e);
401 }
402 if (ctx.shouldComplete()) {
403 break;
404 }
405 }
406 }
407 }
408
409 public void postModifyTableHandler(final TableName tableName,
410 HTableDescriptor htd) throws IOException {
411 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
412 for (MasterEnvironment env : coprocessors) {
413 if (env.getInstance() instanceof MasterObserver) {
414 ctx = ObserverContext.createAndPrepare(env, ctx);
415 try {
416 ((MasterObserver) env.getInstance()).postModifyTableHandler(ctx,
417 tableName, htd);
418 } catch (Throwable e) {
419 handleCoprocessorThrowable(env, e);
420 }
421 if (ctx.shouldComplete()) {
422 break;
423 }
424 }
425 }
426 }
427
428 public boolean preAddColumn(TableName tableName, HColumnDescriptor column)
429 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()).preAddColumn(ctx, tableName, column);
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 public void postAddColumn(TableName tableName, HColumnDescriptor column)
450 throws IOException {
451 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
452 for (MasterEnvironment env: coprocessors) {
453 if (env.getInstance() instanceof MasterObserver) {
454 ctx = ObserverContext.createAndPrepare(env, ctx);
455 try {
456 ((MasterObserver)env.getInstance()).postAddColumn(ctx, tableName,
457 column);
458 } catch (Throwable e) {
459 handleCoprocessorThrowable(env, e);
460 }
461 if (ctx.shouldComplete()) {
462 break;
463 }
464 }
465 }
466 }
467
468 public boolean preAddColumnHandler(TableName tableName, HColumnDescriptor column)
469 throws IOException {
470 boolean bypass = false;
471 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
472 for (MasterEnvironment env : coprocessors) {
473 if (env.getInstance() instanceof MasterObserver) {
474 ctx = ObserverContext.createAndPrepare(env, ctx);
475 try {
476 ((MasterObserver) env.getInstance()).preAddColumnHandler(ctx,
477 tableName, column);
478 } catch (Throwable e) {
479 handleCoprocessorThrowable(env, e);
480 }
481 bypass |= ctx.shouldBypass();
482 if (ctx.shouldComplete()) {
483 break;
484 }
485 }
486 }
487 return bypass;
488 }
489
490 public void postAddColumnHandler(TableName tableName, HColumnDescriptor column)
491 throws IOException {
492 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
493 for (MasterEnvironment env : coprocessors) {
494 if (env.getInstance() instanceof MasterObserver) {
495 ctx = ObserverContext.createAndPrepare(env, ctx);
496 try {
497 ((MasterObserver) env.getInstance()).postAddColumnHandler(ctx,
498 tableName, column);
499 } catch (Throwable e) {
500 handleCoprocessorThrowable(env, e);
501 }
502 if (ctx.shouldComplete()) {
503 break;
504 }
505 }
506 }
507 }
508
509 public boolean preModifyColumn(TableName tableName, HColumnDescriptor descriptor)
510 throws IOException {
511 boolean bypass = false;
512 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
513 for (MasterEnvironment env: coprocessors) {
514 if (env.getInstance() instanceof MasterObserver) {
515 ctx = ObserverContext.createAndPrepare(env, ctx);
516 try {
517 ((MasterObserver)env.getInstance()).preModifyColumn(
518 ctx, tableName, descriptor);
519 } catch (Throwable e) {
520 handleCoprocessorThrowable(env, e);
521 }
522 bypass |= ctx.shouldBypass();
523 if (ctx.shouldComplete()) {
524 break;
525 }
526 }
527 }
528 return bypass;
529 }
530
531 public void postModifyColumn(TableName tableName, HColumnDescriptor descriptor)
532 throws IOException {
533 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
534 for (MasterEnvironment env: coprocessors) {
535 if (env.getInstance() instanceof MasterObserver) {
536 ctx = ObserverContext.createAndPrepare(env, ctx);
537 try {
538 ((MasterObserver)env.getInstance()).postModifyColumn(
539 ctx, tableName, descriptor);
540 } catch (Throwable e) {
541 handleCoprocessorThrowable(env, e);
542 }
543 if (ctx.shouldComplete()) {
544 break;
545 }
546 }
547 }
548 }
549
550 public boolean preModifyColumnHandler(TableName tableName,
551 HColumnDescriptor descriptor) throws IOException {
552 boolean bypass = false;
553 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
554 for (MasterEnvironment env : coprocessors) {
555 if (env.getInstance() instanceof MasterObserver) {
556 ctx = ObserverContext.createAndPrepare(env, ctx);
557 try {
558 ((MasterObserver) env.getInstance()).preModifyColumnHandler(ctx,
559 tableName, descriptor);
560 } catch (Throwable e) {
561 handleCoprocessorThrowable(env, e);
562 }
563 bypass |= ctx.shouldBypass();
564 if (ctx.shouldComplete()) {
565 break;
566 }
567 }
568 }
569 return bypass;
570 }
571
572 public void postModifyColumnHandler(TableName tableName,
573 HColumnDescriptor descriptor) throws IOException {
574 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
575 for (MasterEnvironment env : coprocessors) {
576 if (env.getInstance() instanceof MasterObserver) {
577 ctx = ObserverContext.createAndPrepare(env, ctx);
578 try {
579 ((MasterObserver) env.getInstance()).postModifyColumnHandler(ctx,
580 tableName, descriptor);
581 } catch (Throwable e) {
582 handleCoprocessorThrowable(env, e);
583 }
584 if (ctx.shouldComplete()) {
585 break;
586 }
587 }
588 }
589 }
590
591 boolean preDeleteColumn(final TableName tableName, final byte [] c)
592 throws IOException {
593 boolean bypass = false;
594 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
595 for (MasterEnvironment env: coprocessors) {
596 if (env.getInstance() instanceof MasterObserver) {
597 ctx = ObserverContext.createAndPrepare(env, ctx);
598 try {
599 ((MasterObserver)env.getInstance()).preDeleteColumn(ctx, tableName, c);
600 } catch (Throwable e) {
601 handleCoprocessorThrowable(env, e);
602 }
603 bypass |= ctx.shouldBypass();
604 if (ctx.shouldComplete()) {
605 break;
606 }
607 }
608 }
609 return bypass;
610 }
611
612 public void postDeleteColumn(final TableName tableName, final byte [] c)
613 throws IOException {
614 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
615 for (MasterEnvironment env: coprocessors) {
616 if (env.getInstance() instanceof MasterObserver) {
617 ctx = ObserverContext.createAndPrepare(env, ctx);
618 try {
619 ((MasterObserver)env.getInstance()).postDeleteColumn(ctx, tableName,
620 c);
621 } catch (Throwable e) {
622 handleCoprocessorThrowable(env, e);
623 }
624 if (ctx.shouldComplete()) {
625 break;
626 }
627 }
628 }
629 }
630
631 public boolean preDeleteColumnHandler(final TableName tableName, final byte[] c)
632 throws IOException {
633 boolean bypass = false;
634 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
635 for (MasterEnvironment env : coprocessors) {
636 if (env.getInstance() instanceof MasterObserver) {
637 ctx = ObserverContext.createAndPrepare(env, ctx);
638 try {
639 ((MasterObserver) env.getInstance()).preDeleteColumnHandler(ctx,
640 tableName, c);
641 } catch (Throwable e) {
642 handleCoprocessorThrowable(env, e);
643 }
644 bypass |= ctx.shouldBypass();
645 if (ctx.shouldComplete()) {
646 break;
647 }
648 }
649 }
650 return bypass;
651 }
652
653 public void postDeleteColumnHandler(final TableName tableName, final byte[] c)
654 throws IOException {
655 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
656 for (MasterEnvironment env : coprocessors) {
657 if (env.getInstance() instanceof MasterObserver) {
658 ctx = ObserverContext.createAndPrepare(env, ctx);
659 try {
660 ((MasterObserver) env.getInstance()).postDeleteColumnHandler(ctx,
661 tableName, c);
662 } catch (Throwable e) {
663 handleCoprocessorThrowable(env, e);
664 }
665 if (ctx.shouldComplete()) {
666 break;
667 }
668 }
669 }
670 }
671
672 public void preEnableTable(final TableName tableName) throws IOException {
673 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
674 for (MasterEnvironment env: coprocessors) {
675 if (env.getInstance() instanceof MasterObserver) {
676 ctx = ObserverContext.createAndPrepare(env, ctx);
677 try {
678 ((MasterObserver)env.getInstance()).preEnableTable(ctx, tableName);
679 } catch (Throwable e) {
680 handleCoprocessorThrowable(env, e);
681 }
682 if (ctx.shouldComplete()) {
683 break;
684 }
685 }
686 }
687 }
688
689 public void postEnableTable(final TableName tableName) 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()).postEnableTable(ctx, tableName);
696 } catch (Throwable e) {
697 handleCoprocessorThrowable(env, e);
698 }
699 if (ctx.shouldComplete()) {
700 break;
701 }
702 }
703 }
704 }
705
706 public void preEnableTableHandler(final TableName tableName) throws IOException {
707 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
708 for (MasterEnvironment env : coprocessors) {
709 if (env.getInstance() instanceof MasterObserver) {
710 ctx = ObserverContext.createAndPrepare(env, ctx);
711 try {
712 ((MasterObserver) env.getInstance()).preEnableTableHandler(ctx,
713 tableName);
714 } catch (Throwable e) {
715 handleCoprocessorThrowable(env, e);
716 }
717 if (ctx.shouldComplete()) {
718 break;
719 }
720 }
721 }
722 }
723
724 public void postEnableTableHandler(final TableName tableName) throws IOException {
725 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
726 for (MasterEnvironment env : coprocessors) {
727 if (env.getInstance() instanceof MasterObserver) {
728 ctx = ObserverContext.createAndPrepare(env, ctx);
729 try {
730 ((MasterObserver) env.getInstance()).postEnableTableHandler(ctx,
731 tableName);
732 } catch (Throwable e) {
733 handleCoprocessorThrowable(env, e);
734 }
735 if (ctx.shouldComplete()) {
736 break;
737 }
738 }
739 }
740 }
741
742 public void preDisableTable(final TableName tableName) 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()).preDisableTable(ctx, tableName);
749 } catch (Throwable e) {
750 handleCoprocessorThrowable(env, e);
751 }
752 if (ctx.shouldComplete()) {
753 break;
754 }
755 }
756 }
757 }
758
759 public void postDisableTable(final TableName tableName) 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()).postDisableTable(ctx, tableName);
766 } catch (Throwable e) {
767 handleCoprocessorThrowable(env, e);
768 }
769 if (ctx.shouldComplete()) {
770 break;
771 }
772 }
773 }
774 }
775
776 public void preDisableTableHandler(final TableName tableName) throws IOException {
777 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
778 for (MasterEnvironment env : coprocessors) {
779 if (env.getInstance() instanceof MasterObserver) {
780 ctx = ObserverContext.createAndPrepare(env, ctx);
781 try {
782 ((MasterObserver) env.getInstance()).preDisableTableHandler(ctx,
783 tableName);
784 } catch (Throwable e) {
785 handleCoprocessorThrowable(env, e);
786 }
787 if (ctx.shouldComplete()) {
788 break;
789 }
790 }
791 }
792 }
793
794 public void postDisableTableHandler(final TableName tableName)
795 throws IOException {
796 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
797 for (MasterEnvironment env : coprocessors) {
798 if (env.getInstance() instanceof MasterObserver) {
799 ctx = ObserverContext.createAndPrepare(env, ctx);
800 try {
801 ((MasterObserver) env.getInstance()).postDisableTableHandler(ctx,
802 tableName);
803 } catch (Throwable e) {
804 handleCoprocessorThrowable(env, e);
805 }
806 if (ctx.shouldComplete()) {
807 break;
808 }
809 }
810 }
811 }
812
813 public boolean preMove(final HRegionInfo region, final ServerName srcServer,
814 final ServerName destServer) throws IOException {
815 boolean bypass = false;
816 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
817 for (MasterEnvironment env: coprocessors) {
818 if (env.getInstance() instanceof MasterObserver) {
819 ctx = ObserverContext.createAndPrepare(env, ctx);
820 try {
821 ((MasterObserver)env.getInstance()).preMove(
822 ctx, region, srcServer, destServer);
823 } catch (Throwable e) {
824 handleCoprocessorThrowable(env, e);
825 }
826 bypass |= ctx.shouldBypass();
827 if (ctx.shouldComplete()) {
828 break;
829 }
830 }
831 }
832 return bypass;
833 }
834
835 public void postMove(final HRegionInfo region, final ServerName srcServer,
836 final ServerName destServer) throws IOException {
837 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
838 for (MasterEnvironment env: coprocessors) {
839 if (env.getInstance() instanceof MasterObserver) {
840 ctx = ObserverContext.createAndPrepare(env, ctx);
841 try {
842 ((MasterObserver)env.getInstance()).postMove(
843 ctx, region, srcServer, destServer);
844 } catch (Throwable e) {
845 handleCoprocessorThrowable(env, e);
846 }
847 if (ctx.shouldComplete()) {
848 break;
849 }
850 }
851 }
852 }
853
854 boolean preAssign(final HRegionInfo regionInfo) throws IOException {
855 boolean bypass = false;
856 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
857 for (MasterEnvironment env: coprocessors) {
858 if (env.getInstance() instanceof MasterObserver) {
859 ctx = ObserverContext.createAndPrepare(env, ctx);
860 try {
861 ((MasterObserver) env.getInstance()).preAssign(ctx, regionInfo);
862 } catch (Throwable e) {
863 handleCoprocessorThrowable(env, e);
864 }
865 bypass |= ctx.shouldBypass();
866 if (ctx.shouldComplete()) {
867 break;
868 }
869 }
870 }
871 return bypass;
872 }
873
874 void postAssign(final HRegionInfo regionInfo) throws IOException {
875 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
876 for (MasterEnvironment env: coprocessors) {
877 if (env.getInstance() instanceof MasterObserver) {
878 ctx = ObserverContext.createAndPrepare(env, ctx);
879 try {
880 ((MasterObserver)env.getInstance()).postAssign(ctx, regionInfo);
881 } catch (Throwable e) {
882 handleCoprocessorThrowable(env, e);
883 }
884 if (ctx.shouldComplete()) {
885 break;
886 }
887 }
888 }
889 }
890
891 boolean preUnassign(final HRegionInfo regionInfo, final boolean force)
892 throws IOException {
893 boolean bypass = false;
894 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
895 for (MasterEnvironment env: coprocessors) {
896 if (env.getInstance() instanceof MasterObserver) {
897 ctx = ObserverContext.createAndPrepare(env, ctx);
898 try {
899 ((MasterObserver)env.getInstance()).preUnassign(
900 ctx, regionInfo, force);
901 } catch (Throwable e) {
902 handleCoprocessorThrowable(env, e);
903 }
904 bypass |= ctx.shouldBypass();
905 if (ctx.shouldComplete()) {
906 break;
907 }
908 }
909 }
910 return bypass;
911 }
912
913 void postUnassign(final HRegionInfo regionInfo, final boolean force)
914 throws IOException {
915 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
916 for (MasterEnvironment env: coprocessors) {
917 if (env.getInstance() instanceof MasterObserver) {
918 ctx = ObserverContext.createAndPrepare(env, ctx);
919 try {
920 ((MasterObserver)env.getInstance()).postUnassign(
921 ctx, regionInfo, force);
922 } catch (Throwable e) {
923 handleCoprocessorThrowable(env, e);
924 }
925 if (ctx.shouldComplete()) {
926 break;
927 }
928 }
929 }
930 }
931
932 void preRegionOffline(final HRegionInfo regionInfo) throws IOException {
933 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
934 for (MasterEnvironment env : coprocessors) {
935 if (env.getInstance() instanceof MasterObserver) {
936 ctx = ObserverContext.createAndPrepare(env, ctx);
937 try {
938 ((MasterObserver) env.getInstance()).preRegionOffline(ctx, regionInfo);
939 } catch (Throwable e) {
940 handleCoprocessorThrowable(env, e);
941 }
942 if (ctx.shouldComplete()) {
943 break;
944 }
945 }
946 }
947 }
948
949 void postRegionOffline(final HRegionInfo regionInfo) throws IOException {
950 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
951 for (MasterEnvironment env : coprocessors) {
952 if (env.getInstance() instanceof MasterObserver) {
953 ctx = ObserverContext.createAndPrepare(env, ctx);
954 try {
955 ((MasterObserver) env.getInstance()).postRegionOffline(ctx, regionInfo);
956 } catch (Throwable e) {
957 handleCoprocessorThrowable(env, e);
958 }
959 if (ctx.shouldComplete()) {
960 break;
961 }
962 }
963 }
964 }
965
966 boolean preBalance() throws IOException {
967 boolean bypass = false;
968 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
969 for (MasterEnvironment env: coprocessors) {
970 if (env.getInstance() instanceof MasterObserver) {
971 ctx = ObserverContext.createAndPrepare(env, ctx);
972 try {
973 ((MasterObserver)env.getInstance()).preBalance(ctx);
974 } catch (Throwable e) {
975 handleCoprocessorThrowable(env, e);
976 }
977 bypass |= ctx.shouldBypass();
978 if (ctx.shouldComplete()) {
979 break;
980 }
981 }
982 }
983 return bypass;
984 }
985
986 void postBalance(List<RegionPlan> plans) throws IOException {
987 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
988 for (MasterEnvironment env: coprocessors) {
989 if (env.getInstance() instanceof MasterObserver) {
990 ctx = ObserverContext.createAndPrepare(env, ctx);
991 try {
992 ((MasterObserver)env.getInstance()).postBalance(ctx, plans);
993 } catch (Throwable e) {
994 handleCoprocessorThrowable(env, e);
995 }
996 if (ctx.shouldComplete()) {
997 break;
998 }
999 }
1000 }
1001 }
1002
1003 boolean preBalanceSwitch(final boolean b) throws IOException {
1004 boolean balance = b;
1005 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
1006 for (MasterEnvironment env: coprocessors) {
1007 if (env.getInstance() instanceof MasterObserver) {
1008 ctx = ObserverContext.createAndPrepare(env, ctx);
1009 try {
1010 balance = ((MasterObserver)env.getInstance()).preBalanceSwitch(
1011 ctx, balance);
1012 } catch (Throwable e) {
1013 handleCoprocessorThrowable(env, e);
1014 }
1015 if (ctx.shouldComplete()) {
1016 break;
1017 }
1018 }
1019 }
1020 return balance;
1021 }
1022
1023 void postBalanceSwitch(final boolean oldValue, final boolean newValue)
1024 throws IOException {
1025 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
1026 for (MasterEnvironment env: coprocessors) {
1027 if (env.getInstance() instanceof MasterObserver) {
1028 ctx = ObserverContext.createAndPrepare(env, ctx);
1029 try {
1030 ((MasterObserver)env.getInstance()).postBalanceSwitch(
1031 ctx, oldValue, newValue);
1032 } catch (Throwable e) {
1033 handleCoprocessorThrowable(env, e);
1034 }
1035 if (ctx.shouldComplete()) {
1036 break;
1037 }
1038 }
1039 }
1040 }
1041
1042 void preShutdown() throws IOException {
1043 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
1044 for (MasterEnvironment env: coprocessors) {
1045 if (env.getInstance() instanceof MasterObserver) {
1046 ctx = ObserverContext.createAndPrepare(env, ctx);
1047 try {
1048 ((MasterObserver)env.getInstance()).preShutdown(ctx);
1049 } catch (Throwable e) {
1050 handleCoprocessorThrowable(env, e);
1051 }
1052 if (ctx.shouldComplete()) {
1053 break;
1054 }
1055 }
1056 }
1057 }
1058
1059 void preStopMaster() throws IOException {
1060 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
1061 for (MasterEnvironment env: coprocessors) {
1062 if (env.getInstance() instanceof MasterObserver) {
1063 ctx = ObserverContext.createAndPrepare(env, ctx);
1064 try {
1065 ((MasterObserver)env.getInstance()).preStopMaster(ctx);
1066 } catch (Throwable e) {
1067 handleCoprocessorThrowable(env, e);
1068 }
1069 if (ctx.shouldComplete()) {
1070 break;
1071 }
1072 }
1073 }
1074 }
1075
1076 void postStartMaster() throws IOException {
1077 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
1078 for (MasterEnvironment env: coprocessors) {
1079 if (env.getInstance() instanceof MasterObserver) {
1080 ctx = ObserverContext.createAndPrepare(env, ctx);
1081 try {
1082 ((MasterObserver)env.getInstance()).postStartMaster(ctx);
1083 } catch (Throwable e) {
1084 handleCoprocessorThrowable(env, e);
1085 }
1086 if (ctx.shouldComplete()) {
1087 break;
1088 }
1089 }
1090 }
1091 }
1092
1093 public void preSnapshot(final SnapshotDescription snapshot,
1094 final HTableDescriptor hTableDescriptor) throws IOException {
1095 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
1096 for (MasterEnvironment env: coprocessors) {
1097 if (env.getInstance() instanceof MasterObserver) {
1098 ctx = ObserverContext.createAndPrepare(env, ctx);
1099 try {
1100 ((MasterObserver)env.getInstance()).preSnapshot(ctx, snapshot, hTableDescriptor);
1101 } catch (Throwable e) {
1102 handleCoprocessorThrowable(env, e);
1103 }
1104 if (ctx.shouldComplete()) {
1105 break;
1106 }
1107 }
1108 }
1109 }
1110
1111 public void postSnapshot(final SnapshotDescription snapshot,
1112 final HTableDescriptor hTableDescriptor) throws IOException {
1113 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
1114 for (MasterEnvironment env: coprocessors) {
1115 if (env.getInstance() instanceof MasterObserver) {
1116 ctx = ObserverContext.createAndPrepare(env, ctx);
1117 try {
1118 ((MasterObserver)env.getInstance()).postSnapshot(ctx, snapshot, hTableDescriptor);
1119 } catch (Throwable e) {
1120 handleCoprocessorThrowable(env, e);
1121 }
1122 if (ctx.shouldComplete()) {
1123 break;
1124 }
1125 }
1126 }
1127 }
1128
1129 public void preCloneSnapshot(final SnapshotDescription snapshot,
1130 final HTableDescriptor hTableDescriptor) throws IOException {
1131 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
1132 for (MasterEnvironment env: coprocessors) {
1133 if (env.getInstance() instanceof MasterObserver) {
1134 ctx = ObserverContext.createAndPrepare(env, ctx);
1135 try {
1136 ((MasterObserver)env.getInstance()).preCloneSnapshot(ctx, snapshot, hTableDescriptor);
1137 } catch (Throwable e) {
1138 handleCoprocessorThrowable(env, e);
1139 }
1140 if (ctx.shouldComplete()) {
1141 break;
1142 }
1143 }
1144 }
1145 }
1146
1147 public void postCloneSnapshot(final SnapshotDescription snapshot,
1148 final HTableDescriptor hTableDescriptor) throws IOException {
1149 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
1150 for (MasterEnvironment env: coprocessors) {
1151 if (env.getInstance() instanceof MasterObserver) {
1152 ctx = ObserverContext.createAndPrepare(env, ctx);
1153 try {
1154 ((MasterObserver)env.getInstance()).postCloneSnapshot(ctx, snapshot, hTableDescriptor);
1155 } catch (Throwable e) {
1156 handleCoprocessorThrowable(env, e);
1157 }
1158 if (ctx.shouldComplete()) {
1159 break;
1160 }
1161 }
1162 }
1163 }
1164
1165 public void preRestoreSnapshot(final SnapshotDescription snapshot,
1166 final HTableDescriptor hTableDescriptor) throws IOException {
1167 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
1168 for (MasterEnvironment env: coprocessors) {
1169 if (env.getInstance() instanceof MasterObserver) {
1170 ctx = ObserverContext.createAndPrepare(env, ctx);
1171 try {
1172 ((MasterObserver)env.getInstance()).preRestoreSnapshot(ctx, snapshot, hTableDescriptor);
1173 } catch (Throwable e) {
1174 handleCoprocessorThrowable(env, e);
1175 }
1176 if (ctx.shouldComplete()) {
1177 break;
1178 }
1179 }
1180 }
1181 }
1182
1183 public void postRestoreSnapshot(final SnapshotDescription snapshot,
1184 final HTableDescriptor hTableDescriptor) throws IOException {
1185 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
1186 for (MasterEnvironment env: coprocessors) {
1187 if (env.getInstance() instanceof MasterObserver) {
1188 ctx = ObserverContext.createAndPrepare(env, ctx);
1189 try {
1190 ((MasterObserver)env.getInstance()).postRestoreSnapshot(ctx, snapshot, hTableDescriptor);
1191 } catch (Throwable e) {
1192 handleCoprocessorThrowable(env, e);
1193 }
1194 if (ctx.shouldComplete()) {
1195 break;
1196 }
1197 }
1198 }
1199 }
1200
1201 public void preDeleteSnapshot(final SnapshotDescription snapshot) throws IOException {
1202 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
1203 for (MasterEnvironment env: coprocessors) {
1204 if (env.getInstance() instanceof MasterObserver) {
1205 ctx = ObserverContext.createAndPrepare(env, ctx);
1206 try {
1207 ((MasterObserver)env.getInstance()).preDeleteSnapshot(ctx, snapshot);
1208 } catch (Throwable e) {
1209 handleCoprocessorThrowable(env, e);
1210 }
1211 if (ctx.shouldComplete()) {
1212 break;
1213 }
1214 }
1215 }
1216 }
1217
1218 public void postDeleteSnapshot(final SnapshotDescription snapshot) throws IOException {
1219 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
1220 for (MasterEnvironment env: coprocessors) {
1221 if (env.getInstance() instanceof MasterObserver) {
1222 ctx = ObserverContext.createAndPrepare(env, ctx);
1223 try {
1224 ((MasterObserver)env.getInstance()).postDeleteSnapshot(ctx, snapshot);
1225 } catch (Throwable e) {
1226 handleCoprocessorThrowable(env, e);
1227 }
1228 if (ctx.shouldComplete()) {
1229 break;
1230 }
1231 }
1232 }
1233 }
1234
1235 public boolean preGetTableDescriptors(final List<TableName> tableNamesList,
1236 final List<HTableDescriptor> descriptors) throws IOException {
1237 boolean bypass = false;
1238 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
1239 for (MasterEnvironment env : coprocessors) {
1240 if (env.getInstance() instanceof MasterObserver) {
1241 ctx = ObserverContext.createAndPrepare(env, ctx);
1242 try {
1243 ((MasterObserver) env.getInstance()).preGetTableDescriptors(ctx,
1244 tableNamesList, descriptors);
1245 } catch (Throwable e) {
1246 handleCoprocessorThrowable(env, e);
1247 }
1248 bypass |= ctx.shouldBypass();
1249 if (ctx.shouldComplete()) {
1250 break;
1251 }
1252 }
1253 }
1254 return bypass;
1255 }
1256
1257 public void postGetTableDescriptors(List<HTableDescriptor> descriptors) throws IOException {
1258 ObserverContext<MasterCoprocessorEnvironment> ctx = null;
1259 for (MasterEnvironment env: coprocessors) {
1260 if (env.getInstance() instanceof MasterObserver) {
1261 ctx = ObserverContext.createAndPrepare(env, ctx);
1262 try {
1263 ((MasterObserver)env.getInstance()).postGetTableDescriptors(ctx, descriptors);
1264 } catch (Throwable e) {
1265 handleCoprocessorThrowable(env, e);
1266 }
1267 if (ctx.shouldComplete()) {
1268 break;
1269 }
1270 }
1271 }
1272 }
1273
1274 }