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 */
018package org.apache.hadoop.hbase.master;
019
020import java.io.IOException;
021import java.lang.reflect.InvocationTargetException;
022import java.util.List;
023import java.util.Map;
024import java.util.Set;
025import org.apache.hadoop.conf.Configuration;
026import org.apache.hadoop.hbase.ClusterMetrics;
027import org.apache.hadoop.hbase.MetaMutationAnnotation;
028import org.apache.hadoop.hbase.NamespaceDescriptor;
029import org.apache.hadoop.hbase.ServerName;
030import org.apache.hadoop.hbase.TableName;
031import org.apache.hadoop.hbase.client.BalanceRequest;
032import org.apache.hadoop.hbase.client.BalanceResponse;
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.SharedConnection;
038import org.apache.hadoop.hbase.client.SnapshotDescription;
039import org.apache.hadoop.hbase.client.TableDescriptor;
040import org.apache.hadoop.hbase.coprocessor.BaseEnvironment;
041import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
042import org.apache.hadoop.hbase.coprocessor.CoreCoprocessor;
043import org.apache.hadoop.hbase.coprocessor.HasMasterServices;
044import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
045import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
046import org.apache.hadoop.hbase.coprocessor.MasterObserver;
047import org.apache.hadoop.hbase.coprocessor.MetricsCoprocessor;
048import org.apache.hadoop.hbase.master.locking.LockProcedure;
049import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
050import org.apache.hadoop.hbase.metrics.MetricRegistry;
051import org.apache.hadoop.hbase.net.Address;
052import org.apache.hadoop.hbase.procedure2.LockType;
053import org.apache.hadoop.hbase.procedure2.LockedResource;
054import org.apache.hadoop.hbase.procedure2.Procedure;
055import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
056import org.apache.hadoop.hbase.quotas.GlobalQuotaSettings;
057import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
058import org.apache.hadoop.hbase.replication.SyncReplicationState;
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
066import org.apache.hbase.thirdparty.com.google.protobuf.Service;
067
068/**
069 * Provides the coprocessor framework and environment for master oriented operations.
070 * {@link HMaster} interacts with the loaded coprocessors through this class.
071 */
072@InterfaceAudience.Private
073public class MasterCoprocessorHost
074  extends CoprocessorHost<MasterCoprocessor, MasterCoprocessorEnvironment> {
075
076  private static final Logger LOG = LoggerFactory.getLogger(MasterCoprocessorHost.class);
077
078  /**
079   * Coprocessor environment extension providing access to master related 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 =
155      conf.getBoolean(COPROCESSORS_ENABLED_CONF_KEY, 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 {
180        LOG.error("{} is not of type MasterCoprocessor. Check the configuration of {}",
181          implClass.getName(), CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY);
182        return null;
183      }
184    } catch (NoSuchMethodException | InvocationTargetException e) {
185      throw (InstantiationException) new InstantiationException(implClass.getName()).initCause(e);
186    }
187  }
188
189  private ObserverGetter<MasterCoprocessor, MasterObserver> masterObserverGetter =
190    MasterCoprocessor::getMasterObserver;
191
192  abstract class MasterObserverOperation extends ObserverOperationWithoutResult<MasterObserver> {
193    public MasterObserverOperation() {
194      super(masterObserverGetter);
195    }
196
197    public MasterObserverOperation(boolean bypassable) {
198      this(null, bypassable);
199    }
200
201    public MasterObserverOperation(User user) {
202      super(masterObserverGetter, user);
203    }
204
205    public MasterObserverOperation(User user, boolean bypassable) {
206      super(masterObserverGetter, user, bypassable);
207    }
208  }
209
210  //////////////////////////////////////////////////////////////////////////////////////////////////
211  // MasterObserver operations
212  //////////////////////////////////////////////////////////////////////////////////////////////////
213
214  public void preCreateNamespace(final NamespaceDescriptor ns) throws IOException {
215    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
216      @Override
217      public void call(MasterObserver observer) throws IOException {
218        observer.preCreateNamespace(this, ns);
219      }
220    });
221  }
222
223  public void postCreateNamespace(final NamespaceDescriptor ns) throws IOException {
224    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
225      @Override
226      public void call(MasterObserver observer) throws IOException {
227        observer.postCreateNamespace(this, ns);
228      }
229    });
230  }
231
232  public void preDeleteNamespace(final String namespaceName) throws IOException {
233    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
234      @Override
235      public void call(MasterObserver observer) throws IOException {
236        observer.preDeleteNamespace(this, namespaceName);
237      }
238    });
239  }
240
241  public void postDeleteNamespace(final String namespaceName) throws IOException {
242    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
243      @Override
244      public void call(MasterObserver observer) throws IOException {
245        observer.postDeleteNamespace(this, namespaceName);
246      }
247    });
248  }
249
250  public void preModifyNamespace(final NamespaceDescriptor currentNsDescriptor,
251    final NamespaceDescriptor newNsDescriptor) throws IOException {
252    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
253      @Override
254      public void call(MasterObserver observer) throws IOException {
255        observer.preModifyNamespace(this, currentNsDescriptor, newNsDescriptor);
256      }
257    });
258  }
259
260  public void postModifyNamespace(final NamespaceDescriptor oldNsDescriptor,
261    final NamespaceDescriptor currentNsDescriptor) throws IOException {
262    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
263      @Override
264      public void call(MasterObserver observer) throws IOException {
265        observer.postModifyNamespace(this, oldNsDescriptor, currentNsDescriptor);
266      }
267    });
268  }
269
270  public void preGetNamespaceDescriptor(final String namespaceName) throws IOException {
271    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
272      @Override
273      public void call(MasterObserver observer) throws IOException {
274        observer.preGetNamespaceDescriptor(this, namespaceName);
275      }
276    });
277  }
278
279  public void postGetNamespaceDescriptor(final NamespaceDescriptor ns) throws IOException {
280    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
281      @Override
282      public void call(MasterObserver observer) throws IOException {
283        observer.postGetNamespaceDescriptor(this, ns);
284      }
285    });
286  }
287
288  public void preListNamespaces(final List<String> namespaces) throws IOException {
289    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
290      @Override
291      public void call(MasterObserver oserver) throws IOException {
292        oserver.preListNamespaces(this, namespaces);
293      }
294    });
295  }
296
297  public void postListNamespaces(final List<String> namespaces) throws IOException {
298    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
299      @Override
300      public void call(MasterObserver oserver) throws IOException {
301        oserver.postListNamespaces(this, namespaces);
302      }
303    });
304  }
305
306  public void preListNamespaceDescriptors(final List<NamespaceDescriptor> descriptors)
307    throws IOException {
308    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
309      @Override
310      public void call(MasterObserver observer) throws IOException {
311        observer.preListNamespaceDescriptors(this, descriptors);
312      }
313    });
314  }
315
316  public void postListNamespaceDescriptors(final List<NamespaceDescriptor> descriptors)
317    throws IOException {
318    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
319      @Override
320      public void call(MasterObserver observer) throws IOException {
321        observer.postListNamespaceDescriptors(this, descriptors);
322      }
323    });
324  }
325
326  /* Implementation of hooks for invoking MasterObservers */
327
328  public TableDescriptor preCreateTableRegionsInfos(TableDescriptor desc) throws IOException {
329    if (coprocEnvironments.isEmpty()) {
330      return desc;
331    }
332    return execOperationWithResult(
333      new ObserverOperationWithResult<MasterObserver, TableDescriptor>(masterObserverGetter, desc) {
334
335        @Override
336        protected TableDescriptor call(MasterObserver observer) throws IOException {
337          return observer.preCreateTableRegionsInfos(this, getResult());
338        }
339      });
340  }
341
342  public void preCreateTable(final TableDescriptor htd, final RegionInfo[] regions)
343    throws IOException {
344    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
345      @Override
346      public void call(MasterObserver observer) throws IOException {
347        observer.preCreateTable(this, htd, regions);
348      }
349    });
350  }
351
352  public void postCreateTable(final TableDescriptor htd, final RegionInfo[] regions)
353    throws IOException {
354    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
355      @Override
356      public void call(MasterObserver observer) throws IOException {
357        observer.postCreateTable(this, htd, regions);
358      }
359    });
360  }
361
362  public void preCreateTableAction(final TableDescriptor htd, final RegionInfo[] regions,
363    final User user) throws IOException {
364    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
365      @Override
366      public void call(MasterObserver observer) throws IOException {
367        observer.preCreateTableAction(this, htd, regions);
368      }
369    });
370  }
371
372  public void postCompletedCreateTableAction(final TableDescriptor htd, final RegionInfo[] regions,
373    final User user) throws IOException {
374    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
375      @Override
376      public void call(MasterObserver observer) throws IOException {
377        observer.postCompletedCreateTableAction(this, htd, regions);
378      }
379    });
380  }
381
382  public void preDeleteTable(final TableName tableName) throws IOException {
383    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
384      @Override
385      public void call(MasterObserver observer) throws IOException {
386        observer.preDeleteTable(this, tableName);
387      }
388    });
389  }
390
391  public void postDeleteTable(final TableName tableName) throws IOException {
392    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
393      @Override
394      public void call(MasterObserver observer) throws IOException {
395        observer.postDeleteTable(this, tableName);
396      }
397    });
398  }
399
400  public void preDeleteTableAction(final TableName tableName, final User user) throws IOException {
401    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
402      @Override
403      public void call(MasterObserver observer) throws IOException {
404        observer.preDeleteTableAction(this, tableName);
405      }
406    });
407  }
408
409  public void postCompletedDeleteTableAction(final TableName tableName, final User user)
410    throws IOException {
411    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
412      @Override
413      public void call(MasterObserver observer) throws IOException {
414        observer.postCompletedDeleteTableAction(this, tableName);
415      }
416    });
417  }
418
419  public void preTruncateTable(final TableName tableName) throws IOException {
420    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
421      @Override
422      public void call(MasterObserver observer) throws IOException {
423        observer.preTruncateTable(this, tableName);
424      }
425    });
426  }
427
428  public void postTruncateTable(final TableName tableName) throws IOException {
429    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
430      @Override
431      public void call(MasterObserver observer) throws IOException {
432        observer.postTruncateTable(this, tableName);
433      }
434    });
435  }
436
437  public void preTruncateTableAction(final TableName tableName, final User user)
438    throws IOException {
439    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
440      @Override
441      public void call(MasterObserver observer) throws IOException {
442        observer.preTruncateTableAction(this, tableName);
443      }
444    });
445  }
446
447  public void postCompletedTruncateTableAction(final TableName tableName, final User user)
448    throws IOException {
449    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
450      @Override
451      public void call(MasterObserver observer) throws IOException {
452        observer.postCompletedTruncateTableAction(this, tableName);
453      }
454    });
455  }
456
457  public TableDescriptor preModifyTable(final TableName tableName,
458    final TableDescriptor currentDescriptor, final TableDescriptor newDescriptor)
459    throws IOException {
460    if (coprocEnvironments.isEmpty()) {
461      return newDescriptor;
462    }
463    return execOperationWithResult(new ObserverOperationWithResult<MasterObserver, TableDescriptor>(
464      masterObserverGetter, newDescriptor) {
465      @Override
466      protected TableDescriptor call(MasterObserver observer) throws IOException {
467        return observer.preModifyTable(this, tableName, currentDescriptor, getResult());
468      }
469    });
470  }
471
472  public void postModifyTable(final TableName tableName, final TableDescriptor oldDescriptor,
473    final TableDescriptor currentDescriptor) throws IOException {
474    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
475      @Override
476      public void call(MasterObserver observer) throws IOException {
477        observer.postModifyTable(this, tableName, oldDescriptor, currentDescriptor);
478      }
479    });
480  }
481
482  public String preModifyTableStoreFileTracker(final TableName tableName, final String dstSFT)
483    throws IOException {
484    if (coprocEnvironments.isEmpty()) {
485      return dstSFT;
486    }
487    return execOperationWithResult(
488      new ObserverOperationWithResult<MasterObserver, String>(masterObserverGetter, dstSFT) {
489        @Override
490        protected String call(MasterObserver observer) throws IOException {
491          return observer.preModifyTableStoreFileTracker(this, tableName, getResult());
492        }
493      });
494  }
495
496  public void postModifyTableStoreFileTracker(final TableName tableName, final String dstSFT)
497    throws IOException {
498    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
499      @Override
500      public void call(MasterObserver observer) throws IOException {
501        observer.postModifyTableStoreFileTracker(this, tableName, dstSFT);
502      }
503    });
504  }
505
506  public String preModifyColumnFamilyStoreFileTracker(final TableName tableName,
507    final byte[] family, final String dstSFT) throws IOException {
508    if (coprocEnvironments.isEmpty()) {
509      return dstSFT;
510    }
511    return execOperationWithResult(
512      new ObserverOperationWithResult<MasterObserver, String>(masterObserverGetter, dstSFT) {
513        @Override
514        protected String call(MasterObserver observer) throws IOException {
515          return observer.preModifyColumnFamilyStoreFileTracker(this, tableName, family,
516            getResult());
517        }
518      });
519  }
520
521  public void postModifyColumnFamilyStoreFileTracker(final TableName tableName, final byte[] family,
522    final String dstSFT) throws IOException {
523    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
524      @Override
525      public void call(MasterObserver observer) throws IOException {
526        observer.postModifyColumnFamilyStoreFileTracker(this, tableName, family, dstSFT);
527      }
528    });
529  }
530
531  public void preModifyTableAction(final TableName tableName,
532    final TableDescriptor currentDescriptor, final TableDescriptor newDescriptor, final User user)
533    throws IOException {
534    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
535      @Override
536      public void call(MasterObserver observer) throws IOException {
537        observer.preModifyTableAction(this, tableName, currentDescriptor, newDescriptor);
538      }
539    });
540  }
541
542  public void postCompletedModifyTableAction(final TableName tableName,
543    final TableDescriptor oldDescriptor, final TableDescriptor currentDescriptor, final User user)
544    throws IOException {
545    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
546      @Override
547      public void call(MasterObserver observer) throws IOException {
548        observer.postCompletedModifyTableAction(this, tableName, oldDescriptor, currentDescriptor);
549      }
550    });
551  }
552
553  public void preEnableTable(final TableName tableName) throws IOException {
554    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
555      @Override
556      public void call(MasterObserver observer) throws IOException {
557        observer.preEnableTable(this, tableName);
558      }
559    });
560  }
561
562  public void postEnableTable(final TableName tableName) throws IOException {
563    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
564      @Override
565      public void call(MasterObserver observer) throws IOException {
566        observer.postEnableTable(this, tableName);
567      }
568    });
569  }
570
571  public void preEnableTableAction(final TableName tableName, final User user) throws IOException {
572    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
573      @Override
574      public void call(MasterObserver observer) throws IOException {
575        observer.preEnableTableAction(this, tableName);
576      }
577    });
578  }
579
580  public void postCompletedEnableTableAction(final TableName tableName, final User user)
581    throws IOException {
582    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
583      @Override
584      public void call(MasterObserver observer) throws IOException {
585        observer.postCompletedEnableTableAction(this, tableName);
586      }
587    });
588  }
589
590  public void preDisableTable(final TableName tableName) throws IOException {
591    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
592      @Override
593      public void call(MasterObserver observer) throws IOException {
594        observer.preDisableTable(this, tableName);
595      }
596    });
597  }
598
599  public void postDisableTable(final TableName tableName) throws IOException {
600    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
601      @Override
602      public void call(MasterObserver observer) throws IOException {
603        observer.postDisableTable(this, tableName);
604      }
605    });
606  }
607
608  public void preDisableTableAction(final TableName tableName, final User user) throws IOException {
609    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
610      @Override
611      public void call(MasterObserver observer) throws IOException {
612        observer.preDisableTableAction(this, tableName);
613      }
614    });
615  }
616
617  public void postCompletedDisableTableAction(final TableName tableName, final User user)
618    throws IOException {
619    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
620      @Override
621      public void call(MasterObserver observer) throws IOException {
622        observer.postCompletedDisableTableAction(this, tableName);
623      }
624    });
625  }
626
627  public void preAbortProcedure(final ProcedureExecutor<MasterProcedureEnv> procEnv,
628    final long procId) throws IOException {
629    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
630      @Override
631      public void call(MasterObserver observer) throws IOException {
632        observer.preAbortProcedure(this, procId);
633      }
634    });
635  }
636
637  public void postAbortProcedure() throws IOException {
638    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
639      @Override
640      public void call(MasterObserver observer) throws IOException {
641        observer.postAbortProcedure(this);
642      }
643    });
644  }
645
646  public void preGetProcedures() throws IOException {
647    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
648      @Override
649      public void call(MasterObserver observer) throws IOException {
650        observer.preGetProcedures(this);
651      }
652    });
653  }
654
655  public void postGetProcedures(final List<Procedure<?>> procInfoList) throws IOException {
656    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
657      @Override
658      public void call(MasterObserver observer) throws IOException {
659        observer.postGetProcedures(this);
660      }
661    });
662  }
663
664  public void preGetLocks() throws IOException {
665    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
666      @Override
667      public void call(MasterObserver observer) throws IOException {
668        observer.preGetLocks(this);
669      }
670    });
671  }
672
673  public void postGetLocks(final List<LockedResource> lockedResources) throws IOException {
674    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
675      @Override
676      public void call(MasterObserver observer) throws IOException {
677        observer.postGetLocks(this);
678      }
679    });
680  }
681
682  public void preMove(final RegionInfo region, final ServerName srcServer,
683    final ServerName destServer) throws IOException {
684    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
685      @Override
686      public void call(MasterObserver observer) throws IOException {
687        observer.preMove(this, region, srcServer, destServer);
688      }
689    });
690  }
691
692  public void postMove(final RegionInfo region, final ServerName srcServer,
693    final ServerName destServer) throws IOException {
694    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
695      @Override
696      public void call(MasterObserver observer) throws IOException {
697        observer.postMove(this, region, srcServer, destServer);
698      }
699    });
700  }
701
702  public void preAssign(final RegionInfo regionInfo) throws IOException {
703    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
704      @Override
705      public void call(MasterObserver observer) throws IOException {
706        observer.preAssign(this, regionInfo);
707      }
708    });
709  }
710
711  public void postAssign(final RegionInfo regionInfo) throws IOException {
712    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
713      @Override
714      public void call(MasterObserver observer) throws IOException {
715        observer.postAssign(this, regionInfo);
716      }
717    });
718  }
719
720  public void preUnassign(final RegionInfo regionInfo) throws IOException {
721    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
722      @Override
723      public void call(MasterObserver observer) throws IOException {
724        observer.preUnassign(this, regionInfo);
725      }
726    });
727  }
728
729  public void postUnassign(final RegionInfo regionInfo) throws IOException {
730    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
731      @Override
732      public void call(MasterObserver observer) throws IOException {
733        observer.postUnassign(this, regionInfo);
734      }
735    });
736  }
737
738  public void preRegionOffline(final RegionInfo regionInfo) throws IOException {
739    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
740      @Override
741      public void call(MasterObserver observer) throws IOException {
742        observer.preRegionOffline(this, regionInfo);
743      }
744    });
745  }
746
747  public void postRegionOffline(final RegionInfo regionInfo) throws IOException {
748    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
749      @Override
750      public void call(MasterObserver observer) throws IOException {
751        observer.postRegionOffline(this, regionInfo);
752      }
753    });
754  }
755
756  public void preMergeRegions(final RegionInfo[] regionsToMerge) throws IOException {
757    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
758      @Override
759      public void call(MasterObserver observer) throws IOException {
760        observer.preMergeRegions(this, regionsToMerge);
761      }
762    });
763  }
764
765  public void postMergeRegions(final RegionInfo[] regionsToMerge) throws IOException {
766    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
767      @Override
768      public void call(MasterObserver observer) throws IOException {
769        observer.postMergeRegions(this, regionsToMerge);
770      }
771    });
772  }
773
774  public boolean preBalance(final BalanceRequest request) throws IOException {
775    return execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
776      @Override
777      public void call(MasterObserver observer) throws IOException {
778        observer.preBalance(this, request);
779      }
780    });
781  }
782
783  public void postBalance(final BalanceRequest request, final List<RegionPlan> plans)
784    throws IOException {
785    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
786      @Override
787      public void call(MasterObserver observer) throws IOException {
788        observer.postBalance(this, request, plans);
789      }
790    });
791  }
792
793  public void preSetSplitOrMergeEnabled(final boolean newValue, final MasterSwitchType switchType)
794    throws IOException {
795    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
796      @Override
797      public void call(MasterObserver observer) throws IOException {
798        observer.preSetSplitOrMergeEnabled(this, newValue, switchType);
799      }
800    });
801  }
802
803  public void postSetSplitOrMergeEnabled(final boolean newValue, final MasterSwitchType switchType)
804    throws IOException {
805    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
806      @Override
807      public void call(MasterObserver observer) throws IOException {
808        observer.postSetSplitOrMergeEnabled(this, newValue, switchType);
809      }
810    });
811  }
812
813  /**
814   * Invoked just before calling the split region procedure
815   * @param tableName the table where the region belongs to
816   * @param splitRow  the split point
817   */
818  public void preSplitRegion(final TableName tableName, final byte[] splitRow) throws IOException {
819    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
820      @Override
821      public void call(MasterObserver observer) throws IOException {
822        observer.preSplitRegion(this, tableName, splitRow);
823      }
824    });
825  }
826
827  /**
828   * Invoked just before a split
829   * @param tableName the table where the region belongs to
830   * @param splitRow  the split point
831   * @param user      the user
832   */
833  public void preSplitRegionAction(final TableName tableName, final byte[] splitRow,
834    final User user) throws IOException {
835    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
836      @Override
837      public void call(MasterObserver observer) throws IOException {
838        observer.preSplitRegionAction(this, tableName, splitRow);
839      }
840    });
841  }
842
843  /**
844   * Invoked just after a split
845   * @param regionInfoA the new left-hand daughter region
846   * @param regionInfoB the new right-hand daughter region
847   * @param user        the user
848   */
849  public void postCompletedSplitRegionAction(final RegionInfo regionInfoA,
850    final RegionInfo regionInfoB, final User user) throws IOException {
851    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
852      @Override
853      public void call(MasterObserver observer) throws IOException {
854        observer.postCompletedSplitRegionAction(this, regionInfoA, regionInfoB);
855      }
856    });
857  }
858
859  /**
860   * Invoked just before calling the truncate region procedure
861   * @param regionInfo region being truncated
862   */
863  public void preTruncateRegion(RegionInfo regionInfo) throws IOException {
864    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
865      @Override
866      public void call(MasterObserver observer) {
867        observer.preTruncateRegion(this, regionInfo);
868      }
869    });
870  }
871
872  /**
873   * Invoked after calling the truncate region procedure
874   * @param regionInfo region being truncated
875   */
876  public void postTruncateRegion(RegionInfo regionInfo) throws IOException {
877    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
878      @Override
879      public void call(MasterObserver observer) {
880        observer.postTruncateRegion(this, regionInfo);
881      }
882    });
883  }
884
885  /**
886   * Invoked just before calling the truncate region procedure
887   * @param region Region to be truncated
888   * @param user   The user
889   */
890  public void preTruncateRegionAction(final RegionInfo region, User user) throws IOException {
891    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
892      @Override
893      public void call(MasterObserver observer) throws IOException {
894        observer.preTruncateRegionAction(this, region);
895      }
896    });
897  }
898
899  /**
900   * Invoked after calling the truncate region procedure
901   * @param region Region which was truncated
902   * @param user   The user
903   */
904  public void postTruncateRegionAction(final RegionInfo region, User user) throws IOException {
905    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
906      @Override
907      public void call(MasterObserver observer) throws IOException {
908        observer.postTruncateRegionAction(this, region);
909      }
910    });
911  }
912
913  /**
914   * This will be called before update META step as part of split table region procedure.
915   * @param user the user
916   */
917  public void preSplitBeforeMETAAction(final byte[] splitKey, final List<Mutation> metaEntries,
918    final User user) throws IOException {
919    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
920      @Override
921      public void call(MasterObserver observer) throws IOException {
922        observer.preSplitRegionBeforeMETAAction(this, splitKey, metaEntries);
923      }
924    });
925  }
926
927  /**
928   * This will be called after update META step as part of split table region procedure.
929   * @param user the user
930   */
931  public void preSplitAfterMETAAction(final User user) throws IOException {
932    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
933      @Override
934      public void call(MasterObserver observer) throws IOException {
935        observer.preSplitRegionAfterMETAAction(this);
936      }
937    });
938  }
939
940  /**
941   * Invoked just after the rollback of a failed split
942   * @param user the user
943   */
944  public void postRollBackSplitRegionAction(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.postRollBackSplitRegionAction(this);
949      }
950    });
951  }
952
953  /**
954   * Invoked just before a merge
955   * @param regionsToMerge the regions to merge
956   * @param user           the user
957   */
958  public void preMergeRegionsAction(final RegionInfo[] regionsToMerge, final User user)
959    throws IOException {
960    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
961      @Override
962      public void call(MasterObserver observer) throws IOException {
963        observer.preMergeRegionsAction(this, regionsToMerge);
964      }
965    });
966  }
967
968  /**
969   * Invoked after completing merge regions operation
970   * @param regionsToMerge the regions to merge
971   * @param mergedRegion   the new merged region
972   * @param user           the user
973   */
974  public void postCompletedMergeRegionsAction(final RegionInfo[] regionsToMerge,
975    final RegionInfo mergedRegion, final User user) throws IOException {
976    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
977      @Override
978      public void call(MasterObserver observer) throws IOException {
979        observer.postCompletedMergeRegionsAction(this, regionsToMerge, mergedRegion);
980      }
981    });
982  }
983
984  /**
985   * Invoked before merge regions operation writes the new region to hbase:meta
986   * @param regionsToMerge the regions to merge
987   * @param metaEntries    the meta entry
988   * @param user           the user
989   */
990  public void preMergeRegionsCommit(final RegionInfo[] regionsToMerge,
991    final @MetaMutationAnnotation List<Mutation> metaEntries, final User user) throws IOException {
992    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
993      @Override
994      public void call(MasterObserver observer) throws IOException {
995        observer.preMergeRegionsCommitAction(this, regionsToMerge, metaEntries);
996      }
997    });
998  }
999
1000  /**
1001   * Invoked after merge regions operation writes the new region to hbase:meta
1002   * @param regionsToMerge the regions to merge
1003   * @param mergedRegion   the new merged region
1004   * @param user           the user
1005   */
1006  public void postMergeRegionsCommit(final RegionInfo[] regionsToMerge,
1007    final RegionInfo mergedRegion, final User user) throws IOException {
1008    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
1009      @Override
1010      public void call(MasterObserver observer) throws IOException {
1011        observer.postMergeRegionsCommitAction(this, regionsToMerge, mergedRegion);
1012      }
1013    });
1014  }
1015
1016  /**
1017   * Invoked after rollback merge regions operation
1018   * @param regionsToMerge the regions to merge
1019   * @param user           the user
1020   */
1021  public void postRollBackMergeRegionsAction(final RegionInfo[] regionsToMerge, final User user)
1022    throws IOException {
1023    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
1024      @Override
1025      public void call(MasterObserver observer) throws IOException {
1026        observer.postRollBackMergeRegionsAction(this, regionsToMerge);
1027      }
1028    });
1029  }
1030
1031  // This hook allows Coprocessor change value of balance switch.
1032  public void preBalanceSwitch(final boolean b) throws IOException {
1033    if (this.coprocEnvironments.isEmpty()) {
1034      return;
1035    }
1036    execOperation(new MasterObserverOperation() {
1037      @Override
1038      public void call(MasterObserver observer) throws IOException {
1039        observer.preBalanceSwitch(this, b);
1040      }
1041    });
1042  }
1043
1044  public void postBalanceSwitch(final boolean oldValue, final boolean newValue) throws IOException {
1045    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1046      @Override
1047      public void call(MasterObserver observer) throws IOException {
1048        observer.postBalanceSwitch(this, oldValue, newValue);
1049      }
1050    });
1051  }
1052
1053  public void preShutdown() throws IOException {
1054    // While stopping the cluster all coprocessors method should be executed first then the
1055    // coprocessor should be cleaned up.
1056    if (coprocEnvironments.isEmpty()) {
1057      return;
1058    }
1059    execShutdown(new MasterObserverOperation() {
1060      @Override
1061      public void call(MasterObserver observer) throws IOException {
1062        observer.preShutdown(this);
1063      }
1064
1065      @Override
1066      public void postEnvCall() {
1067        // invoke coprocessor stop method
1068        shutdown(this.getEnvironment());
1069      }
1070    });
1071  }
1072
1073  public void preStopMaster() throws IOException {
1074    // While stopping master all coprocessors method should be executed first then the coprocessor
1075    // environment should be cleaned up.
1076    if (coprocEnvironments.isEmpty()) {
1077      return;
1078    }
1079    execShutdown(new MasterObserverOperation() {
1080      @Override
1081      public void call(MasterObserver observer) throws IOException {
1082        observer.preStopMaster(this);
1083      }
1084
1085      @Override
1086      public void postEnvCall() {
1087        // invoke coprocessor stop method
1088        shutdown(this.getEnvironment());
1089      }
1090    });
1091  }
1092
1093  public void preMasterInitialization() throws IOException {
1094    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1095      @Override
1096      public void call(MasterObserver observer) throws IOException {
1097        observer.preMasterInitialization(this);
1098      }
1099    });
1100  }
1101
1102  public void postStartMaster() throws IOException {
1103    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1104      @Override
1105      public void call(MasterObserver observer) throws IOException {
1106        observer.postStartMaster(this);
1107      }
1108    });
1109  }
1110
1111  public void preSnapshot(final SnapshotDescription snapshot,
1112    final TableDescriptor hTableDescriptor, final User user) throws IOException {
1113    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
1114      @Override
1115      public void call(MasterObserver observer) throws IOException {
1116        observer.preSnapshot(this, snapshot, hTableDescriptor);
1117      }
1118    });
1119  }
1120
1121  public void postSnapshot(final SnapshotDescription snapshot,
1122    final TableDescriptor hTableDescriptor, final User user) throws IOException {
1123    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
1124      @Override
1125      public void call(MasterObserver observer) throws IOException {
1126        observer.postSnapshot(this, snapshot, hTableDescriptor);
1127      }
1128    });
1129  }
1130
1131  public void postCompletedSnapshotAction(SnapshotDescription snapshot,
1132    TableDescriptor hTableDescriptor) throws IOException {
1133    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1134      @Override
1135      public void call(MasterObserver observer) throws IOException {
1136        observer.postCompletedSnapshotAction(this, snapshot, hTableDescriptor);
1137      }
1138    });
1139  }
1140
1141  public void preListSnapshot(final SnapshotDescription snapshot) throws IOException {
1142    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1143      @Override
1144      public void call(MasterObserver observer) throws IOException {
1145        observer.preListSnapshot(this, snapshot);
1146      }
1147    });
1148  }
1149
1150  public void postListSnapshot(final SnapshotDescription snapshot) throws IOException {
1151    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1152      @Override
1153      public void call(MasterObserver observer) throws IOException {
1154        observer.postListSnapshot(this, snapshot);
1155      }
1156    });
1157  }
1158
1159  public void preCloneSnapshot(final SnapshotDescription snapshot,
1160    final TableDescriptor hTableDescriptor) throws IOException {
1161    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1162      @Override
1163      public void call(MasterObserver observer) throws IOException {
1164        observer.preCloneSnapshot(this, snapshot, hTableDescriptor);
1165      }
1166    });
1167  }
1168
1169  public void postCloneSnapshot(final SnapshotDescription snapshot,
1170    final TableDescriptor hTableDescriptor) throws IOException {
1171    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1172      @Override
1173      public void call(MasterObserver observer) throws IOException {
1174        observer.postCloneSnapshot(this, snapshot, hTableDescriptor);
1175      }
1176    });
1177  }
1178
1179  public void preRestoreSnapshot(final SnapshotDescription snapshot,
1180    final TableDescriptor hTableDescriptor) throws IOException {
1181    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1182      @Override
1183      public void call(MasterObserver observer) throws IOException {
1184        observer.preRestoreSnapshot(this, snapshot, hTableDescriptor);
1185      }
1186    });
1187  }
1188
1189  public void postRestoreSnapshot(final SnapshotDescription snapshot,
1190    final TableDescriptor hTableDescriptor) throws IOException {
1191    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1192      @Override
1193      public void call(MasterObserver observer) throws IOException {
1194        observer.postRestoreSnapshot(this, snapshot, hTableDescriptor);
1195      }
1196    });
1197  }
1198
1199  public void preDeleteSnapshot(final SnapshotDescription snapshot) throws IOException {
1200    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1201      @Override
1202      public void call(MasterObserver observer) throws IOException {
1203        observer.preDeleteSnapshot(this, snapshot);
1204      }
1205    });
1206  }
1207
1208  public void postDeleteSnapshot(final SnapshotDescription snapshot) throws IOException {
1209    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1210      @Override
1211      public void call(MasterObserver observer) throws IOException {
1212        observer.postDeleteSnapshot(this, snapshot);
1213      }
1214    });
1215  }
1216
1217  public void preGetTableDescriptors(final List<TableName> tableNamesList,
1218    final List<TableDescriptor> descriptors, final String regex) throws IOException {
1219    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1220      @Override
1221      public void call(MasterObserver observer) throws IOException {
1222        observer.preGetTableDescriptors(this, tableNamesList, descriptors, regex);
1223      }
1224    });
1225  }
1226
1227  public void postGetTableDescriptors(final List<TableName> tableNamesList,
1228    final List<TableDescriptor> descriptors, final String regex) throws IOException {
1229    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1230      @Override
1231      public void call(MasterObserver observer) throws IOException {
1232        observer.postGetTableDescriptors(this, tableNamesList, descriptors, regex);
1233      }
1234    });
1235  }
1236
1237  public void preGetTableNames(final List<TableDescriptor> descriptors, final String regex)
1238    throws IOException {
1239    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1240      @Override
1241      public void call(MasterObserver observer) throws IOException {
1242        observer.preGetTableNames(this, descriptors, regex);
1243      }
1244    });
1245  }
1246
1247  public void postGetTableNames(final List<TableDescriptor> descriptors, final String regex)
1248    throws IOException {
1249    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1250      @Override
1251      public void call(MasterObserver observer) throws IOException {
1252        observer.postGetTableNames(this, descriptors, regex);
1253      }
1254    });
1255  }
1256
1257  public void preTableFlush(final TableName tableName) throws IOException {
1258    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1259      @Override
1260      public void call(MasterObserver observer) throws IOException {
1261        observer.preTableFlush(this, tableName);
1262      }
1263    });
1264  }
1265
1266  public void postTableFlush(final TableName tableName) throws IOException {
1267    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1268      @Override
1269      public void call(MasterObserver observer) throws IOException {
1270        observer.postTableFlush(this, tableName);
1271      }
1272    });
1273  }
1274
1275  public void preMasterStoreFlush() throws IOException {
1276    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1277      @Override
1278      public void call(MasterObserver observer) throws IOException {
1279        observer.preMasterStoreFlush(this);
1280      }
1281    });
1282  }
1283
1284  public void postMasterStoreFlush() throws IOException {
1285    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1286      @Override
1287      public void call(MasterObserver observer) throws IOException {
1288        observer.postMasterStoreFlush(this);
1289      }
1290    });
1291  }
1292
1293  public void preSetUserQuota(final String user, final GlobalQuotaSettings quotas)
1294    throws IOException {
1295    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1296      @Override
1297      public void call(MasterObserver observer) throws IOException {
1298        observer.preSetUserQuota(this, user, quotas);
1299      }
1300    });
1301  }
1302
1303  public void postSetUserQuota(final String user, final GlobalQuotaSettings quotas)
1304    throws IOException {
1305    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1306      @Override
1307      public void call(MasterObserver observer) throws IOException {
1308        observer.postSetUserQuota(this, user, quotas);
1309      }
1310    });
1311  }
1312
1313  public void preSetUserQuota(final String user, final TableName table,
1314    final GlobalQuotaSettings quotas) throws IOException {
1315    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1316      @Override
1317      public void call(MasterObserver observer) throws IOException {
1318        observer.preSetUserQuota(this, user, table, quotas);
1319      }
1320    });
1321  }
1322
1323  public void postSetUserQuota(final String user, final TableName table,
1324    final GlobalQuotaSettings quotas) throws IOException {
1325    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1326      @Override
1327      public void call(MasterObserver observer) throws IOException {
1328        observer.postSetUserQuota(this, user, table, quotas);
1329      }
1330    });
1331  }
1332
1333  public void preSetUserQuota(final String user, final String namespace,
1334    final GlobalQuotaSettings quotas) throws IOException {
1335    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1336      @Override
1337      public void call(MasterObserver observer) throws IOException {
1338        observer.preSetUserQuota(this, user, namespace, quotas);
1339      }
1340    });
1341  }
1342
1343  public void postSetUserQuota(final String user, final String namespace,
1344    final GlobalQuotaSettings quotas) throws IOException {
1345    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1346      @Override
1347      public void call(MasterObserver observer) throws IOException {
1348        observer.postSetUserQuota(this, user, namespace, quotas);
1349      }
1350    });
1351  }
1352
1353  public void preSetTableQuota(final TableName table, final GlobalQuotaSettings quotas)
1354    throws IOException {
1355    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1356      @Override
1357      public void call(MasterObserver observer) throws IOException {
1358        observer.preSetTableQuota(this, table, quotas);
1359      }
1360    });
1361  }
1362
1363  public void postSetTableQuota(final TableName table, final GlobalQuotaSettings quotas)
1364    throws IOException {
1365    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1366      @Override
1367      public void call(MasterObserver observer) throws IOException {
1368        observer.postSetTableQuota(this, table, quotas);
1369      }
1370    });
1371  }
1372
1373  public void preSetNamespaceQuota(final String namespace, final GlobalQuotaSettings quotas)
1374    throws IOException {
1375    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1376      @Override
1377      public void call(MasterObserver observer) throws IOException {
1378        observer.preSetNamespaceQuota(this, namespace, quotas);
1379      }
1380    });
1381  }
1382
1383  public void postSetNamespaceQuota(final String namespace, final GlobalQuotaSettings quotas)
1384    throws IOException {
1385    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1386      @Override
1387      public void call(MasterObserver observer) throws IOException {
1388        observer.postSetNamespaceQuota(this, namespace, quotas);
1389      }
1390    });
1391  }
1392
1393  public void preSetRegionServerQuota(final String regionServer, final GlobalQuotaSettings quotas)
1394    throws IOException {
1395    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1396      @Override
1397      public void call(MasterObserver observer) throws IOException {
1398        observer.preSetRegionServerQuota(this, regionServer, quotas);
1399      }
1400    });
1401  }
1402
1403  public void postSetRegionServerQuota(final String regionServer, final GlobalQuotaSettings quotas)
1404    throws IOException {
1405    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1406      @Override
1407      public void call(MasterObserver observer) throws IOException {
1408        observer.postSetRegionServerQuota(this, regionServer, quotas);
1409      }
1410    });
1411  }
1412
1413  public void preMoveServersAndTables(final Set<Address> servers, final Set<TableName> tables,
1414    final String targetGroup) throws IOException {
1415    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1416      @Override
1417      public void call(MasterObserver observer) throws IOException {
1418        observer.preMoveServersAndTables(this, servers, tables, targetGroup);
1419      }
1420    });
1421  }
1422
1423  public void postMoveServersAndTables(final Set<Address> servers, final Set<TableName> tables,
1424    final String targetGroup) throws IOException {
1425    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1426      @Override
1427      public void call(MasterObserver observer) throws IOException {
1428        observer.postMoveServersAndTables(this, servers, tables, targetGroup);
1429      }
1430    });
1431  }
1432
1433  public void preMoveServers(final Set<Address> servers, final String targetGroup)
1434    throws IOException {
1435    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1436      @Override
1437      public void call(MasterObserver observer) throws IOException {
1438        observer.preMoveServers(this, servers, targetGroup);
1439      }
1440    });
1441  }
1442
1443  public void postMoveServers(final Set<Address> servers, final String targetGroup)
1444    throws IOException {
1445    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1446      @Override
1447      public void call(MasterObserver observer) throws IOException {
1448        observer.postMoveServers(this, servers, targetGroup);
1449      }
1450    });
1451  }
1452
1453  public void preMoveTables(final Set<TableName> tables, final String targetGroup)
1454    throws IOException {
1455    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1456      @Override
1457      public void call(MasterObserver observer) throws IOException {
1458        observer.preMoveTables(this, tables, targetGroup);
1459      }
1460    });
1461  }
1462
1463  public void postMoveTables(final Set<TableName> tables, final String targetGroup)
1464    throws IOException {
1465    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1466      @Override
1467      public void call(MasterObserver observer) throws IOException {
1468        observer.postMoveTables(this, tables, targetGroup);
1469      }
1470    });
1471  }
1472
1473  public void preAddRSGroup(final String name) throws IOException {
1474    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1475      @Override
1476      public void call(MasterObserver observer) throws IOException {
1477        observer.preAddRSGroup(this, name);
1478      }
1479    });
1480  }
1481
1482  public void postAddRSGroup(final String name) throws IOException {
1483    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1484      @Override
1485      public void call(MasterObserver observer) throws IOException {
1486        observer.postAddRSGroup(this, name);
1487      }
1488    });
1489  }
1490
1491  public void preRemoveRSGroup(final String name) throws IOException {
1492    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1493      @Override
1494      public void call(MasterObserver observer) throws IOException {
1495        observer.preRemoveRSGroup(this, name);
1496      }
1497    });
1498  }
1499
1500  public void postRemoveRSGroup(final String name) throws IOException {
1501    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1502      @Override
1503      public void call(MasterObserver observer) throws IOException {
1504        observer.postRemoveRSGroup(this, name);
1505      }
1506    });
1507  }
1508
1509  public void preBalanceRSGroup(final String name, final BalanceRequest request)
1510    throws IOException {
1511    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1512      @Override
1513      public void call(MasterObserver observer) throws IOException {
1514        observer.preBalanceRSGroup(this, name, request);
1515      }
1516    });
1517  }
1518
1519  public void postBalanceRSGroup(final String name, final BalanceRequest request,
1520    final BalanceResponse response) throws IOException {
1521    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1522      @Override
1523      public void call(MasterObserver observer) throws IOException {
1524        observer.postBalanceRSGroup(this, name, request, response);
1525      }
1526    });
1527  }
1528
1529  public void preRemoveServers(final Set<Address> servers) throws IOException {
1530    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1531      @Override
1532      public void call(MasterObserver observer) throws IOException {
1533        observer.preRemoveServers(this, servers);
1534      }
1535    });
1536  }
1537
1538  public void postRemoveServers(final Set<Address> servers) throws IOException {
1539    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1540      @Override
1541      public void call(MasterObserver observer) throws IOException {
1542        observer.postRemoveServers(this, servers);
1543      }
1544    });
1545  }
1546
1547  public void preGetRSGroupInfo(final String groupName) throws IOException {
1548    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1549      @Override
1550      public void call(MasterObserver observer) throws IOException {
1551        observer.preGetRSGroupInfo(this, groupName);
1552      }
1553    });
1554  }
1555
1556  public void postGetRSGroupInfo(final String groupName) throws IOException {
1557    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1558      @Override
1559      public void call(MasterObserver observer) throws IOException {
1560        observer.postGetRSGroupInfo(this, groupName);
1561      }
1562    });
1563  }
1564
1565  public void preGetRSGroupInfoOfTable(final TableName tableName) throws IOException {
1566    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1567      @Override
1568      public void call(MasterObserver observer) throws IOException {
1569        observer.preGetRSGroupInfoOfTable(this, tableName);
1570      }
1571    });
1572  }
1573
1574  public void postGetRSGroupInfoOfTable(final TableName tableName) throws IOException {
1575    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1576      @Override
1577      public void call(MasterObserver observer) throws IOException {
1578        observer.postGetRSGroupInfoOfTable(this, tableName);
1579      }
1580    });
1581  }
1582
1583  public void preListRSGroups() throws IOException {
1584    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1585      @Override
1586      public void call(MasterObserver observer) throws IOException {
1587        observer.preListRSGroups(this);
1588      }
1589    });
1590  }
1591
1592  public void postListRSGroups() throws IOException {
1593    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1594      @Override
1595      public void call(MasterObserver observer) throws IOException {
1596        observer.postListRSGroups(this);
1597      }
1598    });
1599  }
1600
1601  public void preListTablesInRSGroup(final String groupName) throws IOException {
1602    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1603
1604      @Override
1605      protected void call(MasterObserver observer) throws IOException {
1606        observer.preListTablesInRSGroup(this, groupName);
1607      }
1608    });
1609  }
1610
1611  public void postListTablesInRSGroup(final String groupName) throws IOException {
1612    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1613
1614      @Override
1615      protected void call(MasterObserver observer) throws IOException {
1616        observer.postListTablesInRSGroup(this, groupName);
1617      }
1618    });
1619  }
1620
1621  public void preRenameRSGroup(final String oldName, final String newName) throws IOException {
1622    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1623
1624      @Override
1625      protected void call(MasterObserver observer) throws IOException {
1626        observer.preRenameRSGroup(this, oldName, newName);
1627      }
1628    });
1629  }
1630
1631  public void postRenameRSGroup(final String oldName, final String newName) throws IOException {
1632    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1633
1634      @Override
1635      protected void call(MasterObserver observer) throws IOException {
1636        observer.postRenameRSGroup(this, oldName, newName);
1637      }
1638    });
1639  }
1640
1641  public void preUpdateRSGroupConfig(final String groupName,
1642    final Map<String, String> configuration) throws IOException {
1643    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1644      @Override
1645      protected void call(MasterObserver observer) throws IOException {
1646        observer.preUpdateRSGroupConfig(this, groupName, configuration);
1647      }
1648    });
1649  }
1650
1651  public void postUpdateRSGroupConfig(final String groupName,
1652    final Map<String, String> configuration) throws IOException {
1653    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1654      @Override
1655      protected void call(MasterObserver observer) throws IOException {
1656        observer.postUpdateRSGroupConfig(this, groupName, configuration);
1657      }
1658    });
1659  }
1660
1661  public void preGetConfiguredNamespacesAndTablesInRSGroup(final String groupName)
1662    throws IOException {
1663    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1664
1665      @Override
1666      protected void call(MasterObserver observer) throws IOException {
1667        observer.preGetConfiguredNamespacesAndTablesInRSGroup(this, groupName);
1668      }
1669    });
1670  }
1671
1672  public void postGetConfiguredNamespacesAndTablesInRSGroup(final String groupName)
1673    throws IOException {
1674    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1675
1676      @Override
1677      protected void call(MasterObserver observer) throws IOException {
1678        observer.postGetConfiguredNamespacesAndTablesInRSGroup(this, groupName);
1679      }
1680    });
1681  }
1682
1683  public void preGetRSGroupInfoOfServer(final Address server) throws IOException {
1684    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1685      @Override
1686      public void call(MasterObserver observer) throws IOException {
1687        observer.preGetRSGroupInfoOfServer(this, server);
1688      }
1689    });
1690  }
1691
1692  public void postGetRSGroupInfoOfServer(final Address server) throws IOException {
1693    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1694      @Override
1695      public void call(MasterObserver observer) throws IOException {
1696        observer.postGetRSGroupInfoOfServer(this, server);
1697      }
1698    });
1699  }
1700
1701  public void preAddReplicationPeer(final String peerId, final ReplicationPeerConfig peerConfig)
1702    throws IOException {
1703    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1704      @Override
1705      public void call(MasterObserver observer) throws IOException {
1706        observer.preAddReplicationPeer(this, peerId, peerConfig);
1707      }
1708    });
1709  }
1710
1711  public void postAddReplicationPeer(final String peerId, final ReplicationPeerConfig peerConfig)
1712    throws IOException {
1713    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1714      @Override
1715      public void call(MasterObserver observer) throws IOException {
1716        observer.postAddReplicationPeer(this, peerId, peerConfig);
1717      }
1718    });
1719  }
1720
1721  public void preRemoveReplicationPeer(final String peerId) throws IOException {
1722    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1723      @Override
1724      public void call(MasterObserver observer) throws IOException {
1725        observer.preRemoveReplicationPeer(this, peerId);
1726      }
1727    });
1728  }
1729
1730  public void postRemoveReplicationPeer(final String peerId) throws IOException {
1731    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1732      @Override
1733      public void call(MasterObserver observer) throws IOException {
1734        observer.postRemoveReplicationPeer(this, peerId);
1735      }
1736    });
1737  }
1738
1739  public void preEnableReplicationPeer(final String peerId) throws IOException {
1740    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1741      @Override
1742      public void call(MasterObserver observer) throws IOException {
1743        observer.preEnableReplicationPeer(this, peerId);
1744      }
1745    });
1746  }
1747
1748  public void postEnableReplicationPeer(final String peerId) throws IOException {
1749    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1750      @Override
1751      public void call(MasterObserver observer) throws IOException {
1752        observer.postEnableReplicationPeer(this, peerId);
1753      }
1754    });
1755  }
1756
1757  public void preDisableReplicationPeer(final String peerId) throws IOException {
1758    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1759      @Override
1760      public void call(MasterObserver observer) throws IOException {
1761        observer.preDisableReplicationPeer(this, peerId);
1762      }
1763    });
1764  }
1765
1766  public void postDisableReplicationPeer(final String peerId) throws IOException {
1767    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1768      @Override
1769      public void call(MasterObserver observer) throws IOException {
1770        observer.postDisableReplicationPeer(this, peerId);
1771      }
1772    });
1773  }
1774
1775  public void preGetReplicationPeerConfig(final String peerId) throws IOException {
1776    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1777      @Override
1778      public void call(MasterObserver observer) throws IOException {
1779        observer.preGetReplicationPeerConfig(this, peerId);
1780      }
1781    });
1782  }
1783
1784  public void postGetReplicationPeerConfig(final String peerId) throws IOException {
1785    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1786      @Override
1787      public void call(MasterObserver observer) throws IOException {
1788        observer.postGetReplicationPeerConfig(this, peerId);
1789      }
1790    });
1791  }
1792
1793  public void preUpdateReplicationPeerConfig(final String peerId,
1794    final ReplicationPeerConfig peerConfig) throws IOException {
1795    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1796      @Override
1797      public void call(MasterObserver observer) throws IOException {
1798        observer.preUpdateReplicationPeerConfig(this, peerId, peerConfig);
1799      }
1800    });
1801  }
1802
1803  public void postUpdateReplicationPeerConfig(final String peerId,
1804    final ReplicationPeerConfig peerConfig) throws IOException {
1805    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1806      @Override
1807      public void call(MasterObserver observer) throws IOException {
1808        observer.postUpdateReplicationPeerConfig(this, peerId, peerConfig);
1809      }
1810    });
1811  }
1812
1813  public void preListReplicationPeers(final String regex) throws IOException {
1814    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1815      @Override
1816      public void call(MasterObserver observer) throws IOException {
1817        observer.preListReplicationPeers(this, regex);
1818      }
1819    });
1820  }
1821
1822  public void postListReplicationPeers(final String regex) throws IOException {
1823    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1824      @Override
1825      public void call(MasterObserver observer) throws IOException {
1826        observer.postListReplicationPeers(this, regex);
1827      }
1828    });
1829  }
1830
1831  public void preTransitReplicationPeerSyncReplicationState(String peerId,
1832    SyncReplicationState state) throws IOException {
1833    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1834      @Override
1835      public void call(MasterObserver observer) throws IOException {
1836        observer.preTransitReplicationPeerSyncReplicationState(this, peerId, state);
1837      }
1838    });
1839  }
1840
1841  public void postTransitReplicationPeerSyncReplicationState(String peerId,
1842    SyncReplicationState from, SyncReplicationState to) throws IOException {
1843    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1844      @Override
1845      public void call(MasterObserver observer) throws IOException {
1846        observer.postTransitReplicationPeerSyncReplicationState(this, peerId, from, to);
1847      }
1848    });
1849  }
1850
1851  public void preRequestLock(String namespace, TableName tableName, RegionInfo[] regionInfos,
1852    LockType type, String description) throws IOException {
1853    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1854      @Override
1855      public void call(MasterObserver observer) throws IOException {
1856        observer.preRequestLock(this, namespace, tableName, regionInfos, description);
1857      }
1858    });
1859  }
1860
1861  public void postRequestLock(String namespace, TableName tableName, RegionInfo[] regionInfos,
1862    LockType type, String description) throws IOException {
1863    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1864      @Override
1865      public void call(MasterObserver observer) throws IOException {
1866        observer.postRequestLock(this, namespace, tableName, regionInfos, description);
1867      }
1868    });
1869  }
1870
1871  public void preLockHeartbeat(LockProcedure proc, boolean keepAlive) throws IOException {
1872    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1873      @Override
1874      public void call(MasterObserver observer) throws IOException {
1875        observer.preLockHeartbeat(this, proc.getTableName(), proc.getDescription());
1876      }
1877    });
1878  }
1879
1880  public void postLockHeartbeat(LockProcedure proc, boolean keepAlive) throws IOException {
1881    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1882      @Override
1883      public void call(MasterObserver observer) throws IOException {
1884        observer.postLockHeartbeat(this);
1885      }
1886    });
1887  }
1888
1889  public void preGetClusterMetrics() throws IOException {
1890    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1891      @Override
1892      public void call(MasterObserver observer) throws IOException {
1893        observer.preGetClusterMetrics(this);
1894      }
1895    });
1896  }
1897
1898  public void postGetClusterMetrics(ClusterMetrics status) throws IOException {
1899    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1900      @Override
1901      public void call(MasterObserver observer) throws IOException {
1902        observer.postGetClusterMetrics(this, status);
1903      }
1904    });
1905  }
1906
1907  public void preClearDeadServers() throws IOException {
1908    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1909      @Override
1910      public void call(MasterObserver observer) throws IOException {
1911        observer.preClearDeadServers(this);
1912      }
1913    });
1914  }
1915
1916  public void postClearDeadServers(List<ServerName> servers, List<ServerName> notClearedServers)
1917    throws IOException {
1918    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1919      @Override
1920      public void call(MasterObserver observer) throws IOException {
1921        observer.postClearDeadServers(this, servers, notClearedServers);
1922      }
1923    });
1924  }
1925
1926  public void preDecommissionRegionServers(List<ServerName> servers, boolean offload)
1927    throws IOException {
1928    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1929      @Override
1930      public void call(MasterObserver observer) throws IOException {
1931        observer.preDecommissionRegionServers(this, servers, offload);
1932      }
1933    });
1934  }
1935
1936  public void postDecommissionRegionServers(List<ServerName> servers, boolean offload)
1937    throws IOException {
1938    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1939      @Override
1940      public void call(MasterObserver observer) throws IOException {
1941        observer.postDecommissionRegionServers(this, servers, offload);
1942      }
1943    });
1944  }
1945
1946  public void preListDecommissionedRegionServers() throws IOException {
1947    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1948      @Override
1949      public void call(MasterObserver observer) throws IOException {
1950        observer.preListDecommissionedRegionServers(this);
1951      }
1952    });
1953  }
1954
1955  public void postListDecommissionedRegionServers() throws IOException {
1956    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1957      @Override
1958      public void call(MasterObserver observer) throws IOException {
1959        observer.postListDecommissionedRegionServers(this);
1960      }
1961    });
1962  }
1963
1964  public void preRecommissionRegionServer(ServerName server, List<byte[]> encodedRegionNames)
1965    throws IOException {
1966    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1967      @Override
1968      public void call(MasterObserver observer) throws IOException {
1969        observer.preRecommissionRegionServer(this, server, encodedRegionNames);
1970      }
1971    });
1972  }
1973
1974  public void postRecommissionRegionServer(ServerName server, List<byte[]> encodedRegionNames)
1975    throws IOException {
1976    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1977      @Override
1978      public void call(MasterObserver observer) throws IOException {
1979        observer.postRecommissionRegionServer(this, server, encodedRegionNames);
1980      }
1981    });
1982  }
1983
1984  public void preSwitchRpcThrottle(boolean enable) throws IOException {
1985    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1986      @Override
1987      public void call(MasterObserver observer) throws IOException {
1988        observer.preSwitchRpcThrottle(this, enable);
1989      }
1990    });
1991  }
1992
1993  public void postSwitchRpcThrottle(final boolean oldValue, final boolean newValue)
1994    throws IOException {
1995    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1996      @Override
1997      public void call(MasterObserver observer) throws IOException {
1998        observer.postSwitchRpcThrottle(this, oldValue, newValue);
1999      }
2000    });
2001  }
2002
2003  public void preIsRpcThrottleEnabled() throws IOException {
2004    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
2005      @Override
2006      public void call(MasterObserver observer) throws IOException {
2007        observer.preIsRpcThrottleEnabled(this);
2008      }
2009    });
2010  }
2011
2012  public void postIsRpcThrottleEnabled(boolean enabled) throws IOException {
2013    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
2014      @Override
2015      public void call(MasterObserver observer) throws IOException {
2016        observer.postIsRpcThrottleEnabled(this, enabled);
2017      }
2018    });
2019  }
2020
2021  public void preSwitchExceedThrottleQuota(boolean enable) throws IOException {
2022    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
2023      @Override
2024      public void call(MasterObserver observer) throws IOException {
2025        observer.preSwitchExceedThrottleQuota(this, enable);
2026      }
2027    });
2028  }
2029
2030  public void postSwitchExceedThrottleQuota(final boolean oldValue, final boolean newValue)
2031    throws IOException {
2032    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
2033      @Override
2034      public void call(MasterObserver observer) throws IOException {
2035        observer.postSwitchExceedThrottleQuota(this, oldValue, newValue);
2036      }
2037    });
2038  }
2039
2040  public void preGrant(UserPermission userPermission, boolean mergeExistingPermissions)
2041    throws IOException {
2042    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
2043      @Override
2044      public void call(MasterObserver observer) throws IOException {
2045        observer.preGrant(this, userPermission, mergeExistingPermissions);
2046      }
2047    });
2048  }
2049
2050  public void postGrant(UserPermission userPermission, boolean mergeExistingPermissions)
2051    throws IOException {
2052    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
2053      @Override
2054      public void call(MasterObserver observer) throws IOException {
2055        observer.postGrant(this, userPermission, mergeExistingPermissions);
2056      }
2057    });
2058  }
2059
2060  public void preRevoke(UserPermission userPermission) throws IOException {
2061    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
2062      @Override
2063      public void call(MasterObserver observer) throws IOException {
2064        observer.preRevoke(this, userPermission);
2065      }
2066    });
2067  }
2068
2069  public void postRevoke(UserPermission userPermission) throws IOException {
2070    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
2071      @Override
2072      public void call(MasterObserver observer) throws IOException {
2073        observer.postRevoke(this, userPermission);
2074      }
2075    });
2076  }
2077
2078  public void preGetUserPermissions(String userName, String namespace, TableName tableName,
2079    byte[] family, byte[] qualifier) throws IOException {
2080    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
2081      @Override
2082      public void call(MasterObserver observer) throws IOException {
2083        observer.preGetUserPermissions(this, userName, namespace, tableName, family, qualifier);
2084      }
2085    });
2086  }
2087
2088  public void postGetUserPermissions(String userName, String namespace, TableName tableName,
2089    byte[] family, byte[] qualifier) throws IOException {
2090    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
2091      @Override
2092      public void call(MasterObserver observer) throws IOException {
2093        observer.postGetUserPermissions(this, userName, namespace, tableName, family, qualifier);
2094      }
2095    });
2096  }
2097
2098  public void preHasUserPermissions(String userName, List<Permission> permissions)
2099    throws IOException {
2100    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
2101      @Override
2102      public void call(MasterObserver observer) throws IOException {
2103        observer.preHasUserPermissions(this, userName, permissions);
2104      }
2105    });
2106  }
2107
2108  public void postHasUserPermissions(String userName, List<Permission> permissions)
2109    throws IOException {
2110    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
2111      @Override
2112      public void call(MasterObserver observer) throws IOException {
2113        observer.postHasUserPermissions(this, userName, permissions);
2114      }
2115    });
2116  }
2117
2118  public void preUpdateConfiguration(Configuration preReloadConf) throws IOException {
2119    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
2120      @Override
2121      public void call(MasterObserver observer) throws IOException {
2122        observer.preUpdateMasterConfiguration(this, preReloadConf);
2123      }
2124    });
2125  }
2126
2127  public void postUpdateConfiguration(Configuration postReloadConf) throws IOException {
2128    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
2129      @Override
2130      public void call(MasterObserver observer) throws IOException {
2131        observer.postUpdateMasterConfiguration(this, postReloadConf);
2132      }
2133    });
2134  }
2135}