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