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