View Javadoc

1   /*
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   * http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  
21  package org.apache.hadoop.hbase.regionserver;
22  
23  import java.io.IOException;
24  import java.util.ArrayList;
25  import java.util.List;
26  import java.util.Map;
27  import java.util.NavigableSet;
28  import java.util.concurrent.ConcurrentHashMap;
29  import java.util.concurrent.ConcurrentMap;
30  import java.util.regex.Matcher;
31  
32  import org.apache.commons.collections.map.AbstractReferenceMap;
33  import org.apache.commons.collections.map.ReferenceMap;
34  import org.apache.commons.logging.Log;
35  import org.apache.commons.logging.LogFactory;
36  import org.apache.hadoop.conf.Configuration;
37  import org.apache.hadoop.fs.Path;
38  import org.apache.hadoop.hbase.Coprocessor;
39  import org.apache.hadoop.hbase.CoprocessorEnvironment;
40  import org.apache.hadoop.hbase.HConstants;
41  import org.apache.hadoop.hbase.HRegionInfo;
42  import org.apache.hadoop.hbase.HTableDescriptor;
43  import org.apache.hadoop.hbase.KeyValue;
44  import org.apache.hadoop.hbase.client.Append;
45  import org.apache.hadoop.hbase.client.Delete;
46  import org.apache.hadoop.hbase.client.Get;
47  import org.apache.hadoop.hbase.client.Increment;
48  import org.apache.hadoop.hbase.client.Mutation;
49  import org.apache.hadoop.hbase.client.Put;
50  import org.apache.hadoop.hbase.client.Result;
51  import org.apache.hadoop.hbase.client.Scan;
52  import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
53  import org.apache.hadoop.hbase.coprocessor.ObserverContext;
54  import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
55  import org.apache.hadoop.hbase.coprocessor.RegionObserver;
56  import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
57  import org.apache.hadoop.hbase.filter.WritableByteArrayComparable;
58  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
59  import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
60  import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
61  import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
62  import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
63  import org.apache.hadoop.hbase.util.Bytes;
64  import org.apache.hadoop.hbase.util.Pair;
65  import org.apache.hadoop.util.StringUtils;
66  
67  import com.google.common.collect.ImmutableList;
68  
69  /**
70   * Implements the coprocessor environment and runtime support for coprocessors
71   * loaded within a {@link HRegion}.
72   */
73  public class RegionCoprocessorHost
74      extends CoprocessorHost<RegionCoprocessorHost.RegionEnvironment> {
75  
76    private static final Log LOG = LogFactory.getLog(RegionCoprocessorHost.class);
77    // The shared data map
78    private static ReferenceMap sharedDataMap =
79        new ReferenceMap(AbstractReferenceMap.HARD, AbstractReferenceMap.WEAK);
80  
81    /**
82     * Encapsulation of the environment of each coprocessor
83     */
84    static class RegionEnvironment extends CoprocessorHost.Environment
85        implements RegionCoprocessorEnvironment {
86  
87      private HRegion region;
88      private RegionServerServices rsServices;
89      ConcurrentMap<String, Object> sharedData;
90  
91      /**
92       * Constructor
93       * @param impl the coprocessor instance
94       * @param priority chaining priority
95       */
96      public RegionEnvironment(final Coprocessor impl, final int priority,
97          final int seq, final Configuration conf, final HRegion region,
98          final RegionServerServices services, final ConcurrentMap<String, Object> sharedData) {
99        super(impl, priority, seq, conf);
100       this.region = region;
101       this.rsServices = services;
102       this.sharedData = sharedData;
103     }
104 
105     /** @return the region */
106     @Override
107     public HRegion getRegion() {
108       return region;
109     }
110 
111     /** @return reference to the region server services */
112     @Override
113     public RegionServerServices getRegionServerServices() {
114       return rsServices;
115     }
116 
117     public void shutdown() {
118       super.shutdown();
119     }
120 
121     @Override
122     public ConcurrentMap<String, Object> getSharedData() {
123       return sharedData;
124     }
125   }
126 
127   /** The region server services */
128   RegionServerServices rsServices;
129   /** The region */
130   HRegion region;
131 
132   /**
133    * Constructor
134    * @param region the region
135    * @param rsServices interface to available region server functionality
136    * @param conf the configuration
137    */
138   public RegionCoprocessorHost(final HRegion region,
139       final RegionServerServices rsServices, final Configuration conf) {
140     this.conf = conf;
141     this.rsServices = rsServices;
142     this.region = region;
143     this.pathPrefix = Integer.toString(this.region.getRegionInfo().hashCode());
144 
145     // load system default cp's from configuration.
146     loadSystemCoprocessors(conf, REGION_COPROCESSOR_CONF_KEY);
147 
148     // load system default cp's for user tables from configuration.
149     if (!HTableDescriptor.isMetaTable(region.getRegionInfo().getTableName())) {
150       loadSystemCoprocessors(conf, USER_REGION_COPROCESSOR_CONF_KEY);
151     }
152 
153     // load Coprocessor From HDFS
154     loadTableCoprocessors(conf);
155   }
156 
157   void loadTableCoprocessors(final Configuration conf) {
158     // scan the table attributes for coprocessor load specifications
159     // initialize the coprocessors
160     List<RegionEnvironment> configured = new ArrayList<RegionEnvironment>();
161     for (Map.Entry<ImmutableBytesWritable,ImmutableBytesWritable> e:
162         region.getTableDesc().getValues().entrySet()) {
163       String key = Bytes.toString(e.getKey().get()).trim();
164       String spec = Bytes.toString(e.getValue().get()).trim();
165       if (HConstants.CP_HTD_ATTR_KEY_PATTERN.matcher(key).matches()) {
166         // found one
167         try {
168           Matcher matcher = HConstants.CP_HTD_ATTR_VALUE_PATTERN.matcher(spec);
169           if (matcher.matches()) {
170             // jar file path can be empty if the cp class can be loaded
171             // from class loader.
172             Path path = matcher.group(1).trim().isEmpty() ?
173                 null : new Path(matcher.group(1).trim());
174             String className = matcher.group(2).trim();
175             int priority = matcher.group(3).trim().isEmpty() ?
176                 Coprocessor.PRIORITY_USER : Integer.valueOf(matcher.group(3));
177             String cfgSpec = null;
178             try {
179               cfgSpec = matcher.group(4);
180             } catch (IndexOutOfBoundsException ex) {
181               // ignore
182             }
183             if (cfgSpec != null) {
184               cfgSpec = cfgSpec.substring(cfgSpec.indexOf('|') + 1);
185               Configuration newConf = new Configuration(conf);
186               Matcher m = HConstants.CP_HTD_ATTR_VALUE_PARAM_PATTERN.matcher(cfgSpec);
187               while (m.find()) {
188                 newConf.set(m.group(1), m.group(2));
189               }
190               configured.add(load(path, className, priority, newConf));
191             } else {
192               configured.add(load(path, className, priority, conf));
193             }
194             LOG.info("Load coprocessor " + className + " from HTD of " +
195               Bytes.toString(region.getTableDesc().getName()) +
196                 " successfully.");
197           } else {
198             throw new RuntimeException("specification does not match pattern");
199           }
200         } catch (Exception ex) {
201           LOG.warn("attribute '" + key +
202             "' has invalid coprocessor specification '" + spec + "'");
203           LOG.warn(StringUtils.stringifyException(ex));
204         }
205       }
206     }
207     // add together to coprocessor set for COW efficiency
208     coprocessors.addAll(configured);
209   }
210 
211   @Override
212   public RegionEnvironment createEnvironment(Class<?> implClass,
213       Coprocessor instance, int priority, int seq, Configuration conf) {
214     // Check if it's an Endpoint.
215     // Due to current dynamic protocol design, Endpoint
216     // uses a different way to be registered and executed.
217     // It uses a visitor pattern to invoke registered Endpoint
218     // method.
219     for (Class c : implClass.getInterfaces()) {
220       if (CoprocessorProtocol.class.isAssignableFrom(c)) {
221         region.registerProtocol(c, (CoprocessorProtocol)instance);
222         break;
223       }
224     }
225     ConcurrentMap<String, Object> classData;
226     // make sure only one thread can add maps
227     synchronized (sharedDataMap) {
228       // as long as at least one RegionEnvironment holds on to its classData it will
229       // remain in this map
230       classData = (ConcurrentMap<String, Object>)sharedDataMap.get(implClass.getName());
231       if (classData == null) {
232         classData = new ConcurrentHashMap<String, Object>();
233         sharedDataMap.put(implClass.getName(), classData);
234       }
235     }
236     return new RegionEnvironment(instance, priority, seq, conf, region,
237         rsServices, classData);
238   }
239 
240   @Override
241   protected void abortServer(final CoprocessorEnvironment env, final Throwable e) {
242     abortServer("regionserver", rsServices, env, e);
243   }
244 
245   /**
246    * HBASE-4014 : This is used by coprocessor hooks which are not declared to throw exceptions.
247    *
248    * For example, {@link
249    * org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost#preOpen()} and
250    * {@link org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost#postOpen()} are such hooks.
251    *
252    * See also {@link org.apache.hadoop.hbase.master.MasterCoprocessorHost#handleCoprocessorThrowable()}
253    * @param env The coprocessor that threw the exception.
254    * @param e The exception that was thrown.
255    */
256   private void handleCoprocessorThrowableNoRethrow(
257       final CoprocessorEnvironment env, final Throwable e) {
258     try {
259       handleCoprocessorThrowable(env,e);
260     } catch (IOException ioe) {
261       // We cannot throw exceptions from the caller hook, so ignore.
262       LOG.warn("handleCoprocessorThrowable() threw an IOException while attempting to handle Throwable " + e
263         + ". Ignoring.",e);
264     }
265   }
266 
267   /**
268    * Invoked before a region open
269    */
270   public void preOpen(){
271     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
272     for (RegionEnvironment env: coprocessors) {
273       if (env.getInstance() instanceof RegionObserver) {
274         ctx = ObserverContext.createAndPrepare(env, ctx);
275          try {
276           ((RegionObserver)env.getInstance()).preOpen(ctx);
277          } catch (Throwable e) {
278            handleCoprocessorThrowableNoRethrow(env, e);
279          }
280         if (ctx.shouldComplete()) {
281           break;
282         }
283       }
284     }
285   }
286 
287   /**
288    * Invoked after a region open
289    */
290   public void postOpen(){
291     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
292     for (RegionEnvironment env: coprocessors) {
293       if (env.getInstance() instanceof RegionObserver) {
294         ctx = ObserverContext.createAndPrepare(env, ctx);
295         try {
296           ((RegionObserver)env.getInstance()).postOpen(ctx);
297         } catch (Throwable e) {
298           handleCoprocessorThrowableNoRethrow(env, e);
299         }
300         if (ctx.shouldComplete()) {
301           break;
302         }
303       }
304     }
305   }
306 
307   /**
308    * Invoked before a region is closed
309    * @param abortRequested true if the server is aborting
310    */
311   public void preClose(boolean abortRequested) throws IOException {
312     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
313     for (RegionEnvironment env: coprocessors) {
314       if (env.getInstance() instanceof RegionObserver) {
315         ctx = ObserverContext.createAndPrepare(env, ctx);
316         try {
317           ((RegionObserver)env.getInstance()).preClose(ctx, abortRequested);
318         } catch (Throwable e) {
319           handleCoprocessorThrowable(env, e);
320         }
321       }
322     }
323   }
324 
325   /**
326    * Invoked after a region is closed
327    * @param abortRequested true if the server is aborting
328    */
329   public void postClose(boolean abortRequested){
330     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
331     for (RegionEnvironment env: coprocessors) {
332       if (env.getInstance() instanceof RegionObserver) {
333         ctx = ObserverContext.createAndPrepare(env, ctx);
334         try {
335           ((RegionObserver)env.getInstance()).postClose(ctx, abortRequested);
336         } catch (Throwable e) {
337           handleCoprocessorThrowableNoRethrow(env, e);
338         }
339 
340       }
341       shutdown(env);
342     }
343   }
344 
345   /**
346    * See
347    * {@link RegionObserver#preCompactScannerOpen(ObserverContext, Store, List, ScanType, long, InternalScanner, CompactionRequest)}
348    */
349   public InternalScanner preCompactScannerOpen(Store store, List<StoreFileScanner> scanners,
350       ScanType scanType, long earliestPutTs, CompactionRequest request) throws IOException {
351     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
352     InternalScanner s = null;
353     for (RegionEnvironment env: coprocessors) {
354       if (env.getInstance() instanceof RegionObserver) {
355         ctx = ObserverContext.createAndPrepare(env, ctx);
356         try {
357           s = ((RegionObserver) env.getInstance()).preCompactScannerOpen(ctx, store, scanners,
358             scanType, earliestPutTs, s, request);
359         } catch (Throwable e) {
360           handleCoprocessorThrowable(env,e);
361         }
362         if (ctx.shouldComplete()) {
363           break;
364         }
365       }
366     }
367     return s;
368   }
369 
370   /**
371    * Called prior to selecting the {@link StoreFile}s for compaction from the list of currently
372    * available candidates.
373    * @param store The store where compaction is being requested
374    * @param candidates The currently available store files
375    * @param request custom compaction request
376    * @return If {@code true}, skip the normal selection process and use the current list
377    * @throws IOException
378    */
379   public boolean preCompactSelection(Store store, List<StoreFile> candidates,
380       CompactionRequest request) throws IOException {
381     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
382     boolean bypass = false;
383     for (RegionEnvironment env: coprocessors) {
384       if (env.getInstance() instanceof RegionObserver) {
385         ctx = ObserverContext.createAndPrepare(env, ctx);
386         try {
387           ((RegionObserver) env.getInstance()).preCompactSelection(ctx, store, candidates, request);
388         } catch (Throwable e) {
389           handleCoprocessorThrowable(env,e);
390         }
391         bypass |= ctx.shouldBypass();
392         if (ctx.shouldComplete()) {
393           break;
394         }
395       }
396     }
397     return bypass;
398   }
399 
400   /**
401    * Called after the {@link StoreFile}s to be compacted have been selected from the available
402    * candidates.
403    * @param store The store where compaction is being requested
404    * @param selected The store files selected to compact
405    * @param request custom compaction
406    */
407   public void postCompactSelection(Store store, ImmutableList<StoreFile> selected,
408       CompactionRequest request) {
409     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
410     for (RegionEnvironment env: coprocessors) {
411       if (env.getInstance() instanceof RegionObserver) {
412         ctx = ObserverContext.createAndPrepare(env, ctx);
413         try {
414           ((RegionObserver) env.getInstance()).postCompactSelection(ctx, store, selected, request);
415         } catch (Throwable e) {
416           handleCoprocessorThrowableNoRethrow(env,e);
417         }
418         if (ctx.shouldComplete()) {
419           break;
420         }
421       }
422     }
423   }
424 
425   /**
426    * Called prior to rewriting the store files selected for compaction
427    * @param store the store being compacted
428    * @param scanner the scanner used to read store data during compaction
429    * @param request the compaction that will be executed
430    * @throws IOException
431    */
432   public InternalScanner preCompact(Store store, InternalScanner scanner, 
433       CompactionRequest request) throws IOException {
434     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
435     boolean bypass = false;
436     for (RegionEnvironment env : coprocessors) {
437       if (env.getInstance() instanceof RegionObserver) {
438         ctx = ObserverContext.createAndPrepare(env, ctx);
439         try {
440           scanner = ((RegionObserver) env.getInstance()).preCompact(ctx, store, scanner, request);
441         } catch (Throwable e) {
442           handleCoprocessorThrowable(env, e);
443         }
444         bypass |= ctx.shouldBypass();
445         if (ctx.shouldComplete()) {
446           break;
447         }
448       }
449     }
450     return bypass ? null : scanner;
451   }
452 
453   /**
454    * Called after the store compaction has completed.
455    * @param store the store being compacted
456    * @param resultFile the new store file written during compaction
457    * @param request the compaction that is being executed
458    * @throws IOException
459    */
460   public void postCompact(Store store, StoreFile resultFile, CompactionRequest request)
461       throws IOException {
462     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
463     for (RegionEnvironment env: coprocessors) {
464       if (env.getInstance() instanceof RegionObserver) {
465         ctx = ObserverContext.createAndPrepare(env, ctx);
466         try {
467           ((RegionObserver) env.getInstance()).postCompact(ctx, store, resultFile, request);
468         } catch (Throwable e) {
469           handleCoprocessorThrowable(env, e);
470         }
471         if (ctx.shouldComplete()) {
472           break;
473         }
474       }
475     }
476   }
477 
478   /**
479    * Invoked before a memstore flush
480    * @throws IOException
481    */
482   public InternalScanner preFlush(Store store, InternalScanner scanner) throws IOException {
483     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
484     boolean bypass = false;
485     for (RegionEnvironment env: coprocessors) {
486       if (env.getInstance() instanceof RegionObserver) {
487         ctx = ObserverContext.createAndPrepare(env, ctx);
488         try {
489           scanner = ((RegionObserver)env.getInstance()).preFlush(
490               ctx, store, scanner);
491         } catch (Throwable e) {
492           handleCoprocessorThrowable(env,e);
493         }
494         bypass |= ctx.shouldBypass();
495         if (ctx.shouldComplete()) {
496           break;
497         }
498       }
499     }
500     return bypass ? null : scanner;
501   }
502 
503   /**
504    * Invoked before a memstore flush
505    * @throws IOException 
506    */
507   public void preFlush() throws IOException {
508     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
509     for (RegionEnvironment env: coprocessors) {
510       if (env.getInstance() instanceof RegionObserver) {
511         ctx = ObserverContext.createAndPrepare(env, ctx);
512         try {
513           ((RegionObserver)env.getInstance()).preFlush(ctx);
514         } catch (Throwable e) {
515           handleCoprocessorThrowable(env, e);
516         }
517         if (ctx.shouldComplete()) {
518           break;
519         }
520       }
521     }
522   }
523 
524   /**
525    * See
526    * {@link RegionObserver#preFlush(ObserverContext, Store, KeyValueScanner)}
527    */
528   public InternalScanner preFlushScannerOpen(Store store, KeyValueScanner memstoreScanner) throws IOException {
529     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
530     InternalScanner s = null;
531     for (RegionEnvironment env : coprocessors) {
532       if (env.getInstance() instanceof RegionObserver) {
533         ctx = ObserverContext.createAndPrepare(env, ctx);
534         try {
535           s = ((RegionObserver) env.getInstance()).preFlushScannerOpen(ctx, store, memstoreScanner, s);
536         } catch (Throwable e) {
537           handleCoprocessorThrowable(env, e);
538         }
539         if (ctx.shouldComplete()) {
540           break;
541         }
542       }
543     }
544     return s;
545   }
546 
547   /**
548    * Invoked after a memstore flush
549    * @throws IOException
550    */
551   public void postFlush() throws IOException {
552     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
553     for (RegionEnvironment env: coprocessors) {
554       if (env.getInstance() instanceof RegionObserver) {
555         ctx = ObserverContext.createAndPrepare(env, ctx);
556         try {
557           ((RegionObserver)env.getInstance()).postFlush(ctx);
558         } catch (Throwable e) {
559           handleCoprocessorThrowable(env, e);
560         }
561         if (ctx.shouldComplete()) {
562           break;
563         }
564       }
565     }
566   }
567 
568   /**
569    * Invoked after a memstore flush
570    * @throws IOException
571    */
572   public void postFlush(final Store store, final StoreFile storeFile) throws IOException {
573     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
574     for (RegionEnvironment env: coprocessors) {
575       if (env.getInstance() instanceof RegionObserver) {
576         ctx = ObserverContext.createAndPrepare(env, ctx);
577         try {
578           ((RegionObserver)env.getInstance()).postFlush(ctx, store, storeFile);
579         } catch (Throwable e) {
580           handleCoprocessorThrowable(env, e);
581         }
582         if (ctx.shouldComplete()) {
583           break;
584         }
585       }
586     }
587   }
588 
589   /**
590    * Invoked just before a split
591    * @throws IOException
592    */
593   public void preSplit() throws IOException {
594     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
595     for (RegionEnvironment env: coprocessors) {
596       if (env.getInstance() instanceof RegionObserver) {
597         ctx = ObserverContext.createAndPrepare(env, ctx);
598         try {
599           ((RegionObserver)env.getInstance()).preSplit(ctx);
600         } catch (Throwable e) {
601           handleCoprocessorThrowable(env, e);
602         }
603         if (ctx.shouldComplete()) {
604           break;
605         }
606       }
607     }
608   }
609 
610   /**
611    * Invoked just after a split
612    * @param l the new left-hand daughter region
613    * @param r the new right-hand daughter region
614    * @throws IOException 
615    */
616   public void postSplit(HRegion l, HRegion r) throws IOException {
617     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
618     for (RegionEnvironment env: coprocessors) {
619       if (env.getInstance() instanceof RegionObserver) {
620         ctx = ObserverContext.createAndPrepare(env, ctx);
621         try {
622           ((RegionObserver)env.getInstance()).postSplit(ctx, l, r);
623         } catch (Throwable e) {
624           handleCoprocessorThrowable(env, e);
625         }
626         if (ctx.shouldComplete()) {
627           break;
628         }
629       }
630     }
631   }
632 
633   // RegionObserver support
634 
635   /**
636    * @param row the row key
637    * @param family the family
638    * @param result the result set from the region
639    * @return true if default processing should be bypassed
640    * @exception IOException Exception
641    */
642   public boolean preGetClosestRowBefore(final byte[] row, final byte[] family,
643       final Result result) throws IOException {
644     boolean bypass = false;
645     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
646     for (RegionEnvironment env: coprocessors) {
647       if (env.getInstance() instanceof RegionObserver) {
648         ctx = ObserverContext.createAndPrepare(env, ctx);
649         try {
650           ((RegionObserver)env.getInstance()).preGetClosestRowBefore(ctx, row,
651               family, result);
652         } catch (Throwable e) {
653           handleCoprocessorThrowable(env, e);
654         }
655         bypass |= ctx.shouldBypass();
656         if (ctx.shouldComplete()) {
657           break;
658         }
659       }
660     }
661     return bypass;
662   }
663 
664   /**
665    * @param row the row key
666    * @param family the family
667    * @param result the result set from the region
668    * @exception IOException Exception
669    */
670   public void postGetClosestRowBefore(final byte[] row, final byte[] family,
671       final Result result) throws IOException {
672     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
673     for (RegionEnvironment env: coprocessors) {
674       if (env.getInstance() instanceof RegionObserver) {
675         ctx = ObserverContext.createAndPrepare(env, ctx);
676         try {
677           ((RegionObserver)env.getInstance()).postGetClosestRowBefore(ctx, row,
678               family, result);
679         } catch (Throwable e) {
680           handleCoprocessorThrowable(env, e);
681         }
682         if (ctx.shouldComplete()) {
683           break;
684         }
685       }
686     }
687   }
688 
689   /**
690    * @param get the Get request
691    * @return true if default processing should be bypassed
692    * @exception IOException Exception
693    */
694   public boolean preGet(final Get get, final List<KeyValue> results)
695       throws IOException {
696     boolean bypass = false;
697     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
698     for (RegionEnvironment env: coprocessors) {
699       if (env.getInstance() instanceof RegionObserver) {
700         ctx = ObserverContext.createAndPrepare(env, ctx);
701         try {
702           ((RegionObserver)env.getInstance()).preGet(ctx, get, results);
703         } catch (Throwable e) {
704           handleCoprocessorThrowable(env, e);
705         }
706         bypass |= ctx.shouldBypass();
707         if (ctx.shouldComplete()) {
708           break;
709         }
710       }
711     }
712     return bypass;
713   }
714 
715   /**
716    * @param get the Get request
717    * @param results the result set
718    * @exception IOException Exception
719    */
720   public void postGet(final Get get, final List<KeyValue> results)
721   throws IOException {
722     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
723     for (RegionEnvironment env: coprocessors) {
724       if (env.getInstance() instanceof RegionObserver) {
725         ctx = ObserverContext.createAndPrepare(env, ctx);
726         try {
727           ((RegionObserver)env.getInstance()).postGet(ctx, get, results);
728         } catch (Throwable e) {
729           handleCoprocessorThrowable(env, e);
730         }
731         if (ctx.shouldComplete()) {
732           break;
733         }
734       }
735     }
736   }
737 
738   /**
739    * @param get the Get request
740    * @return true or false to return to client if bypassing normal operation,
741    * or null otherwise
742    * @exception IOException Exception
743    */
744   public Boolean preExists(final Get get) throws IOException {
745     boolean bypass = false;
746     boolean exists = false;
747     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
748     for (RegionEnvironment env: coprocessors) {
749       if (env.getInstance() instanceof RegionObserver) {
750         ctx = ObserverContext.createAndPrepare(env, ctx);
751         try {
752           exists = ((RegionObserver)env.getInstance()).preExists(ctx, get, exists);
753         } catch (Throwable e) {
754           handleCoprocessorThrowable(env, e);
755         }
756         bypass |= ctx.shouldBypass();
757         if (ctx.shouldComplete()) {
758           break;
759         }
760       }
761     }
762     return bypass ? exists : null;
763   }
764 
765   /**
766    * @param get the Get request
767    * @param exists the result returned by the region server
768    * @return the result to return to the client
769    * @exception IOException Exception
770    */
771   public boolean postExists(final Get get, boolean exists)
772       throws IOException {
773     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
774     for (RegionEnvironment env: coprocessors) {
775       if (env.getInstance() instanceof RegionObserver) {
776         ctx = ObserverContext.createAndPrepare(env, ctx);
777         try {
778           exists = ((RegionObserver)env.getInstance()).postExists(ctx, get, exists);
779         } catch (Throwable e) {
780           handleCoprocessorThrowable(env, e);
781         }
782         if (ctx.shouldComplete()) {
783           break;
784         }
785       }
786     }
787     return exists;
788   }
789 
790   /**
791    * @param put The Put object
792    * @param edit The WALEdit object.
793    * @param writeToWAL true if the change should be written to the WAL
794    * @return true if default processing should be bypassed
795    * @exception IOException Exception
796    */
797   public boolean prePut(Put put, WALEdit edit,
798       final boolean writeToWAL) throws IOException {
799     boolean bypass = false;
800     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
801     for (RegionEnvironment env: coprocessors) {
802       if (env.getInstance() instanceof RegionObserver) {
803         ctx = ObserverContext.createAndPrepare(env, ctx);
804         try {
805           ((RegionObserver)env.getInstance()).prePut(ctx, put, edit, writeToWAL);
806         } catch (Throwable e) {
807           handleCoprocessorThrowable(env, e);
808         }
809         bypass |= ctx.shouldBypass();
810         if (ctx.shouldComplete()) {
811           break;
812         }
813       }
814     }
815     return bypass;
816   }
817 
818   /**
819    * @param put The Put object
820    * @param edit The WALEdit object.
821    * @param writeToWAL true if the change should be written to the WAL
822    * @exception IOException Exception
823    */
824   public void postPut(Put put, WALEdit edit,
825       final boolean writeToWAL) throws IOException {
826     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
827     for (RegionEnvironment env: coprocessors) {
828       if (env.getInstance() instanceof RegionObserver) {
829         ctx = ObserverContext.createAndPrepare(env, ctx);
830         try {
831           ((RegionObserver)env.getInstance()).postPut(ctx, put, edit, writeToWAL);
832         } catch (Throwable e) {
833           handleCoprocessorThrowable(env, e);
834         }
835         if (ctx.shouldComplete()) {
836           break;
837         }
838       }
839     }
840   }
841 
842   /**
843    * @param delete The Delete object
844    * @param edit The WALEdit object.
845    * @param writeToWAL true if the change should be written to the WAL
846    * @return true if default processing should be bypassed
847    * @exception IOException Exception
848    */
849   public boolean preDelete(Delete delete, WALEdit edit,
850       final boolean writeToWAL) throws IOException {
851     boolean bypass = false;
852     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
853     for (RegionEnvironment env: coprocessors) {
854       if (env.getInstance() instanceof RegionObserver) {
855         ctx = ObserverContext.createAndPrepare(env, ctx);
856         try {
857           ((RegionObserver)env.getInstance()).preDelete(ctx, delete, edit, writeToWAL);
858         } catch (Throwable e) {
859           handleCoprocessorThrowable(env, e);
860         }
861         bypass |= ctx.shouldBypass();
862         if (ctx.shouldComplete()) {
863           break;
864         }
865       }
866     }
867     return bypass;
868   }
869 
870   /**
871    * @param delete The Delete object
872    * @param edit The WALEdit object.
873    * @param writeToWAL true if the change should be written to the WAL
874    * @exception IOException Exception
875    */
876   public void postDelete(Delete delete, WALEdit edit,
877       final boolean writeToWAL) throws IOException {
878     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
879     for (RegionEnvironment env: coprocessors) {
880       if (env.getInstance() instanceof RegionObserver) {
881         ctx = ObserverContext.createAndPrepare(env, ctx);
882         try {
883           ((RegionObserver)env.getInstance()).postDelete(ctx, delete, edit, writeToWAL);
884         } catch (Throwable e) {
885           handleCoprocessorThrowable(env, e);
886         }
887         if (ctx.shouldComplete()) {
888           break;
889         }
890       }
891     }
892   }
893   
894   /**
895    * @param miniBatchOp
896    * @return true if default processing should be bypassed
897    * @throws IOException
898    */
899   public boolean preBatchMutate(
900       final MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) throws IOException {
901     boolean bypass = false;
902     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
903     for (RegionEnvironment env : coprocessors) {
904       if (env.getInstance() instanceof RegionObserver) {
905         ctx = ObserverContext.createAndPrepare(env, ctx);
906         try {
907           ((RegionObserver) env.getInstance()).preBatchMutate(ctx, miniBatchOp);
908         } catch (Throwable e) {
909           handleCoprocessorThrowable(env, e);
910         }
911         bypass |= ctx.shouldBypass();
912         if (ctx.shouldComplete()) {
913           break;
914         }
915       }
916     }
917     return bypass;
918   }
919 
920   /**
921    * @param miniBatchOp
922    * @throws IOException
923    */
924   public void postBatchMutate(
925       final MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) throws IOException {
926     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
927     for (RegionEnvironment env : coprocessors) {
928       if (env.getInstance() instanceof RegionObserver) {
929         ctx = ObserverContext.createAndPrepare(env, ctx);
930         try {
931           ((RegionObserver) env.getInstance()).postBatchMutate(ctx, miniBatchOp);
932         } catch (Throwable e) {
933           handleCoprocessorThrowable(env, e);
934         }
935         if (ctx.shouldComplete()) {
936           break;
937         }
938       }
939     }
940   }
941 
942   /**
943    * @param row row to check
944    * @param family column family
945    * @param qualifier column qualifier
946    * @param compareOp the comparison operation
947    * @param comparator the comparator
948    * @param put data to put if check succeeds
949    * @return true or false to return to client if default processing should
950    * be bypassed, or null otherwise
951    * @throws IOException e
952    */
953   public Boolean preCheckAndPut(final byte [] row, final byte [] family,
954       final byte [] qualifier, final CompareOp compareOp,
955       final WritableByteArrayComparable comparator, Put put)
956     throws IOException {
957     boolean bypass = false;
958     boolean result = false;
959     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
960     for (RegionEnvironment env: coprocessors) {
961       if (env.getInstance() instanceof RegionObserver) {
962         ctx = ObserverContext.createAndPrepare(env, ctx);
963         try {
964           result = ((RegionObserver)env.getInstance()).preCheckAndPut(ctx, row, family,
965             qualifier, compareOp, comparator, put, result);
966         } catch (Throwable e) {
967           handleCoprocessorThrowable(env, e);
968         }
969 
970 
971         bypass |= ctx.shouldBypass();
972         if (ctx.shouldComplete()) {
973           break;
974         }
975       }
976     }
977     return bypass ? result : null;
978   }
979 
980   /**
981    * @param row row to check
982    * @param family column family
983    * @param qualifier column qualifier
984    * @param compareOp the comparison operation
985    * @param comparator the comparator
986    * @param put data to put if check succeeds
987    * @throws IOException e
988    */
989   public boolean postCheckAndPut(final byte [] row, final byte [] family,
990       final byte [] qualifier, final CompareOp compareOp,
991       final WritableByteArrayComparable comparator, final Put put,
992       boolean result)
993     throws IOException {
994     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
995     for (RegionEnvironment env: coprocessors) {
996       if (env.getInstance() instanceof RegionObserver) {
997         ctx = ObserverContext.createAndPrepare(env, ctx);
998         try {
999           result = ((RegionObserver)env.getInstance()).postCheckAndPut(ctx, row,
1000             family, qualifier, compareOp, comparator, put, result);
1001         } catch (Throwable e) {
1002           handleCoprocessorThrowable(env, e);
1003         }
1004         if (ctx.shouldComplete()) {
1005           break;
1006         }
1007       }
1008     }
1009     return result;
1010   }
1011 
1012   /**
1013    * @param row row to check
1014    * @param family column family
1015    * @param qualifier column qualifier
1016    * @param compareOp the comparison operation
1017    * @param comparator the comparator
1018    * @param delete delete to commit if check succeeds
1019    * @return true or false to return to client if default processing should
1020    * be bypassed, or null otherwise
1021    * @throws IOException e
1022    */
1023   public Boolean preCheckAndDelete(final byte [] row, final byte [] family,
1024       final byte [] qualifier, final CompareOp compareOp,
1025       final WritableByteArrayComparable comparator, Delete delete)
1026       throws IOException {
1027     boolean bypass = false;
1028     boolean result = false;
1029     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
1030     for (RegionEnvironment env: coprocessors) {
1031       if (env.getInstance() instanceof RegionObserver) {
1032         ctx = ObserverContext.createAndPrepare(env, ctx);
1033         try {
1034           result = ((RegionObserver)env.getInstance()).preCheckAndDelete(ctx, row,
1035             family, qualifier, compareOp, comparator, delete, result);
1036         } catch (Throwable e) {
1037           handleCoprocessorThrowable(env, e);
1038         }
1039         bypass |= ctx.shouldBypass();
1040         if (ctx.shouldComplete()) {
1041           break;
1042         }
1043       }
1044     }
1045     return bypass ? result : null;
1046   }
1047 
1048   /**
1049    * @param row row to check
1050    * @param family column family
1051    * @param qualifier column qualifier
1052    * @param compareOp the comparison operation
1053    * @param comparator the comparator
1054    * @param delete delete to commit if check succeeds
1055    * @throws IOException e
1056    */
1057   public boolean postCheckAndDelete(final byte [] row, final byte [] family,
1058       final byte [] qualifier, final CompareOp compareOp,
1059       final WritableByteArrayComparable comparator, final Delete delete,
1060       boolean result)
1061     throws IOException {
1062     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
1063     for (RegionEnvironment env: coprocessors) {
1064       if (env.getInstance() instanceof RegionObserver) {
1065         ctx = ObserverContext.createAndPrepare(env, ctx);
1066         try {
1067           result = ((RegionObserver)env.getInstance())
1068             .postCheckAndDelete(ctx, row, family, qualifier, compareOp,
1069               comparator, delete, result);
1070         } catch (Throwable e) {
1071           handleCoprocessorThrowable(env, e);
1072         }
1073         if (ctx.shouldComplete()) {
1074           break;
1075         }
1076       }
1077     }
1078     return result;
1079   }
1080 
1081   /**
1082    * @param row row to check
1083    * @param family column family
1084    * @param qualifier column qualifier
1085    * @param amount long amount to increment
1086    * @param writeToWAL true if the change should be written to the WAL
1087    * @return return value for client if default operation should be bypassed,
1088    * or null otherwise
1089    * @throws IOException if an error occurred on the coprocessor
1090    */
1091   public Long preIncrementColumnValue(final byte [] row, final byte [] family,
1092       final byte [] qualifier, long amount, final boolean writeToWAL)
1093       throws IOException {
1094     boolean bypass = false;
1095     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
1096     for (RegionEnvironment env: coprocessors) {
1097       if (env.getInstance() instanceof RegionObserver) {
1098         ctx = ObserverContext.createAndPrepare(env, ctx);
1099         try {
1100           amount = ((RegionObserver)env.getInstance()).preIncrementColumnValue(ctx,
1101               row, family, qualifier, amount, writeToWAL);
1102         } catch (Throwable e) {
1103           handleCoprocessorThrowable(env, e);
1104         }
1105         bypass |= ctx.shouldBypass();
1106         if (ctx.shouldComplete()) {
1107           break;
1108         }
1109       }
1110     }
1111     return bypass ? amount : null;
1112   }
1113 
1114   /**
1115    * @param row row to check
1116    * @param family column family
1117    * @param qualifier column qualifier
1118    * @param amount long amount to increment
1119    * @param writeToWAL true if the change should be written to the WAL
1120    * @param result the result returned by incrementColumnValue
1121    * @return the result to return to the client
1122    * @throws IOException if an error occurred on the coprocessor
1123    */
1124   public long postIncrementColumnValue(final byte [] row, final byte [] family,
1125       final byte [] qualifier, final long amount, final boolean writeToWAL,
1126       long result) throws IOException {
1127     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
1128     for (RegionEnvironment env: coprocessors) {
1129       if (env.getInstance() instanceof RegionObserver) {
1130         ctx = ObserverContext.createAndPrepare(env, ctx);
1131         try {
1132           result = ((RegionObserver)env.getInstance()).postIncrementColumnValue(ctx,
1133               row, family, qualifier, amount, writeToWAL, result);
1134         } catch (Throwable e) {
1135           handleCoprocessorThrowable(env, e);
1136         }
1137         if (ctx.shouldComplete()) {
1138           break;
1139         }
1140       }
1141     }
1142     return result;
1143   }
1144 
1145   /**
1146    * @param append append object
1147    * @return result to return to client if default operation should be
1148    * bypassed, null otherwise
1149    * @throws IOException if an error occurred on the coprocessor
1150    */
1151   public Result preAppend(Append append)
1152       throws IOException {
1153     boolean bypass = false;
1154     Result result = null;
1155     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
1156     for (RegionEnvironment env: coprocessors) {
1157       if (env.getInstance() instanceof RegionObserver) {
1158         ctx = ObserverContext.createAndPrepare(env, ctx);
1159         try {
1160           result = ((RegionObserver)env.getInstance()).preAppend(ctx, append);
1161         } catch (Throwable e) {
1162           handleCoprocessorThrowable(env, e);
1163         }
1164         bypass |= ctx.shouldBypass();
1165         if (ctx.shouldComplete()) {
1166           break;
1167         }
1168       }
1169     }
1170     return bypass ? result : null;
1171   }
1172 
1173   /**
1174    * @param increment increment object
1175    * @return result to return to client if default operation should be
1176    * bypassed, null otherwise
1177    * @throws IOException if an error occurred on the coprocessor
1178    */
1179   public Result preIncrement(Increment increment)
1180       throws IOException {
1181     boolean bypass = false;
1182     Result result = null;
1183     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
1184     for (RegionEnvironment env: coprocessors) {
1185       if (env.getInstance() instanceof RegionObserver) {
1186         ctx = ObserverContext.createAndPrepare(env, ctx);
1187         try {
1188           result = ((RegionObserver)env.getInstance()).preIncrement(ctx, increment);
1189         } catch (Throwable e) {
1190           handleCoprocessorThrowable(env, e);
1191         }
1192         bypass |= ctx.shouldBypass();
1193         if (ctx.shouldComplete()) {
1194           break;
1195         }
1196       }
1197     }
1198     return bypass ? result : null;
1199   }
1200 
1201   /**
1202    * @param append Append object
1203    * @param result the result returned by postAppend
1204    * @throws IOException if an error occurred on the coprocessor
1205    */
1206   public void postAppend(final Append append, Result result)
1207       throws IOException {
1208     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
1209     for (RegionEnvironment env: coprocessors) {
1210       if (env.getInstance() instanceof RegionObserver) {
1211         ctx = ObserverContext.createAndPrepare(env, ctx);
1212         try {
1213           ((RegionObserver)env.getInstance()).postAppend(ctx, append, result);
1214         } catch (Throwable e) {
1215           handleCoprocessorThrowable(env, e);
1216         }
1217         if (ctx.shouldComplete()) {
1218           break;
1219         }
1220       }
1221     }
1222   }
1223 
1224   /**
1225    * @param increment increment object
1226    * @param result the result returned by postIncrement
1227    * @throws IOException if an error occurred on the coprocessor
1228    */
1229   public Result postIncrement(final Increment increment, Result result)
1230       throws IOException {
1231     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
1232     for (RegionEnvironment env: coprocessors) {
1233       if (env.getInstance() instanceof RegionObserver) {
1234         ctx = ObserverContext.createAndPrepare(env, ctx);
1235         try {
1236           result = ((RegionObserver)env.getInstance()).postIncrement(ctx, increment, result);
1237         } catch (Throwable e) {
1238           handleCoprocessorThrowable(env, e);
1239         }
1240         if (ctx.shouldComplete()) {
1241           break;
1242         }
1243       }
1244     }
1245     return result;
1246   }
1247 
1248   /**
1249    * @param scan the Scan specification
1250    * @return scanner id to return to client if default operation should be
1251    * bypassed, false otherwise
1252    * @exception IOException Exception
1253    */
1254   public RegionScanner preScannerOpen(Scan scan) throws IOException {
1255     boolean bypass = false;
1256     RegionScanner s = null;
1257     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
1258     for (RegionEnvironment env: coprocessors) {
1259       if (env.getInstance() instanceof RegionObserver) {
1260         ctx = ObserverContext.createAndPrepare(env, ctx);
1261         try {
1262           s = ((RegionObserver)env.getInstance()).preScannerOpen(ctx, scan, s);
1263         } catch (Throwable e) {
1264           handleCoprocessorThrowable(env, e);
1265         }
1266         bypass |= ctx.shouldBypass();
1267         if (ctx.shouldComplete()) {
1268           break;
1269         }
1270       }
1271     }
1272     return bypass ? s : null;
1273   }
1274 
1275   /**
1276    * See
1277    * {@link RegionObserver#preStoreScannerOpen(ObserverContext, Store, Scan, NavigableSet, KeyValueScanner)}
1278    */
1279   public KeyValueScanner preStoreScannerOpen(Store store, Scan scan,
1280       final NavigableSet<byte[]> targetCols) throws IOException {
1281     KeyValueScanner s = null;
1282     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
1283     for (RegionEnvironment env: coprocessors) {
1284       if (env.getInstance() instanceof RegionObserver) {
1285         ctx = ObserverContext.createAndPrepare(env, ctx);
1286         try {
1287           s = ((RegionObserver) env.getInstance()).preStoreScannerOpen(ctx, store, scan,
1288               targetCols, s);
1289         } catch (Throwable e) {
1290           handleCoprocessorThrowable(env, e);
1291         }
1292         if (ctx.shouldComplete()) {
1293           break;
1294         }
1295       }
1296     }
1297     return s;
1298   }
1299 
1300   /**
1301    * @param scan the Scan specification
1302    * @param s the scanner
1303    * @return the scanner instance to use
1304    * @exception IOException Exception
1305    */
1306   public RegionScanner postScannerOpen(final Scan scan, RegionScanner s)
1307       throws IOException {
1308     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
1309     for (RegionEnvironment env: coprocessors) {
1310       if (env.getInstance() instanceof RegionObserver) {
1311         ctx = ObserverContext.createAndPrepare(env, ctx);
1312         try {
1313           s = ((RegionObserver)env.getInstance()).postScannerOpen(ctx, scan, s);
1314         } catch (Throwable e) {
1315           handleCoprocessorThrowable(env, e);
1316         }
1317         if (ctx.shouldComplete()) {
1318           break;
1319         }
1320       }
1321     }
1322     return s;
1323   }
1324 
1325   /**
1326    * @param s the scanner
1327    * @param results the result set returned by the region server
1328    * @param limit the maximum number of results to return
1329    * @return 'has next' indication to client if bypassing default behavior, or
1330    * null otherwise
1331    * @exception IOException Exception
1332    */
1333   public Boolean preScannerNext(final InternalScanner s,
1334       final List<Result> results, int limit) throws IOException {
1335     boolean bypass = false;
1336     boolean hasNext = false;
1337     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
1338     for (RegionEnvironment env: coprocessors) {
1339       if (env.getInstance() instanceof RegionObserver) {
1340         ctx = ObserverContext.createAndPrepare(env, ctx);
1341         try {
1342           hasNext = ((RegionObserver)env.getInstance()).preScannerNext(ctx, s, results,
1343             limit, hasNext);
1344         } catch (Throwable e) {
1345           handleCoprocessorThrowable(env, e);
1346         }
1347         bypass |= ctx.shouldBypass();
1348         if (ctx.shouldComplete()) {
1349           break;
1350         }
1351       }
1352     }
1353     return bypass ? hasNext : null;
1354   }
1355 
1356   /**
1357    * @param s the scanner
1358    * @param results the result set returned by the region server
1359    * @param limit the maximum number of results to return
1360    * @param hasMore
1361    * @return 'has more' indication to give to client
1362    * @exception IOException Exception
1363    */
1364   public boolean postScannerNext(final InternalScanner s,
1365       final List<Result> results, final int limit, boolean hasMore)
1366       throws IOException {
1367     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
1368     for (RegionEnvironment env: coprocessors) {
1369       if (env.getInstance() instanceof RegionObserver) {
1370         ctx = ObserverContext.createAndPrepare(env, ctx);
1371         try {
1372           hasMore = ((RegionObserver)env.getInstance()).postScannerNext(ctx, s,
1373             results, limit, hasMore);
1374         } catch (Throwable e) {
1375           handleCoprocessorThrowable(env, e);
1376         }
1377         if (ctx.shouldComplete()) {
1378           break;
1379         }
1380       }
1381     }
1382     return hasMore;
1383   }
1384 
1385   /**
1386    * This will be called by the scan flow when the current scanned row is being filtered out by the
1387    * filter.
1388    * @param s the scanner
1389    * @param currentRow The current rowkey which got filtered out
1390    * @return whether more rows are available for the scanner or not
1391    * @throws IOException
1392    */
1393   public boolean postScannerFilterRow(final InternalScanner s, final byte[] currentRow)
1394       throws IOException {
1395     boolean hasMore = true; // By default assume more rows there.
1396     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
1397     for (RegionEnvironment env : coprocessors) {
1398       if (env.getInstance() instanceof RegionObserver) {
1399         ctx = ObserverContext.createAndPrepare(env, ctx);
1400         try {
1401           hasMore = ((RegionObserver) env.getInstance()).postScannerFilterRow(ctx, s, currentRow,
1402               hasMore);
1403         } catch (Throwable e) {
1404           handleCoprocessorThrowable(env, e);
1405         }
1406         if (ctx.shouldComplete()) {
1407           break;
1408         }
1409       }
1410     }
1411     return hasMore;
1412   }
1413  
1414   /**
1415    * @param s the scanner
1416    * @return true if default behavior should be bypassed, false otherwise
1417    * @exception IOException Exception
1418    */
1419   public boolean preScannerClose(final InternalScanner s)
1420       throws IOException {
1421     boolean bypass = false;
1422     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
1423     for (RegionEnvironment env: coprocessors) {
1424       if (env.getInstance() instanceof RegionObserver) {
1425         ctx = ObserverContext.createAndPrepare(env, ctx);
1426         try {
1427           ((RegionObserver)env.getInstance()).preScannerClose(ctx, s);
1428         } catch (Throwable e) {
1429           handleCoprocessorThrowable(env, e);
1430         }
1431         bypass |= ctx.shouldBypass();
1432         if (ctx.shouldComplete()) {
1433           break;
1434         }
1435       }
1436     }
1437     return bypass;
1438   }
1439 
1440   /**
1441    * @param s the scanner
1442    * @exception IOException Exception
1443    */
1444   public void postScannerClose(final InternalScanner s)
1445       throws IOException {
1446     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
1447     for (RegionEnvironment env: coprocessors) {
1448       if (env.getInstance() instanceof RegionObserver) {
1449         ctx = ObserverContext.createAndPrepare(env, ctx);
1450         try {
1451           ((RegionObserver)env.getInstance()).postScannerClose(ctx, s);
1452         } catch (Throwable e) {
1453           handleCoprocessorThrowable(env, e);
1454         }
1455         if (ctx.shouldComplete()) {
1456           break;
1457         }
1458       }
1459     }
1460   }
1461 
1462   /**
1463    * @param info
1464    * @param logKey
1465    * @param logEdit
1466    * @return true if default behavior should be bypassed, false otherwise
1467    * @throws IOException
1468    */
1469   public boolean preWALRestore(HRegionInfo info, HLogKey logKey,
1470       WALEdit logEdit) throws IOException {
1471     boolean bypass = false;
1472     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
1473     for (RegionEnvironment env: coprocessors) {
1474       if (env.getInstance() instanceof RegionObserver) {
1475         ctx = ObserverContext.createAndPrepare(env, ctx);
1476         try {
1477           ((RegionObserver)env.getInstance()).preWALRestore(ctx, info, logKey,
1478               logEdit);
1479         } catch (Throwable e) {
1480           handleCoprocessorThrowable(env, e);
1481         }
1482         bypass |= ctx.shouldBypass();
1483         if (ctx.shouldComplete()) {
1484           break;
1485         }
1486       }
1487     }
1488     return bypass;
1489   }
1490 
1491   /**
1492    * @param info
1493    * @param logKey
1494    * @param logEdit
1495    * @throws IOException
1496    */
1497   public void postWALRestore(HRegionInfo info, HLogKey logKey,
1498       WALEdit logEdit) throws IOException {
1499     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
1500     for (RegionEnvironment env: coprocessors) {
1501       if (env.getInstance() instanceof RegionObserver) {
1502         ctx = ObserverContext.createAndPrepare(env, ctx);
1503         try {
1504           ((RegionObserver)env.getInstance()).postWALRestore(ctx, info,
1505               logKey, logEdit);
1506         } catch (Throwable e) {
1507           handleCoprocessorThrowable(env, e);
1508         }
1509         if (ctx.shouldComplete()) {
1510           break;
1511         }
1512       }
1513     }
1514   }
1515 
1516   /**
1517    * @param familyPaths pairs of { CF, file path } submitted for bulk load
1518    * @return true if the default operation should be bypassed
1519    * @throws IOException
1520    */
1521   public boolean preBulkLoadHFile(List<Pair<byte[], String>> familyPaths) throws IOException {
1522     boolean bypass = false;
1523     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
1524     for (RegionEnvironment env: coprocessors) {
1525       if (env.getInstance() instanceof RegionObserver) {
1526         ctx = ObserverContext.createAndPrepare(env, ctx);
1527         try {
1528           ((RegionObserver)env.getInstance()).preBulkLoadHFile(ctx, familyPaths);
1529         } catch (Throwable e) {
1530           handleCoprocessorThrowable(env, e);
1531         }
1532         bypass |= ctx.shouldBypass();
1533         if (ctx.shouldComplete()) {
1534           break;
1535         }
1536       }
1537     }
1538 
1539     return bypass;
1540   }
1541 
1542   /**
1543    * @param familyPaths pairs of { CF, file path } submitted for bulk load
1544    * @param hasLoaded whether load was successful or not
1545    * @return the possibly modified value of hasLoaded
1546    * @throws IOException
1547    */
1548   public boolean postBulkLoadHFile(List<Pair<byte[], String>> familyPaths, boolean hasLoaded)
1549       throws IOException {
1550     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
1551     for (RegionEnvironment env: coprocessors) {
1552       if (env.getInstance() instanceof RegionObserver) {
1553         ctx = ObserverContext.createAndPrepare(env, ctx);
1554         try {
1555           hasLoaded = ((RegionObserver)env.getInstance()).postBulkLoadHFile(ctx,
1556             familyPaths, hasLoaded);
1557         } catch (Throwable e) {
1558           handleCoprocessorThrowable(env, e);
1559         }
1560         if (ctx.shouldComplete()) {
1561           break;
1562         }
1563       }
1564     }
1565 
1566     return hasLoaded;
1567   }
1568   
1569   public void preLockRow(byte[] regionName, byte[] row) throws IOException {
1570     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
1571     for (RegionEnvironment env : coprocessors) {
1572       if (env.getInstance() instanceof RegionObserver) {
1573         ctx = ObserverContext.createAndPrepare(env, ctx);
1574         ((RegionObserver) env.getInstance()).preLockRow(ctx, regionName, row);
1575         if (ctx.shouldComplete()) {
1576           break;
1577         }
1578       }
1579     }
1580   }
1581 
1582   public void preUnLockRow(byte[] regionName, long lockId) throws IOException {
1583     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
1584     for (RegionEnvironment env : coprocessors) {
1585       if (env.getInstance() instanceof RegionObserver) {
1586         ctx = ObserverContext.createAndPrepare(env, ctx);
1587         ((RegionObserver) env.getInstance()).preUnlockRow(ctx, regionName, lockId);
1588         if (ctx.shouldComplete()) {
1589           break;
1590         }
1591       }
1592     }
1593   }
1594 
1595 }