001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 * http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018
019package org.apache.hadoop.hbase.master;
020
021import com.google.protobuf.Service;
022import java.io.IOException;
023import java.lang.reflect.InvocationTargetException;
024import java.util.List;
025import java.util.Set;
026import org.apache.hadoop.conf.Configuration;
027import org.apache.hadoop.hbase.ClusterMetrics;
028import org.apache.hadoop.hbase.MetaMutationAnnotation;
029import org.apache.hadoop.hbase.NamespaceDescriptor;
030import org.apache.hadoop.hbase.ServerName;
031import org.apache.hadoop.hbase.SharedConnection;
032import org.apache.hadoop.hbase.TableName;
033import org.apache.hadoop.hbase.client.Connection;
034import org.apache.hadoop.hbase.client.MasterSwitchType;
035import org.apache.hadoop.hbase.client.Mutation;
036import org.apache.hadoop.hbase.client.RegionInfo;
037import org.apache.hadoop.hbase.client.SnapshotDescription;
038import org.apache.hadoop.hbase.client.TableDescriptor;
039import org.apache.hadoop.hbase.coprocessor.BaseEnvironment;
040import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
041import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
042import org.apache.hadoop.hbase.coprocessor.CoprocessorServiceBackwardCompatiblity;
043import org.apache.hadoop.hbase.coprocessor.CoreCoprocessor;
044import org.apache.hadoop.hbase.coprocessor.HasMasterServices;
045import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
046import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
047import org.apache.hadoop.hbase.coprocessor.MasterObserver;
048import org.apache.hadoop.hbase.coprocessor.MetricsCoprocessor;
049import org.apache.hadoop.hbase.master.locking.LockProcedure;
050import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
051import org.apache.hadoop.hbase.metrics.MetricRegistry;
052import org.apache.hadoop.hbase.net.Address;
053import org.apache.hadoop.hbase.procedure2.LockType;
054import org.apache.hadoop.hbase.procedure2.LockedResource;
055import org.apache.hadoop.hbase.procedure2.Procedure;
056import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
057import org.apache.hadoop.hbase.quotas.GlobalQuotaSettings;
058import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
059import org.apache.hadoop.hbase.security.User;
060import org.apache.hadoop.hbase.security.access.Permission;
061import org.apache.hadoop.hbase.security.access.UserPermission;
062import org.apache.yetus.audience.InterfaceAudience;
063import org.slf4j.Logger;
064import org.slf4j.LoggerFactory;
065
066/**
067 * Provides the coprocessor framework and environment for master oriented
068 * operations.  {@link HMaster} interacts with the loaded coprocessors
069 * through this class.
070 */
071@InterfaceAudience.Private
072public class MasterCoprocessorHost
073    extends CoprocessorHost<MasterCoprocessor, MasterCoprocessorEnvironment> {
074
075  private static final Logger LOG = LoggerFactory.getLogger(MasterCoprocessorHost.class);
076
077  /**
078   * Coprocessor environment extension providing access to master related
079   * services.
080   */
081  private static class MasterEnvironment extends BaseEnvironment<MasterCoprocessor>
082      implements MasterCoprocessorEnvironment {
083    private final MetricRegistry metricRegistry;
084    private final MasterServices services;
085
086    public MasterEnvironment(final MasterCoprocessor impl, final int priority, final int seq,
087        final Configuration conf, final MasterServices services) {
088      super(impl, priority, seq, conf);
089      this.services = services;
090      this.metricRegistry =
091          MetricsCoprocessor.createRegistryForMasterCoprocessor(impl.getClass().getName());
092    }
093
094    @Override
095    public ServerName getServerName() {
096      return this.services.getServerName();
097    }
098
099    @Override
100    public Connection getConnection() {
101      return new SharedConnection(this.services.getConnection());
102    }
103
104    @Override
105    public Connection createConnection(Configuration conf) throws IOException {
106      return this.services.createConnection(conf);
107    }
108
109    @Override
110    public MetricRegistry getMetricRegistryForMaster() {
111      return metricRegistry;
112    }
113
114    @Override
115    public void shutdown() {
116      super.shutdown();
117      MetricsCoprocessor.removeRegistry(this.metricRegistry);
118    }
119  }
120
121  /**
122   * Special version of MasterEnvironment that exposes MasterServices for Core Coprocessors only.
123   * Temporary hack until Core Coprocessors are integrated into Core.
124   */
125  private static class MasterEnvironmentForCoreCoprocessors extends MasterEnvironment
126      implements HasMasterServices {
127    private final MasterServices masterServices;
128
129    public MasterEnvironmentForCoreCoprocessors(final MasterCoprocessor impl, final int priority,
130        final int seq, final Configuration conf, final MasterServices services) {
131      super(impl, priority, seq, conf, services);
132      this.masterServices = services;
133    }
134
135    /**
136     * @return An instance of MasterServices, an object NOT for general user-space Coprocessor
137     * consumption.
138     */
139    @Override
140    public MasterServices getMasterServices() {
141      return this.masterServices;
142    }
143  }
144
145  private MasterServices masterServices;
146
147  public MasterCoprocessorHost(final MasterServices services, final Configuration conf) {
148    super(services);
149    this.conf = conf;
150    this.masterServices = services;
151    // Log the state of coprocessor loading here; should appear only once or
152    // twice in the daemon log, depending on HBase version, because there is
153    // only one MasterCoprocessorHost instance in the master process
154    boolean coprocessorsEnabled = conf.getBoolean(COPROCESSORS_ENABLED_CONF_KEY,
155      DEFAULT_COPROCESSORS_ENABLED);
156    LOG.trace("System coprocessor loading is {}",  (coprocessorsEnabled ? "enabled" : "disabled"));
157    loadSystemCoprocessors(conf, MASTER_COPROCESSOR_CONF_KEY);
158  }
159
160  @Override
161  public MasterEnvironment createEnvironment(final MasterCoprocessor instance, final int priority,
162      final int seq, final Configuration conf) {
163    // If coprocessor exposes any services, register them.
164    for (Service service : instance.getServices()) {
165      masterServices.registerService(service);
166    }
167    // If a CoreCoprocessor, return a 'richer' environment, one laden with MasterServices.
168    return instance.getClass().isAnnotationPresent(CoreCoprocessor.class)?
169        new MasterEnvironmentForCoreCoprocessors(instance, priority, seq, conf, masterServices):
170        new MasterEnvironment(instance, priority, seq, conf, masterServices);
171  }
172
173  @Override
174  public MasterCoprocessor checkAndGetInstance(Class<?> implClass)
175      throws InstantiationException, IllegalAccessException {
176    try {
177      if (MasterCoprocessor.class.isAssignableFrom(implClass)) {
178        return implClass.asSubclass(MasterCoprocessor.class).getDeclaredConstructor().newInstance();
179      } else if (CoprocessorService.class.isAssignableFrom(implClass)) {
180        // For backward compatibility with old CoprocessorService impl which don't extend
181        // MasterCoprocessor.
182        CoprocessorService cs;
183        cs = implClass.asSubclass(CoprocessorService.class).getDeclaredConstructor().newInstance();
184        return new CoprocessorServiceBackwardCompatiblity.MasterCoprocessorService(cs);
185      } else {
186        LOG.error("{} is not of type MasterCoprocessor. Check the configuration of {}",
187            implClass.getName(), CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY);
188        return null;
189      }
190    } catch (NoSuchMethodException | InvocationTargetException e) {
191      throw (InstantiationException) new InstantiationException(implClass.getName()).initCause(e);
192    }
193  }
194
195  private ObserverGetter<MasterCoprocessor, MasterObserver> masterObserverGetter =
196      MasterCoprocessor::getMasterObserver;
197
198  abstract class MasterObserverOperation extends
199      ObserverOperationWithoutResult<MasterObserver> {
200    public MasterObserverOperation(){
201      super(masterObserverGetter);
202    }
203
204    public MasterObserverOperation(boolean bypassable) {
205      this(null, bypassable);
206    }
207
208    public MasterObserverOperation(User user) {
209      super(masterObserverGetter, user);
210    }
211
212    public MasterObserverOperation(User user, boolean bypassable) {
213      super(masterObserverGetter, user, bypassable);
214    }
215  }
216
217
218  //////////////////////////////////////////////////////////////////////////////////////////////////
219  // MasterObserver operations
220  //////////////////////////////////////////////////////////////////////////////////////////////////
221
222
223  public void preCreateNamespace(final NamespaceDescriptor ns) throws IOException {
224    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
225      @Override
226      public void call(MasterObserver observer) throws IOException {
227        observer.preCreateNamespace(this, ns);
228      }
229    });
230  }
231
232  public void postCreateNamespace(final NamespaceDescriptor ns) throws IOException {
233    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
234      @Override
235      public void call(MasterObserver observer) throws IOException {
236        observer.postCreateNamespace(this, ns);
237      }
238    });
239  }
240
241  public void preDeleteNamespace(final String namespaceName) throws IOException {
242    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
243      @Override
244      public void call(MasterObserver observer) throws IOException {
245        observer.preDeleteNamespace(this, namespaceName);
246      }
247    });
248  }
249
250  public void postDeleteNamespace(final String namespaceName) throws IOException {
251    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
252      @Override
253      public void call(MasterObserver observer) throws IOException {
254        observer.postDeleteNamespace(this, namespaceName);
255      }
256    });
257  }
258
259  public void preModifyNamespace(final NamespaceDescriptor currentNsDescriptor,
260    final NamespaceDescriptor newNsDescriptor) throws IOException {
261    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
262      @Override
263      public void call(MasterObserver observer) throws IOException {
264        observer.preModifyNamespace(this, currentNsDescriptor, newNsDescriptor);
265      }
266    });
267  }
268
269  public void postModifyNamespace(final NamespaceDescriptor oldNsDescriptor,
270    final NamespaceDescriptor currentNsDescriptor) throws IOException {
271    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
272      @Override
273      public void call(MasterObserver observer) throws IOException {
274        observer.postModifyNamespace(this, oldNsDescriptor, currentNsDescriptor);
275      }
276    });
277  }
278
279  public void preGetNamespaceDescriptor(final String namespaceName)
280      throws IOException {
281    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
282      @Override
283      public void call(MasterObserver observer) throws IOException {
284        observer.preGetNamespaceDescriptor(this, namespaceName);
285      }
286    });
287  }
288
289  public void postGetNamespaceDescriptor(final NamespaceDescriptor ns)
290      throws IOException {
291    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
292      @Override
293      public void call(MasterObserver observer) throws IOException {
294        observer.postGetNamespaceDescriptor(this, ns);
295      }
296    });
297  }
298
299  public void preListNamespaces(final List<String> namespaces) throws IOException {
300    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
301      @Override
302      public void call(MasterObserver oserver) throws IOException {
303        oserver.preListNamespaces(this, namespaces);
304      }
305    });
306  }
307
308  public void postListNamespaces(final List<String> namespaces) throws IOException {
309    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
310      @Override
311      public void call(MasterObserver oserver) throws IOException {
312        oserver.postListNamespaces(this, namespaces);
313      }
314    });
315  }
316
317  public void preListNamespaceDescriptors(final List<NamespaceDescriptor> descriptors)
318      throws IOException {
319    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
320      @Override
321      public void call(MasterObserver observer) throws IOException {
322        observer.preListNamespaceDescriptors(this, descriptors);
323      }
324    });
325  }
326
327  public void postListNamespaceDescriptors(final List<NamespaceDescriptor> descriptors)
328      throws IOException {
329    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
330      @Override
331      public void call(MasterObserver observer) throws IOException {
332        observer.postListNamespaceDescriptors(this, descriptors);
333      }
334    });
335  }
336
337  /* Implementation of hooks for invoking MasterObservers */
338
339  public TableDescriptor preCreateTableRegionsInfos(TableDescriptor desc) throws IOException {
340    if (coprocEnvironments.isEmpty()) {
341      return desc;
342    }
343    return execOperationWithResult(
344      new ObserverOperationWithResult<MasterObserver, TableDescriptor>(masterObserverGetter, desc) {
345
346        @Override
347        protected TableDescriptor call(MasterObserver observer) throws IOException {
348          return observer.preCreateTableRegionsInfos(this, getResult());
349        }
350      });
351  }
352
353  public void preCreateTable(final TableDescriptor htd, final RegionInfo[] regions)
354      throws IOException {
355    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
356      @Override
357      public void call(MasterObserver observer) throws IOException {
358        observer.preCreateTable(this, htd, regions);
359      }
360    });
361  }
362
363  public void postCreateTable(final TableDescriptor htd, final RegionInfo[] regions)
364      throws IOException {
365    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
366      @Override
367      public void call(MasterObserver observer) throws IOException {
368        observer.postCreateTable(this, htd, regions);
369      }
370    });
371  }
372
373  public void preCreateTableAction(final TableDescriptor htd, final RegionInfo[] regions,
374      final User user) throws IOException {
375    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
376      @Override
377      public void call(MasterObserver observer) throws IOException {
378        observer.preCreateTableAction(this, htd, regions);
379      }
380    });
381  }
382
383  public void postCompletedCreateTableAction(
384      final TableDescriptor htd, final RegionInfo[] regions, final User user) throws IOException {
385    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
386      @Override
387      public void call(MasterObserver observer) throws IOException {
388        observer.postCompletedCreateTableAction(this, htd, regions);
389      }
390    });
391  }
392
393  public void preDeleteTable(final TableName tableName) throws IOException {
394    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
395      @Override
396      public void call(MasterObserver observer) throws IOException {
397        observer.preDeleteTable(this, tableName);
398      }
399    });
400  }
401
402  public void postDeleteTable(final TableName tableName) throws IOException {
403    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
404      @Override
405      public void call(MasterObserver observer) throws IOException {
406        observer.postDeleteTable(this, tableName);
407      }
408    });
409  }
410
411  public void preDeleteTableAction(final TableName tableName, final User user) throws IOException {
412    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
413      @Override
414      public void call(MasterObserver observer) throws IOException {
415        observer.preDeleteTableAction(this, tableName);
416      }
417    });
418  }
419
420  public void postCompletedDeleteTableAction(final TableName tableName, final User user)
421      throws IOException {
422    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
423      @Override
424      public void call(MasterObserver observer) throws IOException {
425        observer.postCompletedDeleteTableAction(this, tableName);
426      }
427    });
428  }
429
430  public void preTruncateTable(final TableName tableName) throws IOException {
431    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
432      @Override
433      public void call(MasterObserver observer) throws IOException {
434        observer.preTruncateTable(this, tableName);
435      }
436    });
437  }
438
439  public void postTruncateTable(final TableName tableName) throws IOException {
440    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
441      @Override
442      public void call(MasterObserver observer) throws IOException {
443        observer.postTruncateTable(this, tableName);
444      }
445    });
446  }
447
448  public void preTruncateTableAction(final TableName tableName, final User user)
449      throws IOException {
450    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
451      @Override
452      public void call(MasterObserver observer) throws IOException {
453        observer.preTruncateTableAction(this, tableName);
454      }
455    });
456  }
457
458  public void postCompletedTruncateTableAction(final TableName tableName, final User user)
459      throws IOException {
460    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
461      @Override
462      public void call(MasterObserver observer) throws IOException {
463        observer.postCompletedTruncateTableAction(this, tableName);
464      }
465    });
466  }
467
468  public TableDescriptor preModifyTable(final TableName tableName,
469      final TableDescriptor currentDescriptor, final TableDescriptor newDescriptor)
470      throws IOException {
471    if (coprocEnvironments.isEmpty()) {
472      return newDescriptor;
473    }
474    return execOperationWithResult(new ObserverOperationWithResult<MasterObserver, TableDescriptor>(
475        masterObserverGetter, newDescriptor) {
476      @Override
477      protected TableDescriptor call(MasterObserver observer) throws IOException {
478        return observer.preModifyTable(this, tableName, currentDescriptor, getResult());
479      }
480    });
481  }
482
483  public void postModifyTable(final TableName tableName, final TableDescriptor oldDescriptor,
484    final TableDescriptor currentDescriptor) throws IOException {
485    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
486      @Override
487      public void call(MasterObserver observer) throws IOException {
488        observer.postModifyTable(this, tableName, oldDescriptor, currentDescriptor);
489      }
490    });
491  }
492
493  public void preModifyTableAction(final TableName tableName,
494    final TableDescriptor currentDescriptor, final TableDescriptor newDescriptor, final User user)
495    throws IOException {
496    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
497      @Override
498      public void call(MasterObserver observer) throws IOException {
499        observer.preModifyTableAction(this, tableName, currentDescriptor, newDescriptor);
500      }
501    });
502  }
503
504  public void postCompletedModifyTableAction(final TableName tableName,
505    final TableDescriptor oldDescriptor, final TableDescriptor currentDescriptor, final User user)
506    throws IOException {
507    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
508      @Override
509      public void call(MasterObserver observer) throws IOException {
510        observer.postCompletedModifyTableAction(this, tableName, oldDescriptor, currentDescriptor);
511      }
512    });
513  }
514
515  public void preEnableTable(final TableName tableName) throws IOException {
516    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
517      @Override
518      public void call(MasterObserver observer) throws IOException {
519        observer.preEnableTable(this, tableName);
520      }
521    });
522  }
523
524  public void postEnableTable(final TableName tableName) throws IOException {
525    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
526      @Override
527      public void call(MasterObserver observer) throws IOException {
528        observer.postEnableTable(this, tableName);
529      }
530    });
531  }
532
533  public void preEnableTableAction(final TableName tableName, final User user) throws IOException {
534    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
535      @Override
536      public void call(MasterObserver observer) throws IOException {
537        observer.preEnableTableAction(this, tableName);
538      }
539    });
540  }
541
542  public void postCompletedEnableTableAction(final TableName tableName, final User user)
543      throws IOException {
544    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
545      @Override
546      public void call(MasterObserver observer) throws IOException {
547        observer.postCompletedEnableTableAction(this, tableName);
548      }
549    });
550  }
551
552  public void preDisableTable(final TableName tableName) throws IOException {
553    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
554      @Override
555      public void call(MasterObserver observer) throws IOException {
556        observer.preDisableTable(this, tableName);
557      }
558    });
559  }
560
561  public void postDisableTable(final TableName tableName) throws IOException {
562    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
563      @Override
564      public void call(MasterObserver observer) throws IOException {
565        observer.postDisableTable(this, tableName);
566      }
567    });
568  }
569
570  public void preDisableTableAction(final TableName tableName, final User user) throws IOException {
571    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
572      @Override
573      public void call(MasterObserver observer) throws IOException {
574        observer.preDisableTableAction(this, tableName);
575      }
576    });
577  }
578
579  public void postCompletedDisableTableAction(final TableName tableName, final User user)
580      throws IOException {
581    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
582      @Override
583      public void call(MasterObserver observer) throws IOException {
584        observer.postCompletedDisableTableAction(this, tableName);
585      }
586    });
587  }
588
589  public void preAbortProcedure(
590      final ProcedureExecutor<MasterProcedureEnv> procEnv,
591      final long procId) throws IOException {
592    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
593      @Override
594      public void call(MasterObserver observer) throws IOException {
595        observer.preAbortProcedure(this,  procId);
596      }
597    });
598  }
599
600  public void postAbortProcedure() throws IOException {
601    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
602      @Override
603      public void call(MasterObserver observer) throws IOException {
604        observer.postAbortProcedure(this);
605      }
606    });
607  }
608
609  public void preGetProcedures() throws IOException {
610    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
611      @Override
612      public void call(MasterObserver observer) throws IOException {
613        observer.preGetProcedures(this);
614      }
615    });
616  }
617
618  public void postGetProcedures(final List<Procedure<?>> procInfoList) throws IOException {
619    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
620      @Override
621      public void call(MasterObserver observer) throws IOException {
622        observer.postGetProcedures(this);
623      }
624    });
625  }
626
627  public void preGetLocks() throws IOException {
628    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
629      @Override
630      public void call(MasterObserver observer) throws IOException {
631        observer.preGetLocks(this);
632      }
633    });
634  }
635
636  public void postGetLocks(final List<LockedResource> lockedResources) throws IOException {
637    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
638      @Override
639      public void call(MasterObserver observer) throws IOException {
640        observer.postGetLocks(this);
641      }
642    });
643  }
644
645  public void preMove(final RegionInfo region, final ServerName srcServer,
646      final ServerName destServer) throws IOException {
647    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
648      @Override
649      public void call(MasterObserver observer) throws IOException {
650        observer.preMove(this, region, srcServer, destServer);
651      }
652    });
653  }
654
655  public void postMove(final RegionInfo region, final ServerName srcServer,
656      final ServerName destServer) throws IOException {
657    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
658      @Override
659      public void call(MasterObserver observer) throws IOException {
660        observer.postMove(this, region, srcServer, destServer);
661      }
662    });
663  }
664
665  public void preAssign(final RegionInfo regionInfo) throws IOException {
666    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
667      @Override
668      public void call(MasterObserver observer) throws IOException {
669        observer.preAssign(this, regionInfo);
670      }
671    });
672  }
673
674  public void postAssign(final RegionInfo regionInfo) throws IOException {
675    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
676      @Override
677      public void call(MasterObserver observer) throws IOException {
678        observer.postAssign(this, regionInfo);
679      }
680    });
681  }
682
683  public void preUnassign(final RegionInfo regionInfo, final boolean force)
684      throws IOException {
685    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
686      @Override
687      public void call(MasterObserver observer) throws IOException {
688        observer.preUnassign(this, regionInfo, force);
689      }
690    });
691  }
692
693  public void postUnassign(final RegionInfo regionInfo, final boolean force) throws IOException {
694    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
695      @Override
696      public void call(MasterObserver observer) throws IOException {
697        observer.postUnassign(this, regionInfo, force);
698      }
699    });
700  }
701
702  public void preRegionOffline(final RegionInfo regionInfo) throws IOException {
703    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
704      @Override
705      public void call(MasterObserver observer) throws IOException {
706        observer.preRegionOffline(this, regionInfo);
707      }
708    });
709  }
710
711  public void postRegionOffline(final RegionInfo regionInfo) throws IOException {
712    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
713      @Override
714      public void call(MasterObserver observer) throws IOException {
715        observer.postRegionOffline(this, regionInfo);
716      }
717    });
718  }
719
720  public void preMergeRegions(final RegionInfo[] regionsToMerge)
721      throws IOException {
722    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
723      @Override
724      public void call(MasterObserver observer) throws IOException {
725        observer.preMergeRegions(this, regionsToMerge);
726      }
727    });
728  }
729
730  public void postMergeRegions(final RegionInfo[] regionsToMerge)
731      throws IOException {
732    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
733      @Override
734      public void call(MasterObserver observer) throws IOException {
735        observer.postMergeRegions(this, regionsToMerge);
736      }
737    });
738  }
739
740  public boolean preBalance() throws IOException {
741    return execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
742      @Override
743      public void call(MasterObserver observer) throws IOException {
744        observer.preBalance(this);
745      }
746    });
747  }
748
749  public void postBalance(final List<RegionPlan> plans) throws IOException {
750    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
751      @Override
752      public void call(MasterObserver observer) throws IOException {
753        observer.postBalance(this, plans);
754      }
755    });
756  }
757
758  public void preSetSplitOrMergeEnabled(final boolean newValue,
759      final MasterSwitchType switchType) throws IOException {
760    execOperation(coprocEnvironments.isEmpty()? null: new MasterObserverOperation() {
761      @Override
762      public void call(MasterObserver observer) throws IOException {
763        observer.preSetSplitOrMergeEnabled(this, newValue, switchType);
764      }
765    });
766  }
767
768  public void postSetSplitOrMergeEnabled(final boolean newValue,
769      final MasterSwitchType switchType) throws IOException {
770    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
771      @Override
772      public void call(MasterObserver observer) throws IOException {
773        observer.postSetSplitOrMergeEnabled(this, newValue, switchType);
774      }
775    });
776  }
777
778  /**
779   * Invoked just before calling the split region procedure
780   * @param tableName the table where the region belongs to
781   * @param splitRow the split point
782   * @throws IOException
783   */
784  public void preSplitRegion(
785      final TableName tableName,
786      final byte[] splitRow) throws IOException {
787    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
788      @Override
789      public void call(MasterObserver observer) throws IOException {
790        observer.preSplitRegion(this, tableName, splitRow);
791      }
792    });
793  }
794
795  /**
796   * Invoked just before a split
797   * @param tableName the table where the region belongs to
798   * @param splitRow the split point
799   * @param user the user
800   * @throws IOException
801   */
802  public void preSplitRegionAction(
803      final TableName tableName,
804      final byte[] splitRow,
805      final User user) throws IOException {
806    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
807      @Override
808      public void call(MasterObserver observer) throws IOException {
809        observer.preSplitRegionAction(this, tableName, splitRow);
810      }
811    });
812  }
813
814  /**
815   * Invoked just after a split
816   * @param regionInfoA the new left-hand daughter region
817   * @param regionInfoB the new right-hand daughter region
818   * @param user the user
819   * @throws IOException
820   */
821  public void postCompletedSplitRegionAction(
822      final RegionInfo regionInfoA,
823      final RegionInfo regionInfoB,
824      final User user) throws IOException {
825    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
826      @Override
827      public void call(MasterObserver observer) throws IOException {
828        observer.postCompletedSplitRegionAction(this, regionInfoA, regionInfoB);
829      }
830    });
831  }
832
833  /**
834   * This will be called before update META step as part of split table region procedure.
835   * @param splitKey
836   * @param metaEntries
837   * @param user the user
838   * @throws IOException
839   */
840  public void preSplitBeforeMETAAction(
841      final byte[] splitKey,
842      final List<Mutation> metaEntries,
843      final User user) throws IOException {
844    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
845      @Override
846      public void call(MasterObserver observer) throws IOException {
847        observer.preSplitRegionBeforeMETAAction(this, splitKey, metaEntries);
848      }
849    });
850  }
851
852  /**
853   * This will be called after update META step as part of split table region procedure.
854   * @param user the user
855   * @throws IOException
856   */
857  public void preSplitAfterMETAAction(final User user) throws IOException {
858    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
859      @Override
860      public void call(MasterObserver observer) throws IOException {
861        observer.preSplitRegionAfterMETAAction(this);
862      }
863    });
864  }
865
866  /**
867   * Invoked just after the rollback of a failed split
868   * @param user the user
869   * @throws IOException
870   */
871  public void postRollBackSplitRegionAction(final User user) throws IOException {
872    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
873      @Override
874      public void call(MasterObserver observer) throws IOException {
875        observer.postRollBackSplitRegionAction(this);
876      }
877    });
878  }
879
880  /**
881   * Invoked just before a merge
882   * @param regionsToMerge the regions to merge
883   * @param user the user
884   * @throws IOException
885   */
886  public void preMergeRegionsAction(
887      final RegionInfo[] regionsToMerge, final User user) throws IOException {
888    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
889      @Override
890      public void call(MasterObserver observer) throws IOException {
891        observer.preMergeRegionsAction(this, regionsToMerge);
892      }
893    });
894  }
895
896  /**
897   * Invoked after completing merge regions operation
898   * @param regionsToMerge the regions to merge
899   * @param mergedRegion the new merged region
900   * @param user the user
901   * @throws IOException
902   */
903  public void postCompletedMergeRegionsAction(
904      final RegionInfo[] regionsToMerge,
905      final RegionInfo mergedRegion,
906      final User user) throws IOException {
907    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
908      @Override
909      public void call(MasterObserver observer) throws IOException {
910        observer.postCompletedMergeRegionsAction(this, regionsToMerge, mergedRegion);
911      }
912    });
913  }
914
915  /**
916   * Invoked before merge regions operation writes the new region to hbase:meta
917   * @param regionsToMerge the regions to merge
918   * @param metaEntries the meta entry
919   * @param user the user
920   * @throws IOException
921   */
922  public void preMergeRegionsCommit(
923      final RegionInfo[] regionsToMerge,
924      final @MetaMutationAnnotation List<Mutation> metaEntries,
925      final User user) throws IOException {
926    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
927      @Override
928      public void call(MasterObserver observer) throws IOException {
929        observer.preMergeRegionsCommitAction(this, regionsToMerge, metaEntries);
930      }
931    });
932  }
933
934  /**
935   * Invoked after merge regions operation writes the new region to hbase:meta
936   * @param regionsToMerge the regions to merge
937   * @param mergedRegion the new merged region
938   * @param user the user
939   * @throws IOException
940   */
941  public void postMergeRegionsCommit(
942      final RegionInfo[] regionsToMerge,
943      final RegionInfo mergedRegion,
944      final User user) throws IOException {
945    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
946      @Override
947      public void call(MasterObserver observer) throws IOException {
948        observer.postMergeRegionsCommitAction(this, regionsToMerge, mergedRegion);
949      }
950    });
951  }
952
953  /**
954   * Invoked after rollback merge regions operation
955   * @param regionsToMerge the regions to merge
956   * @param user the user
957   * @throws IOException
958   */
959  public void postRollBackMergeRegionsAction(
960      final RegionInfo[] regionsToMerge, final User user) throws IOException {
961    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
962      @Override
963      public void call(MasterObserver observer) throws IOException {
964        observer.postRollBackMergeRegionsAction(this, regionsToMerge);
965      }
966    });
967  }
968
969  // This hook allows Coprocessor change value of balance switch.
970  public void preBalanceSwitch(final boolean b) throws IOException {
971    if (this.coprocEnvironments.isEmpty()) {
972      return;
973    }
974    execOperation(new MasterObserverOperation() {
975      @Override
976      public void call(MasterObserver observer) throws IOException {
977        observer.preBalanceSwitch(this, b);
978      }
979    });
980  }
981
982  public void postBalanceSwitch(final boolean oldValue, final boolean newValue)
983      throws IOException {
984    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
985      @Override
986      public void call(MasterObserver observer) throws IOException {
987        observer.postBalanceSwitch(this, oldValue, newValue);
988      }
989    });
990  }
991
992  public void preShutdown() throws IOException {
993    // While stopping the cluster all coprocessors method should be executed first then the
994    // coprocessor should be cleaned up.
995    if (coprocEnvironments.isEmpty()) {
996      return;
997    }
998    execShutdown(new MasterObserverOperation() {
999      @Override
1000      public void call(MasterObserver observer) throws IOException {
1001        observer.preShutdown(this);
1002      }
1003      @Override
1004      public void postEnvCall() {
1005        // invoke coprocessor stop method
1006        shutdown(this.getEnvironment());
1007      }
1008    });
1009  }
1010
1011  public void preStopMaster() throws IOException {
1012    // While stopping master all coprocessors method should be executed first then the coprocessor
1013    // environment should be cleaned up.
1014    if (coprocEnvironments.isEmpty()) {
1015      return;
1016    }
1017    execShutdown(new MasterObserverOperation() {
1018      @Override
1019      public void call(MasterObserver observer) throws IOException {
1020        observer.preStopMaster(this);
1021      }
1022      @Override
1023      public void postEnvCall() {
1024        // invoke coprocessor stop method
1025        shutdown(this.getEnvironment());
1026      }
1027    });
1028  }
1029
1030  public void preMasterInitialization() throws IOException {
1031    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1032      @Override
1033      public void call(MasterObserver observer) throws IOException {
1034        observer.preMasterInitialization(this);
1035      }
1036    });
1037  }
1038
1039  public void postStartMaster() throws IOException {
1040    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1041      @Override
1042      public void call(MasterObserver observer) throws IOException {
1043        observer.postStartMaster(this);
1044      }
1045    });
1046  }
1047
1048  public void preSnapshot(final SnapshotDescription snapshot,
1049      final TableDescriptor hTableDescriptor) throws IOException {
1050    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1051      @Override
1052      public void call(MasterObserver observer) throws IOException {
1053        observer.preSnapshot(this, snapshot, hTableDescriptor);
1054      }
1055    });
1056  }
1057
1058  public void postSnapshot(final SnapshotDescription snapshot,
1059      final TableDescriptor hTableDescriptor) throws IOException {
1060    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1061      @Override
1062      public void call(MasterObserver observer) throws IOException {
1063        observer.postSnapshot(this, snapshot, hTableDescriptor);
1064      }
1065    });
1066  }
1067
1068  public void postCompletedSnapshotAction(SnapshotDescription snapshot,
1069      TableDescriptor hTableDescriptor) throws IOException {
1070    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1071      @Override
1072      public void call(MasterObserver observer) throws IOException {
1073        observer.postCompletedSnapshotAction(this, snapshot, hTableDescriptor);
1074      }
1075    });
1076  }
1077
1078  public void preListSnapshot(final SnapshotDescription snapshot) throws IOException {
1079    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1080      @Override
1081      public void call(MasterObserver observer) throws IOException {
1082        observer.preListSnapshot(this, snapshot);
1083      }
1084    });
1085  }
1086
1087  public void postListSnapshot(final SnapshotDescription snapshot) throws IOException {
1088    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1089      @Override
1090      public void call(MasterObserver observer) throws IOException {
1091        observer.postListSnapshot(this, snapshot);
1092      }
1093    });
1094  }
1095
1096  public void preCloneSnapshot(final SnapshotDescription snapshot,
1097      final TableDescriptor hTableDescriptor) throws IOException {
1098    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1099      @Override
1100      public void call(MasterObserver observer) throws IOException {
1101        observer.preCloneSnapshot(this, snapshot, hTableDescriptor);
1102      }
1103    });
1104  }
1105
1106  public void postCloneSnapshot(final SnapshotDescription snapshot,
1107      final TableDescriptor hTableDescriptor) throws IOException {
1108    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1109      @Override
1110      public void call(MasterObserver observer) throws IOException {
1111        observer.postCloneSnapshot(this, snapshot, hTableDescriptor);
1112      }
1113    });
1114  }
1115
1116  public void preRestoreSnapshot(final SnapshotDescription snapshot,
1117      final TableDescriptor hTableDescriptor) throws IOException {
1118    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1119      @Override
1120      public void call(MasterObserver observer) throws IOException {
1121        observer.preRestoreSnapshot(this, snapshot, hTableDescriptor);
1122      }
1123    });
1124  }
1125
1126  public void postRestoreSnapshot(final SnapshotDescription snapshot,
1127      final TableDescriptor hTableDescriptor) throws IOException {
1128    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1129      @Override
1130      public void call(MasterObserver observer) throws IOException {
1131        observer.postRestoreSnapshot(this, snapshot, hTableDescriptor);
1132      }
1133    });
1134  }
1135
1136  public void preDeleteSnapshot(final SnapshotDescription snapshot) throws IOException {
1137    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1138      @Override
1139      public void call(MasterObserver observer) throws IOException {
1140        observer.preDeleteSnapshot(this, snapshot);
1141      }
1142    });
1143  }
1144
1145  public void postDeleteSnapshot(final SnapshotDescription snapshot) throws IOException {
1146    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1147      @Override
1148      public void call(MasterObserver observer) throws IOException {
1149        observer.postDeleteSnapshot(this, snapshot);
1150      }
1151    });
1152  }
1153
1154  public void preGetTableDescriptors(final List<TableName> tableNamesList,
1155      final List<TableDescriptor> descriptors, final String regex) throws IOException {
1156    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1157      @Override
1158      public void call(MasterObserver observer) throws IOException {
1159        observer.preGetTableDescriptors(this, tableNamesList, descriptors, regex);
1160      }
1161    });
1162  }
1163
1164  public void postGetTableDescriptors(final List<TableName> tableNamesList,
1165      final List<TableDescriptor> descriptors, final String regex) throws IOException {
1166    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1167      @Override
1168      public void call(MasterObserver observer) throws IOException {
1169        observer.postGetTableDescriptors(this, tableNamesList, descriptors, regex);
1170      }
1171    });
1172  }
1173
1174  public void preGetTableNames(final List<TableDescriptor> descriptors,
1175      final String regex) throws IOException {
1176    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1177      @Override
1178      public void call(MasterObserver observer) throws IOException {
1179        observer.preGetTableNames(this, descriptors, regex);
1180      }
1181    });
1182  }
1183
1184  public void postGetTableNames(final List<TableDescriptor> descriptors,
1185      final String regex) throws IOException {
1186    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1187      @Override
1188      public void call(MasterObserver observer) throws IOException {
1189        observer.postGetTableNames(this, descriptors, regex);
1190      }
1191    });
1192  }
1193
1194  public void preTableFlush(final TableName tableName) throws IOException {
1195    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1196      @Override
1197      public void call(MasterObserver observer) throws IOException {
1198        observer.preTableFlush(this, tableName);
1199      }
1200    });
1201  }
1202
1203  public void postTableFlush(final TableName tableName) throws IOException {
1204    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1205      @Override
1206      public void call(MasterObserver observer) throws IOException {
1207        observer.postTableFlush(this, tableName);
1208      }
1209    });
1210  }
1211
1212  public void preSetUserQuota(
1213      final String user, final GlobalQuotaSettings quotas) throws IOException {
1214    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1215      @Override
1216      public void call(MasterObserver observer) throws IOException {
1217        observer.preSetUserQuota(this, user, quotas);
1218      }
1219    });
1220  }
1221
1222  public void postSetUserQuota(
1223      final String user, final GlobalQuotaSettings quotas) throws IOException {
1224    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1225      @Override
1226      public void call(MasterObserver observer) throws IOException {
1227        observer.postSetUserQuota(this, user, quotas);
1228      }
1229    });
1230  }
1231
1232  public void preSetUserQuota(
1233      final String user, final TableName table, final GlobalQuotaSettings quotas)
1234          throws IOException {
1235    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1236      @Override
1237      public void call(MasterObserver observer) throws IOException {
1238        observer.preSetUserQuota(this, user, table, quotas);
1239      }
1240    });
1241  }
1242
1243  public void postSetUserQuota(
1244      final String user, final TableName table, final GlobalQuotaSettings quotas)
1245          throws IOException {
1246    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1247      @Override
1248      public void call(MasterObserver observer) throws IOException {
1249        observer.postSetUserQuota(this, user, table, quotas);
1250      }
1251    });
1252  }
1253
1254  public void preSetUserQuota(
1255      final String user, final String namespace, final GlobalQuotaSettings quotas)
1256          throws IOException {
1257    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1258      @Override
1259      public void call(MasterObserver observer) throws IOException {
1260        observer.preSetUserQuota(this, user, namespace, quotas);
1261      }
1262    });
1263  }
1264
1265  public void postSetUserQuota(
1266      final String user, final String namespace, final GlobalQuotaSettings quotas)
1267          throws IOException {
1268    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1269      @Override
1270      public void call(MasterObserver observer) throws IOException {
1271        observer.postSetUserQuota(this, user, namespace, quotas);
1272      }
1273    });
1274  }
1275
1276  public void preSetTableQuota(
1277      final TableName table, final GlobalQuotaSettings quotas) throws IOException {
1278    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1279      @Override
1280      public void call(MasterObserver observer) throws IOException {
1281        observer.preSetTableQuota(this, table, quotas);
1282      }
1283    });
1284  }
1285
1286  public void postSetTableQuota(
1287      final TableName table, final GlobalQuotaSettings quotas) throws IOException {
1288    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1289      @Override
1290      public void call(MasterObserver observer) throws IOException {
1291        observer.postSetTableQuota(this, table, quotas);
1292      }
1293    });
1294  }
1295
1296  public void preSetNamespaceQuota(
1297      final String namespace, final GlobalQuotaSettings quotas) throws IOException {
1298    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1299      @Override
1300      public void call(MasterObserver observer) throws IOException {
1301        observer.preSetNamespaceQuota(this, namespace, quotas);
1302      }
1303    });
1304  }
1305
1306  public void postSetNamespaceQuota(
1307      final String namespace, final GlobalQuotaSettings quotas) throws IOException{
1308    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1309      @Override
1310      public void call(MasterObserver observer) throws IOException {
1311        observer.postSetNamespaceQuota(this, namespace, quotas);
1312      }
1313    });
1314  }
1315
1316  public void preSetRegionServerQuota(final String regionServer, final GlobalQuotaSettings quotas)
1317      throws IOException {
1318    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1319      @Override
1320      public void call(MasterObserver observer) throws IOException {
1321        observer.preSetRegionServerQuota(this, regionServer, quotas);
1322      }
1323    });
1324  }
1325
1326  public void postSetRegionServerQuota(final String regionServer, final GlobalQuotaSettings quotas)
1327      throws IOException {
1328    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1329      @Override
1330      public void call(MasterObserver observer) throws IOException {
1331        observer.postSetRegionServerQuota(this, regionServer, quotas);
1332      }
1333    });
1334  }
1335
1336  public void preMoveServersAndTables(final Set<Address> servers, final Set<TableName> tables,
1337      final String targetGroup) throws IOException {
1338    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1339      @Override
1340      public void call(MasterObserver observer) throws IOException {
1341        observer.preMoveServersAndTables(this, servers, tables, targetGroup);
1342      }
1343    });
1344  }
1345
1346  public void postMoveServersAndTables(final Set<Address> servers, final Set<TableName> tables,
1347      final String targetGroup) throws IOException {
1348    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1349      @Override
1350      public void call(MasterObserver observer) throws IOException {
1351        observer.postMoveServersAndTables(this, servers, tables, targetGroup);
1352      }
1353    });
1354  }
1355
1356  public void preMoveServers(final Set<Address> servers, final String targetGroup)
1357      throws IOException {
1358    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1359      @Override
1360      public void call(MasterObserver observer) throws IOException {
1361        observer.preMoveServers(this, servers, targetGroup);
1362      }
1363    });
1364  }
1365
1366  public void postMoveServers(final Set<Address> servers, final String targetGroup)
1367      throws IOException {
1368    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1369      @Override
1370      public void call(MasterObserver observer) throws IOException {
1371        observer.postMoveServers(this, servers, targetGroup);
1372      }
1373    });
1374  }
1375
1376  public void preMoveTables(final Set<TableName> tables, final String targetGroup)
1377      throws IOException {
1378    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1379      @Override
1380      public void call(MasterObserver observer) throws IOException {
1381        observer.preMoveTables(this, tables, targetGroup);
1382      }
1383    });
1384  }
1385
1386  public void postMoveTables(final Set<TableName> tables, final String targetGroup)
1387      throws IOException {
1388    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1389      @Override
1390      public void call(MasterObserver observer) throws IOException {
1391        observer.postMoveTables(this, tables, targetGroup);
1392      }
1393    });
1394  }
1395
1396  public void preAddRSGroup(final String name)
1397      throws IOException {
1398    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1399      @Override
1400      public void call(MasterObserver observer) throws IOException {
1401        observer.preAddRSGroup(this, name);
1402      }
1403    });
1404  }
1405
1406  public void postAddRSGroup(final String name)
1407      throws IOException {
1408    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1409      @Override
1410      public void call(MasterObserver observer) throws IOException {
1411        observer.postAddRSGroup(this, name);
1412      }
1413    });
1414  }
1415
1416  public void preRemoveRSGroup(final String name)
1417      throws IOException {
1418    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1419      @Override
1420      public void call(MasterObserver observer) throws IOException {
1421        observer.preRemoveRSGroup(this, name);
1422      }
1423    });
1424  }
1425
1426  public void postRemoveRSGroup(final String name)
1427      throws IOException {
1428    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1429      @Override
1430      public void call(MasterObserver observer) throws IOException {
1431        observer.postRemoveRSGroup(this, name);
1432      }
1433    });
1434  }
1435
1436  public void preBalanceRSGroup(final String name)
1437      throws IOException {
1438    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1439      @Override
1440      public void call(MasterObserver observer) throws IOException {
1441        observer.preBalanceRSGroup(this, name);
1442      }
1443    });
1444  }
1445
1446  public void postBalanceRSGroup(final String name, final boolean balanceRan)
1447      throws IOException {
1448    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1449      @Override
1450      public void call(MasterObserver observer) throws IOException {
1451        observer.postBalanceRSGroup(this, name, balanceRan);
1452      }
1453    });
1454  }
1455
1456  public void preRenameRSGroup(final String oldName, final String newName)
1457      throws IOException {
1458    execOperation(coprocEnvironments.isEmpty() ? null: new MasterObserverOperation() {
1459      @Override
1460      public void call(MasterObserver observer) throws IOException {
1461        observer.preRenameRSGroup(this, oldName, newName);
1462      }
1463    });
1464  }
1465
1466  public void postRenameRSGroup(final String oldName, final String newName)
1467      throws IOException {
1468    execOperation(coprocEnvironments.isEmpty() ? null: new MasterObserverOperation() {
1469      @Override
1470      public void call(MasterObserver observer) throws IOException {
1471        observer.postRenameRSGroup(this, oldName, newName);
1472      }
1473    });
1474  }
1475
1476  public void preRemoveServers(final Set<Address> servers)
1477      throws IOException {
1478    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1479      @Override
1480      public void call(MasterObserver observer) throws IOException {
1481        observer.preRemoveServers(this, servers);
1482      }
1483    });
1484  }
1485
1486  public void postRemoveServers(final Set<Address> servers)
1487      throws IOException {
1488    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1489      @Override
1490      public void call(MasterObserver observer) throws IOException {
1491        observer.postRemoveServers(this, servers);
1492      }
1493    });
1494  }
1495
1496  public void preAddReplicationPeer(final String peerId, final ReplicationPeerConfig peerConfig)
1497      throws IOException {
1498    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1499      @Override
1500      public void call(MasterObserver observer) throws IOException {
1501        observer.preAddReplicationPeer(this, peerId, peerConfig);
1502      }
1503    });
1504  }
1505
1506  public void postAddReplicationPeer(final String peerId, final ReplicationPeerConfig peerConfig)
1507      throws IOException {
1508    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1509      @Override
1510      public void call(MasterObserver observer) throws IOException {
1511        observer.postAddReplicationPeer(this, peerId, peerConfig);
1512      }
1513    });
1514  }
1515
1516  public void preRemoveReplicationPeer(final String peerId) throws IOException {
1517    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1518      @Override
1519      public void call(MasterObserver observer) throws IOException {
1520        observer.preRemoveReplicationPeer(this, peerId);
1521      }
1522    });
1523  }
1524
1525  public void postRemoveReplicationPeer(final String peerId) throws IOException {
1526    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1527      @Override
1528      public void call(MasterObserver observer) throws IOException {
1529        observer.postRemoveReplicationPeer(this, peerId);
1530      }
1531    });
1532  }
1533
1534  public void preEnableReplicationPeer(final String peerId) throws IOException {
1535    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1536      @Override
1537      public void call(MasterObserver observer) throws IOException {
1538        observer.preEnableReplicationPeer(this, peerId);
1539      }
1540    });
1541  }
1542
1543  public void postEnableReplicationPeer(final String peerId) throws IOException {
1544    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1545      @Override
1546      public void call(MasterObserver observer) throws IOException {
1547        observer.postEnableReplicationPeer(this, peerId);
1548      }
1549    });
1550  }
1551
1552  public void preDisableReplicationPeer(final String peerId) throws IOException {
1553    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1554      @Override
1555      public void call(MasterObserver observer) throws IOException {
1556        observer.preDisableReplicationPeer(this, peerId);
1557      }
1558    });
1559  }
1560
1561  public void postDisableReplicationPeer(final String peerId) throws IOException {
1562    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1563      @Override
1564      public void call(MasterObserver observer) throws IOException {
1565        observer.postDisableReplicationPeer(this, peerId);
1566      }
1567    });
1568  }
1569
1570  public void preGetReplicationPeerConfig(final String peerId) throws IOException {
1571    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1572      @Override
1573      public void call(MasterObserver observer) throws IOException {
1574        observer.preGetReplicationPeerConfig(this, peerId);
1575      }
1576    });
1577  }
1578
1579  public void postGetReplicationPeerConfig(final String peerId) throws IOException {
1580    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1581      @Override
1582      public void call(MasterObserver observer) throws IOException {
1583        observer.postGetReplicationPeerConfig(this, peerId);
1584      }
1585    });
1586  }
1587
1588  public void preUpdateReplicationPeerConfig(final String peerId,
1589      final ReplicationPeerConfig peerConfig) throws IOException {
1590    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1591      @Override
1592      public void call(MasterObserver observer) throws IOException {
1593        observer.preUpdateReplicationPeerConfig(this, peerId, peerConfig);
1594      }
1595    });
1596  }
1597
1598  public void postUpdateReplicationPeerConfig(final String peerId,
1599      final ReplicationPeerConfig peerConfig) throws IOException {
1600    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1601      @Override
1602      public void call(MasterObserver observer) throws IOException {
1603        observer.postUpdateReplicationPeerConfig(this, peerId, peerConfig);
1604      }
1605    });
1606  }
1607
1608  public void preListReplicationPeers(final String regex) throws IOException {
1609    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1610      @Override
1611      public void call(MasterObserver observer) throws IOException {
1612        observer.preListReplicationPeers(this, regex);
1613      }
1614    });
1615  }
1616
1617  public void postListReplicationPeers(final String regex) throws IOException {
1618    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1619      @Override
1620      public void call(MasterObserver observer) throws IOException {
1621        observer.postListReplicationPeers(this, regex);
1622      }
1623    });
1624  }
1625
1626  public void preRequestLock(String namespace, TableName tableName, RegionInfo[] regionInfos,
1627      LockType type, String description) throws IOException {
1628    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1629      @Override
1630      public void call(MasterObserver observer) throws IOException {
1631        observer.preRequestLock(this, namespace, tableName, regionInfos, description);
1632      }
1633    });
1634  }
1635
1636  public void postRequestLock(String namespace, TableName tableName, RegionInfo[] regionInfos,
1637      LockType type, String description) throws IOException {
1638    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1639      @Override
1640      public void call(MasterObserver observer) throws IOException {
1641        observer.postRequestLock(this, namespace, tableName, regionInfos, description);
1642      }
1643    });
1644  }
1645
1646  public void preLockHeartbeat(LockProcedure proc, boolean keepAlive) throws IOException {
1647    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1648      @Override
1649      public void call(MasterObserver observer) throws IOException {
1650        observer.preLockHeartbeat(this, proc.getTableName(), proc.getDescription());
1651      }
1652    });
1653  }
1654
1655  public void postLockHeartbeat(LockProcedure proc, boolean keepAlive) throws IOException {
1656    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1657      @Override
1658      public void call(MasterObserver observer) throws IOException {
1659        observer.postLockHeartbeat(this);
1660      }
1661    });
1662  }
1663
1664  public void preGetClusterMetrics() throws IOException {
1665    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1666      @Override
1667      public void call(MasterObserver observer) throws IOException {
1668        observer.preGetClusterMetrics(this);
1669      }
1670    });
1671  }
1672
1673  public void postGetClusterMetrics(ClusterMetrics status) throws IOException {
1674    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1675      @Override
1676      public void call(MasterObserver observer) throws IOException {
1677        observer.postGetClusterMetrics(this, status);
1678      }
1679    });
1680  }
1681
1682  public void preClearDeadServers() throws IOException {
1683    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1684      @Override
1685      public void call(MasterObserver observer) throws IOException {
1686        observer.preClearDeadServers(this);
1687      }
1688    });
1689  }
1690
1691  public void postClearDeadServers(List<ServerName> servers,
1692      List<ServerName> notClearedServers) throws IOException {
1693    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1694      @Override
1695      public void call(MasterObserver observer) throws IOException {
1696        observer.postClearDeadServers(this, servers, notClearedServers);
1697      }
1698    });
1699  }
1700
1701  public void preDecommissionRegionServers(List<ServerName> servers, boolean offload) throws IOException {
1702    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1703      @Override
1704      public void call(MasterObserver observer) throws IOException {
1705        observer.preDecommissionRegionServers(this, servers, offload);
1706      }
1707    });
1708  }
1709
1710  public void postDecommissionRegionServers(List<ServerName> servers, boolean offload) throws IOException {
1711    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1712      @Override
1713      public void call(MasterObserver observer) throws IOException {
1714        observer.postDecommissionRegionServers(this, servers, offload);
1715      }
1716    });
1717  }
1718
1719  public void preListDecommissionedRegionServers() throws IOException {
1720    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1721      @Override
1722      public void call(MasterObserver observer) throws IOException {
1723        observer.preListDecommissionedRegionServers(this);
1724      }
1725    });
1726  }
1727
1728  public void postListDecommissionedRegionServers() throws IOException {
1729    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1730      @Override
1731      public void call(MasterObserver observer) throws IOException {
1732        observer.postListDecommissionedRegionServers(this);
1733      }
1734    });
1735  }
1736
1737  public void preRecommissionRegionServer(ServerName server, List<byte[]> encodedRegionNames)
1738      throws IOException {
1739    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1740      @Override
1741      public void call(MasterObserver observer) throws IOException {
1742        observer.preRecommissionRegionServer(this, server, encodedRegionNames);
1743      }
1744    });
1745  }
1746
1747  public void postRecommissionRegionServer(ServerName server, List<byte[]> encodedRegionNames)
1748      throws IOException {
1749    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1750      @Override
1751      public void call(MasterObserver observer) throws IOException {
1752        observer.postRecommissionRegionServer(this, server, encodedRegionNames);
1753      }
1754    });
1755  }
1756
1757  public void preSwitchRpcThrottle(boolean enable) throws IOException {
1758    execOperation(coprocEnvironments.isEmpty() ? null :new MasterObserverOperation() {
1759      @Override
1760      public void call(MasterObserver observer) throws IOException {
1761        observer.preSwitchRpcThrottle(this, enable);
1762      }
1763    });
1764  }
1765
1766  public void postSwitchRpcThrottle(final boolean oldValue, final boolean newValue)
1767      throws IOException {
1768    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1769      @Override
1770      public void call(MasterObserver observer) throws IOException {
1771        observer.postSwitchRpcThrottle(this, oldValue, newValue);
1772      }
1773    });
1774  }
1775
1776  public void preIsRpcThrottleEnabled() throws IOException {
1777    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1778      @Override
1779      public void call(MasterObserver observer) throws IOException {
1780        observer.preIsRpcThrottleEnabled(this);
1781      }
1782    });
1783  }
1784
1785  public void postIsRpcThrottleEnabled(boolean enabled) throws IOException {
1786    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1787      @Override
1788      public void call(MasterObserver observer) throws IOException {
1789        observer.postIsRpcThrottleEnabled(this, enabled);
1790      }
1791    });
1792  }
1793
1794  public void preSwitchExceedThrottleQuota(boolean enable) throws IOException {
1795    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1796      @Override
1797      public void call(MasterObserver observer) throws IOException {
1798        observer.preSwitchExceedThrottleQuota(this, enable);
1799      }
1800    });
1801  }
1802
1803  public void postSwitchExceedThrottleQuota(final boolean oldValue, final boolean newValue)
1804      throws IOException {
1805    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1806      @Override
1807      public void call(MasterObserver observer) throws IOException {
1808        observer.postSwitchExceedThrottleQuota(this, oldValue, newValue);
1809      }
1810    });
1811  }
1812
1813  public void preGrant(UserPermission userPermission, boolean mergeExistingPermissions)
1814      throws IOException {
1815    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1816      @Override
1817      public void call(MasterObserver observer) throws IOException {
1818        observer.preGrant(this, userPermission, mergeExistingPermissions);
1819      }
1820    });
1821  }
1822
1823  public void postGrant(UserPermission userPermission, boolean mergeExistingPermissions)
1824      throws IOException {
1825    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1826      @Override
1827      public void call(MasterObserver observer) throws IOException {
1828        observer.postGrant(this, userPermission, mergeExistingPermissions);
1829      }
1830    });
1831  }
1832
1833  public void preRevoke(UserPermission userPermission) throws IOException {
1834    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1835      @Override
1836      public void call(MasterObserver observer) throws IOException {
1837        observer.preRevoke(this, userPermission);
1838      }
1839    });
1840  }
1841
1842  public void postRevoke(UserPermission userPermission) throws IOException {
1843    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1844      @Override
1845      public void call(MasterObserver observer) throws IOException {
1846        observer.postRevoke(this, userPermission);
1847      }
1848    });
1849  }
1850
1851  public void preGetUserPermissions(String userName, String namespace, TableName tableName,
1852      byte[] family, byte[] qualifier) throws IOException {
1853    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1854      @Override
1855      public void call(MasterObserver observer) throws IOException {
1856        observer.preGetUserPermissions(this, userName, namespace, tableName, family, qualifier);
1857      }
1858    });
1859  }
1860
1861  public void postGetUserPermissions(String userName, String namespace, TableName tableName,
1862      byte[] family, byte[] qualifier) throws IOException {
1863    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1864      @Override
1865      public void call(MasterObserver observer) throws IOException {
1866        observer.postGetUserPermissions(this, userName, namespace, tableName, family, qualifier);
1867      }
1868    });
1869  }
1870
1871  public void preHasUserPermissions(String userName, List<Permission> permissions)
1872      throws IOException {
1873    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1874      @Override
1875      public void call(MasterObserver observer) throws IOException {
1876        observer.preHasUserPermissions(this, userName, permissions);
1877      }
1878    });
1879  }
1880
1881  public void postHasUserPermissions(String userName, List<Permission> permissions)
1882      throws IOException {
1883    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1884      @Override
1885      public void call(MasterObserver observer) throws IOException {
1886        observer.postHasUserPermissions(this, userName, permissions);
1887      }
1888    });
1889  }
1890}