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