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  
29  import java.io.IOException;
30  
31  /**
32   * Provides the coprocessor framework and environment for master oriented
33   * operations.  {@link HMaster} interacts with the loaded coprocessors
34   * through this class.
35   */
36  public class MasterCoprocessorHost
37      extends CoprocessorHost<MasterCoprocessorHost.MasterEnvironment> {
38  
39    private static final Log LOG = LogFactory.getLog(MasterCoprocessorHost.class);
40  
41    /**
42     * Coprocessor environment extension providing access to master related
43     * services.
44     */
45    static class MasterEnvironment extends CoprocessorHost.Environment
46        implements MasterCoprocessorEnvironment {
47      private MasterServices masterServices;
48  
49      public MasterEnvironment(final Class<?> implClass, final Coprocessor impl,
50          final int priority, final int seq, final Configuration conf,
51          final MasterServices services) {
52        super(impl, priority, seq, conf);
53        this.masterServices = services;
54      }
55  
56      public MasterServices getMasterServices() {
57        return masterServices;
58      }
59    }
60  
61    private MasterServices masterServices;
62  
63    MasterCoprocessorHost(final MasterServices services, final Configuration conf) {
64      this.masterServices = services;
65      loadSystemCoprocessors(conf, MASTER_COPROCESSOR_CONF_KEY);
66    }
67  
68    @Override
69    public MasterEnvironment createEnvironment(final Class<?> implClass,
70        final Coprocessor instance, final int priority, final int seq,
71        final Configuration conf) {
72      return new MasterEnvironment(implClass, instance, priority, seq, conf,
73          masterServices);
74    }
75  
76    @Override
77    protected void abortServer(final CoprocessorEnvironment env, final Throwable e) {
78      abortServer("master", masterServices, env, e);
79    }
80  
81    /* Implementation of hooks for invoking MasterObservers */
82    void preCreateTable(HTableDescriptor htd, HRegionInfo[] regions)
83      throws IOException {
84      ObserverContext<MasterCoprocessorEnvironment> ctx = null;
85      for (MasterEnvironment env: coprocessors) {
86        if (env.getInstance() instanceof MasterObserver) {
87          ctx = ObserverContext.createAndPrepare(env, ctx);
88          try {
89            ((MasterObserver)env.getInstance()).preCreateTable(ctx, htd, regions);
90          } catch (Throwable e) {
91            handleCoprocessorThrowable(env, e);
92          }
93          if (ctx.shouldComplete()) {
94            break;
95          }
96        }
97      }
98    }
99  
100   void postCreateTable(HTableDescriptor htd, HRegionInfo[] regions)
101     throws IOException {
102     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
103     for (MasterEnvironment env: coprocessors) {
104       if (env.getInstance() instanceof MasterObserver) {
105         ctx = ObserverContext.createAndPrepare(env, ctx);
106         try {
107             ((MasterObserver)env.getInstance()).postCreateTable(ctx, htd, regions);
108         } catch (Throwable e) {
109           handleCoprocessorThrowable(env, e);
110         }
111         if (ctx.shouldComplete()) {
112           break;
113         }
114       }
115     }
116   }
117 
118   void preDeleteTable(byte[] tableName) throws IOException {
119     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
120     for (MasterEnvironment env: coprocessors) {
121       if (env.getInstance() instanceof MasterObserver) {
122         ctx = ObserverContext.createAndPrepare(env, ctx);
123         try {
124           ((MasterObserver)env.getInstance()).preDeleteTable(ctx, tableName);
125         } catch (Throwable e) {
126           handleCoprocessorThrowable(env, e);
127         }
128         if (ctx.shouldComplete()) {
129           break;
130         }
131       }
132     }
133   }
134 
135   void postDeleteTable(byte[] tableName) throws IOException {
136     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
137     for (MasterEnvironment env: coprocessors) {
138       if (env.getInstance() instanceof MasterObserver) {
139         ctx = ObserverContext.createAndPrepare(env, ctx);
140         try {
141           ((MasterObserver)env.getInstance()).postDeleteTable(ctx, tableName);
142         } catch (Throwable e) {
143           handleCoprocessorThrowable(env, e);
144         }
145         if (ctx.shouldComplete()) {
146           break;
147         }
148       }
149     }
150   }
151 
152   void preModifyTable(final byte[] tableName, HTableDescriptor htd)
153       throws IOException {
154     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
155     for (MasterEnvironment env: coprocessors) {
156       if (env.getInstance() instanceof MasterObserver) {
157         ctx = ObserverContext.createAndPrepare(env, ctx);
158         try {
159           ((MasterObserver)env.getInstance()).preModifyTable(ctx, tableName,
160               htd);
161         } catch (Throwable e) {
162           handleCoprocessorThrowable(env, e);
163         }
164         if (ctx.shouldComplete()) {
165           break;
166         }
167       }
168     }
169   }
170 
171   void postModifyTable(final byte[] tableName, HTableDescriptor htd)
172       throws IOException {
173     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
174     for (MasterEnvironment env: coprocessors) {
175       if (env.getInstance() instanceof MasterObserver) {
176         ctx = ObserverContext.createAndPrepare(env, ctx);
177         try {
178           ((MasterObserver)env.getInstance()).postModifyTable(ctx, tableName,
179               htd);
180         } catch (Throwable e) {
181           handleCoprocessorThrowable(env, e);
182         }
183         if (ctx.shouldComplete()) {
184           break;
185         }
186       }
187     }
188   }
189 
190   boolean preAddColumn(byte [] tableName, HColumnDescriptor column)
191       throws IOException {
192     boolean bypass = false;
193     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
194     for (MasterEnvironment env: coprocessors) {
195       if (env.getInstance() instanceof MasterObserver) {
196         ctx = ObserverContext.createAndPrepare(env, ctx);
197         try {
198           ((MasterObserver)env.getInstance()).preAddColumn(ctx, tableName, column);
199         } catch (Throwable e) {
200           handleCoprocessorThrowable(env, e);
201         }
202         bypass |= ctx.shouldBypass();
203         if (ctx.shouldComplete()) {
204           break;
205         }
206       }
207     }
208     return bypass;
209   }
210 
211   void postAddColumn(byte [] tableName, HColumnDescriptor column)
212       throws IOException {
213     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
214     for (MasterEnvironment env: coprocessors) {
215       if (env.getInstance() instanceof MasterObserver) {
216         ctx = ObserverContext.createAndPrepare(env, ctx);
217         try {
218           ((MasterObserver)env.getInstance()).postAddColumn(ctx, tableName,
219               column);
220         } catch (Throwable e) {
221           handleCoprocessorThrowable(env, e);
222         }
223         if (ctx.shouldComplete()) {
224           break;
225         }
226       }
227     }
228   }
229 
230   boolean preModifyColumn(byte [] tableName, HColumnDescriptor descriptor)
231       throws IOException {
232     boolean bypass = false;
233     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
234     for (MasterEnvironment env: coprocessors) {
235       if (env.getInstance() instanceof MasterObserver) {
236         ctx = ObserverContext.createAndPrepare(env, ctx);
237         try {
238           ((MasterObserver)env.getInstance()).preModifyColumn(
239             ctx, tableName, descriptor);
240         } catch (Throwable e) {
241           handleCoprocessorThrowable(env, e);
242         }
243         bypass |= ctx.shouldBypass();
244         if (ctx.shouldComplete()) {
245           break;
246         }
247       }
248     }
249     return bypass;
250   }
251 
252   void postModifyColumn(byte [] tableName, HColumnDescriptor descriptor)
253       throws IOException {
254     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
255     for (MasterEnvironment env: coprocessors) {
256       if (env.getInstance() instanceof MasterObserver) {
257         ctx = ObserverContext.createAndPrepare(env, ctx);
258         try {
259           ((MasterObserver)env.getInstance()).postModifyColumn(
260               ctx, tableName, descriptor);
261         } catch (Throwable e) {
262           handleCoprocessorThrowable(env, e);
263         }
264         if (ctx.shouldComplete()) {
265           break;
266         }
267       }
268     }
269   }
270 
271   boolean preDeleteColumn(final byte [] tableName, final byte [] c)
272       throws IOException {
273     boolean bypass = false;
274     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
275     for (MasterEnvironment env: coprocessors) {
276       if (env.getInstance() instanceof MasterObserver) {
277         ctx = ObserverContext.createAndPrepare(env, ctx);
278         try {
279           ((MasterObserver)env.getInstance()).preDeleteColumn(ctx, tableName, c);
280         } catch (Throwable e) {
281           handleCoprocessorThrowable(env, e);
282         }
283         bypass |= ctx.shouldBypass();
284         if (ctx.shouldComplete()) {
285           break;
286         }
287       }
288     }
289     return bypass;
290   }
291 
292   void postDeleteColumn(final byte [] tableName, final byte [] c)
293       throws IOException {
294     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
295     for (MasterEnvironment env: coprocessors) {
296       if (env.getInstance() instanceof MasterObserver) {
297         ctx = ObserverContext.createAndPrepare(env, ctx);
298         try {
299           ((MasterObserver)env.getInstance()).postDeleteColumn(ctx, tableName,
300               c);
301         } catch (Throwable e) {
302           handleCoprocessorThrowable(env, e);
303         }
304         if (ctx.shouldComplete()) {
305           break;
306         }
307       }
308     }
309   }
310 
311   void preEnableTable(final byte [] tableName) throws IOException {
312     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
313     for (MasterEnvironment env: coprocessors) {
314       if (env.getInstance() instanceof MasterObserver) {
315         ctx = ObserverContext.createAndPrepare(env, ctx);
316         try {
317           ((MasterObserver)env.getInstance()).preEnableTable(ctx, tableName);
318         } catch (Throwable e) {
319           handleCoprocessorThrowable(env, e);
320         }
321         if (ctx.shouldComplete()) {
322           break;
323         }
324       }
325     }
326   }
327 
328   void postEnableTable(final byte [] tableName) throws IOException {
329     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
330     for (MasterEnvironment env: coprocessors) {
331       if (env.getInstance() instanceof MasterObserver) {
332         ctx = ObserverContext.createAndPrepare(env, ctx);
333         try {
334           ((MasterObserver)env.getInstance()).postEnableTable(ctx, tableName);
335         } catch (Throwable e) {
336           handleCoprocessorThrowable(env, e);
337         }
338         if (ctx.shouldComplete()) {
339           break;
340         }
341       }
342     }
343   }
344 
345   void preDisableTable(final byte [] tableName) throws IOException {
346     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
347     for (MasterEnvironment env: coprocessors) {
348       if (env.getInstance() instanceof MasterObserver) {
349         ctx = ObserverContext.createAndPrepare(env, ctx);
350         try {
351           ((MasterObserver)env.getInstance()).preDisableTable(ctx, tableName);
352         } catch (Throwable e) {
353           handleCoprocessorThrowable(env, e);
354         }
355         if (ctx.shouldComplete()) {
356           break;
357         }
358       }
359     }
360   }
361 
362   void postDisableTable(final byte [] tableName) throws IOException {
363     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
364     for (MasterEnvironment env: coprocessors) {
365       if (env.getInstance() instanceof MasterObserver) {
366         ctx = ObserverContext.createAndPrepare(env, ctx);
367         try {
368           ((MasterObserver)env.getInstance()).postDisableTable(ctx, tableName);
369         } catch (Throwable e) {
370           handleCoprocessorThrowable(env, e);
371         }
372         if (ctx.shouldComplete()) {
373           break;
374         }
375       }
376     }
377   }
378 
379   boolean preMove(final HRegionInfo region, final ServerName srcServer, final ServerName destServer)
380       throws IOException {
381     boolean bypass = false;
382     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
383     for (MasterEnvironment env: coprocessors) {
384       if (env.getInstance() instanceof MasterObserver) {
385         ctx = ObserverContext.createAndPrepare(env, ctx);
386         try {
387           ((MasterObserver)env.getInstance()).preMove(
388               ctx, region, srcServer, destServer);
389         } catch (Throwable e) {
390           handleCoprocessorThrowable(env, e);
391         }
392         bypass |= ctx.shouldBypass();
393         if (ctx.shouldComplete()) {
394           break;
395         }
396       }
397     }
398     return bypass;
399   }
400 
401   void postMove(final HRegionInfo region, final ServerName srcServer, final ServerName destServer)
402       throws IOException {
403     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
404     for (MasterEnvironment env: coprocessors) {
405       if (env.getInstance() instanceof MasterObserver) {
406         ctx = ObserverContext.createAndPrepare(env, ctx);
407         try {
408           ((MasterObserver)env.getInstance()).postMove(
409               ctx, region, srcServer, destServer);
410         } catch (Throwable e) {
411           handleCoprocessorThrowable(env, e);
412         }
413         if (ctx.shouldComplete()) {
414           break;
415         }
416       }
417     }
418   }
419 
420   boolean preAssign(final HRegionInfo regionInfo) throws IOException {
421     boolean bypass = false;
422     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
423     for (MasterEnvironment env: coprocessors) {
424       if (env.getInstance() instanceof MasterObserver) {
425         ctx = ObserverContext.createAndPrepare(env, ctx);
426         try {
427           ((MasterObserver) env.getInstance()).preAssign(ctx, regionInfo);
428         } catch (Throwable e) {
429           handleCoprocessorThrowable(env, e);
430         }
431         bypass |= ctx.shouldBypass();
432         if (ctx.shouldComplete()) {
433           break;
434         }
435       }
436     }
437     return bypass;
438   }
439 
440   void postAssign(final HRegionInfo regionInfo) throws IOException {
441     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
442     for (MasterEnvironment env: coprocessors) {
443       if (env.getInstance() instanceof MasterObserver) {
444         ctx = ObserverContext.createAndPrepare(env, ctx);
445         try {
446           ((MasterObserver)env.getInstance()).postAssign(ctx, regionInfo);
447         } catch (Throwable e) {
448           handleCoprocessorThrowable(env, e);
449         }
450         if (ctx.shouldComplete()) {
451           break;
452         }
453       }
454     }
455   }
456 
457   boolean preUnassign(final HRegionInfo regionInfo, final boolean force)
458       throws IOException {
459     boolean bypass = false;
460     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
461     for (MasterEnvironment env: coprocessors) {
462       if (env.getInstance() instanceof MasterObserver) {
463         ctx = ObserverContext.createAndPrepare(env, ctx);
464         try {
465           ((MasterObserver)env.getInstance()).preUnassign(
466               ctx, regionInfo, force);
467         } catch (Throwable e) {
468           handleCoprocessorThrowable(env, e);
469         }
470         bypass |= ctx.shouldBypass();
471         if (ctx.shouldComplete()) {
472           break;
473         }
474       }
475     }
476     return bypass;
477   }
478 
479   void postUnassign(final HRegionInfo regionInfo, final boolean force)
480       throws IOException {
481     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
482     for (MasterEnvironment env: coprocessors) {
483       if (env.getInstance() instanceof MasterObserver) {
484         ctx = ObserverContext.createAndPrepare(env, ctx);
485         try {
486           ((MasterObserver)env.getInstance()).postUnassign(
487               ctx, regionInfo, force);
488         } catch (Throwable e) {
489           handleCoprocessorThrowable(env, e);
490         }
491         if (ctx.shouldComplete()) {
492           break;
493         }
494       }
495     }
496   }
497 
498   boolean preBalance() throws IOException {
499     boolean bypass = false;
500     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
501     for (MasterEnvironment env: coprocessors) {
502       if (env.getInstance() instanceof MasterObserver) {
503         ctx = ObserverContext.createAndPrepare(env, ctx);
504         try {
505           ((MasterObserver)env.getInstance()).preBalance(ctx);
506         } catch (Throwable e) {
507           handleCoprocessorThrowable(env, e);
508         }
509         bypass |= ctx.shouldBypass();
510         if (ctx.shouldComplete()) {
511           break;
512         }
513       }
514     }
515     return bypass;
516   }
517 
518   void postBalance() throws IOException {
519     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
520     for (MasterEnvironment env: coprocessors) {
521       if (env.getInstance() instanceof MasterObserver) {
522         ctx = ObserverContext.createAndPrepare(env, ctx);
523         try {
524           ((MasterObserver)env.getInstance()).postBalance(ctx);
525         } catch (Throwable e) {
526           handleCoprocessorThrowable(env, e);
527         }
528         if (ctx.shouldComplete()) {
529           break;
530         }
531       }
532     }
533   }
534 
535   boolean preBalanceSwitch(final boolean b) throws IOException {
536     boolean balance = b;
537     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
538     for (MasterEnvironment env: coprocessors) {
539       if (env.getInstance() instanceof MasterObserver) {
540         ctx = ObserverContext.createAndPrepare(env, ctx);
541         try {
542           balance = ((MasterObserver)env.getInstance()).preBalanceSwitch(
543               ctx, balance);
544         } catch (Throwable e) {
545           handleCoprocessorThrowable(env, e);
546         }
547         if (ctx.shouldComplete()) {
548           break;
549         }
550       }
551     }
552     return balance;
553   }
554 
555   void postBalanceSwitch(final boolean oldValue, final boolean newValue)
556       throws IOException {
557     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
558     for (MasterEnvironment env: coprocessors) {
559       if (env.getInstance() instanceof MasterObserver) {
560         ctx = ObserverContext.createAndPrepare(env, ctx);
561         try {
562           ((MasterObserver)env.getInstance()).postBalanceSwitch(
563               ctx, oldValue, newValue);
564         } catch (Throwable e) {
565           handleCoprocessorThrowable(env, e);
566         }
567         if (ctx.shouldComplete()) {
568           break;
569         }
570       }
571     }
572   }
573 
574   void preShutdown() throws IOException {
575     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
576     for (MasterEnvironment env: coprocessors) {
577       if (env.getInstance() instanceof MasterObserver) {
578         ctx = ObserverContext.createAndPrepare(env, ctx);
579         try {
580           ((MasterObserver)env.getInstance()).preShutdown(ctx);
581         } catch (Throwable e) {
582           handleCoprocessorThrowable(env, e);
583         }
584         if (ctx.shouldComplete()) {
585           break;
586         }
587       }
588     }
589   }
590 
591   void preStopMaster() throws IOException {
592     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
593     for (MasterEnvironment env: coprocessors) {
594       if (env.getInstance() instanceof MasterObserver) {
595         ctx = ObserverContext.createAndPrepare(env, ctx);
596         try {
597           ((MasterObserver)env.getInstance()).preStopMaster(ctx);
598         } catch (Throwable e) {
599           handleCoprocessorThrowable(env, e);
600         }
601         if (ctx.shouldComplete()) {
602           break;
603         }
604       }
605     }
606   }
607 
608   void postStartMaster() throws IOException {
609     ObserverContext<MasterCoprocessorEnvironment> ctx = null;
610     for (MasterEnvironment env: coprocessors) {
611       if (env.getInstance() instanceof MasterObserver) {
612         ctx = ObserverContext.createAndPrepare(env, ctx);
613         try {
614           ((MasterObserver)env.getInstance()).postStartMaster(ctx);
615         } catch (Throwable e) {
616           handleCoprocessorThrowable(env, e);
617         }
618         if (ctx.shouldComplete()) {
619           break;
620         }
621       }
622     }
623   }
624 }