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