You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2017/09/27 19:45:36 UTC

[5/7] hbase git commit: HBASE-17732 Coprocessor Design Improvements

http://git-wip-us.apache.org/repos/asf/hbase/blob/97513466/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
index 30d8015..56cf496 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
@@ -23,11 +23,9 @@ import java.io.IOException;
 import java.util.List;
 import java.util.Set;
 
-import org.apache.commons.lang3.ClassUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.MetaMutationAnnotation;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ServerName;
@@ -38,13 +36,15 @@ import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.SnapshotDescription;
 import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.coprocessor.BaseEnvironment;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorServiceBackwardCompatiblity;
+import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.MasterObserver;
 import org.apache.hadoop.hbase.coprocessor.MetricsCoprocessor;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
-import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.master.locking.LockProcedure;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.metrics.MetricRegistry;
@@ -65,7 +65,7 @@ import org.apache.yetus.audience.InterfaceAudience;
  */
 @InterfaceAudience.Private
 public class MasterCoprocessorHost
-    extends CoprocessorHost<MasterCoprocessorHost.MasterEnvironment> {
+    extends CoprocessorHost<MasterCoprocessor, MasterCoprocessorEnvironment> {
 
   private static final Log LOG = LogFactory.getLog(MasterCoprocessorHost.class);
 
@@ -73,21 +73,20 @@ public class MasterCoprocessorHost
    * Coprocessor environment extension providing access to master related
    * services.
    */
-  static class MasterEnvironment extends CoprocessorHost.Environment
+  private static class MasterEnvironment extends BaseEnvironment<MasterCoprocessor>
       implements MasterCoprocessorEnvironment {
     private final MasterServices masterServices;
     private final boolean supportGroupCPs;
     private final MetricRegistry metricRegistry;
 
-    public MasterEnvironment(final Class<?> implClass, final Coprocessor impl,
-        final int priority, final int seq, final Configuration conf,
-        final MasterServices services) {
+    public MasterEnvironment(final MasterCoprocessor impl, final int priority, final int seq,
+        final Configuration conf, final MasterServices services) {
       super(impl, priority, seq, conf);
       this.masterServices = services;
       supportGroupCPs = !useLegacyMethod(impl.getClass(),
           "preBalanceRSGroup", ObserverContext.class, String.class);
       this.metricRegistry =
-          MetricsCoprocessor.createRegistryForMasterCoprocessor(implClass.getName());
+          MetricsCoprocessor.createRegistryForMasterCoprocessor(impl.getClass().getName());
     }
 
     @Override
@@ -101,7 +100,7 @@ public class MasterCoprocessorHost
     }
 
     @Override
-    protected void shutdown() {
+    public void shutdown() {
       super.shutdown();
       MetricsCoprocessor.removeRegistry(this.metricRegistry);
     }
@@ -122,120 +121,142 @@ public class MasterCoprocessorHost
     loadSystemCoprocessors(conf, MASTER_COPROCESSOR_CONF_KEY);
   }
 
+
+
   @Override
-  public MasterEnvironment createEnvironment(final Class<?> implClass,
-      final Coprocessor instance, final int priority, final int seq,
-      final Configuration conf) {
-    for (Object itf : ClassUtils.getAllInterfaces(implClass)) {
-      Class<?> c = (Class<?>) itf;
-      if (CoprocessorService.class.isAssignableFrom(c)) {
-        masterServices.registerService(((CoprocessorService)instance).getService());
-      }
+  public MasterEnvironment createEnvironment(final MasterCoprocessor instance, final int priority,
+      final int seq, final Configuration conf) {
+    instance.getService().ifPresent(masterServices::registerService);
+    return new MasterEnvironment(instance, priority, seq, conf, masterServices);
+  }
+
+  @Override
+  public MasterCoprocessor checkAndGetInstance(Class<?> implClass)
+      throws InstantiationException, IllegalAccessException {
+    if (MasterCoprocessor.class.isAssignableFrom(implClass)) {
+      return (MasterCoprocessor)implClass.newInstance();
+    } else if (CoprocessorService.class.isAssignableFrom(implClass)) {
+      // For backward compatibility with old CoprocessorService impl which don't extend
+      // MasterCoprocessor.
+      return new CoprocessorServiceBackwardCompatiblity.MasterCoprocessorService(
+          (CoprocessorService)implClass.newInstance());
+    } else {
+      LOG.error(implClass.getName() + " is not of type MasterCoprocessor. Check the "
+          + "configuration " + CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY);
+      return null;
+    }
+  }
+
+  private ObserverGetter<MasterCoprocessor, MasterObserver> masterObserverGetter =
+      MasterCoprocessor::getMasterObserver;
+
+  abstract class MasterObserverOperation extends
+      ObserverOperationWithoutResult<MasterObserver> {
+    public MasterObserverOperation(){
+      super(masterObserverGetter);
+    }
+
+    public MasterObserverOperation(User user) {
+      super(masterObserverGetter, user);
     }
-    return new MasterEnvironment(implClass, instance, priority, seq, conf,
-        masterServices);
   }
 
+
+  //////////////////////////////////////////////////////////////////////////////////////////////////
+  // MasterObserver operations
+  //////////////////////////////////////////////////////////////////////////////////////////////////
+
+
   public boolean preCreateNamespace(final NamespaceDescriptor ns) throws IOException {
-    return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    return execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preCreateNamespace(ctx, ns);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preCreateNamespace(this, ns);
       }
     });
   }
 
   public void postCreateNamespace(final NamespaceDescriptor ns) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postCreateNamespace(ctx, ns);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postCreateNamespace(this, ns);
       }
     });
   }
 
   public boolean preDeleteNamespace(final String namespaceName) throws IOException {
-    return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    return execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preDeleteNamespace(ctx, namespaceName);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preDeleteNamespace(this, namespaceName);
       }
     });
   }
 
   public void postDeleteNamespace(final String namespaceName) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postDeleteNamespace(ctx, namespaceName);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postDeleteNamespace(this, namespaceName);
       }
     });
   }
 
   public boolean preModifyNamespace(final NamespaceDescriptor ns) throws IOException {
-    return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    return execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preModifyNamespace(ctx, ns);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preModifyNamespace(this, ns);
       }
     });
   }
 
   public void postModifyNamespace(final NamespaceDescriptor ns) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postModifyNamespace(ctx, ns);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postModifyNamespace(this, ns);
       }
     });
   }
 
   public void preGetNamespaceDescriptor(final String namespaceName)
       throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preGetNamespaceDescriptor(ctx, namespaceName);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preGetNamespaceDescriptor(this, namespaceName);
       }
     });
   }
 
   public void postGetNamespaceDescriptor(final NamespaceDescriptor ns)
       throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postGetNamespaceDescriptor(ctx, ns);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postGetNamespaceDescriptor(this, ns);
       }
     });
   }
 
   public boolean preListNamespaceDescriptors(final List<NamespaceDescriptor> descriptors)
       throws IOException {
-    return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    return execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preListNamespaceDescriptors(ctx, descriptors);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preListNamespaceDescriptors(this, descriptors);
       }
     });
   }
 
   public void postListNamespaceDescriptors(final List<NamespaceDescriptor> descriptors)
       throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postListNamespaceDescriptors(ctx, descriptors);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postListNamespaceDescriptors(this, descriptors);
       }
     });
   }
@@ -244,195 +265,175 @@ public class MasterCoprocessorHost
 
   public void preCreateTable(final TableDescriptor htd, final RegionInfo[] regions)
       throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preCreateTable(ctx, htd, regions);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preCreateTable(this, htd, regions);
       }
     });
   }
 
   public void postCreateTable(final TableDescriptor htd, final RegionInfo[] regions)
       throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postCreateTable(ctx, htd, regions);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postCreateTable(this, htd, regions);
       }
     });
   }
 
   public void preCreateTableAction(final TableDescriptor htd, final RegionInfo[] regions,
-                                   final User user)
-      throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+      final User user) throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preCreateTableAction(ctx, htd, regions);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preCreateTableAction(this, htd, regions);
       }
     });
   }
 
   public void postCompletedCreateTableAction(
       final TableDescriptor htd, final RegionInfo[] regions, final User user) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postCompletedCreateTableAction(ctx, htd, regions);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postCompletedCreateTableAction(this, htd, regions);
       }
     });
   }
 
   public void preDeleteTable(final TableName tableName) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preDeleteTable(ctx, tableName);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preDeleteTable(this, tableName);
       }
     });
   }
 
   public void postDeleteTable(final TableName tableName) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postDeleteTable(ctx, tableName);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postDeleteTable(this, tableName);
       }
     });
   }
 
   public void preDeleteTableAction(final TableName tableName, final User user) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preDeleteTableAction(ctx, tableName);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preDeleteTableAction(this, tableName);
       }
     });
   }
 
   public void postCompletedDeleteTableAction(final TableName tableName, final User user)
       throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postCompletedDeleteTableAction(ctx, tableName);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postCompletedDeleteTableAction(this, tableName);
       }
     });
   }
 
   public void preTruncateTable(final TableName tableName) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preTruncateTable(ctx, tableName);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preTruncateTable(this, tableName);
       }
     });
   }
 
   public void postTruncateTable(final TableName tableName) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postTruncateTable(ctx, tableName);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postTruncateTable(this, tableName);
       }
     });
   }
 
-  public void preTruncateTableAction(final TableName tableName, final User user) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+  public void preTruncateTableAction(final TableName tableName, final User user)
+      throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preTruncateTableAction(ctx, tableName);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preTruncateTableAction(this, tableName);
       }
     });
   }
 
   public void postCompletedTruncateTableAction(final TableName tableName, final User user)
       throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postCompletedTruncateTableAction(ctx, tableName);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postCompletedTruncateTableAction(this, tableName);
       }
     });
   }
 
   public void preModifyTable(final TableName tableName, final TableDescriptor htd)
       throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preModifyTable(ctx, tableName, htd);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preModifyTable(this, tableName, htd);
       }
     });
   }
 
   public void postModifyTable(final TableName tableName, final TableDescriptor htd)
       throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postModifyTable(ctx, tableName, htd);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postModifyTable(this, tableName, htd);
       }
     });
   }
 
   public void preModifyTableAction(final TableName tableName, final TableDescriptor htd,
-                                   final User user)
-      throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+      final User user) throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preModifyTableAction(ctx, tableName, htd);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preModifyTableAction(this, tableName, htd);
       }
     });
   }
 
   public void postCompletedModifyTableAction(final TableName tableName, final TableDescriptor htd,
-                                             final User user)
-      throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+      final User user) throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postCompletedModifyTableAction(ctx, tableName, htd);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postCompletedModifyTableAction(this, tableName, htd);
       }
     });
   }
 
   public boolean preAddColumn(final TableName tableName, final ColumnFamilyDescriptor columnFamily)
       throws IOException {
-    return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    return execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preAddColumnFamily(ctx, tableName, columnFamily);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preAddColumnFamily(this, tableName, columnFamily);
       }
     });
   }
 
   public void postAddColumn(final TableName tableName, final ColumnFamilyDescriptor columnFamily)
       throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postAddColumnFamily(ctx, tableName, columnFamily);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postAddColumnFamily(this, tableName, columnFamily);
       }
     });
   }
@@ -442,11 +443,10 @@ public class MasterCoprocessorHost
       final ColumnFamilyDescriptor columnFamily,
       final User user)
       throws IOException {
-    return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+    return execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preAddColumnFamilyAction(ctx, tableName, columnFamily);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preAddColumnFamilyAction(this, tableName, columnFamily);
       }
     });
   }
@@ -456,33 +456,30 @@ public class MasterCoprocessorHost
       final ColumnFamilyDescriptor columnFamily,
       final User user)
       throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postCompletedAddColumnFamilyAction(ctx, tableName, columnFamily);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postCompletedAddColumnFamilyAction(this, tableName, columnFamily);
       }
     });
   }
 
-  public boolean preModifyColumn(final TableName tableName, final ColumnFamilyDescriptor columnFamily)
-      throws IOException {
-    return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+  public boolean preModifyColumn(final TableName tableName,
+      final ColumnFamilyDescriptor columnFamily) throws IOException {
+    return execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preModifyColumnFamily(ctx, tableName, columnFamily);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preModifyColumnFamily(this, tableName, columnFamily);
       }
     });
   }
 
   public void postModifyColumn(final TableName tableName, final ColumnFamilyDescriptor columnFamily)
       throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postModifyColumnFamily(ctx, tableName, columnFamily);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postModifyColumnFamily(this, tableName, columnFamily);
       }
     });
   }
@@ -491,11 +488,10 @@ public class MasterCoprocessorHost
       final TableName tableName,
       final ColumnFamilyDescriptor columnFamily,
       final User user) throws IOException {
-    return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+    return execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preModifyColumnFamilyAction(ctx, tableName, columnFamily);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preModifyColumnFamilyAction(this, tableName, columnFamily);
       }
     });
   }
@@ -504,33 +500,30 @@ public class MasterCoprocessorHost
       final TableName tableName,
       final ColumnFamilyDescriptor columnFamily,
       final User user) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postCompletedModifyColumnFamilyAction(ctx, tableName, columnFamily);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postCompletedModifyColumnFamilyAction(this, tableName, columnFamily);
       }
     });
   }
 
   public boolean preDeleteColumn(final TableName tableName, final byte[] columnFamily)
       throws IOException {
-    return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    return execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preDeleteColumnFamily(ctx, tableName, columnFamily);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preDeleteColumnFamily(this, tableName, columnFamily);
       }
     });
   }
 
   public void postDeleteColumn(final TableName tableName, final byte[] columnFamily)
       throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postDeleteColumnFamily(ctx, tableName, columnFamily);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postDeleteColumnFamily(this, tableName, columnFamily);
       }
     });
   }
@@ -540,104 +533,94 @@ public class MasterCoprocessorHost
       final byte[] columnFamily,
       final User user)
       throws IOException {
-    return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+    return execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preDeleteColumnFamilyAction(ctx, tableName, columnFamily);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preDeleteColumnFamilyAction(this, tableName, columnFamily);
       }
     });
   }
 
   public void postCompletedDeleteColumnFamilyAction(
       final TableName tableName, final byte[] columnFamily, final User user) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postCompletedDeleteColumnFamilyAction(ctx, tableName, columnFamily);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postCompletedDeleteColumnFamilyAction(this, tableName, columnFamily);
       }
     });
   }
 
   public void preEnableTable(final TableName tableName) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preEnableTable(ctx, tableName);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preEnableTable(this, tableName);
       }
     });
   }
 
   public void postEnableTable(final TableName tableName) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postEnableTable(ctx, tableName);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postEnableTable(this, tableName);
       }
     });
   }
 
   public void preEnableTableAction(final TableName tableName, final User user) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preEnableTableAction(ctx, tableName);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preEnableTableAction(this, tableName);
       }
     });
   }
 
   public void postCompletedEnableTableAction(final TableName tableName, final User user)
       throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postCompletedEnableTableAction(ctx, tableName);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postCompletedEnableTableAction(this, tableName);
       }
     });
   }
 
   public void preDisableTable(final TableName tableName) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preDisableTable(ctx, tableName);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preDisableTable(this, tableName);
       }
     });
   }
 
   public void postDisableTable(final TableName tableName) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postDisableTable(ctx, tableName);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postDisableTable(this, tableName);
       }
     });
   }
 
   public void preDisableTableAction(final TableName tableName, final User user) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preDisableTableAction(ctx, tableName);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preDisableTableAction(this, tableName);
       }
     });
   }
 
   public void postCompletedDisableTableAction(final TableName tableName, final User user)
       throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postCompletedDisableTableAction(ctx, tableName);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postCompletedDisableTableAction(this, tableName);
       }
     });
   }
@@ -645,208 +628,188 @@ public class MasterCoprocessorHost
   public boolean preAbortProcedure(
       final ProcedureExecutor<MasterProcedureEnv> procEnv,
       final long procId) throws IOException {
-    return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    return execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preAbortProcedure(ctx, procEnv, procId);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preAbortProcedure(this, procEnv, procId);
       }
     });
   }
 
   public void postAbortProcedure() throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postAbortProcedure(ctx);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postAbortProcedure(this);
       }
     });
   }
 
   public boolean preGetProcedures() throws IOException {
-    return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    return execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preGetProcedures(ctx);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preGetProcedures(this);
       }
     });
   }
 
   public void postGetProcedures(final List<Procedure<?>> procInfoList) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postGetProcedures(ctx, procInfoList);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postGetProcedures(this, procInfoList);
       }
     });
   }
 
   public boolean preGetLocks() throws IOException {
-    return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    return execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preGetLocks(ctx);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preGetLocks(this);
       }
     });
   }
 
   public void postGetLocks(final List<LockedResource> lockedResources) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postGetLocks(ctx, lockedResources);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postGetLocks(this, lockedResources);
       }
     });
   }
 
   public boolean preMove(final RegionInfo region, final ServerName srcServer,
-                         final ServerName destServer) throws IOException {
-    return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+      final ServerName destServer) throws IOException {
+    return execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preMove(ctx, region, srcServer, destServer);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preMove(this, region, srcServer, destServer);
       }
     });
   }
 
   public void postMove(final RegionInfo region, final ServerName srcServer,
       final ServerName destServer) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postMove(ctx, region, srcServer, destServer);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postMove(this, region, srcServer, destServer);
       }
     });
   }
 
   public boolean preAssign(final RegionInfo regionInfo) throws IOException {
-    return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    return execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preAssign(ctx, regionInfo);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preAssign(this, regionInfo);
       }
     });
   }
 
   public void postAssign(final RegionInfo regionInfo) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postAssign(ctx, regionInfo);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postAssign(this, regionInfo);
       }
     });
   }
 
   public boolean preUnassign(final RegionInfo regionInfo, final boolean force)
       throws IOException {
-    return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    return execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preUnassign(ctx, regionInfo, force);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preUnassign(this, regionInfo, force);
       }
     });
   }
 
   public void postUnassign(final RegionInfo regionInfo, final boolean force) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postUnassign(ctx, regionInfo, force);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postUnassign(this, regionInfo, force);
       }
     });
   }
 
   public void preRegionOffline(final RegionInfo regionInfo) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preRegionOffline(ctx, regionInfo);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preRegionOffline(this, regionInfo);
       }
     });
   }
 
   public void postRegionOffline(final RegionInfo regionInfo) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postRegionOffline(ctx, regionInfo);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postRegionOffline(this, regionInfo);
       }
     });
   }
 
   public void preMergeRegions(final RegionInfo[] regionsToMerge)
       throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preMergeRegions(ctx, regionsToMerge);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preMergeRegions(this, regionsToMerge);
       }
     });
   }
 
   public void postMergeRegions(final RegionInfo[] regionsToMerge)
       throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postMergeRegions(ctx, regionsToMerge);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postMergeRegions(this, regionsToMerge);
       }
     });
   }
 
   public boolean preBalance() throws IOException {
-    return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    return execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preBalance(ctx);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preBalance(this);
       }
     });
   }
 
   public void postBalance(final List<RegionPlan> plans) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postBalance(ctx, plans);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postBalance(this, plans);
       }
     });
   }
 
   public boolean preSetSplitOrMergeEnabled(final boolean newValue,
       final MasterSwitchType switchType) throws IOException {
-    return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    return execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preSetSplitOrMergeEnabled(ctx, newValue, switchType);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preSetSplitOrMergeEnabled(this, newValue, switchType);
       }
     });
   }
 
   public void postSetSplitOrMergeEnabled(final boolean newValue,
       final MasterSwitchType switchType) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postSetSplitOrMergeEnabled(ctx, newValue, switchType);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postSetSplitOrMergeEnabled(this, newValue, switchType);
       }
     });
   }
@@ -860,11 +823,10 @@ public class MasterCoprocessorHost
   public void preSplitRegion(
       final TableName tableName,
       final byte[] splitRow) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preSplitRegion(ctx, tableName, splitRow);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preSplitRegion(this, tableName, splitRow);
       }
     });
   }
@@ -880,11 +842,10 @@ public class MasterCoprocessorHost
       final TableName tableName,
       final byte[] splitRow,
       final User user) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preSplitRegionAction(ctx, tableName, splitRow);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preSplitRegionAction(this, tableName, splitRow);
       }
     });
   }
@@ -900,11 +861,10 @@ public class MasterCoprocessorHost
       final RegionInfo regionInfoA,
       final RegionInfo regionInfoB,
       final User user) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postCompletedSplitRegionAction(ctx, regionInfoA, regionInfoB);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postCompletedSplitRegionAction(this, regionInfoA, regionInfoB);
       }
     });
   }
@@ -920,11 +880,10 @@ public class MasterCoprocessorHost
       final byte[] splitKey,
       final List<Mutation> metaEntries,
       final User user) throws IOException {
-    return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+    return execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preSplitRegionBeforePONRAction(ctx, splitKey, metaEntries);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preSplitRegionBeforePONRAction(this, splitKey, metaEntries);
       }
     });
   }
@@ -935,11 +894,10 @@ public class MasterCoprocessorHost
    * @throws IOException
    */
   public void preSplitAfterPONRAction(final User user) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preSplitRegionAfterPONRAction(ctx);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preSplitRegionAfterPONRAction(this);
       }
     });
   }
@@ -950,11 +908,10 @@ public class MasterCoprocessorHost
    * @throws IOException
    */
   public void postRollBackSplitRegionAction(final User user) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postRollBackSplitRegionAction(ctx);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postRollBackSplitRegionAction(this);
       }
     });
   }
@@ -967,11 +924,10 @@ public class MasterCoprocessorHost
    */
   public boolean preMergeRegionsAction(
       final RegionInfo[] regionsToMerge, final User user) throws IOException {
-    return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+    return execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
       @Override
-      public void call(MasterObserver oserver,
-          ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
-        oserver.preMergeRegionsAction(ctx, regionsToMerge);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preMergeRegionsAction(this, regionsToMerge);
       }
     });
   }
@@ -987,11 +943,10 @@ public class MasterCoprocessorHost
       final RegionInfo[] regionsToMerge,
       final RegionInfo mergedRegion,
       final User user) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
       @Override
-      public void call(MasterObserver oserver,
-          ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
-        oserver.postCompletedMergeRegionsAction(ctx, regionsToMerge, mergedRegion);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postCompletedMergeRegionsAction(this, regionsToMerge, mergedRegion);
       }
     });
   }
@@ -1007,11 +962,10 @@ public class MasterCoprocessorHost
       final RegionInfo[] regionsToMerge,
       final @MetaMutationAnnotation List<Mutation> metaEntries,
       final User user) throws IOException {
-    return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+    return execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
       @Override
-      public void call(MasterObserver oserver,
-          ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
-        oserver.preMergeRegionsCommitAction(ctx, regionsToMerge, metaEntries);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preMergeRegionsCommitAction(this, regionsToMerge, metaEntries);
       }
     });
   }
@@ -1027,11 +981,10 @@ public class MasterCoprocessorHost
       final RegionInfo[] regionsToMerge,
       final RegionInfo mergedRegion,
       final User user) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
       @Override
-      public void call(MasterObserver oserver,
-          ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
-        oserver.postMergeRegionsCommitAction(ctx, regionsToMerge, mergedRegion);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postMergeRegionsCommitAction(this, regionsToMerge, mergedRegion);
       }
     });
   }
@@ -1044,33 +997,30 @@ public class MasterCoprocessorHost
    */
   public void postRollBackMergeRegionsAction(
       final RegionInfo[] regionsToMerge, final User user) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
       @Override
-      public void call(MasterObserver oserver,
-          ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
-        oserver.postRollBackMergeRegionsAction(ctx, regionsToMerge);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postRollBackMergeRegionsAction(this, regionsToMerge);
       }
     });
   }
 
   public boolean preBalanceSwitch(final boolean b) throws IOException {
-    return execOperationWithResult(b, coprocessors.isEmpty() ? null :
-        new CoprocessorOperationWithResult<Boolean>() {
-      @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        setResult(oserver.preBalanceSwitch(ctx, getResult()));
-      }
-    });
+    return execOperationWithResult(b, coprocEnvironments.isEmpty() ? null :
+        new ObserverOperationWithResult<MasterObserver, Boolean>(masterObserverGetter) {
+          @Override
+          public Boolean call(MasterObserver observer) throws IOException {
+            return observer.preBalanceSwitch(this, getResult());
+          }
+        });
   }
 
   public void postBalanceSwitch(final boolean oldValue, final boolean newValue)
       throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postBalanceSwitch(ctx, oldValue, newValue);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postBalanceSwitch(this, oldValue, newValue);
       }
     });
   }
@@ -1078,16 +1028,15 @@ public class MasterCoprocessorHost
   public void preShutdown() throws IOException {
     // While stopping the cluster all coprocessors method should be executed first then the
     // coprocessor should be cleaned up.
-    execShutdown(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execShutdown(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preShutdown(ctx);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preShutdown(this);
       }
       @Override
-      public void postEnvCall(MasterEnvironment env) {
+      public void postEnvCall() {
         // invoke coprocessor stop method
-        shutdown(env);
+        shutdown(this.getEnvironment());
       }
     });
   }
@@ -1095,228 +1044,207 @@ public class MasterCoprocessorHost
   public void preStopMaster() throws IOException {
     // While stopping master all coprocessors method should be executed first then the coprocessor
     // environment should be cleaned up.
-    execShutdown(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execShutdown(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preStopMaster(ctx);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preStopMaster(this);
       }
       @Override
-      public void postEnvCall(MasterEnvironment env) {
+      public void postEnvCall() {
         // invoke coprocessor stop method
-        shutdown(env);
+        shutdown(this.getEnvironment());
       }
     });
   }
 
   public void preMasterInitialization() throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preMasterInitialization(ctx);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preMasterInitialization(this);
       }
     });
   }
 
   public void postStartMaster() throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postStartMaster(ctx);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postStartMaster(this);
       }
     });
   }
 
   public void preSnapshot(final SnapshotDescription snapshot,
       final TableDescriptor hTableDescriptor) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preSnapshot(ctx, snapshot, hTableDescriptor);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preSnapshot(this, snapshot, hTableDescriptor);
       }
     });
   }
 
   public void postSnapshot(final SnapshotDescription snapshot,
       final TableDescriptor hTableDescriptor) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postSnapshot(ctx, snapshot, hTableDescriptor);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postSnapshot(this, snapshot, hTableDescriptor);
       }
     });
   }
 
   public void preListSnapshot(final SnapshotDescription snapshot) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver observer, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        observer.preListSnapshot(ctx, snapshot);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preListSnapshot(this, snapshot);
       }
     });
   }
 
   public void postListSnapshot(final SnapshotDescription snapshot) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver observer, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        observer.postListSnapshot(ctx, snapshot);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postListSnapshot(this, snapshot);
       }
     });
   }
 
   public void preCloneSnapshot(final SnapshotDescription snapshot,
       final TableDescriptor hTableDescriptor) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preCloneSnapshot(ctx, snapshot, hTableDescriptor);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preCloneSnapshot(this, snapshot, hTableDescriptor);
       }
     });
   }
 
   public void postCloneSnapshot(final SnapshotDescription snapshot,
       final TableDescriptor hTableDescriptor) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postCloneSnapshot(ctx, snapshot, hTableDescriptor);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postCloneSnapshot(this, snapshot, hTableDescriptor);
       }
     });
   }
 
   public void preRestoreSnapshot(final SnapshotDescription snapshot,
       final TableDescriptor hTableDescriptor) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preRestoreSnapshot(ctx, snapshot, hTableDescriptor);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preRestoreSnapshot(this, snapshot, hTableDescriptor);
       }
     });
   }
 
   public void postRestoreSnapshot(final SnapshotDescription snapshot,
       final TableDescriptor hTableDescriptor) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postRestoreSnapshot(ctx, snapshot, hTableDescriptor);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postRestoreSnapshot(this, snapshot, hTableDescriptor);
       }
     });
   }
 
   public void preDeleteSnapshot(final SnapshotDescription snapshot) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preDeleteSnapshot(ctx, snapshot);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preDeleteSnapshot(this, snapshot);
       }
     });
   }
 
   public void postDeleteSnapshot(final SnapshotDescription snapshot) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postDeleteSnapshot(ctx, snapshot);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postDeleteSnapshot(this, snapshot);
       }
     });
   }
 
   public boolean preGetTableDescriptors(final List<TableName> tableNamesList,
       final List<TableDescriptor> descriptors, final String regex) throws IOException {
-    return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    return execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preGetTableDescriptors(ctx, tableNamesList, descriptors, regex);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preGetTableDescriptors(this, tableNamesList, descriptors, regex);
       }
     });
   }
 
   public void postGetTableDescriptors(final List<TableName> tableNamesList,
       final List<TableDescriptor> descriptors, final String regex) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postGetTableDescriptors(ctx, tableNamesList, descriptors, regex);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postGetTableDescriptors(this, tableNamesList, descriptors, regex);
       }
     });
   }
 
   public boolean preGetTableNames(final List<TableDescriptor> descriptors,
       final String regex) throws IOException {
-    return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    return execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preGetTableNames(ctx, descriptors, regex);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preGetTableNames(this, descriptors, regex);
       }
     });
   }
 
   public void postGetTableNames(final List<TableDescriptor> descriptors,
       final String regex) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postGetTableNames(ctx, descriptors, regex);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postGetTableNames(this, descriptors, regex);
       }
     });
   }
 
   public void preTableFlush(final TableName tableName) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preTableFlush(ctx, tableName);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preTableFlush(this, tableName);
       }
     });
   }
 
   public void postTableFlush(final TableName tableName) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postTableFlush(ctx, tableName);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postTableFlush(this, tableName);
       }
     });
   }
 
   public void preSetUserQuota(
       final String user, final GlobalQuotaSettings quotas) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preSetUserQuota(ctx, user, quotas);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preSetUserQuota(this, user, quotas);
       }
     });
   }
 
   public void postSetUserQuota(
       final String user, final GlobalQuotaSettings quotas) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postSetUserQuota(ctx, user, quotas);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postSetUserQuota(this, user, quotas);
       }
     });
   }
@@ -1324,11 +1252,10 @@ public class MasterCoprocessorHost
   public void preSetUserQuota(
       final String user, final TableName table, final GlobalQuotaSettings quotas)
           throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preSetUserQuota(ctx, user, table, quotas);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preSetUserQuota(this, user, table, quotas);
       }
     });
   }
@@ -1336,11 +1263,10 @@ public class MasterCoprocessorHost
   public void postSetUserQuota(
       final String user, final TableName table, final GlobalQuotaSettings quotas)
           throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postSetUserQuota(ctx, user, table, quotas);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postSetUserQuota(this, user, table, quotas);
       }
     });
   }
@@ -1348,11 +1274,10 @@ public class MasterCoprocessorHost
   public void preSetUserQuota(
       final String user, final String namespace, final GlobalQuotaSettings quotas)
           throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preSetUserQuota(ctx, user, namespace, quotas);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preSetUserQuota(this, user, namespace, quotas);
       }
     });
   }
@@ -1360,186 +1285,73 @@ public class MasterCoprocessorHost
   public void postSetUserQuota(
       final String user, final String namespace, final GlobalQuotaSettings quotas)
           throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postSetUserQuota(ctx, user, namespace, quotas);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postSetUserQuota(this, user, namespace, quotas);
       }
     });
   }
 
   public void preSetTableQuota(
       final TableName table, final GlobalQuotaSettings quotas) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preSetTableQuota(ctx, table, quotas);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preSetTableQuota(this, table, quotas);
       }
     });
   }
 
   public void postSetTableQuota(
       final TableName table, final GlobalQuotaSettings quotas) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postSetTableQuota(ctx, table, quotas);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postSetTableQuota(this, table, quotas);
       }
     });
   }
 
   public void preSetNamespaceQuota(
       final String namespace, final GlobalQuotaSettings quotas) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preSetNamespaceQuota(ctx, namespace, quotas);
+      public void call(MasterObserver observer) throws IOException {
+        observer.preSetNamespaceQuota(this, namespace, quotas);
       }
     });
   }
 
   public void postSetNamespaceQuota(
       final String namespace, final GlobalQuotaSettings quotas) throws IOException{
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postSetNamespaceQuota(ctx, namespace, quotas);
+      public void call(MasterObserver observer) throws IOException {
+        observer.postSetNamespaceQuota(this, namespace, quotas);
       }
     });
   }
 
-  private static abstract class CoprocessorOperation
-      extends ObserverContext<MasterCoprocessorEnvironment> {
-    public CoprocessorOperation() {
-      this(RpcServer.getRequestUser());
-    }
-
-    public CoprocessorOperation(User user) {
-      super(user);
-    }
-
-    public abstract void call(MasterObserver oserver,
-        ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException;
-
-    public void postEnvCall(MasterEnvironment env) {
-    }
-  }
-
-  private static abstract class CoprocessorOperationWithResult<T> extends CoprocessorOperation {
-    private T result = null;
-    public void setResult(final T result) { this.result = result; }
-    public T getResult() { return this.result; }
-  }
-
-  private <T> T execOperationWithResult(final T defaultValue,
-      final CoprocessorOperationWithResult<T> ctx) throws IOException {
-    if (ctx == null) return defaultValue;
-    ctx.setResult(defaultValue);
-    execOperation(ctx);
-    return ctx.getResult();
-  }
-
-  private boolean execOperation(final CoprocessorOperation ctx) throws IOException {
-    if (ctx == null) return false;
-    boolean bypass = false;
-    List<MasterEnvironment> envs = coprocessors.get();
-    for (int i = 0; i < envs.size(); i++) {
-      MasterEnvironment env = envs.get(i);
-      if (env.getInstance() instanceof MasterObserver) {
-        ctx.prepare(env);
-        Thread currentThread = Thread.currentThread();
-        ClassLoader cl = currentThread.getContextClassLoader();
-        try {
-          currentThread.setContextClassLoader(env.getClassLoader());
-          ctx.call((MasterObserver)env.getInstance(), ctx);
-        } catch (Throwable e) {
-          handleCoprocessorThrowable(env, e);
-        } finally {
-          currentThread.setContextClassLoader(cl);
-        }
-        bypass |= ctx.shouldBypass();
-        if (ctx.shouldComplete()) {
-          break;
-        }
-      }
-      ctx.postEnvCall(env);
-    }
-    return bypass;
-  }
-
-  /**
-   * Master coprocessor classes can be configured in any order, based on that priority is set and
-   * chained in a sorted order. For preStopMaster()/preShutdown(), coprocessor methods are invoked
-   * in call() and environment is shutdown in postEnvCall(). <br>
-   * Need to execute all coprocessor methods first then postEnvCall(), otherwise some coprocessors
-   * may remain shutdown if any exception occurs during next coprocessor execution which prevent
-   * Master stop or cluster shutdown. (Refer:
-   * <a href="https://issues.apache.org/jira/browse/HBASE-16663">HBASE-16663</a>
-   * @param ctx CoprocessorOperation
-   * @return true if bypaas coprocessor execution, false if not.
-   * @throws IOException
-   */
-  private boolean execShutdown(final CoprocessorOperation ctx) throws IOException {
-    if (ctx == null) return false;
-    boolean bypass = false;
-    List<MasterEnvironment> envs = coprocessors.get();
-    int envsSize = envs.size();
-    // Iterate the coprocessors and execute CoprocessorOperation's call()
-    for (int i = 0; i < envsSize; i++) {
-      MasterEnvironment env = envs.get(i);
-      if (env.getInstance() instanceof MasterObserver) {
-        ctx.prepare(env);
-        Thread currentThread = Thread.currentThread();
-        ClassLoader cl = currentThread.getContextClassLoader();
-        try {
-          currentThread.setContextClassLoader(env.getClassLoader());
-          ctx.call((MasterObserver) env.getInstance(), ctx);
-        } catch (Throwable e) {
-          handleCoprocessorThrowable(env, e);
-        } finally {
-          currentThread.setContextClassLoader(cl);
-        }
-        bypass |= ctx.shouldBypass();
-        if (ctx.shouldComplete()) {
-          break;
-        }
-      }
-    }
-
-    // Iterate the coprocessors and execute CoprocessorOperation's postEnvCall()
-    for (int i = 0; i < envsSize; i++) {
-      MasterEnvironment env = envs.get(i);
-      ctx.postEnvCall(env);
-    }
-    return bypass;
-  }
-
-  public void preMoveServersAndTables(final Set<Address> servers, final Set<TableName> tables, final String targetGroup)
-          throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+  public void preMoveServersAndTables(final Set<Address> servers, final Set<TableName> tables,
+      final String targetGroup) throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver,
-                       ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
-        if(((MasterEnvironment)ctx.getEnvironment()).supportGroupCPs) {
-          oserver.preMoveServersAndTables(ctx, servers, tables, targetGroup);
+      public void call(MasterObserver observer) throws IOException {
+        if(((MasterEnvironment)getEnvironment()).supportGroupCPs) {
+          observer.preMoveServersAndTables(this, servers, tables, targetGroup);
         }
       }
     });
   }
 
-  public void postMoveServersAndTables(final Set<Address> servers, final Set<TableName> tables, final String targetGroup)
-          throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+  public void postMoveServersAndTables(final Set<Address> servers, final Set<TableName> tables,
+      final String targetGroup) throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver,
-                       ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
-        if(((MasterEnvironment)ctx.getEnvironment()).supportGroupCPs) {
-          oserver.postMoveServersAndTables(ctx, servers, tables, targetGroup);
+      public void call(MasterObserver observer) throws IOException {
+        if(((MasterEnvironment)getEnvironment()).supportGroupCPs) {
+          observer.postMoveServersAndTables(this, servers, tables, targetGroup);
         }
       }
     });
@@ -1547,12 +1359,11 @@ public class MasterCoprocessorHost
 
   public void preMoveServers(final Set<Address> servers, final String targetGroup)
       throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver,
-          ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
-        if(((MasterEnvironment)ctx.getEnvironment()).supportGroupCPs) {
-          oserver.preMoveServers(ctx, servers, targetGroup);
+      public void call(MasterObserver observer) throws IOException {
+        if(((MasterEnvironment)getEnvironment()).supportGroupCPs) {
+          observer.preMoveServers(this, servers, targetGroup);
         }
       }
     });
@@ -1560,12 +1371,11 @@ public class MasterCoprocessorHost
 
   public void postMoveServers(final Set<Address> servers, final String targetGroup)
       throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver,
-          ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
-        if(((MasterEnvironment)ctx.getEnvironment()).supportGroupCPs) {
-          oserver.postMoveServers(ctx, servers, targetGroup);
+      public void call(MasterObserver observer) throws IOException {
+        if(((MasterEnvironment)getEnvironment()).supportGroupCPs) {
+          observer.postMoveServers(this, servers, targetGroup);
         }
       }
     });
@@ -1573,12 +1383,11 @@ public class MasterCoprocessorHost
 
   public void preMoveTables(final Set<TableName> tables, final String targetGroup)
       throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver,
-          ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
-        if(((MasterEnvironment)ctx.getEnvironment()).supportGroupCPs) {
-          oserver.preMoveTables(ctx, tables, targetGroup);
+      public void call(MasterObserver observer) throws IOException {
+        if(((MasterEnvironment)getEnvironment()).supportGroupCPs) {
+          observer.preMoveTables(this, tables, targetGroup);
         }
       }
     });
@@ -1586,12 +1395,11 @@ public class MasterCoprocessorHost
 
   public void postMoveTables(final Set<TableName> tables, final String targetGroup)
       throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
-      public void call(MasterObserver oserver,
-          ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
-        if(((MasterEnvironment)ctx.getEnvironment()).supportGroupCPs) {
-          oserver.postMoveTables(ctx, tables, targetGroup);
+      public void call(MasterObserver observer) throws IOException {
+        if(((MasterEnvironment)getEnvironment()).supportGroupCPs) {
+          observer.postMoveTables(this, tables, targetGroup);
         }
       }
     });
@@ -1599,12 +1407,11 @@ public class MasterCoprocessorHost
 
   public void preAddRSGroup(final String name)
       throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    execOp

<TRUNCATED>