You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ga...@apache.org on 2011/04/13 07:41:26 UTC

svn commit: r1091655 [1/2] - in /hbase/trunk: ./ src/main/java/org/apache/hadoop/hbase/coprocessor/ src/main/java/org/apache/hadoop/hbase/master/ src/main/java/org/apache/hadoop/hbase/regionserver/ src/main/java/org/apache/hadoop/hbase/regionserver/wal...

Author: garyh
Date: Wed Apr 13 05:41:25 2011
New Revision: 1091655

URL: http://svn.apache.org/viewvc?rev=1091655&view=rev
Log:
HBASE-3759  Eliminate use of ThreadLocals for CoprocessorEnvironment bypass() and complete()

Added:
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/ObserverContext.java
Modified:
    hbase/trunk/CHANGES.txt
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserverCoprocessor.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorEnvironment.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/coprocessor/SampleRegionWALObserver.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java

Modified: hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/trunk/CHANGES.txt?rev=1091655&r1=1091654&r2=1091655&view=diff
==============================================================================
--- hbase/trunk/CHANGES.txt (original)
+++ hbase/trunk/CHANGES.txt Wed Apr 13 05:41:25 2011
@@ -157,6 +157,8 @@ Release 0.91.0 - Unreleased
                (Erik Onnen via Stack)
    HBASE-3765  metrics.xml - small format change and adding nav to hbase
                book metrics section (Doug Meil)
+   HBASE-3759  Eliminate use of ThreadLocals for CoprocessorEnvironment
+               bypass() and complete() 
 
   TASKS
    HBASE-3559  Move report of split to master OFF the heartbeat channel

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java?rev=1091655&r1=1091654&r2=1091655&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java Wed Apr 13 05:41:25 2011
@@ -30,142 +30,145 @@ import java.io.IOException;
 
 public class BaseMasterObserver implements MasterObserver {
   @Override
-  public void preCreateTable(MasterCoprocessorEnvironment env,
+  public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> env,
       HTableDescriptor desc, byte[][] splitKeys) throws IOException {
   }
 
   @Override
-  public void postCreateTable(MasterCoprocessorEnvironment env,
+  public void postCreateTable(ObserverContext<MasterCoprocessorEnvironment> env,
       HRegionInfo[] regions, boolean sync) throws IOException {
   }
 
   @Override
-  public void preDeleteTable(MasterCoprocessorEnvironment env, byte[] tableName)
-      throws IOException {
+  public void preDeleteTable(ObserverContext<MasterCoprocessorEnvironment> env,
+      byte[] tableName) throws IOException {
   }
 
   @Override
-  public void postDeleteTable(MasterCoprocessorEnvironment env, byte[] tableName)
-      throws IOException {
+  public void postDeleteTable(ObserverContext<MasterCoprocessorEnvironment> env,
+      byte[] tableName) throws IOException {
   }
 
   @Override
-  public void preModifyTable(MasterCoprocessorEnvironment env,
+  public void preModifyTable(ObserverContext<MasterCoprocessorEnvironment> env,
       byte[] tableName, HTableDescriptor htd) throws IOException {
   }
 
   @Override
-  public void postModifyTable(MasterCoprocessorEnvironment env,
+  public void postModifyTable(ObserverContext<MasterCoprocessorEnvironment> env,
       byte[] tableName, HTableDescriptor htd) throws IOException {
   }
 
   @Override
-  public void preAddColumn(MasterCoprocessorEnvironment env,
+  public void preAddColumn(ObserverContext<MasterCoprocessorEnvironment> env,
       byte[] tableName, HColumnDescriptor column) throws IOException {
   }
 
   @Override
-  public void postAddColumn(MasterCoprocessorEnvironment env, byte[] tableName,
-      HColumnDescriptor column) throws IOException {
+  public void postAddColumn(ObserverContext<MasterCoprocessorEnvironment> env,
+      byte[] tableName, HColumnDescriptor column) throws IOException {
   }
 
   @Override
-  public void preModifyColumn(MasterCoprocessorEnvironment env,
+  public void preModifyColumn(ObserverContext<MasterCoprocessorEnvironment> env,
       byte[] tableName, HColumnDescriptor descriptor) throws IOException {
   }
 
   @Override
-  public void postModifyColumn(MasterCoprocessorEnvironment env,
+  public void postModifyColumn(ObserverContext<MasterCoprocessorEnvironment> env,
       byte[] tableName, HColumnDescriptor descriptor) throws IOException {
   }
 
   @Override
-  public void preDeleteColumn(MasterCoprocessorEnvironment env,
+  public void preDeleteColumn(ObserverContext<MasterCoprocessorEnvironment> env,
       byte[] tableName, byte[] c) throws IOException {
   }
 
   @Override
-  public void postDeleteColumn(MasterCoprocessorEnvironment env,
+  public void postDeleteColumn(ObserverContext<MasterCoprocessorEnvironment> env,
       byte[] tableName, byte[] c) throws IOException {
   }
 
   @Override
-  public void preEnableTable(MasterCoprocessorEnvironment env, byte[] tableName)
-      throws IOException {
+  public void preEnableTable(ObserverContext<MasterCoprocessorEnvironment> env,
+      byte[] tableName) throws IOException {
   }
 
   @Override
-  public void postEnableTable(MasterCoprocessorEnvironment env,
+  public void postEnableTable(ObserverContext<MasterCoprocessorEnvironment> env,
       byte[] tableName) throws IOException {
   }
 
   @Override
-  public void preDisableTable(MasterCoprocessorEnvironment env,
+  public void preDisableTable(ObserverContext<MasterCoprocessorEnvironment> env,
       byte[] tableName) throws IOException {
   }
 
   @Override
-  public void postDisableTable(MasterCoprocessorEnvironment env,
+  public void postDisableTable(ObserverContext<MasterCoprocessorEnvironment> env,
       byte[] tableName) throws IOException {
   }
 
   @Override
-  public void preMove(MasterCoprocessorEnvironment env, HRegionInfo region,
-      HServerInfo srcServer, HServerInfo destServer)
+  public void preMove(ObserverContext<MasterCoprocessorEnvironment> env,
+      HRegionInfo region, HServerInfo srcServer, HServerInfo destServer)
   throws UnknownRegionException {
   }
 
   @Override
-  public void postMove(MasterCoprocessorEnvironment env, HRegionInfo region,
-      HServerInfo srcServer, HServerInfo destServer)
+  public void postMove(ObserverContext<MasterCoprocessorEnvironment> env,
+      HRegionInfo region, HServerInfo srcServer, HServerInfo destServer)
   throws UnknownRegionException {
   }
 
   @Override
-  public void preAssign(MasterCoprocessorEnvironment env, byte[] regionName,
-      boolean force) throws IOException {
+  public void preAssign(ObserverContext<MasterCoprocessorEnvironment> env,
+      byte[] regionName, boolean force) throws IOException {
   }
 
   @Override
-  public void postAssign(MasterCoprocessorEnvironment env,
+  public void postAssign(ObserverContext<MasterCoprocessorEnvironment> env,
       HRegionInfo regionInfo) throws IOException {
   }
 
   @Override
-  public void preUnassign(MasterCoprocessorEnvironment env, byte[] regionName,
-      boolean force) throws IOException {
+  public void preUnassign(ObserverContext<MasterCoprocessorEnvironment> env,
+      byte[] regionName, boolean force) throws IOException {
   }
 
   @Override
-  public void postUnassign(MasterCoprocessorEnvironment env,
+  public void postUnassign(ObserverContext<MasterCoprocessorEnvironment> env,
       HRegionInfo regionInfo, boolean force) throws IOException {
   }
 
   @Override
-  public void preBalance(MasterCoprocessorEnvironment env) throws IOException {
+  public void preBalance(ObserverContext<MasterCoprocessorEnvironment> env)
+      throws IOException {
   }
 
   @Override
-  public void postBalance(MasterCoprocessorEnvironment env) throws IOException {
+  public void postBalance(ObserverContext<MasterCoprocessorEnvironment> env)
+      throws IOException {
   }
 
   @Override
-  public boolean preBalanceSwitch(MasterCoprocessorEnvironment env, boolean b)
-      throws IOException {
+  public boolean preBalanceSwitch(ObserverContext<MasterCoprocessorEnvironment> env,
+      boolean b) throws IOException {
     return b;
   }
 
   @Override
-  public void postBalanceSwitch(MasterCoprocessorEnvironment env,
+  public void postBalanceSwitch(ObserverContext<MasterCoprocessorEnvironment> env,
       boolean oldValue, boolean newValue) throws IOException {
   }
 
   @Override
-  public void preShutdown(MasterCoprocessorEnvironment env) throws IOException {
+  public void preShutdown(ObserverContext<MasterCoprocessorEnvironment> env)
+      throws IOException {
   }
 
   @Override
-  public void preStopMaster(MasterCoprocessorEnvironment env)
+  public void preStopMaster(ObserverContext<MasterCoprocessorEnvironment> env)
       throws IOException {
   }
 

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserverCoprocessor.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserverCoprocessor.java?rev=1091655&r1=1091654&r2=1091655&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserverCoprocessor.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserverCoprocessor.java Wed Apr 13 05:41:25 2011
@@ -49,94 +49,97 @@ public abstract class BaseRegionObserver
   public void stop(CoprocessorEnvironment e) { }
 
   @Override
-  public void preOpen(RegionCoprocessorEnvironment e) { }
+  public void preOpen(ObserverContext<RegionCoprocessorEnvironment> e) { }
 
   @Override
-  public void postOpen(RegionCoprocessorEnvironment e) { }
+  public void postOpen(ObserverContext<RegionCoprocessorEnvironment> e) { }
 
   @Override
-  public void preClose(RegionCoprocessorEnvironment e, boolean abortRequested)
-    { }
+  public void preClose(ObserverContext<RegionCoprocessorEnvironment> e,
+      boolean abortRequested) { }
 
   @Override
-  public void postClose(RegionCoprocessorEnvironment e, boolean abortRequested)
-    { }
+  public void postClose(ObserverContext<RegionCoprocessorEnvironment> e,
+      boolean abortRequested) { }
 
   @Override
-  public void preFlush(RegionCoprocessorEnvironment e) { }
+  public void preFlush(ObserverContext<RegionCoprocessorEnvironment> e) { }
 
   @Override
-  public void postFlush(RegionCoprocessorEnvironment e) { }
+  public void postFlush(ObserverContext<RegionCoprocessorEnvironment> e) { }
 
   @Override
-  public void preSplit(RegionCoprocessorEnvironment e) { }
+  public void preSplit(ObserverContext<RegionCoprocessorEnvironment> e) { }
 
   @Override
-  public void postSplit(RegionCoprocessorEnvironment e, HRegion l, HRegion r) { }
+  public void postSplit(ObserverContext<RegionCoprocessorEnvironment> e,
+      HRegion l, HRegion r) { }
 
   @Override
-  public void preCompact(RegionCoprocessorEnvironment e, boolean willSplit) { }
+  public void preCompact(ObserverContext<RegionCoprocessorEnvironment> e,
+      boolean willSplit) { }
 
   @Override
-  public void postCompact(RegionCoprocessorEnvironment e, boolean willSplit) { }
+  public void postCompact(ObserverContext<RegionCoprocessorEnvironment> e,
+      boolean willSplit) { }
 
   @Override
-  public void preGetClosestRowBefore(final RegionCoprocessorEnvironment e,
+  public void preGetClosestRowBefore(final ObserverContext<RegionCoprocessorEnvironment> e,
       final byte [] row, final byte [] family, final Result result)
     throws IOException {
   }
 
   @Override
-  public void postGetClosestRowBefore(final RegionCoprocessorEnvironment e,
+  public void postGetClosestRowBefore(final ObserverContext<RegionCoprocessorEnvironment> e,
       final byte [] row, final byte [] family, final Result result)
       throws IOException {
   }
 
   @Override
-  public void preGet(final RegionCoprocessorEnvironment e, final Get get,
-      final List<KeyValue> results) throws IOException {
+  public void preGet(final ObserverContext<RegionCoprocessorEnvironment> e,
+      final Get get, final List<KeyValue> results) throws IOException {
   }
 
   @Override
-  public void postGet(final RegionCoprocessorEnvironment e, final Get get,
-      final List<KeyValue> results) throws IOException {
+  public void postGet(final ObserverContext<RegionCoprocessorEnvironment> e,
+      final Get get, final List<KeyValue> results) throws IOException {
   }
 
   @Override
-  public boolean preExists(final RegionCoprocessorEnvironment e, final Get get,
-      final boolean exists) throws IOException {
+  public boolean preExists(final ObserverContext<RegionCoprocessorEnvironment> e,
+      final Get get, final boolean exists) throws IOException {
     return exists;
   }
 
   @Override
-  public boolean postExists(final RegionCoprocessorEnvironment e, final Get get,
-      boolean exists) throws IOException {
+  public boolean postExists(final ObserverContext<RegionCoprocessorEnvironment> e,
+      final Get get, boolean exists) throws IOException {
     return exists;
   }
 
   @Override
-  public void prePut(final RegionCoprocessorEnvironment e, final Map<byte[],
+  public void prePut(final ObserverContext<RegionCoprocessorEnvironment> e, final Map<byte[],
       List<KeyValue>> familyMap, final boolean writeToWAL) throws IOException {
   }
 
   @Override
-  public void postPut(final RegionCoprocessorEnvironment e, final Map<byte[],
+  public void postPut(final ObserverContext<RegionCoprocessorEnvironment> e, final Map<byte[],
       List<KeyValue>> familyMap, final boolean writeToWAL) throws IOException {
   }
 
   @Override
-  public void preDelete(final RegionCoprocessorEnvironment e, final Map<byte[],
+  public void preDelete(final ObserverContext<RegionCoprocessorEnvironment> e, final Map<byte[],
       List<KeyValue>> familyMap, final boolean writeToWAL) throws IOException {
   }
 
   @Override
-  public void postDelete(final RegionCoprocessorEnvironment e,
+  public void postDelete(final ObserverContext<RegionCoprocessorEnvironment> e,
       final Map<byte[], List<KeyValue>> familyMap, final boolean writeToWAL)
       throws IOException {
   }
 
   @Override
-  public boolean preCheckAndPut(final RegionCoprocessorEnvironment e,
+  public boolean preCheckAndPut(final ObserverContext<RegionCoprocessorEnvironment> e,
       final byte [] row, final byte [] family, final byte [] qualifier,
       final CompareOp compareOp, final WritableByteArrayComparable comparator,
       final Put put, final boolean result) throws IOException {
@@ -144,7 +147,7 @@ public abstract class BaseRegionObserver
   }
 
   @Override
-  public boolean postCheckAndPut(final RegionCoprocessorEnvironment e,
+  public boolean postCheckAndPut(final ObserverContext<RegionCoprocessorEnvironment> e,
       final byte [] row, final byte [] family, final byte [] qualifier,
       final CompareOp compareOp, final WritableByteArrayComparable comparator,
       final Put put, final boolean result) throws IOException {
@@ -152,7 +155,7 @@ public abstract class BaseRegionObserver
   }
 
   @Override
-  public boolean preCheckAndDelete(final RegionCoprocessorEnvironment e,
+  public boolean preCheckAndDelete(final ObserverContext<RegionCoprocessorEnvironment> e,
       final byte [] row, final byte [] family, final byte [] qualifier,
       final CompareOp compareOp, final WritableByteArrayComparable comparator,
       final Delete delete, final boolean result) throws IOException {
@@ -160,7 +163,7 @@ public abstract class BaseRegionObserver
   }
 
   @Override
-  public boolean postCheckAndDelete(final RegionCoprocessorEnvironment e,
+  public boolean postCheckAndDelete(final ObserverContext<RegionCoprocessorEnvironment> e,
       final byte [] row, final byte [] family, final byte [] qualifier,
       final CompareOp compareOp, final WritableByteArrayComparable comparator,
       final Delete delete, final boolean result) throws IOException {
@@ -168,14 +171,14 @@ public abstract class BaseRegionObserver
   }
 
   @Override
-  public long preIncrementColumnValue(final RegionCoprocessorEnvironment e,
+  public long preIncrementColumnValue(final ObserverContext<RegionCoprocessorEnvironment> e,
       final byte [] row, final byte [] family, final byte [] qualifier,
       final long amount, final boolean writeToWAL) throws IOException {
     return amount;
   }
 
   @Override
-  public long postIncrementColumnValue(final RegionCoprocessorEnvironment e,
+  public long postIncrementColumnValue(final ObserverContext<RegionCoprocessorEnvironment> e,
       final byte [] row, final byte [] family, final byte [] qualifier,
       final long amount, final boolean writeToWAL, long result)
       throws IOException {
@@ -183,58 +186,58 @@ public abstract class BaseRegionObserver
   }
 
   @Override
-  public void preIncrement(final RegionCoprocessorEnvironment e,
+  public void preIncrement(final ObserverContext<RegionCoprocessorEnvironment> e,
       final Increment increment, final Result result) throws IOException {
   }
 
   @Override
-  public void postIncrement(final RegionCoprocessorEnvironment e,
+  public void postIncrement(final ObserverContext<RegionCoprocessorEnvironment> e,
       final Increment increment, final Result result) throws IOException {
   }
 
   @Override
-  public InternalScanner preScannerOpen(final RegionCoprocessorEnvironment e,
+  public InternalScanner preScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> e,
       final Scan scan, final InternalScanner s) throws IOException {
     return s;
   }
 
   @Override
-  public InternalScanner postScannerOpen(final RegionCoprocessorEnvironment e,
+  public InternalScanner postScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> e,
       final Scan scan, final InternalScanner s) throws IOException {
     return s;
   }
 
   @Override
-  public boolean preScannerNext(final RegionCoprocessorEnvironment e,
+  public boolean preScannerNext(final ObserverContext<RegionCoprocessorEnvironment> e,
       final InternalScanner s, final List<Result> results,
       final int limit, final boolean hasMore) throws IOException {
     return hasMore;
   }
 
   @Override
-  public boolean postScannerNext(final RegionCoprocessorEnvironment e,
+  public boolean postScannerNext(final ObserverContext<RegionCoprocessorEnvironment> e,
       final InternalScanner s, final List<Result> results, final int limit,
       final boolean hasMore) throws IOException {
     return hasMore;
   }
 
   @Override
-  public void preScannerClose(final RegionCoprocessorEnvironment e,
+  public void preScannerClose(final ObserverContext<RegionCoprocessorEnvironment> e,
       final InternalScanner s) throws IOException {
   }
 
   @Override
-  public void postScannerClose(final RegionCoprocessorEnvironment e,
+  public void postScannerClose(final ObserverContext<RegionCoprocessorEnvironment> e,
       final InternalScanner s) throws IOException {
   }
 
   @Override
-  public void preWALRestore(RegionCoprocessorEnvironment env, HRegionInfo info,
+  public void preWALRestore(ObserverContext<RegionCoprocessorEnvironment> env, HRegionInfo info,
       HLogKey logKey, WALEdit logEdit) throws IOException {
   }
 
   @Override
-  public void postWALRestore(RegionCoprocessorEnvironment env,
+  public void postWALRestore(ObserverContext<RegionCoprocessorEnvironment> env,
       HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException {
   }
 }

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorEnvironment.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorEnvironment.java?rev=1091655&r1=1091654&r2=1091655&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorEnvironment.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorEnvironment.java Wed Apr 13 05:41:25 2011
@@ -45,18 +45,4 @@ public interface CoprocessorEnvironment 
    * @throws IOException
    */
   public HTableInterface getTable(byte[] tableName) throws IOException;
-
-  /* Control flow changes */
-
-  /**
-   * Causes framework to bypass default actions and return with the results
-   * from a preXXX chain.
-   */
-  public void bypass();
-
-  /**
-   * Mark coprocessor chain processing as complete. Causes framework to return
-   * immediately without calling any additional chained coprocessors.
-   */
-  public void complete();
 }

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java?rev=1091655&r1=1091654&r2=1091655&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java Wed Apr 13 05:41:25 2011
@@ -443,16 +443,6 @@ public abstract class CoprocessorHost<E 
     protected List<HTableInterface> openTables =
       Collections.synchronizedList(new ArrayList<HTableInterface>());
     private int seq;
-    static final ThreadLocal<Boolean> bypass = new ThreadLocal<Boolean>() {
-      @Override protected Boolean initialValue() {
-        return Boolean.FALSE;
-      }
-    };
-    static final ThreadLocal<Boolean> complete = new ThreadLocal<Boolean>() {
-      @Override protected Boolean initialValue() {
-        return Boolean.FALSE;
-      }
-    };
 
     /**
      * Constructor
@@ -509,18 +499,6 @@ public abstract class CoprocessorHost<E 
       }
     }
 
-    public boolean shouldBypass() {
-      boolean current = bypass.get();
-      bypass.set(false);
-      return current;
-    }
-
-    public boolean shouldComplete() {
-      boolean current = complete.get();
-      complete.set(false);
-      return current;
-    }
-
     @Override
     public Coprocessor getInstance() {
       return impl;
@@ -558,14 +536,5 @@ public abstract class CoprocessorHost<E 
     public HTableInterface getTable(byte[] tableName) throws IOException {
       return new HTableWrapper(tableName);
     }
-
-    @Override
-    public void complete() {
-      complete.set(true);
-    }
-
-    @Override
-    public void bypass() {
-      bypass.set(true);
-    }
-  }}
+  }
+}

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java?rev=1091655&r1=1091654&r2=1091655&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java Wed Apr 13 05:41:25 2011
@@ -34,182 +34,188 @@ public interface MasterObserver extends 
    * Called before a new table is created by
    * {@link org.apache.hadoop.hbase.master.HMaster}.
    */
-  void preCreateTable(MasterCoprocessorEnvironment env,
+  void preCreateTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
       HTableDescriptor desc, byte[][] splitKeys) throws IOException;
 
   /**
    * Called after the initial table regions have been created.
-   * @param env the environment to interact with the framework and master
+   * @param ctx the environment to interact with the framework and master
    * @param regions the initial regions created for the table
    * @param sync whether the client call is waiting for region assignment to
    * complete before returning
    * @throws IOException
    */
-  void postCreateTable(MasterCoprocessorEnvironment env,
+  void postCreateTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
       HRegionInfo[] regions, boolean sync) throws IOException;
 
   /**
    * Called before {@link org.apache.hadoop.hbase.master.HMaster} deletes a
    * table
    */
-  void preDeleteTable(MasterCoprocessorEnvironment env, byte[] tableName)
-      throws IOException;
+  void preDeleteTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      byte[] tableName) throws IOException;
 
   /**
    * Called after the table has been deleted, before returning to the client.
    */
-  void postDeleteTable(MasterCoprocessorEnvironment env, byte[] tableName)
-      throws IOException;
+  void postDeleteTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      byte[] tableName) throws IOException;
 
   /**
    * Called prior to modifying a table's properties.
    */
-  void preModifyTable(MasterCoprocessorEnvironment env, final byte[] tableName,
-      HTableDescriptor htd) throws IOException;
+  void preModifyTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final byte[] tableName, HTableDescriptor htd) throws IOException;
 
   /**
    * Called after {@link org.apache.hadoop.hbase.master.HMaster} has modified
    * the table's properties in all the table regions.
    */
-  void postModifyTable(MasterCoprocessorEnvironment env, final byte[] tableName,
-      HTableDescriptor htd) throws IOException;
+  void postModifyTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final byte[] tableName, HTableDescriptor htd) throws IOException;
 
   /**
    * Called prior to adding a new column family to the table.
    */
-  void preAddColumn(MasterCoprocessorEnvironment env, byte[] tableName,
-      HColumnDescriptor column) throws IOException;
+  void preAddColumn(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      byte[] tableName, HColumnDescriptor column) throws IOException;
 
   /**
    * Called after the new column family has been created.
    */
-  void postAddColumn(MasterCoprocessorEnvironment env, byte[] tableName,
-      HColumnDescriptor column) throws IOException;
+  void postAddColumn(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      byte[] tableName, HColumnDescriptor column) throws IOException;
 
   /**
    * Called prior to modifying a column family's attributes.
    */
-  void preModifyColumn(MasterCoprocessorEnvironment env,
+  void preModifyColumn(final ObserverContext<MasterCoprocessorEnvironment> ctx,
       byte [] tableName, HColumnDescriptor descriptor) throws IOException;
 
   /**
    * Called after the column family has been updated.
    */
-  void postModifyColumn(MasterCoprocessorEnvironment env, byte[] tableName,
-      HColumnDescriptor descriptor) throws IOException;
+  void postModifyColumn(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      byte[] tableName, HColumnDescriptor descriptor) throws IOException;
 
   /**
    * Called prior to deleting the entire column family.
    */
-  void preDeleteColumn(MasterCoprocessorEnvironment env,
+  void preDeleteColumn(final ObserverContext<MasterCoprocessorEnvironment> ctx,
       final byte [] tableName, final byte[] c) throws IOException;
 
   /**
    * Called after the column family has been deleted.
    */
-  void postDeleteColumn(MasterCoprocessorEnvironment env,
+  void postDeleteColumn(final ObserverContext<MasterCoprocessorEnvironment> ctx,
       final byte [] tableName, final byte[] c) throws IOException;
 
   /**
    * Called prior to enabling a table.
    */
-  void preEnableTable(MasterCoprocessorEnvironment env, final byte[] tableName)
-      throws IOException;
+  void preEnableTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final byte[] tableName) throws IOException;
 
   /**
    * Called after the table has been enabled.
    */
-  void postEnableTable(MasterCoprocessorEnvironment env, final byte[] tableName)
-      throws IOException;
+  void postEnableTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final byte[] tableName) throws IOException;
 
   /**
    * Called prior to disabling a table.
    */
-  void preDisableTable(MasterCoprocessorEnvironment env, final byte[] tableName)
-      throws IOException;
+  void preDisableTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final byte[] tableName) throws IOException;
 
   /**
    * Called after the table has been disabled.
    */
-  void postDisableTable(MasterCoprocessorEnvironment env, final byte[] tableName)
-      throws IOException;
+  void postDisableTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final byte[] tableName) throws IOException;
 
   /**
    * Called prior to moving a given region from one region server to another.
    */
-  void preMove(MasterCoprocessorEnvironment env, final HRegionInfo region,
-      final HServerInfo srcServer, final HServerInfo destServer)
+  void preMove(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final HRegionInfo region, final HServerInfo srcServer,
+      final HServerInfo destServer)
     throws UnknownRegionException;
 
   /**
    * Called after the region move has been requested.
    */
-  void postMove(MasterCoprocessorEnvironment env, final HRegionInfo region,
-      final HServerInfo srcServer, final HServerInfo destServer)
+  void postMove(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final HRegionInfo region, final HServerInfo srcServer,
+      final HServerInfo destServer)
     throws UnknownRegionException;
 
   /**
    * Called prior to assigning a specific region.
    */
-  void preAssign(MasterCoprocessorEnvironment env, final byte [] regionName,
-      final boolean force) throws IOException;
+  void preAssign(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final byte [] regionName, final boolean force) throws IOException;
 
   /**
    * Called after the region assignment has been requested.
    */
-  void postAssign(MasterCoprocessorEnvironment env, final HRegionInfo regionInfo)
-      throws IOException;
+  void postAssign(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final HRegionInfo regionInfo) throws IOException;
 
   /**
    * Called prior to unassigning a given region.
    */
-  void preUnassign(MasterCoprocessorEnvironment env, final byte [] regionName,
-      final boolean force) throws IOException;
+  void preUnassign(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final byte [] regionName, final boolean force) throws IOException;
 
   /**
    * Called after the region unassignment has been requested.
    */
-  void postUnassign(MasterCoprocessorEnvironment env,
+  void postUnassign(final ObserverContext<MasterCoprocessorEnvironment> ctx,
       final HRegionInfo regionInfo, final boolean force) throws IOException;
 
   /**
    * Called prior to requesting rebalancing of the cluster regions, though after
    * the initial checks for regions in transition and the balance switch flag.
    */
-  void preBalance(MasterCoprocessorEnvironment env) throws IOException;
+  void preBalance(final ObserverContext<MasterCoprocessorEnvironment> ctx)
+      throws IOException;
 
   /**
    * Called after the balancing plan has been submitted.
    */
-  void postBalance(MasterCoprocessorEnvironment env) throws IOException;
+  void postBalance(final ObserverContext<MasterCoprocessorEnvironment> ctx)
+      throws IOException;
 
   /**
    * Called prior to modifying the flag used to enable/disable region balancing.
-   * @param env the coprocessor instance's environment
+   * @param ctx the coprocessor instance's environment
    * @param newValue the new flag value submitted in the call
    */
-  boolean preBalanceSwitch(MasterCoprocessorEnvironment env,
+  boolean preBalanceSwitch(final ObserverContext<MasterCoprocessorEnvironment> ctx,
       final boolean newValue) throws IOException;
 
   /**
    * Called after the flag to enable/disable balancing has changed.
-   * @param env the coprocessor instance's environment
+   * @param ctx the coprocessor instance's environment
    * @param oldValue the previously set balanceSwitch value
    * @param newValue the newly set balanceSwitch value
    */
-  void postBalanceSwitch(MasterCoprocessorEnvironment env,
+  void postBalanceSwitch(final ObserverContext<MasterCoprocessorEnvironment> ctx,
       final boolean oldValue, final boolean newValue) throws IOException;
 
   /**
    * Called prior to shutting down the full HBase cluster, including this
    * {@link org.apache.hadoop.hbase.master.HMaster} process.
    */
-  void preShutdown(MasterCoprocessorEnvironment env) throws IOException;
+  void preShutdown(final ObserverContext<MasterCoprocessorEnvironment> ctx)
+      throws IOException;
 
 
   /**
    * Called immediatly prior to stopping this
    * {@link org.apache.hadoop.hbase.master.HMaster} process.
    */
-  void preStopMaster(MasterCoprocessorEnvironment env) throws IOException;
+  void preStopMaster(final ObserverContext<MasterCoprocessorEnvironment> ctx)
+      throws IOException;
 }

Added: hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/ObserverContext.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/ObserverContext.java?rev=1091655&view=auto
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/ObserverContext.java (added)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/ObserverContext.java Wed Apr 13 05:41:25 2011
@@ -0,0 +1,108 @@
+/*
+ * Copyright 2011 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.coprocessor;
+
+/**
+ * Carries the execution state for a given invocation of an Observer coprocessor
+ * ({@link RegionObserver}, {@link MasterObserver}, or {@link WALObserver})
+ * method.  The same ObserverContext instance is passed sequentially to all loaded
+ * coprocessors for a given Observer method trigger, with the
+ * <code>CoprocessorEnvironment</code> reference swapped out for each
+ * coprocessor.
+ * @param <E> The {@link CoprocessorEnvironment} subclass applicable to the
+ *     revelant Observer interface.
+ */
+public class ObserverContext<E extends CoprocessorEnvironment> {
+  private E env;
+  private boolean bypass;
+  private boolean complete;
+
+  public ObserverContext() {
+  }
+
+  public E getEnvironment() {
+    return env;
+  }
+
+  public void prepare(E env) {
+    this.env = env;
+  }
+
+  /**
+   * Call to indicate that the current coprocessor's return value should be
+   * used in place of the normal HBase obtained value.
+   */
+  public void bypass() {
+    bypass = true;
+  }
+
+  /**
+   * Call to indicate that additional coprocessors further down the execution
+   * chain do not need to be invoked.  Implies that this coprocessor's response
+   * is definitive.
+   */
+  public void complete() {
+    complete = true;
+  }
+
+  /**
+   * For use by the coprocessor framework.
+   * @return <code>true</code> if {@link ObserverContext#bypass()}
+   *     was called by one of the loaded coprocessors, <code>false</code> otherwise.
+   */
+  public boolean shouldBypass() {
+    boolean current = bypass;
+    bypass = false;
+    return current;
+  }
+
+  /**
+   * For use by the coprocessor framework.
+   * @return <code>true</code> if {@link ObserverContext#complete()}
+   *     was called by one of the loaded coprocessors, <code>false</code> otherwise.
+   */
+  public boolean shouldComplete() {
+    boolean current = complete;
+    complete = false;
+    return current;
+  }
+
+  /**
+   * Instantiates a new ObserverContext instance if the passed reference is
+   * <code>null</code> and sets the environment in the new or existing instance.
+   * This allows deferring the instantiation of a ObserverContext until it is
+   * actually needed.
+   *
+   * @param env The coprocessor environment to set
+   * @param context An existing ObserverContext instance to use, or <code>null</code>
+   *     to create a new instance
+   * @param <T> The environment type for the context
+   * @return An instance of <code>ObserverContext</code> with the environment set
+   */
+  public static <T extends CoprocessorEnvironment> ObserverContext<T> createAndPrepare(
+      T env, ObserverContext<T> context) {
+    if (context == null) {
+      context = new ObserverContext<T>();
+    }
+    context.prepare(env);
+    return context;
+  }
+}

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java?rev=1091655&r1=1091654&r2=1091655&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java Wed Apr 13 05:41:25 2011
@@ -44,77 +44,77 @@ public interface RegionObserver extends 
 
   /**
    * Called before the region is reported as open to the master.
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    */
-  public void preOpen(final RegionCoprocessorEnvironment e);
+  public void preOpen(final ObserverContext<RegionCoprocessorEnvironment> c);
 
   /**
    * Called after the region is reported as open to the master.
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    */
-  public void postOpen(final RegionCoprocessorEnvironment e);
+  public void postOpen(final ObserverContext<RegionCoprocessorEnvironment> c);
 
   /**
    * Called before the memstore is flushed to disk.
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    */
-  public void preFlush(final RegionCoprocessorEnvironment e);
+  public void preFlush(final ObserverContext<RegionCoprocessorEnvironment> c);
 
   /**
    * Called after the memstore is flushed to disk.
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    */
-  public void postFlush(final RegionCoprocessorEnvironment e);
+  public void postFlush(final ObserverContext<RegionCoprocessorEnvironment> c);
 
   /**
    * Called before compaction.
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    * @param willSplit true if compaction will result in a split, false
    * otherwise
    */
-  public void preCompact(final RegionCoprocessorEnvironment e,
+  public void preCompact(final ObserverContext<RegionCoprocessorEnvironment> c,
     final boolean willSplit);
 
   /**
    * Called after compaction.
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    * @param willSplit true if compaction will result in a split, false
    * otherwise
    */
-  public void postCompact(final RegionCoprocessorEnvironment e,
+  public void postCompact(final ObserverContext<RegionCoprocessorEnvironment> c,
     final boolean willSplit);
 
   /**
    * Called before the region is split.
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    * (e.getRegion() returns the parent region)
    */
-  public void preSplit(final RegionCoprocessorEnvironment e);
+  public void preSplit(final ObserverContext<RegionCoprocessorEnvironment> c);
 
   /**
    * Called after the region is split.
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    * (e.getRegion() returns the parent region)
    * @param l the left daughter region
    * @param r the right daughter region
    */
-  public void postSplit(final RegionCoprocessorEnvironment e, final HRegion l,
+  public void postSplit(final ObserverContext<RegionCoprocessorEnvironment> c, final HRegion l,
     final HRegion r);
 
   /**
    * Called before the region is reported as closed to the master.
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    * @param abortRequested true if the region server is aborting
    */
-  public void preClose(final RegionCoprocessorEnvironment e,
+  public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
       boolean abortRequested);
 
   /**
    * Called after the region is reported as closed to the master.
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    * @param abortRequested true if the region server is aborting
    */
-  public void postClose(final RegionCoprocessorEnvironment e,
+  public void postClose(final ObserverContext<RegionCoprocessorEnvironment> c,
       boolean abortRequested);
 
   /**
@@ -124,7 +124,7 @@ public interface RegionObserver extends 
    * <p>
    * Call CoprocessorEnvironment#complete to skip any subsequent chained
    * coprocessors
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    * @param row the row
    * @param family the family
    * @param result The result to return to the client if default processing
@@ -132,7 +132,7 @@ public interface RegionObserver extends 
    * is not bypassed.
    * @throws IOException if an error occurred on the coprocessor
    */
-  public void preGetClosestRowBefore(final RegionCoprocessorEnvironment e,
+  public void preGetClosestRowBefore(final ObserverContext<RegionCoprocessorEnvironment> c,
       final byte [] row, final byte [] family, final Result result)
     throws IOException;
 
@@ -141,13 +141,13 @@ public interface RegionObserver extends 
    * <p>
    * Call CoprocessorEnvironment#complete to skip any subsequent chained
    * coprocessors
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    * @param row the row
    * @param family the desired family
    * @param result the result to return to the client, modify as necessary
    * @throws IOException if an error occurred on the coprocessor
    */
-  public void postGetClosestRowBefore(final RegionCoprocessorEnvironment e,
+  public void postGetClosestRowBefore(final ObserverContext<RegionCoprocessorEnvironment> c,
       final byte [] row, final byte [] family, final Result result)
     throws IOException;
 
@@ -158,14 +158,14 @@ public interface RegionObserver extends 
    * <p>
    * Call CoprocessorEnvironment#complete to skip any subsequent chained
    * coprocessors
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    * @param get the Get request
    * @param result The result to return to the client if default processing
    * is bypassed. Can be modified. Will not be used if default processing
    * is not bypassed.
    * @throws IOException if an error occurred on the coprocessor
    */
-  public void preGet(final RegionCoprocessorEnvironment e, final Get get,
+  public void preGet(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
       final List<KeyValue> result)
     throws IOException;
 
@@ -174,12 +174,12 @@ public interface RegionObserver extends 
    * <p>
    * Call CoprocessorEnvironment#complete to skip any subsequent chained
    * coprocessors
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    * @param get the Get request
    * @param result the result to return to the client, modify as necessary
    * @throws IOException if an error occurred on the coprocessor
    */
-  public void postGet(final RegionCoprocessorEnvironment e, final Get get,
+  public void postGet(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
       final List<KeyValue> result)
     throws IOException;
 
@@ -190,13 +190,13 @@ public interface RegionObserver extends 
    * <p>
    * Call CoprocessorEnvironment#complete to skip any subsequent chained
    * coprocessors
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    * @param get the Get request
    * @param exists
    * @return the value to return to the client if bypassing default processing
    * @throws IOException if an error occurred on the coprocessor
    */
-  public boolean preExists(final RegionCoprocessorEnvironment e, final Get get,
+  public boolean preExists(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
       final boolean exists)
     throws IOException;
 
@@ -205,13 +205,13 @@ public interface RegionObserver extends 
    * <p>
    * Call CoprocessorEnvironment#complete to skip any subsequent chained
    * coprocessors
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    * @param get the Get request
    * @param exists the result returned by the region server
    * @return the result to return to the client
    * @throws IOException if an error occurred on the coprocessor
    */
-  public boolean postExists(final RegionCoprocessorEnvironment e, final Get get,
+  public boolean postExists(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
       final boolean exists)
     throws IOException;
 
@@ -222,12 +222,12 @@ public interface RegionObserver extends 
    * <p>
    * Call CoprocessorEnvironment#complete to skip any subsequent chained
    * coprocessors
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    * @param familyMap map of family to edits for the given family
    * @param writeToWAL true if the change should be written to the WAL
    * @throws IOException if an error occurred on the coprocessor
    */
-  public void prePut(final RegionCoprocessorEnvironment e, final Map<byte[],
+  public void prePut(final ObserverContext<RegionCoprocessorEnvironment> c, final Map<byte[],
       List<KeyValue>> familyMap, final boolean writeToWAL)
     throws IOException;
 
@@ -236,12 +236,12 @@ public interface RegionObserver extends 
    * <p>
    * Call CoprocessorEnvironment#complete to skip any subsequent chained
    * coprocessors
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    * @param familyMap map of family to edits for the given family
    * @param writeToWAL true if the change should be written to the WAL
    * @throws IOException if an error occurred on the coprocessor
    */
-  public void postPut(final RegionCoprocessorEnvironment e, final Map<byte[],
+  public void postPut(final ObserverContext<RegionCoprocessorEnvironment> c, final Map<byte[],
       List<KeyValue>> familyMap, final boolean writeToWAL)
     throws IOException;
 
@@ -252,12 +252,12 @@ public interface RegionObserver extends 
    * <p>
    * Call CoprocessorEnvironment#complete to skip any subsequent chained
    * coprocessors
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    * @param familyMap map of family to edits for the given family
    * @param writeToWAL true if the change should be written to the WAL
    * @throws IOException if an error occurred on the coprocessor
    */
-  public void preDelete(final RegionCoprocessorEnvironment e, final Map<byte[],
+  public void preDelete(final ObserverContext<RegionCoprocessorEnvironment> c, final Map<byte[],
       List<KeyValue>> familyMap, final boolean writeToWAL)
     throws IOException;
 
@@ -266,12 +266,12 @@ public interface RegionObserver extends 
    * <p>
    * Call CoprocessorEnvironment#complete to skip any subsequent chained
    * coprocessors
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    * @param familyMap map of family to edits for the given family
    * @param writeToWAL true if the change should be written to the WAL
    * @throws IOException if an error occurred on the coprocessor
    */
-  public void postDelete(final RegionCoprocessorEnvironment e,
+  public void postDelete(final ObserverContext<RegionCoprocessorEnvironment> c,
       final Map<byte[], List<KeyValue>> familyMap, final boolean writeToWAL)
     throws IOException;
 
@@ -282,7 +282,7 @@ public interface RegionObserver extends 
    * <p>
    * Call CoprocessorEnvironment#complete to skip any subsequent chained
    * coprocessors
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    * @param row row to check
    * @param family column family
    * @param qualifier column qualifier
@@ -294,7 +294,7 @@ public interface RegionObserver extends 
    * processing
    * @throws IOException if an error occurred on the coprocessor
    */
-  public boolean preCheckAndPut(final RegionCoprocessorEnvironment e,
+  public boolean preCheckAndPut(final ObserverContext<RegionCoprocessorEnvironment> c,
       final byte [] row, final byte [] family, final byte [] qualifier,
       final CompareOp compareOp, final WritableByteArrayComparable comparator,
       final Put put, final boolean result)
@@ -305,7 +305,7 @@ public interface RegionObserver extends 
    * <p>
    * Call CoprocessorEnvironment#complete to skip any subsequent chained
    * coprocessors
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    * @param row row to check
    * @param family column family
    * @param qualifier column qualifier
@@ -316,7 +316,7 @@ public interface RegionObserver extends 
    * @return the possibly transformed return value to return to client
    * @throws IOException if an error occurred on the coprocessor
    */
-  public boolean postCheckAndPut(final RegionCoprocessorEnvironment e,
+  public boolean postCheckAndPut(final ObserverContext<RegionCoprocessorEnvironment> c,
       final byte [] row, final byte [] family, final byte [] qualifier,
       final CompareOp compareOp, final WritableByteArrayComparable comparator,
       final Put put, final boolean result)
@@ -329,7 +329,7 @@ public interface RegionObserver extends 
    * <p>
    * Call CoprocessorEnvironment#complete to skip any subsequent chained
    * coprocessors
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    * @param row row to check
    * @param family column family
    * @param qualifier column qualifier
@@ -340,7 +340,7 @@ public interface RegionObserver extends 
    * @return the value to return to client if bypassing default processing
    * @throws IOException if an error occurred on the coprocessor
    */
-  public boolean preCheckAndDelete(final RegionCoprocessorEnvironment e,
+  public boolean preCheckAndDelete(final ObserverContext<RegionCoprocessorEnvironment> c,
       final byte [] row, final byte [] family, final byte [] qualifier,
       final CompareOp compareOp, final WritableByteArrayComparable comparator,
       final Delete delete, final boolean result)
@@ -351,7 +351,7 @@ public interface RegionObserver extends 
    * <p>
    * Call CoprocessorEnvironment#complete to skip any subsequent chained
    * coprocessors
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    * @param row row to check
    * @param family column family
    * @param qualifier column qualifier
@@ -362,7 +362,7 @@ public interface RegionObserver extends 
    * @return the possibly transformed returned value to return to client
    * @throws IOException if an error occurred on the coprocessor
    */
-  public boolean postCheckAndDelete(final RegionCoprocessorEnvironment e,
+  public boolean postCheckAndDelete(final ObserverContext<RegionCoprocessorEnvironment> c,
       final byte [] row, final byte [] family, final byte [] qualifier,
       final CompareOp compareOp, final WritableByteArrayComparable comparator,
       final Delete delete, final boolean result)
@@ -375,7 +375,7 @@ public interface RegionObserver extends 
    * <p>
    * Call CoprocessorEnvironment#complete to skip any subsequent chained
    * coprocessors
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    * @param row row to check
    * @param family column family
    * @param qualifier column qualifier
@@ -384,7 +384,7 @@ public interface RegionObserver extends 
    * @return value to return to the client if bypassing default processing
    * @throws IOException if an error occurred on the coprocessor
    */
-  public long preIncrementColumnValue(final RegionCoprocessorEnvironment e,
+  public long preIncrementColumnValue(final ObserverContext<RegionCoprocessorEnvironment> c,
       final byte [] row, final byte [] family, final byte [] qualifier,
       final long amount, final boolean writeToWAL)
     throws IOException;
@@ -394,7 +394,7 @@ public interface RegionObserver extends 
    * <p>
    * Call CoprocessorEnvironment#complete to skip any subsequent chained
    * coprocessors
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    * @param row row to check
    * @param family column family
    * @param qualifier column qualifier
@@ -404,7 +404,7 @@ public interface RegionObserver extends 
    * @return the result to return to the client
    * @throws IOException if an error occurred on the coprocessor
    */
-  public long postIncrementColumnValue(final RegionCoprocessorEnvironment e,
+  public long postIncrementColumnValue(final ObserverContext<RegionCoprocessorEnvironment> c,
       final byte [] row, final byte [] family, final byte [] qualifier,
       final long amount, final boolean writeToWAL, final long result)
     throws IOException;
@@ -416,14 +416,14 @@ public interface RegionObserver extends 
    * <p>
    * Call CoprocessorEnvironment#complete to skip any subsequent chained
    * coprocessors
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    * @param increment increment object
    * @param result The result to return to the client if default processing
    * is bypassed. Can be modified. Will not be used if default processing
    * is not bypassed.
    * @throws IOException if an error occurred on the coprocessor
    */
-  public void preIncrement(final RegionCoprocessorEnvironment e,
+  public void preIncrement(final ObserverContext<RegionCoprocessorEnvironment> c,
       final Increment increment, final Result result)
     throws IOException;
 
@@ -432,12 +432,12 @@ public interface RegionObserver extends 
    * <p>
    * Call CoprocessorEnvironment#complete to skip any subsequent chained
    * coprocessors
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    * @param increment increment object
    * @param result the result returned by increment, can be modified
    * @throws IOException if an error occurred on the coprocessor
    */
-  public void postIncrement(final RegionCoprocessorEnvironment e,
+  public void postIncrement(final ObserverContext<RegionCoprocessorEnvironment> c,
       final Increment increment, final Result result)
     throws IOException;
 
@@ -448,14 +448,14 @@ public interface RegionObserver extends 
    * <p>
    * Call CoprocessorEnvironment#complete to skip any subsequent chained
    * coprocessors
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    * @param scan the Scan specification
    * @param s if not null, the base scanner
    * @return an InternalScanner instance to use instead of the base scanner if
    * overriding default behavior, null otherwise
    * @throws IOException if an error occurred on the coprocessor
    */
-  public InternalScanner preScannerOpen(final RegionCoprocessorEnvironment e,
+  public InternalScanner preScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
       final Scan scan, final InternalScanner s)
     throws IOException;
 
@@ -464,13 +464,13 @@ public interface RegionObserver extends 
    * <p>
    * Call CoprocessorEnvironment#complete to skip any subsequent chained
    * coprocessors
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    * @param scan the Scan specification
    * @param s if not null, the base scanner
    * @return the scanner instance to use
    * @throws IOException if an error occurred on the coprocessor
    */
-  public InternalScanner postScannerOpen(final RegionCoprocessorEnvironment e,
+  public InternalScanner postScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
       final Scan scan, final InternalScanner s)
     throws IOException;
 
@@ -481,7 +481,7 @@ public interface RegionObserver extends 
    * <p>
    * Call CoprocessorEnvironment#complete to skip any subsequent chained
    * coprocessors
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    * @param s the scanner
    * @param result The result to return to the client if default processing
    * is bypassed. Can be modified. Will not be returned if default processing
@@ -491,7 +491,7 @@ public interface RegionObserver extends 
    * @return 'has more' indication that should be sent to client
    * @throws IOException if an error occurred on the coprocessor
    */
-  public boolean preScannerNext(final RegionCoprocessorEnvironment e,
+  public boolean preScannerNext(final ObserverContext<RegionCoprocessorEnvironment> c,
       final InternalScanner s, final List<Result> result,
       final int limit, final boolean hasNext)
     throws IOException;
@@ -501,7 +501,7 @@ public interface RegionObserver extends 
    * <p>
    * Call CoprocessorEnvironment#complete to skip any subsequent chained
    * coprocessors
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    * @param s the scanner
    * @param result the result to return to the client, can be modified
    * @param limit the maximum number of results to return
@@ -509,7 +509,7 @@ public interface RegionObserver extends 
    * @return 'has more' indication that should be sent to client
    * @throws IOException if an error occurred on the coprocessor
    */
-  public boolean postScannerNext(final RegionCoprocessorEnvironment e,
+  public boolean postScannerNext(final ObserverContext<RegionCoprocessorEnvironment> c,
       final InternalScanner s, final List<Result> result, final int limit,
       final boolean hasNext)
     throws IOException;
@@ -521,11 +521,11 @@ public interface RegionObserver extends 
    * <p>
    * Call CoprocessorEnvironment#complete to skip any subsequent chained
    * coprocessors
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    * @param s the scanner
    * @throws IOException if an error occurred on the coprocessor
    */
-  public void preScannerClose(final RegionCoprocessorEnvironment e,
+  public void preScannerClose(final ObserverContext<RegionCoprocessorEnvironment> c,
       final InternalScanner s)
     throws IOException;
 
@@ -534,11 +534,11 @@ public interface RegionObserver extends 
    * <p>
    * Call CoprocessorEnvironment#complete to skip any subsequent chained
    * coprocessors
-   * @param e the environment provided by the region server
+   * @param c the environment provided by the region server
    * @param s the scanner
    * @throws IOException if an error occurred on the coprocessor
    */
-  public void postScannerClose(final RegionCoprocessorEnvironment e,
+  public void postScannerClose(final ObserverContext<RegionCoprocessorEnvironment> c,
       final InternalScanner s)
     throws IOException;
 
@@ -546,25 +546,25 @@ public interface RegionObserver extends 
    * Called before a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit}
    * replayed for this region.
    *
-   * @param env
+   * @param ctx
    * @param info
    * @param logKey
    * @param logEdit
    * @throws IOException
    */
-  void preWALRestore(final RegionCoprocessorEnvironment env,
+  void preWALRestore(final ObserverContext<RegionCoprocessorEnvironment> ctx,
       HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException;
 
   /**
    * Called after a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit}
    * replayed for this region.
    *
-   * @param env
+   * @param ctx
    * @param info
    * @param logKey
    * @param logEdit
    * @throws IOException
    */
-  void postWALRestore(final RegionCoprocessorEnvironment env,
+  void postWALRestore(final ObserverContext<RegionCoprocessorEnvironment> ctx,
       HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException;
 }

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java?rev=1091655&r1=1091654&r2=1091655&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java Wed Apr 13 05:41:25 2011
@@ -42,26 +42,26 @@ public interface WALObserver extends Cop
    * Called before a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit}
    * is writen to WAL.
    *
-   * @param env
+   * @param ctx
    * @param info
    * @param logKey
    * @param logEdit
    * @return true if default behavior should be bypassed, false otherwise
    * @throws IOException
    */
-  boolean preWALWrite(CoprocessorEnvironment env,
+  boolean preWALWrite(ObserverContext<WALCoprocessorEnvironment> ctx,
       HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException;
 
   /**
    * Called after a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit}
    * is writen to WAL.
    *
-   * @param env
+   * @param ctx
    * @param info
    * @param logKey
    * @param logEdit
    * @throws IOException
    */
-  void postWALWrite(CoprocessorEnvironment env,
+  void postWALWrite(ObserverContext<WALCoprocessorEnvironment> ctx,
       HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException;
 }

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java?rev=1091655&r1=1091654&r2=1091655&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java Wed Apr 13 05:41:25 2011
@@ -70,10 +70,12 @@ public class MasterCoprocessorHost
   /* Implementation of hooks for invoking MasterObservers */
   void preCreateTable(HTableDescriptor desc, byte[][] splitKeys)
       throws IOException {
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
     for (MasterEnvironment env: coprocessors) {
       if (env.getInstance() instanceof MasterObserver) {
-        ((MasterObserver)env.getInstance()).preCreateTable(env, desc, splitKeys);
-        if (env.shouldComplete()) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
+        ((MasterObserver)env.getInstance()).preCreateTable(ctx, desc, splitKeys);
+        if (ctx.shouldComplete()) {
           break;
         }
       }
@@ -81,10 +83,12 @@ public class MasterCoprocessorHost
   }
 
   void postCreateTable(HRegionInfo[] regions, boolean sync) throws IOException {
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
     for (MasterEnvironment env: coprocessors) {
       if (env.getInstance() instanceof MasterObserver) {
-        ((MasterObserver)env.getInstance()).postCreateTable(env, regions, sync);
-        if (env.shouldComplete()) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
+        ((MasterObserver)env.getInstance()).postCreateTable(ctx, regions, sync);
+        if (ctx.shouldComplete()) {
           break;
         }
       }
@@ -92,10 +96,12 @@ public class MasterCoprocessorHost
   }
 
   void preDeleteTable(byte[] tableName) throws IOException {
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
     for (MasterEnvironment env: coprocessors) {
       if (env.getInstance() instanceof MasterObserver) {
-        ((MasterObserver)env.getInstance()).preDeleteTable(env, tableName);
-        if (env.shouldComplete()) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
+        ((MasterObserver)env.getInstance()).preDeleteTable(ctx, tableName);
+        if (ctx.shouldComplete()) {
           break;
         }
       }
@@ -103,10 +109,12 @@ public class MasterCoprocessorHost
   }
 
   void postDeleteTable(byte[] tableName) throws IOException {
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
     for (MasterEnvironment env: coprocessors) {
       if (env.getInstance() instanceof MasterObserver) {
-        ((MasterObserver)env.getInstance()).postDeleteTable(env, tableName);
-        if (env.shouldComplete()) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
+        ((MasterObserver)env.getInstance()).postDeleteTable(ctx, tableName);
+        if (ctx.shouldComplete()) {
           break;
         }
       }
@@ -115,10 +123,12 @@ public class MasterCoprocessorHost
 
   void preModifyTable(final byte[] tableName, HTableDescriptor htd)
       throws IOException {
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
     for (MasterEnvironment env: coprocessors) {
       if (env.getInstance() instanceof MasterObserver) {
-        ((MasterObserver)env.getInstance()).preModifyTable(env, tableName, htd);
-        if (env.shouldComplete()) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
+        ((MasterObserver)env.getInstance()).preModifyTable(ctx, tableName, htd);
+        if (ctx.shouldComplete()) {
           break;
         }
       }
@@ -127,10 +137,12 @@ public class MasterCoprocessorHost
 
   void postModifyTable(final byte[] tableName, HTableDescriptor htd)
       throws IOException {
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
     for (MasterEnvironment env: coprocessors) {
       if (env.getInstance() instanceof MasterObserver) {
-        ((MasterObserver)env.getInstance()).postModifyTable(env, tableName, htd);
-        if (env.shouldComplete()) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
+        ((MasterObserver)env.getInstance()).postModifyTable(ctx, tableName, htd);
+        if (ctx.shouldComplete()) {
           break;
         }
       }
@@ -139,10 +151,12 @@ public class MasterCoprocessorHost
 
   void preAddColumn(byte [] tableName, HColumnDescriptor column)
       throws IOException {
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
     for (MasterEnvironment env: coprocessors) {
       if (env.getInstance() instanceof MasterObserver) {
-        ((MasterObserver)env.getInstance()).preAddColumn(env, tableName, column);
-        if (env.shouldComplete()) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
+        ((MasterObserver)env.getInstance()).preAddColumn(ctx, tableName, column);
+        if (ctx.shouldComplete()) {
           break;
         }
       }
@@ -151,10 +165,12 @@ public class MasterCoprocessorHost
 
   void postAddColumn(byte [] tableName, HColumnDescriptor column)
       throws IOException {
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
     for (MasterEnvironment env: coprocessors) {
       if (env.getInstance() instanceof MasterObserver) {
-        ((MasterObserver)env.getInstance()).postAddColumn(env, tableName, column);
-        if (env.shouldComplete()) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
+        ((MasterObserver)env.getInstance()).postAddColumn(ctx, tableName, column);
+        if (ctx.shouldComplete()) {
           break;
         }
       }
@@ -163,11 +179,13 @@ public class MasterCoprocessorHost
 
   void preModifyColumn(byte [] tableName, HColumnDescriptor descriptor)
       throws IOException {
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
     for (MasterEnvironment env: coprocessors) {
       if (env.getInstance() instanceof MasterObserver) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
         ((MasterObserver)env.getInstance()).preModifyColumn(
-            env, tableName, descriptor);
-        if (env.shouldComplete()) {
+            ctx, tableName, descriptor);
+        if (ctx.shouldComplete()) {
           break;
         }
       }
@@ -176,11 +194,13 @@ public class MasterCoprocessorHost
 
   void postModifyColumn(byte [] tableName, HColumnDescriptor descriptor)
       throws IOException {
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
     for (MasterEnvironment env: coprocessors) {
       if (env.getInstance() instanceof MasterObserver) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
         ((MasterObserver)env.getInstance()).postModifyColumn(
-            env, tableName, descriptor);
-        if (env.shouldComplete()) {
+            ctx, tableName, descriptor);
+        if (ctx.shouldComplete()) {
           break;
         }
       }
@@ -189,10 +209,12 @@ public class MasterCoprocessorHost
 
   void preDeleteColumn(final byte [] tableName, final byte [] c)
       throws IOException {
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
     for (MasterEnvironment env: coprocessors) {
       if (env.getInstance() instanceof MasterObserver) {
-        ((MasterObserver)env.getInstance()).preDeleteColumn(env, tableName, c);
-        if (env.shouldComplete()) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
+        ((MasterObserver)env.getInstance()).preDeleteColumn(ctx, tableName, c);
+        if (ctx.shouldComplete()) {
           break;
         }
       }
@@ -201,10 +223,12 @@ public class MasterCoprocessorHost
 
   void postDeleteColumn(final byte [] tableName, final byte [] c)
       throws IOException {
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
     for (MasterEnvironment env: coprocessors) {
       if (env.getInstance() instanceof MasterObserver) {
-        ((MasterObserver)env.getInstance()).postDeleteColumn(env, tableName, c);
-        if (env.shouldComplete()) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
+        ((MasterObserver)env.getInstance()).postDeleteColumn(ctx, tableName, c);
+        if (ctx.shouldComplete()) {
           break;
         }
       }
@@ -212,10 +236,12 @@ public class MasterCoprocessorHost
   }
 
   void preEnableTable(final byte [] tableName) throws IOException {
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
     for (MasterEnvironment env: coprocessors) {
       if (env.getInstance() instanceof MasterObserver) {
-        ((MasterObserver)env.getInstance()).preEnableTable(env, tableName);
-        if (env.shouldComplete()) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
+        ((MasterObserver)env.getInstance()).preEnableTable(ctx, tableName);
+        if (ctx.shouldComplete()) {
           break;
         }
       }
@@ -223,10 +249,12 @@ public class MasterCoprocessorHost
   }
 
   void postEnableTable(final byte [] tableName) throws IOException {
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
     for (MasterEnvironment env: coprocessors) {
       if (env.getInstance() instanceof MasterObserver) {
-        ((MasterObserver)env.getInstance()).postEnableTable(env, tableName);
-        if (env.shouldComplete()) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
+        ((MasterObserver)env.getInstance()).postEnableTable(ctx, tableName);
+        if (ctx.shouldComplete()) {
           break;
         }
       }
@@ -234,10 +262,12 @@ public class MasterCoprocessorHost
   }
 
   void preDisableTable(final byte [] tableName) throws IOException {
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
     for (MasterEnvironment env: coprocessors) {
       if (env.getInstance() instanceof MasterObserver) {
-        ((MasterObserver)env.getInstance()).preDisableTable(env, tableName);
-        if (env.shouldComplete()) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
+        ((MasterObserver)env.getInstance()).preDisableTable(ctx, tableName);
+        if (ctx.shouldComplete()) {
           break;
         }
       }
@@ -245,10 +275,12 @@ public class MasterCoprocessorHost
   }
 
   void postDisableTable(final byte [] tableName) throws IOException {
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
     for (MasterEnvironment env: coprocessors) {
       if (env.getInstance() instanceof MasterObserver) {
-        ((MasterObserver)env.getInstance()).postDisableTable(env, tableName);
-        if (env.shouldComplete()) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
+        ((MasterObserver)env.getInstance()).postDisableTable(ctx, tableName);
+        if (ctx.shouldComplete()) {
           break;
         }
       }
@@ -257,11 +289,13 @@ public class MasterCoprocessorHost
 
   void preMove(final HRegionInfo region, final HServerInfo srcServer, final HServerInfo destServer)
       throws UnknownRegionException {
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
     for (MasterEnvironment env: coprocessors) {
       if (env.getInstance() instanceof MasterObserver) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
         ((MasterObserver)env.getInstance()).preMove(
-            env, region, srcServer, destServer);
-        if (env.shouldComplete()) {
+            ctx, region, srcServer, destServer);
+        if (ctx.shouldComplete()) {
           break;
         }
       }
@@ -270,11 +304,13 @@ public class MasterCoprocessorHost
 
   void postMove(final HRegionInfo region, final HServerInfo srcServer, final HServerInfo destServer)
       throws UnknownRegionException {
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
     for (MasterEnvironment env: coprocessors) {
       if (env.getInstance() instanceof MasterObserver) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
         ((MasterObserver)env.getInstance()).postMove(
-            env, region, srcServer, destServer);
-        if (env.shouldComplete()) {
+            ctx, region, srcServer, destServer);
+        if (ctx.shouldComplete()) {
           break;
         }
       }
@@ -284,11 +320,13 @@ public class MasterCoprocessorHost
   boolean preAssign(final byte [] regionName, final boolean force)
       throws IOException {
     boolean bypass = false;
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
     for (MasterEnvironment env: coprocessors) {
       if (env.getInstance() instanceof MasterObserver) {
-        ((MasterObserver)env.getInstance()).preAssign(env, regionName, force);
-        bypass |= env.shouldBypass();
-        if (env.shouldComplete()) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
+        ((MasterObserver)env.getInstance()).preAssign(ctx, regionName, force);
+        bypass |= ctx.shouldBypass();
+        if (ctx.shouldComplete()) {
           break;
         }
       }
@@ -297,10 +335,12 @@ public class MasterCoprocessorHost
   }
 
   void postAssign(final HRegionInfo regionInfo) throws IOException {
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
     for (MasterEnvironment env: coprocessors) {
       if (env.getInstance() instanceof MasterObserver) {
-        ((MasterObserver)env.getInstance()).postAssign(env, regionInfo);
-        if (env.shouldComplete()) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
+        ((MasterObserver)env.getInstance()).postAssign(ctx, regionInfo);
+        if (ctx.shouldComplete()) {
           break;
         }
       }
@@ -310,12 +350,14 @@ public class MasterCoprocessorHost
   boolean preUnassign(final byte [] regionName, final boolean force)
       throws IOException {
     boolean bypass = false;
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
     for (MasterEnvironment env: coprocessors) {
       if (env.getInstance() instanceof MasterObserver) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
         ((MasterObserver)env.getInstance()).preUnassign(
-            env, regionName, force);
-        bypass |= env.shouldBypass();
-        if (env.shouldComplete()) {
+            ctx, regionName, force);
+        bypass |= ctx.shouldBypass();
+        if (ctx.shouldComplete()) {
           break;
         }
       }
@@ -325,11 +367,13 @@ public class MasterCoprocessorHost
 
   void postUnassign(final HRegionInfo regionInfo, final boolean force)
       throws IOException {
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
     for (MasterEnvironment env: coprocessors) {
       if (env.getInstance() instanceof MasterObserver) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
         ((MasterObserver)env.getInstance()).postUnassign(
-            env, regionInfo, force);
-        if (env.shouldComplete()) {
+            ctx, regionInfo, force);
+        if (ctx.shouldComplete()) {
           break;
         }
       }
@@ -338,11 +382,13 @@ public class MasterCoprocessorHost
 
   boolean preBalance() throws IOException {
     boolean bypass = false;
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
     for (MasterEnvironment env: coprocessors) {
       if (env.getInstance() instanceof MasterObserver) {
-        ((MasterObserver)env.getInstance()).preBalance(env);
-        bypass |= env.shouldBypass();
-        if (env.shouldComplete()) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
+        ((MasterObserver)env.getInstance()).preBalance(ctx);
+        bypass |= ctx.shouldBypass();
+        if (ctx.shouldComplete()) {
           break;
         }
       }
@@ -351,10 +397,12 @@ public class MasterCoprocessorHost
   }
 
   void postBalance() throws IOException {
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
     for (MasterEnvironment env: coprocessors) {
       if (env.getInstance() instanceof MasterObserver) {
-        ((MasterObserver)env.getInstance()).postBalance(env);
-        if (env.shouldComplete()) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
+        ((MasterObserver)env.getInstance()).postBalance(ctx);
+        if (ctx.shouldComplete()) {
           break;
         }
       }
@@ -363,11 +411,13 @@ public class MasterCoprocessorHost
 
   boolean preBalanceSwitch(final boolean b) throws IOException {
     boolean balance = b;
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
     for (MasterEnvironment env: coprocessors) {
       if (env.getInstance() instanceof MasterObserver) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
         balance = ((MasterObserver)env.getInstance()).preBalanceSwitch(
-            env, balance);
-        if (env.shouldComplete()) {
+            ctx, balance);
+        if (ctx.shouldComplete()) {
           break;
         }
       }
@@ -377,11 +427,13 @@ public class MasterCoprocessorHost
 
   void postBalanceSwitch(final boolean oldValue, final boolean newValue)
       throws IOException {
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
     for (MasterEnvironment env: coprocessors) {
       if (env.getInstance() instanceof MasterObserver) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
         ((MasterObserver)env.getInstance()).postBalanceSwitch(
-            env, oldValue, newValue);
-        if (env.shouldComplete()) {
+            ctx, oldValue, newValue);
+        if (ctx.shouldComplete()) {
           break;
         }
       }
@@ -389,10 +441,12 @@ public class MasterCoprocessorHost
   }
 
   void preShutdown() throws IOException {
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
     for (MasterEnvironment env: coprocessors) {
       if (env.getInstance() instanceof MasterObserver) {
-        ((MasterObserver)env.getInstance()).preShutdown(env);
-        if (env.shouldComplete()) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
+        ((MasterObserver)env.getInstance()).preShutdown(ctx);
+        if (ctx.shouldComplete()) {
           break;
         }
       }
@@ -400,10 +454,12 @@ public class MasterCoprocessorHost
   }
 
   void preStopMaster() throws IOException {
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
     for (MasterEnvironment env: coprocessors) {
       if (env.getInstance() instanceof MasterObserver) {
-        ((MasterObserver)env.getInstance()).preStopMaster(env);
-        if (env.shouldComplete()) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
+        ((MasterObserver)env.getInstance()).preStopMaster(ctx);
+        if (ctx.shouldComplete()) {
           break;
         }
       }