View Javadoc

1   /*
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   * http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
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  import java.util.List;
33  
34  /**
35   * Provides the coprocessor framework and environment for master oriented
36   * operations.  {@link HMaster} interacts with the loaded coprocessors
37   * through this class.
38   */
39  public class MasterCoprocessorHost
40      extends CoprocessorHost<MasterCoprocessorHost.MasterEnvironment> {
41  
42    private static final Log LOG = LogFactory.getLog(MasterCoprocessorHost.class);
43  
44    /**
45     * Coprocessor environment extension providing access to master related
46     * services.
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 (CoprocessorProtocol.class.isAssignableFrom(c)) {
78          masterServices.registerProtocol(c, (CoprocessorProtocol)instance);
79          break;
80        }
81      }
82      return new MasterEnvironment(implClass, instance, priority, seq, conf,
83          masterServices);
84    }
85  
86    @Override
87    protected void abortServer(final CoprocessorEnvironment env, final Throwable e) {
88      abortServer("master", masterServices, env, e);
89    }
90  
91    /* Implementation of hooks for invoking MasterObservers */
92    void preCreateTable(HTableDescriptor htd, HRegionInfo[] regions)
93      throws IOException {
94      ObserverContext<MasterCoprocessorEnvironment> ctx = null;
95      for (MasterEnvironment env: coprocessors) {
96        if (env.getInstance() instanceof MasterObserver) {
97          ctx = ObserverContext.createAndPrepare(env, ctx);
98          try {
99            ((MasterObserver)env.getInstance()).preCreateTable(ctx, htd, regions);
100         } catch (Throwable e) {
101           handleCoprocessorThrowable(env, e);
102         }
103         if (ctx.shouldComplete()) {
104           break;
105         }
106       }
107     }
108   }
109 
110   void postCreateTable(HTableDescriptor htd, HRegionInfo[] regions)
111     throws IOException {
112     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
113     for (MasterEnvironment env: coprocessors) {
114       if (env.getInstance() instanceof MasterObserver) {
115         ctx = ObserverContext.createAndPrepare(env, ctx);
116         try {
117             ((MasterObserver)env.getInstance()).postCreateTable(ctx, htd, regions);
118         } catch (Throwable e) {
119           handleCoprocessorThrowable(env, e);
120         }
121         if (ctx.shouldComplete()) {
122           break;
123         }
124       }
125     }
126   }
127 
128   void preDeleteTable(byte[] tableName) 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()).preDeleteTable(ctx, tableName);
135         } catch (Throwable e) {
136           handleCoprocessorThrowable(env, e);
137         }
138         if (ctx.shouldComplete()) {
139           break;
140         }
141       }
142     }
143   }
144 
145   void postDeleteTable(byte[] tableName) throws IOException {
146     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
147     for (MasterEnvironment env: coprocessors) {
148       if (env.getInstance() instanceof MasterObserver) {
149         ctx = ObserverContext.createAndPrepare(env, ctx);
150         try {
151           ((MasterObserver)env.getInstance()).postDeleteTable(ctx, tableName);
152         } catch (Throwable e) {
153           handleCoprocessorThrowable(env, e);
154         }
155         if (ctx.shouldComplete()) {
156           break;
157         }
158       }
159     }
160   }
161 
162   void preModifyTable(final byte[] tableName, HTableDescriptor htd)
163       throws IOException {
164     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
165     for (MasterEnvironment env: coprocessors) {
166       if (env.getInstance() instanceof MasterObserver) {
167         ctx = ObserverContext.createAndPrepare(env, ctx);
168         try {
169           ((MasterObserver)env.getInstance()).preModifyTable(ctx, tableName,
170               htd);
171         } catch (Throwable e) {
172           handleCoprocessorThrowable(env, e);
173         }
174         if (ctx.shouldComplete()) {
175           break;
176         }
177       }
178     }
179   }
180 
181   void postModifyTable(final byte[] tableName, HTableDescriptor htd)
182       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()).postModifyTable(ctx, tableName,
189               htd);
190         } catch (Throwable e) {
191           handleCoprocessorThrowable(env, e);
192         }
193         if (ctx.shouldComplete()) {
194           break;
195         }
196       }
197     }
198   }
199 
200   boolean preAddColumn(byte [] tableName, HColumnDescriptor column)
201       throws IOException {
202     boolean bypass = false;
203     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
204     for (MasterEnvironment env: coprocessors) {
205       if (env.getInstance() instanceof MasterObserver) {
206         ctx = ObserverContext.createAndPrepare(env, ctx);
207         try {
208           ((MasterObserver)env.getInstance()).preAddColumn(ctx, tableName, column);
209         } catch (Throwable e) {
210           handleCoprocessorThrowable(env, e);
211         }
212         bypass |= ctx.shouldBypass();
213         if (ctx.shouldComplete()) {
214           break;
215         }
216       }
217     }
218     return bypass;
219   }
220 
221   void postAddColumn(byte [] tableName, HColumnDescriptor column)
222       throws IOException {
223     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
224     for (MasterEnvironment env: coprocessors) {
225       if (env.getInstance() instanceof MasterObserver) {
226         ctx = ObserverContext.createAndPrepare(env, ctx);
227         try {
228           ((MasterObserver)env.getInstance()).postAddColumn(ctx, tableName,
229               column);
230         } catch (Throwable e) {
231           handleCoprocessorThrowable(env, e);
232         }
233         if (ctx.shouldComplete()) {
234           break;
235         }
236       }
237     }
238   }
239 
240   boolean preModifyColumn(byte [] tableName, HColumnDescriptor descriptor)
241       throws IOException {
242     boolean bypass = false;
243     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
244     for (MasterEnvironment env: coprocessors) {
245       if (env.getInstance() instanceof MasterObserver) {
246         ctx = ObserverContext.createAndPrepare(env, ctx);
247         try {
248           ((MasterObserver)env.getInstance()).preModifyColumn(
249             ctx, tableName, descriptor);
250         } catch (Throwable e) {
251           handleCoprocessorThrowable(env, e);
252         }
253         bypass |= ctx.shouldBypass();
254         if (ctx.shouldComplete()) {
255           break;
256         }
257       }
258     }
259     return bypass;
260   }
261 
262   void postModifyColumn(byte [] tableName, HColumnDescriptor descriptor)
263       throws IOException {
264     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
265     for (MasterEnvironment env: coprocessors) {
266       if (env.getInstance() instanceof MasterObserver) {
267         ctx = ObserverContext.createAndPrepare(env, ctx);
268         try {
269           ((MasterObserver)env.getInstance()).postModifyColumn(
270               ctx, tableName, descriptor);
271         } catch (Throwable e) {
272           handleCoprocessorThrowable(env, e);
273         }
274         if (ctx.shouldComplete()) {
275           break;
276         }
277       }
278     }
279   }
280 
281   boolean preDeleteColumn(final byte [] tableName, final byte [] c)
282       throws IOException {
283     boolean bypass = false;
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()).preDeleteColumn(ctx, tableName, c);
290         } catch (Throwable e) {
291           handleCoprocessorThrowable(env, e);
292         }
293         bypass |= ctx.shouldBypass();
294         if (ctx.shouldComplete()) {
295           break;
296         }
297       }
298     }
299     return bypass;
300   }
301 
302   void postDeleteColumn(final byte [] tableName, final byte [] c)
303       throws IOException {
304     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
305     for (MasterEnvironment env: coprocessors) {
306       if (env.getInstance() instanceof MasterObserver) {
307         ctx = ObserverContext.createAndPrepare(env, ctx);
308         try {
309           ((MasterObserver)env.getInstance()).postDeleteColumn(ctx, tableName,
310               c);
311         } catch (Throwable e) {
312           handleCoprocessorThrowable(env, e);
313         }
314         if (ctx.shouldComplete()) {
315           break;
316         }
317       }
318     }
319   }
320 
321   void preEnableTable(final byte [] tableName) throws IOException {
322     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
323     for (MasterEnvironment env: coprocessors) {
324       if (env.getInstance() instanceof MasterObserver) {
325         ctx = ObserverContext.createAndPrepare(env, ctx);
326         try {
327           ((MasterObserver)env.getInstance()).preEnableTable(ctx, tableName);
328         } catch (Throwable e) {
329           handleCoprocessorThrowable(env, e);
330         }
331         if (ctx.shouldComplete()) {
332           break;
333         }
334       }
335     }
336   }
337 
338   void postEnableTable(final byte [] tableName) throws IOException {
339     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
340     for (MasterEnvironment env: coprocessors) {
341       if (env.getInstance() instanceof MasterObserver) {
342         ctx = ObserverContext.createAndPrepare(env, ctx);
343         try {
344           ((MasterObserver)env.getInstance()).postEnableTable(ctx, tableName);
345         } catch (Throwable e) {
346           handleCoprocessorThrowable(env, e);
347         }
348         if (ctx.shouldComplete()) {
349           break;
350         }
351       }
352     }
353   }
354 
355   void preDisableTable(final byte [] tableName) throws IOException {
356     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
357     for (MasterEnvironment env: coprocessors) {
358       if (env.getInstance() instanceof MasterObserver) {
359         ctx = ObserverContext.createAndPrepare(env, ctx);
360         try {
361           ((MasterObserver)env.getInstance()).preDisableTable(ctx, tableName);
362         } catch (Throwable e) {
363           handleCoprocessorThrowable(env, e);
364         }
365         if (ctx.shouldComplete()) {
366           break;
367         }
368       }
369     }
370   }
371 
372   void postDisableTable(final byte [] tableName) 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()).postDisableTable(ctx, tableName);
379         } catch (Throwable e) {
380           handleCoprocessorThrowable(env, e);
381         }
382         if (ctx.shouldComplete()) {
383           break;
384         }
385       }
386     }
387   }
388 
389   boolean preMove(final HRegionInfo region, final ServerName srcServer, final ServerName destServer)
390       throws IOException {
391     boolean bypass = false;
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()).preMove(
398               ctx, region, srcServer, destServer);
399         } catch (Throwable e) {
400           handleCoprocessorThrowable(env, e);
401         }
402         bypass |= ctx.shouldBypass();
403         if (ctx.shouldComplete()) {
404           break;
405         }
406       }
407     }
408     return bypass;
409   }
410 
411   void postMove(final HRegionInfo region, final ServerName srcServer, final ServerName destServer)
412       throws IOException {
413     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
414     for (MasterEnvironment env: coprocessors) {
415       if (env.getInstance() instanceof MasterObserver) {
416         ctx = ObserverContext.createAndPrepare(env, ctx);
417         try {
418           ((MasterObserver)env.getInstance()).postMove(
419               ctx, region, srcServer, destServer);
420         } catch (Throwable e) {
421           handleCoprocessorThrowable(env, e);
422         }
423         if (ctx.shouldComplete()) {
424           break;
425         }
426       }
427     }
428   }
429 
430   boolean preAssign(final HRegionInfo regionInfo) throws IOException {
431     boolean bypass = false;
432     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
433     for (MasterEnvironment env: coprocessors) {
434       if (env.getInstance() instanceof MasterObserver) {
435         ctx = ObserverContext.createAndPrepare(env, ctx);
436         try {
437           ((MasterObserver) env.getInstance()).preAssign(ctx, regionInfo);
438         } catch (Throwable e) {
439           handleCoprocessorThrowable(env, e);
440         }
441         bypass |= ctx.shouldBypass();
442         if (ctx.shouldComplete()) {
443           break;
444         }
445       }
446     }
447     return bypass;
448   }
449 
450   void postAssign(final HRegionInfo regionInfo) 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()).postAssign(ctx, regionInfo);
457         } catch (Throwable e) {
458           handleCoprocessorThrowable(env, e);
459         }
460         if (ctx.shouldComplete()) {
461           break;
462         }
463       }
464     }
465   }
466 
467   boolean preUnassign(final HRegionInfo regionInfo, final boolean force)
468       throws IOException {
469     boolean bypass = false;
470     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
471     for (MasterEnvironment env: coprocessors) {
472       if (env.getInstance() instanceof MasterObserver) {
473         ctx = ObserverContext.createAndPrepare(env, ctx);
474         try {
475           ((MasterObserver)env.getInstance()).preUnassign(
476               ctx, regionInfo, force);
477         } catch (Throwable e) {
478           handleCoprocessorThrowable(env, e);
479         }
480         bypass |= ctx.shouldBypass();
481         if (ctx.shouldComplete()) {
482           break;
483         }
484       }
485     }
486     return bypass;
487   }
488 
489   void postUnassign(final HRegionInfo regionInfo, final boolean force)
490       throws IOException {
491     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
492     for (MasterEnvironment env: coprocessors) {
493       if (env.getInstance() instanceof MasterObserver) {
494         ctx = ObserverContext.createAndPrepare(env, ctx);
495         try {
496           ((MasterObserver)env.getInstance()).postUnassign(
497               ctx, regionInfo, force);
498         } catch (Throwable e) {
499           handleCoprocessorThrowable(env, e);
500         }
501         if (ctx.shouldComplete()) {
502           break;
503         }
504       }
505     }
506   }
507 
508   boolean preBalance() throws IOException {
509     boolean bypass = false;
510     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
511     for (MasterEnvironment env: coprocessors) {
512       if (env.getInstance() instanceof MasterObserver) {
513         ctx = ObserverContext.createAndPrepare(env, ctx);
514         try {
515           ((MasterObserver)env.getInstance()).preBalance(ctx);
516         } catch (Throwable e) {
517           handleCoprocessorThrowable(env, e);
518         }
519         bypass |= ctx.shouldBypass();
520         if (ctx.shouldComplete()) {
521           break;
522         }
523       }
524     }
525     return bypass;
526   }
527 
528   void postBalance() throws IOException {
529     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
530     for (MasterEnvironment env: coprocessors) {
531       if (env.getInstance() instanceof MasterObserver) {
532         ctx = ObserverContext.createAndPrepare(env, ctx);
533         try {
534           ((MasterObserver)env.getInstance()).postBalance(ctx);
535         } catch (Throwable e) {
536           handleCoprocessorThrowable(env, e);
537         }
538         if (ctx.shouldComplete()) {
539           break;
540         }
541       }
542     }
543   }
544 
545   boolean preBalanceSwitch(final boolean b) throws IOException {
546     boolean balance = b;
547     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
548     for (MasterEnvironment env: coprocessors) {
549       if (env.getInstance() instanceof MasterObserver) {
550         ctx = ObserverContext.createAndPrepare(env, ctx);
551         try {
552           balance = ((MasterObserver)env.getInstance()).preBalanceSwitch(
553               ctx, balance);
554         } catch (Throwable e) {
555           handleCoprocessorThrowable(env, e);
556         }
557         if (ctx.shouldComplete()) {
558           break;
559         }
560       }
561     }
562     return balance;
563   }
564 
565   void postBalanceSwitch(final boolean oldValue, final boolean newValue)
566       throws IOException {
567     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
568     for (MasterEnvironment env: coprocessors) {
569       if (env.getInstance() instanceof MasterObserver) {
570         ctx = ObserverContext.createAndPrepare(env, ctx);
571         try {
572           ((MasterObserver)env.getInstance()).postBalanceSwitch(
573               ctx, oldValue, newValue);
574         } catch (Throwable e) {
575           handleCoprocessorThrowable(env, e);
576         }
577         if (ctx.shouldComplete()) {
578           break;
579         }
580       }
581     }
582   }
583 
584   void preShutdown() throws IOException {
585     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
586     for (MasterEnvironment env: coprocessors) {
587       if (env.getInstance() instanceof MasterObserver) {
588         ctx = ObserverContext.createAndPrepare(env, ctx);
589         try {
590           ((MasterObserver)env.getInstance()).preShutdown(ctx);
591         } catch (Throwable e) {
592           handleCoprocessorThrowable(env, e);
593         }
594         if (ctx.shouldComplete()) {
595           break;
596         }
597       }
598     }
599   }
600 
601   void preStopMaster() throws IOException {
602     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
603     for (MasterEnvironment env: coprocessors) {
604       if (env.getInstance() instanceof MasterObserver) {
605         ctx = ObserverContext.createAndPrepare(env, ctx);
606         try {
607           ((MasterObserver)env.getInstance()).preStopMaster(ctx);
608         } catch (Throwable e) {
609           handleCoprocessorThrowable(env, e);
610         }
611         if (ctx.shouldComplete()) {
612           break;
613         }
614       }
615     }
616   }
617 
618   void postStartMaster() throws IOException {
619     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
620     for (MasterEnvironment env: coprocessors) {
621       if (env.getInstance() instanceof MasterObserver) {
622         ctx = ObserverContext.createAndPrepare(env, ctx);
623         try {
624           ((MasterObserver)env.getInstance()).postStartMaster(ctx);
625         } catch (Throwable e) {
626           handleCoprocessorThrowable(env, e);
627         }
628         if (ctx.shouldComplete()) {
629           break;
630         }
631       }
632     }
633   }
634 
635   public void preSnapshot(final SnapshotDescription snapshot,
636       final HTableDescriptor hTableDescriptor) throws IOException {
637     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
638     for (MasterEnvironment env: coprocessors) {
639       if (env.getInstance() instanceof MasterObserver) {
640         ctx = ObserverContext.createAndPrepare(env, ctx);
641         try {
642           ((MasterObserver)env.getInstance()).preSnapshot(ctx, snapshot, hTableDescriptor);
643         } catch (Throwable e) {
644           handleCoprocessorThrowable(env, e);
645         }
646         if (ctx.shouldComplete()) {
647           break;
648         }
649       }
650     }
651   }
652 
653   public void postSnapshot(final SnapshotDescription snapshot,
654       final HTableDescriptor hTableDescriptor) 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()).postSnapshot(ctx, snapshot, hTableDescriptor);
661         } catch (Throwable e) {
662           handleCoprocessorThrowable(env, e);
663         }
664         if (ctx.shouldComplete()) {
665           break;
666         }
667       }
668     }
669   }
670 
671   public void preCloneSnapshot(final SnapshotDescription snapshot,
672       final HTableDescriptor hTableDescriptor) 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()).preCloneSnapshot(ctx, snapshot, hTableDescriptor);
679         } catch (Throwable e) {
680           handleCoprocessorThrowable(env, e);
681         }
682         if (ctx.shouldComplete()) {
683           break;
684         }
685       }
686     }
687   }
688 
689   public void postCloneSnapshot(final SnapshotDescription snapshot,
690       final HTableDescriptor hTableDescriptor) throws IOException {
691     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
692     for (MasterEnvironment env: coprocessors) {
693       if (env.getInstance() instanceof MasterObserver) {
694         ctx = ObserverContext.createAndPrepare(env, ctx);
695         try {
696           ((MasterObserver)env.getInstance()).postCloneSnapshot(ctx, snapshot, hTableDescriptor);
697         } catch (Throwable e) {
698           handleCoprocessorThrowable(env, e);
699         }
700         if (ctx.shouldComplete()) {
701           break;
702         }
703       }
704     }
705   }
706 
707   public void preRestoreSnapshot(final SnapshotDescription snapshot,
708       final HTableDescriptor hTableDescriptor) throws IOException {
709     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
710     for (MasterEnvironment env: coprocessors) {
711       if (env.getInstance() instanceof MasterObserver) {
712         ctx = ObserverContext.createAndPrepare(env, ctx);
713         try {
714           ((MasterObserver)env.getInstance()).preRestoreSnapshot(ctx, snapshot, hTableDescriptor);
715         } catch (Throwable e) {
716           handleCoprocessorThrowable(env, e);
717         }
718         if (ctx.shouldComplete()) {
719           break;
720         }
721       }
722     }
723   }
724 
725   public void postRestoreSnapshot(final SnapshotDescription snapshot,
726       final HTableDescriptor hTableDescriptor) throws IOException {
727     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
728     for (MasterEnvironment env: coprocessors) {
729       if (env.getInstance() instanceof MasterObserver) {
730         ctx = ObserverContext.createAndPrepare(env, ctx);
731         try {
732           ((MasterObserver)env.getInstance()).postRestoreSnapshot(ctx, snapshot, hTableDescriptor);
733         } catch (Throwable e) {
734           handleCoprocessorThrowable(env, e);
735         }
736         if (ctx.shouldComplete()) {
737           break;
738         }
739       }
740     }
741   }
742 
743   public void preDeleteSnapshot(final SnapshotDescription snapshot) throws IOException {
744     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
745     for (MasterEnvironment env: coprocessors) {
746       if (env.getInstance() instanceof MasterObserver) {
747         ctx = ObserverContext.createAndPrepare(env, ctx);
748         try {
749           ((MasterObserver)env.getInstance()).preDeleteSnapshot(ctx, snapshot);
750         } catch (Throwable e) {
751           handleCoprocessorThrowable(env, e);
752         }
753         if (ctx.shouldComplete()) {
754           break;
755         }
756       }
757     }
758   }
759 
760   public void postDeleteSnapshot(final SnapshotDescription snapshot) throws IOException {
761     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
762     for (MasterEnvironment env: coprocessors) {
763       if (env.getInstance() instanceof MasterObserver) {
764         ctx = ObserverContext.createAndPrepare(env, ctx);
765         try {
766           ((MasterObserver)env.getInstance()).postDeleteSnapshot(ctx, snapshot);
767         } catch (Throwable e) {
768           handleCoprocessorThrowable(env, e);
769         }
770         if (ctx.shouldComplete()) {
771           break;
772         }
773       }
774     }
775   }
776 
777   public boolean preGetTableDescriptors(final List<String> tableNamesList,
778       final List<HTableDescriptor> descriptors) throws IOException {
779     boolean bypass = false;
780     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
781     for (MasterEnvironment env : coprocessors) {
782       if (env.getInstance() instanceof MasterObserver) {
783         ctx = ObserverContext.createAndPrepare(env, ctx);
784         try {
785           ((MasterObserver) env.getInstance()).preGetTableDescriptors(ctx,
786             tableNamesList, descriptors);
787         } catch (Throwable e) {
788           handleCoprocessorThrowable(env, e);
789         }
790         bypass |= ctx.shouldBypass();
791         if (ctx.shouldComplete()) {
792           break;
793         }
794       }
795     }
796     return bypass;
797   }
798 
799   public void postGetTableDescriptors(List<HTableDescriptor> descriptors) throws IOException {
800     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
801     for (MasterEnvironment env: coprocessors) {
802       if (env.getInstance() instanceof MasterObserver) {
803         ctx = ObserverContext.createAndPrepare(env, ctx);
804         try {
805           ((MasterObserver)env.getInstance()).postGetTableDescriptors(ctx, descriptors);
806         } catch (Throwable e) {
807           handleCoprocessorThrowable(env, e);
808         }
809         if (ctx.shouldComplete()) {
810           break;
811         }
812       }
813     }
814   }
815 
816 }