View Javadoc

1   /*
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   * http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  
20  package org.apache.hadoop.hbase.master;
21  
22  import java.io.IOException;
23  import java.util.List;
24  
25  import org.apache.commons.lang.ClassUtils;
26  import org.apache.commons.logging.Log;
27  import org.apache.commons.logging.LogFactory;
28  import org.apache.hadoop.conf.Configuration;
29  import org.apache.hadoop.hbase.Coprocessor;
30  import org.apache.hadoop.hbase.HColumnDescriptor;
31  import org.apache.hadoop.hbase.HRegionInfo;
32  import org.apache.hadoop.hbase.HTableDescriptor;
33  import org.apache.hadoop.hbase.NamespaceDescriptor;
34  import org.apache.hadoop.hbase.ProcedureInfo;
35  import org.apache.hadoop.hbase.ServerName;
36  import org.apache.hadoop.hbase.TableName;
37  import org.apache.hadoop.hbase.classification.InterfaceAudience;
38  import org.apache.hadoop.hbase.coprocessor.*;
39  import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
40  import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
41  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
42  import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas;
43  
44  /**
45   * Provides the coprocessor framework and environment for master oriented
46   * operations.  {@link HMaster} interacts with the loaded coprocessors
47   * through this class.
48   */
49  @InterfaceAudience.Private
50  public class MasterCoprocessorHost
51      extends CoprocessorHost<MasterCoprocessorHost.MasterEnvironment> {
52  
53    private static final Log LOG = LogFactory.getLog(MasterCoprocessorHost.class);
54  
55    /**
56     * Coprocessor environment extension providing access to master related
57     * services.
58     */
59    static class MasterEnvironment extends CoprocessorHost.Environment
60        implements MasterCoprocessorEnvironment {
61      private MasterServices masterServices;
62  
63      public MasterEnvironment(final Class<?> implClass, final Coprocessor impl,
64          final int priority, final int seq, final Configuration conf,
65          final MasterServices services) {
66        super(impl, priority, seq, conf);
67        this.masterServices = services;
68      }
69  
70      public MasterServices getMasterServices() {
71        return masterServices;
72      }
73    }
74  
75    private MasterServices masterServices;
76  
77    public MasterCoprocessorHost(final MasterServices services, final Configuration conf) {
78      super(services);
79      this.conf = conf;
80      this.masterServices = services;
81      // Log the state of coprocessor loading here; should appear only once or
82      // twice in the daemon log, depending on HBase version, because there is
83      // only one MasterCoprocessorHost instance in the master process
84      boolean coprocessorsEnabled = conf.getBoolean(COPROCESSORS_ENABLED_CONF_KEY,
85        DEFAULT_COPROCESSORS_ENABLED);
86      LOG.info("System coprocessor loading is " + (coprocessorsEnabled ? "enabled" : "disabled"));
87      loadSystemCoprocessors(conf, MASTER_COPROCESSOR_CONF_KEY);
88    }
89  
90    @Override
91    public MasterEnvironment createEnvironment(final Class<?> implClass,
92        final Coprocessor instance, final int priority, final int seq,
93        final Configuration conf) {
94      for (Object itf : ClassUtils.getAllInterfaces(implClass)) {
95        Class<?> c = (Class<?>) itf;
96        if (CoprocessorService.class.isAssignableFrom(c)) {
97          masterServices.registerService(((CoprocessorService)instance).getService());
98        }
99      }
100     return new MasterEnvironment(implClass, instance, priority, seq, conf,
101         masterServices);
102   }
103 
104   public boolean preCreateNamespace(final NamespaceDescriptor ns) throws IOException {
105     return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
106       @Override
107       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
108           throws IOException {
109         oserver.preCreateNamespace(ctx, ns);
110       }
111     });
112   }
113 
114   public void postCreateNamespace(final NamespaceDescriptor ns) throws IOException {
115     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
116       @Override
117       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
118           throws IOException {
119         oserver.postCreateNamespace(ctx, ns);
120       }
121     });
122   }
123 
124   public boolean preDeleteNamespace(final String namespaceName) throws IOException {
125     return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
126       @Override
127       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
128           throws IOException {
129         oserver.preDeleteNamespace(ctx, namespaceName);
130       }
131     });
132   }
133 
134   public void postDeleteNamespace(final String namespaceName) throws IOException {
135     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
136       @Override
137       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
138           throws IOException {
139         oserver.postDeleteNamespace(ctx, namespaceName);
140       }
141     });
142   }
143 
144   public boolean preModifyNamespace(final NamespaceDescriptor ns) throws IOException {
145     return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
146       @Override
147       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
148           throws IOException {
149         oserver.preModifyNamespace(ctx, ns);
150       }
151     });
152   }
153 
154   public void postModifyNamespace(final NamespaceDescriptor ns) throws IOException {
155     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
156       @Override
157       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
158           throws IOException {
159         oserver.postModifyNamespace(ctx, ns);
160       }
161     });
162   }
163 
164   public void preGetNamespaceDescriptor(final String namespaceName)
165       throws IOException {
166     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
167       @Override
168       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
169           throws IOException {
170         oserver.preGetNamespaceDescriptor(ctx, namespaceName);
171       }
172     });
173   }
174 
175   public void postGetNamespaceDescriptor(final NamespaceDescriptor ns)
176       throws IOException {
177     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
178       @Override
179       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
180           throws IOException {
181         oserver.postGetNamespaceDescriptor(ctx, ns);
182       }
183     });
184   }
185 
186   public boolean preListNamespaceDescriptors(final List<NamespaceDescriptor> descriptors)
187       throws IOException {
188     return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
189       @Override
190       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
191           throws IOException {
192         oserver.preListNamespaceDescriptors(ctx, descriptors);
193       }
194     });
195   }
196 
197   public void postListNamespaceDescriptors(final List<NamespaceDescriptor> descriptors)
198       throws IOException {
199     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
200       @Override
201       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
202           throws IOException {
203         oserver.postListNamespaceDescriptors(ctx, descriptors);
204       }
205     });
206   }
207 
208   /* Implementation of hooks for invoking MasterObservers */
209 
210   public void preCreateTable(final HTableDescriptor htd, final HRegionInfo[] regions)
211       throws IOException {
212     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
213       @Override
214       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
215           throws IOException {
216         oserver.preCreateTable(ctx, htd, regions);
217       }
218     });
219   }
220 
221   public void postCreateTable(final HTableDescriptor htd, final HRegionInfo[] regions)
222       throws IOException {
223     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
224       @Override
225       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
226           throws IOException {
227         oserver.postCreateTable(ctx, htd, regions);
228       }
229     });
230   }
231 
232   public void preCreateTableHandler(final HTableDescriptor htd, final HRegionInfo[] regions)
233       throws IOException {
234     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
235       @Override
236       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
237           throws IOException {
238         oserver.preCreateTableHandler(ctx, htd, regions);
239       }
240     });
241   }
242 
243   public void postCreateTableHandler(final HTableDescriptor htd, final HRegionInfo[] regions)
244       throws IOException {
245     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
246       @Override
247       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
248           throws IOException {
249         oserver.postCreateTableHandler(ctx, htd, regions);
250       }
251     });
252   }
253 
254   public void preDeleteTable(final TableName tableName) throws IOException {
255     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
256       @Override
257       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
258           throws IOException {
259         oserver.preDeleteTable(ctx, tableName);
260       }
261     });
262   }
263 
264   public void postDeleteTable(final TableName tableName) throws IOException {
265     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
266       @Override
267       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
268           throws IOException {
269         oserver.postDeleteTable(ctx, tableName);
270       }
271     });
272   }
273 
274   public void preDeleteTableHandler(final TableName tableName) throws IOException {
275     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
276       @Override
277       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
278           throws IOException {
279         oserver.preDeleteTableHandler(ctx, tableName);
280       }
281     });
282   }
283 
284   public void postDeleteTableHandler(final TableName tableName) throws IOException {
285     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
286       @Override
287       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
288           throws IOException {
289         oserver.postDeleteTableHandler(ctx, tableName);
290       }
291     });
292   }
293 
294   public void preTruncateTable(final TableName tableName) throws IOException {
295     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
296       @Override
297       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
298           throws IOException {
299         oserver.preTruncateTable(ctx, tableName);
300       }
301     });
302   }
303 
304   public void postTruncateTable(final TableName tableName) throws IOException {
305     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
306       @Override
307       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
308           throws IOException {
309         oserver.postTruncateTable(ctx, tableName);
310       }
311     });
312   }
313 
314   public void preTruncateTableHandler(final TableName tableName) throws IOException {
315     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
316       @Override
317       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
318           throws IOException {
319         oserver.preTruncateTableHandler(ctx, tableName);
320       }
321     });
322   }
323 
324   public void postTruncateTableHandler(final TableName tableName) throws IOException {
325     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
326       @Override
327       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
328           throws IOException {
329         oserver.postTruncateTableHandler(ctx, tableName);
330       }
331     });
332   }
333 
334   public void preModifyTable(final TableName tableName, final HTableDescriptor htd)
335       throws IOException {
336     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
337       @Override
338       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
339           throws IOException {
340         oserver.preModifyTable(ctx, tableName, htd);
341       }
342     });
343   }
344 
345   public void postModifyTable(final TableName tableName, final HTableDescriptor htd)
346       throws IOException {
347     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
348       @Override
349       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
350           throws IOException {
351         oserver.postModifyTable(ctx, tableName, htd);
352       }
353     });
354   }
355 
356   public void preModifyTableHandler(final TableName tableName, final HTableDescriptor htd)
357       throws IOException {
358     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
359       @Override
360       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
361           throws IOException {
362         oserver.preModifyTableHandler(ctx, tableName, htd);
363       }
364     });
365   }
366 
367   public void postModifyTableHandler(final TableName tableName, final HTableDescriptor htd)
368       throws IOException {
369     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
370       @Override
371       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
372           throws IOException {
373         oserver.postModifyTableHandler(ctx, tableName, htd);
374       }
375     });
376   }
377 
378   public boolean preAddColumn(final TableName tableName, final HColumnDescriptor column)
379       throws IOException {
380     return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
381       @Override
382       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
383           throws IOException {
384         oserver.preAddColumn(ctx, tableName, column);
385       }
386     });
387   }
388 
389   public void postAddColumn(final TableName tableName, final HColumnDescriptor column)
390       throws IOException {
391     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
392       @Override
393       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
394           throws IOException {
395         oserver.postAddColumn(ctx, tableName, column);
396       }
397     });
398   }
399 
400   public boolean preAddColumnHandler(final TableName tableName, final HColumnDescriptor column)
401       throws IOException {
402     return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
403       @Override
404       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
405           throws IOException {
406         oserver.preAddColumnHandler(ctx, tableName, column);
407       }
408     });
409   }
410 
411   public void postAddColumnHandler(final TableName tableName, final HColumnDescriptor column)
412       throws IOException {
413     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
414       @Override
415       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
416           throws IOException {
417         oserver.postAddColumnHandler(ctx, tableName, column);
418       }
419     });
420   }
421 
422   public boolean preModifyColumn(final TableName tableName, final HColumnDescriptor descriptor)
423       throws IOException {
424     return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
425       @Override
426       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
427           throws IOException {
428         oserver.preModifyColumn(ctx, tableName, descriptor);
429       }
430     });
431   }
432 
433   public void postModifyColumn(final TableName tableName, final HColumnDescriptor descriptor)
434       throws IOException {
435     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
436       @Override
437       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
438           throws IOException {
439         oserver.postModifyColumn(ctx, tableName, descriptor);
440       }
441     });
442   }
443 
444   public boolean preModifyColumnHandler(final TableName tableName,
445       final HColumnDescriptor descriptor) throws IOException {
446     return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
447       @Override
448       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
449           throws IOException {
450         oserver.preModifyColumnHandler(ctx, tableName, descriptor);
451       }
452     });
453   }
454 
455   public void postModifyColumnHandler(final TableName tableName,
456       final HColumnDescriptor descriptor) throws IOException {
457     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
458       @Override
459       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
460           throws IOException {
461         oserver.postModifyColumnHandler(ctx, tableName, descriptor);
462       }
463     });
464   }
465 
466   public boolean preDeleteColumn(final TableName tableName, final byte [] c) throws IOException {
467     return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
468       @Override
469       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
470           throws IOException {
471         oserver.preDeleteColumn(ctx, tableName, c);
472       }
473     });
474   }
475 
476   public void postDeleteColumn(final TableName tableName, final byte [] c) throws IOException {
477     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
478       @Override
479       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
480           throws IOException {
481         oserver.postDeleteColumn(ctx, tableName, c);
482       }
483     });
484   }
485 
486   public boolean preDeleteColumnHandler(final TableName tableName, final byte[] c)
487       throws IOException {
488     return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
489       @Override
490       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
491           throws IOException {
492         oserver.preDeleteColumnHandler(ctx, tableName, c);
493       }
494     });
495   }
496 
497   public void postDeleteColumnHandler(final TableName tableName, final byte[] c)
498       throws IOException {
499     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
500       @Override
501       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
502           throws IOException {
503         oserver.postDeleteColumnHandler(ctx, tableName, c);
504       }
505     });
506   }
507 
508   public void preEnableTable(final TableName tableName) throws IOException {
509     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
510       @Override
511       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
512           throws IOException {
513         oserver.preEnableTable(ctx, tableName);
514       }
515     });
516   }
517 
518   public void postEnableTable(final TableName tableName) throws IOException {
519     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
520       @Override
521       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
522           throws IOException {
523         oserver.postEnableTable(ctx, tableName);
524       }
525     });
526   }
527 
528   public void preEnableTableHandler(final TableName tableName) throws IOException {
529     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
530       @Override
531       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
532           throws IOException {
533         oserver.preEnableTableHandler(ctx, tableName);
534       }
535     });
536   }
537 
538   public void postEnableTableHandler(final TableName tableName) throws IOException {
539     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
540       @Override
541       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
542           throws IOException {
543         oserver.postEnableTableHandler(ctx, tableName);
544       }
545     });
546   }
547 
548   public void preDisableTable(final TableName tableName) throws IOException {
549     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
550       @Override
551       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
552           throws IOException {
553         oserver.preDisableTable(ctx, tableName);
554       }
555     });
556   }
557 
558   public void postDisableTable(final TableName tableName) throws IOException {
559     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
560       @Override
561       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
562           throws IOException {
563         oserver.postDisableTable(ctx, tableName);
564       }
565     });
566   }
567 
568   public void preDisableTableHandler(final TableName tableName) throws IOException {
569     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
570       @Override
571       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
572           throws IOException {
573         oserver.preDisableTableHandler(ctx, tableName);
574       }
575     });
576   }
577 
578   public void postDisableTableHandler(final TableName tableName) throws IOException {
579     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
580       @Override
581       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
582           throws IOException {
583         oserver.postDisableTableHandler(ctx, tableName);
584       }
585     });
586   }
587 
588   public boolean preAbortProcedure(
589       final ProcedureExecutor<MasterProcedureEnv> procEnv,
590       final long procId) throws IOException {
591     return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
592       @Override
593       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
594           throws IOException {
595         oserver.preAbortProcedure(ctx, procEnv, procId);
596       }
597     });
598   }
599 
600   public void postAbortProcedure() throws IOException {
601     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
602       @Override
603       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
604           throws IOException {
605         oserver.postAbortProcedure(ctx);
606       }
607     });
608   }
609 
610   public boolean preListProcedures() throws IOException {
611     return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
612       @Override
613       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
614           throws IOException {
615         oserver.preListProcedures(ctx);
616       }
617     });
618   }
619 
620   public void postListProcedures(final List<ProcedureInfo> procInfoList) throws IOException {
621     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
622       @Override
623       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
624           throws IOException {
625         oserver.postListProcedures(ctx, procInfoList);
626       }
627     });
628   }
629 
630   public boolean preMove(final HRegionInfo region, final ServerName srcServer,
631       final ServerName destServer) throws IOException {
632     return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
633       @Override
634       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
635           throws IOException {
636         oserver.preMove(ctx, region, srcServer, destServer);
637       }
638     });
639   }
640 
641   public void postMove(final HRegionInfo region, final ServerName srcServer,
642       final ServerName destServer) throws IOException {
643     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
644       @Override
645       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
646           throws IOException {
647         oserver.postMove(ctx, region, srcServer, destServer);
648       }
649     });
650   }
651 
652   public boolean preAssign(final HRegionInfo regionInfo) throws IOException {
653     return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
654       @Override
655       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
656           throws IOException {
657         oserver.preAssign(ctx, regionInfo);
658       }
659     });
660   }
661 
662   public void postAssign(final HRegionInfo regionInfo) throws IOException {
663     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
664       @Override
665       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
666           throws IOException {
667         oserver.postAssign(ctx, regionInfo);
668       }
669     });
670   }
671 
672   public boolean preUnassign(final HRegionInfo regionInfo, final boolean force)
673       throws IOException {
674     return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
675       @Override
676       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
677           throws IOException {
678         oserver.preUnassign(ctx, regionInfo, force);
679       }
680     });
681   }
682 
683   public void postUnassign(final HRegionInfo regionInfo, final boolean force) throws IOException {
684     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
685       @Override
686       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
687           throws IOException {
688         oserver.postUnassign(ctx, regionInfo, force);
689       }
690     });
691   }
692 
693   public void preRegionOffline(final HRegionInfo regionInfo) throws IOException {
694     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
695       @Override
696       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
697           throws IOException {
698         oserver.preRegionOffline(ctx, regionInfo);
699       }
700     });
701   }
702 
703   public void postRegionOffline(final HRegionInfo regionInfo) throws IOException {
704     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
705       @Override
706       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
707           throws IOException {
708         oserver.postRegionOffline(ctx, regionInfo);
709       }
710     });
711   }
712 
713   public boolean preBalance() throws IOException {
714     return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
715       @Override
716       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
717           throws IOException {
718         oserver.preBalance(ctx);
719       }
720     });
721   }
722 
723   public void postBalance(final List<RegionPlan> plans) throws IOException {
724     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
725       @Override
726       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
727           throws IOException {
728         oserver.postBalance(ctx, plans);
729       }
730     });
731   }
732 
733   public boolean preBalanceSwitch(final boolean b) throws IOException {
734     return execOperationWithResult(b, coprocessors.isEmpty() ? null :
735         new CoprocessorOperationWithResult<Boolean>() {
736       @Override
737       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
738           throws IOException {
739         setResult(oserver.preBalanceSwitch(ctx, getResult()));
740       }
741     });
742   }
743 
744   public void postBalanceSwitch(final boolean oldValue, final boolean newValue)
745       throws IOException {
746     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
747       @Override
748       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
749           throws IOException {
750         oserver.postBalanceSwitch(ctx, oldValue, newValue);
751       }
752     });
753   }
754 
755   public void preShutdown() throws IOException {
756     // While stopping the cluster all coprocessors method should be executed first then the
757     // coprocessor should be cleaned up.
758     execShutdown(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
759       @Override
760       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
761           throws IOException {
762         oserver.preShutdown(ctx);
763       }
764       @Override
765       public void postEnvCall(MasterEnvironment env) {
766         // invoke coprocessor stop method
767         shutdown(env);
768       }
769     });
770   }
771 
772   public void preStopMaster() throws IOException {
773     // While stopping master all coprocessors method should be executed first then the coprocessor
774     // environment should be cleaned up.
775     execShutdown(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
776       @Override
777       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
778           throws IOException {
779         oserver.preStopMaster(ctx);
780       }
781       @Override
782       public void postEnvCall(MasterEnvironment env) {
783         // invoke coprocessor stop method
784         shutdown(env);
785       }
786     });
787   }
788 
789   public void preMasterInitialization() throws IOException {
790     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
791       @Override
792       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
793           throws IOException {
794         oserver.preMasterInitialization(ctx);
795       }
796     });
797   }
798 
799   public void postStartMaster() throws IOException {
800     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
801       @Override
802       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
803           throws IOException {
804         oserver.postStartMaster(ctx);
805       }
806     });
807   }
808 
809   public void preSnapshot(final SnapshotDescription snapshot,
810       final HTableDescriptor hTableDescriptor) throws IOException {
811     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
812       @Override
813       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
814           throws IOException {
815         oserver.preSnapshot(ctx, snapshot, hTableDescriptor);
816       }
817     });
818   }
819 
820   public void postSnapshot(final SnapshotDescription snapshot,
821       final HTableDescriptor hTableDescriptor) throws IOException {
822     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
823       @Override
824       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
825           throws IOException {
826         oserver.postSnapshot(ctx, snapshot, hTableDescriptor);
827       }
828     });
829   }
830 
831   public void preListSnapshot(final SnapshotDescription snapshot) throws IOException {
832     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
833       @Override
834       public void call(MasterObserver observer, ObserverContext<MasterCoprocessorEnvironment> ctx)
835           throws IOException {
836         observer.preListSnapshot(ctx, snapshot);
837       }
838     });
839   }
840 
841   public void postListSnapshot(final SnapshotDescription snapshot) throws IOException {
842     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
843       @Override
844       public void call(MasterObserver observer, ObserverContext<MasterCoprocessorEnvironment> ctx)
845           throws IOException {
846         observer.postListSnapshot(ctx, snapshot);
847       }
848     });
849   }
850   
851   public void preCloneSnapshot(final SnapshotDescription snapshot,
852       final HTableDescriptor hTableDescriptor) throws IOException {
853     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
854       @Override
855       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
856           throws IOException {
857         oserver.preCloneSnapshot(ctx, snapshot, hTableDescriptor);
858       }
859     });
860   }
861 
862   public void postCloneSnapshot(final SnapshotDescription snapshot,
863       final HTableDescriptor hTableDescriptor) throws IOException {
864     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
865       @Override
866       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
867           throws IOException {
868         oserver.postCloneSnapshot(ctx, snapshot, hTableDescriptor);
869       }
870     });
871   }
872 
873   public void preRestoreSnapshot(final SnapshotDescription snapshot,
874       final HTableDescriptor hTableDescriptor) throws IOException {
875     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
876       @Override
877       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
878           throws IOException {
879         oserver.preRestoreSnapshot(ctx, snapshot, hTableDescriptor);
880       }
881     });
882   }
883 
884   public void postRestoreSnapshot(final SnapshotDescription snapshot,
885       final HTableDescriptor hTableDescriptor) throws IOException {
886     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
887       @Override
888       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
889           throws IOException {
890         oserver.postRestoreSnapshot(ctx, snapshot, hTableDescriptor);
891       }
892     });
893   }
894 
895   public void preDeleteSnapshot(final SnapshotDescription snapshot) throws IOException {
896     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
897       @Override
898       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
899           throws IOException {
900         oserver.preDeleteSnapshot(ctx, snapshot);
901       }
902     });
903   }
904 
905   public void postDeleteSnapshot(final SnapshotDescription snapshot) throws IOException {
906     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
907       @Override
908       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
909           throws IOException {
910         oserver.postDeleteSnapshot(ctx, snapshot);
911       }
912     });
913   }
914 
915   @Deprecated
916   public boolean preGetTableDescriptors(final List<TableName> tableNamesList,
917     final List<HTableDescriptor> descriptors) throws IOException {
918     return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
919       @Override
920       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
921           throws IOException {
922         oserver.preGetTableDescriptors(ctx, tableNamesList, descriptors);
923       }
924     });
925   }
926 
927   @Deprecated
928   public void postGetTableDescriptors(final List<HTableDescriptor> descriptors)
929       throws IOException {
930     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
931       @Override
932       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
933           throws IOException {
934         oserver.postGetTableDescriptors(ctx, descriptors);
935       }
936     });
937   }
938 
939   public boolean preGetTableDescriptors(final List<TableName> tableNamesList,
940       final List<HTableDescriptor> descriptors, final String regex) throws IOException {
941     return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
942       @Override
943       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
944           throws IOException {
945         oserver.preGetTableDescriptors(ctx, tableNamesList, descriptors, regex);
946       }
947     });
948   }
949 
950   public void postGetTableDescriptors(final List<TableName> tableNamesList,
951       final List<HTableDescriptor> descriptors, final String regex) throws IOException {
952     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
953       @Override
954       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
955           throws IOException {
956         oserver.postGetTableDescriptors(ctx, tableNamesList, descriptors, regex);
957       }
958     });
959   }
960 
961   public boolean preGetTableNames(final List<HTableDescriptor> descriptors,
962       final String regex) throws IOException {
963     return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
964       @Override
965       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
966           throws IOException {
967         oserver.preGetTableNames(ctx, descriptors, regex);
968       }
969     });
970   }
971 
972   public void postGetTableNames(final List<HTableDescriptor> descriptors,
973       final String regex) throws IOException {
974     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
975       @Override
976       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
977           throws IOException {
978         oserver.postGetTableNames(ctx, descriptors, regex);
979       }
980     });
981   }
982 
983   public void preTableFlush(final TableName tableName) throws IOException {
984     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
985       @Override
986       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
987           throws IOException {
988         oserver.preTableFlush(ctx, tableName);
989       }
990     });
991   }
992 
993   public void postTableFlush(final TableName tableName) throws IOException {
994     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
995       @Override
996       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
997           throws IOException {
998         oserver.postTableFlush(ctx, tableName);
999       }
1000     });
1001   }
1002   
1003   public void preSetUserQuota(final String user, final Quotas quotas) throws IOException {
1004     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
1005       @Override
1006       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
1007           throws IOException {
1008         oserver.preSetUserQuota(ctx, user, quotas);
1009       }
1010     });
1011   }
1012 
1013   public void postSetUserQuota(final String user, final Quotas quotas) throws IOException {
1014     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
1015       @Override
1016       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
1017           throws IOException {
1018         oserver.postSetUserQuota(ctx, user, quotas);
1019       }
1020     });
1021   }
1022 
1023   public void preSetUserQuota(final String user, final TableName table, final Quotas quotas)
1024       throws IOException {
1025     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
1026       @Override
1027       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
1028           throws IOException {
1029         oserver.preSetUserQuota(ctx, user, table, quotas);
1030       }
1031     });
1032   }
1033 
1034   public void postSetUserQuota(final String user, final TableName table, final Quotas quotas)
1035       throws IOException {
1036     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
1037       @Override
1038       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
1039           throws IOException {
1040         oserver.postSetUserQuota(ctx, user, table, quotas);
1041       }
1042     });
1043   }
1044 
1045   public void preSetUserQuota(final String user, final String namespace, final Quotas quotas)
1046       throws IOException {
1047     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
1048       @Override
1049       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
1050           throws IOException {
1051         oserver.preSetUserQuota(ctx, user, namespace, quotas);
1052       }
1053     });
1054   }
1055 
1056   public void postSetUserQuota(final String user, final String namespace, final Quotas quotas)
1057       throws IOException {
1058     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
1059       @Override
1060       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
1061           throws IOException {
1062         oserver.postSetUserQuota(ctx, user, namespace, quotas);
1063       }
1064     });
1065   }
1066 
1067   public void preSetTableQuota(final TableName table, final Quotas quotas) throws IOException {
1068     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
1069       @Override
1070       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
1071           throws IOException {
1072         oserver.preSetTableQuota(ctx, table, quotas);
1073       }
1074     });
1075   }
1076 
1077   public void postSetTableQuota(final TableName table, final Quotas quotas) throws IOException {
1078     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
1079       @Override
1080       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
1081           throws IOException {
1082         oserver.postSetTableQuota(ctx, table, quotas);
1083       }
1084     });
1085   }
1086 
1087   public void preSetNamespaceQuota(final String namespace, final Quotas quotas) throws IOException {
1088     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
1089       @Override
1090       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
1091           throws IOException {
1092         oserver.preSetNamespaceQuota(ctx, namespace, quotas);
1093       }
1094     });
1095   }
1096 
1097   public void postSetNamespaceQuota(final String namespace, final Quotas quotas) 
1098       throws IOException {
1099     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
1100       @Override
1101       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
1102           throws IOException {
1103         oserver.postSetNamespaceQuota(ctx, namespace, quotas);
1104       }
1105     });
1106   }
1107 
1108   private static abstract class CoprocessorOperation
1109       extends ObserverContext<MasterCoprocessorEnvironment> {
1110     public CoprocessorOperation() {
1111     }
1112 
1113     public abstract void call(MasterObserver oserver,
1114         ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException;
1115 
1116     public void postEnvCall(MasterEnvironment env) {
1117     }
1118   }
1119 
1120   private static abstract class CoprocessorOperationWithResult<T> extends CoprocessorOperation {
1121     private T result = null;
1122     public void setResult(final T result) { this.result = result; }
1123     public T getResult() { return this.result; }
1124   }
1125 
1126   private <T> T execOperationWithResult(final T defaultValue,
1127       final CoprocessorOperationWithResult<T> ctx) throws IOException {
1128     if (ctx == null) return defaultValue;
1129     ctx.setResult(defaultValue);
1130     execOperation(ctx);
1131     return ctx.getResult();
1132   }
1133 
1134   private boolean execOperation(final CoprocessorOperation ctx) throws IOException {
1135     if (ctx == null) return false;
1136     boolean bypass = false;
1137     List<MasterEnvironment> envs = coprocessors.get();
1138     for (int i = 0; i < envs.size(); i++) {
1139       MasterEnvironment env = envs.get(i);
1140       if (env.getInstance() instanceof MasterObserver) {
1141         ctx.prepare(env);
1142         Thread currentThread = Thread.currentThread();
1143         ClassLoader cl = currentThread.getContextClassLoader();
1144         try {
1145           currentThread.setContextClassLoader(env.getClassLoader());
1146           ctx.call((MasterObserver)env.getInstance(), ctx);
1147         } catch (Throwable e) {
1148           handleCoprocessorThrowable(env, e);
1149         } finally {
1150           currentThread.setContextClassLoader(cl);
1151         }
1152         bypass |= ctx.shouldBypass();
1153         if (ctx.shouldComplete()) {
1154           break;
1155         }
1156       }
1157       ctx.postEnvCall(env);
1158     }
1159     return bypass;
1160   }
1161 
1162   /**
1163    * Master coprocessor classes can be configured in any order, based on that priority is set and
1164    * chained in a sorted order. For preStopMaster()/preShutdown(), coprocessor methods are invoked
1165    * in call() and environment is shutdown in postEnvCall(). <br>
1166    * Need to execute all coprocessor methods first then postEnvCall(), otherwise some coprocessors
1167    * may remain shutdown if any exception occurs during next coprocessor execution which prevent
1168    * Master stop or cluster shutdown. (Refer:
1169    * <a href="https://issues.apache.org/jira/browse/HBASE-16663">HBASE-16663</a>
1170    * @param ctx CoprocessorOperation
1171    * @return true if bypaas coprocessor execution, false if not.
1172    * @throws IOException
1173    */
1174   private boolean execShutdown(final CoprocessorOperation ctx) throws IOException {
1175     if (ctx == null) return false;
1176     boolean bypass = false;
1177     List<MasterEnvironment> envs = coprocessors.get();
1178     int envsSize = envs.size();
1179     // Iterate the coprocessors and execute CoprocessorOperation's call()
1180     for (int i = 0; i < envsSize; i++) {
1181       MasterEnvironment env = envs.get(i);
1182       if (env.getInstance() instanceof MasterObserver) {
1183         ctx.prepare(env);
1184         Thread currentThread = Thread.currentThread();
1185         ClassLoader cl = currentThread.getContextClassLoader();
1186         try {
1187           currentThread.setContextClassLoader(env.getClassLoader());
1188           ctx.call((MasterObserver) env.getInstance(), ctx);
1189         } catch (Throwable e) {
1190           handleCoprocessorThrowable(env, e);
1191         } finally {
1192           currentThread.setContextClassLoader(cl);
1193         }
1194         bypass |= ctx.shouldBypass();
1195         if (ctx.shouldComplete()) {
1196           break;
1197         }
1198       }
1199     }
1200 
1201     // Iterate the coprocessors and execute CoprocessorOperation's postEnvCall()
1202     for (int i = 0; i < envsSize; i++) {
1203       MasterEnvironment env = envs.get(i);
1204       ctx.postEnvCall(env);
1205     }
1206     return bypass;
1207   }
1208 
1209 }