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 com.google.common.collect.ImmutableList;
24  import org.apache.commons.logging.Log;
25  import org.apache.commons.logging.LogFactory;
26  
27  import org.apache.hadoop.conf.Configuration;
28  import org.apache.hadoop.fs.Path;
29  import org.apache.hadoop.hbase.HBaseConfiguration;
30  import org.apache.hadoop.hbase.HRegionInfo;
31  import org.apache.hadoop.hbase.HTableDescriptor;
32  import org.apache.hadoop.hbase.KeyValue;
33  import org.apache.hadoop.hbase.Coprocessor;
34  import org.apache.hadoop.hbase.CoprocessorEnvironment;
35  import org.apache.hadoop.hbase.HConstants;
36  import org.apache.hadoop.hbase.client.*;
37  import org.apache.hadoop.hbase.coprocessor.*;
38  import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
39  import org.apache.hadoop.hbase.filter.WritableByteArrayComparable;
40  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
41  import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
42  import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
43  import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
44  import org.apache.hadoop.hbase.util.Bytes;
45  import org.apache.hadoop.util.StringUtils;
46  
47  import java.io.IOException;
48  import java.util.*;
49  import java.util.regex.Matcher;
50  
51  /**
52   * Implements the coprocessor environment and runtime support for coprocessors
53   * loaded within a {@link HRegion}.
54   */
55  public class RegionCoprocessorHost
56      extends CoprocessorHost<RegionCoprocessorHost.RegionEnvironment> {
57  
58    private static final Log LOG = LogFactory.getLog(RegionCoprocessorHost.class);
59  
60    /**
61     * Encapsulation of the environment of each coprocessor
62     */
63    static class RegionEnvironment extends CoprocessorHost.Environment
64        implements RegionCoprocessorEnvironment {
65  
66      private HRegion region;
67      private RegionServerServices rsServices;
68  
69      /**
70       * Constructor
71       * @param impl the coprocessor instance
72       * @param priority chaining priority
73       */
74      public RegionEnvironment(final Coprocessor impl, final int priority,
75          final int seq, final Configuration conf, final HRegion region,
76          final RegionServerServices services) {
77        super(impl, priority, seq, conf);
78        this.region = region;
79        this.rsServices = services;
80      }
81  
82      /** @return the region */
83      @Override
84      public HRegion getRegion() {
85        return region;
86      }
87  
88      /** @return reference to the region server services */
89      @Override
90      public RegionServerServices getRegionServerServices() {
91        return rsServices;
92      }
93  
94      public void shutdown() {
95        super.shutdown();
96      }
97    }
98  
99    /** The region server services */
100   RegionServerServices rsServices;
101   /** The region */
102   HRegion region;
103 
104   /**
105    * Constructor
106    * @param region the region
107    * @param rsServices interface to available region server functionality
108    * @param conf the configuration
109    */
110   public RegionCoprocessorHost(final HRegion region,
111       final RegionServerServices rsServices, final Configuration conf) {
112     this.rsServices = rsServices;
113     this.region = region;
114     this.pathPrefix = this.region.getRegionNameAsString().replace(',', '_');
115 
116     // load system default cp's from configuration.
117     loadSystemCoprocessors(conf, REGION_COPROCESSOR_CONF_KEY);
118 
119     // load system default cp's for user tables from configuration.
120     if (!HTableDescriptor.isMetaTable(region.getRegionInfo().getTableName())) {
121       loadSystemCoprocessors(conf, USER_REGION_COPROCESSOR_CONF_KEY);
122     }
123 
124     // load Coprocessor From HDFS
125     loadTableCoprocessors(conf);
126   }
127 
128   void loadTableCoprocessors(final Configuration conf) {
129     // scan the table attributes for coprocessor load specifications
130     // initialize the coprocessors
131     List<RegionEnvironment> configured = new ArrayList<RegionEnvironment>();
132     for (Map.Entry<ImmutableBytesWritable,ImmutableBytesWritable> e:
133         region.getTableDesc().getValues().entrySet()) {
134       String key = Bytes.toString(e.getKey().get()).trim();
135       String spec = Bytes.toString(e.getValue().get()).trim();
136       if (HConstants.CP_HTD_ATTR_KEY_PATTERN.matcher(key).matches()) {
137         // found one
138         try {
139           Matcher matcher = HConstants.CP_HTD_ATTR_VALUE_PATTERN.matcher(spec);
140           if (matcher.matches()) {
141             // jar file path can be empty if the cp class can be loaded
142             // from class loader.
143             Path path = matcher.group(1).trim().isEmpty() ?
144                 null : new Path(matcher.group(1).trim());
145             String className = matcher.group(2).trim();
146             int priority = matcher.group(3).trim().isEmpty() ?
147                 Coprocessor.PRIORITY_USER : Integer.valueOf(matcher.group(3));
148             String cfgSpec = null;
149             try {
150               cfgSpec = matcher.group(4);
151             } catch (IndexOutOfBoundsException ex) {
152               // ignore
153             }
154             if (cfgSpec != null) {
155               cfgSpec = cfgSpec.substring(cfgSpec.indexOf('|') + 1);
156               Configuration newConf = HBaseConfiguration.create(conf);
157               Matcher m = HConstants.CP_HTD_ATTR_VALUE_PARAM_PATTERN.matcher(cfgSpec);
158               while (m.find()) {
159                 newConf.set(m.group(1), m.group(2));
160               }
161               configured.add(load(path, className, priority, newConf));
162             } else {
163               configured.add(load(path, className, priority, conf));
164             }
165             LOG.info("Load coprocessor " + className + " from HTD of " +
166               Bytes.toString(region.getTableDesc().getName()) +
167                 " successfully.");
168           } else {
169             throw new RuntimeException("specification does not match pattern");
170           }
171         } catch (Exception ex) {
172           LOG.warn("attribute '" + key +
173             "' has invalid coprocessor specification '" + spec + "'");
174           LOG.warn(StringUtils.stringifyException(ex));
175         }
176       }
177     }
178     // add together to coprocessor set for COW efficiency
179     coprocessors.addAll(configured);
180   }
181 
182   @Override
183   public RegionEnvironment createEnvironment(Class<?> implClass,
184       Coprocessor instance, int priority, int seq, Configuration conf) {
185     // Check if it's an Endpoint.
186     // Due to current dynamic protocol design, Endpoint
187     // uses a different way to be registered and executed.
188     // It uses a visitor pattern to invoke registered Endpoint
189     // method.
190     for (Class c : implClass.getInterfaces()) {
191       if (CoprocessorProtocol.class.isAssignableFrom(c)) {
192         region.registerProtocol(c, (CoprocessorProtocol)instance);
193         break;
194       }
195     }
196     return new RegionEnvironment(instance, priority, seq, conf, region,
197         rsServices);
198   }
199 
200   @Override
201   protected void abortServer(final CoprocessorEnvironment env, final Throwable e) {
202     abortServer("regionserver", rsServices, env, e);
203   }
204 
205   /**
206    * HBASE-4014 : This is used by coprocessor hooks which are not declared to throw exceptions.
207    *
208    * For example, {@link
209    * org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost#preOpen()} and
210    * {@link org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost#postOpen()} are such hooks.
211    *
212    * See also {@link org.apache.hadoop.hbase.master.MasterCoprocessorHost#handleCoprocessorThrowable()}
213    * @param env The coprocessor that threw the exception.
214    * @param e The exception that was thrown.
215    */
216   private void handleCoprocessorThrowableNoRethrow(
217       final CoprocessorEnvironment env, final Throwable e) {
218     try {
219       handleCoprocessorThrowable(env,e);
220     } catch (IOException ioe) {
221       // We cannot throw exceptions from the caller hook, so ignore.
222       LOG.warn("handleCoprocessorThrowable() threw an IOException while attempting to handle Throwable " + e
223         + ". Ignoring.",e);
224     }
225   }
226 
227   /**
228    * Invoked before a region open
229    */
230   public void preOpen() {
231     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
232     for (RegionEnvironment env: coprocessors) {
233       if (env.getInstance() instanceof RegionObserver) {
234         ctx = ObserverContext.createAndPrepare(env, ctx);
235          try {
236           ((RegionObserver)env.getInstance()).preOpen(ctx);
237          } catch (Throwable e) {
238            handleCoprocessorThrowableNoRethrow(env, e);
239          }
240         if (ctx.shouldComplete()) {
241           break;
242         }
243       }
244     }
245   }
246 
247   /**
248    * Invoked after a region open
249    */
250   public void postOpen() {
251     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
252     for (RegionEnvironment env: coprocessors) {
253       if (env.getInstance() instanceof RegionObserver) {
254         ctx = ObserverContext.createAndPrepare(env, ctx);
255         try {
256           ((RegionObserver)env.getInstance()).postOpen(ctx);
257         } catch (Throwable e) {
258           handleCoprocessorThrowableNoRethrow(env, e);
259         }
260         if (ctx.shouldComplete()) {
261           break;
262         }
263       }
264     }
265   }
266 
267   /**
268    * Invoked before a region is closed
269    * @param abortRequested true if the server is aborting
270    */
271   public void preClose(boolean abortRequested) {
272     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
273     for (RegionEnvironment env: coprocessors) {
274       if (env.getInstance() instanceof RegionObserver) {
275         ctx = ObserverContext.createAndPrepare(env, ctx);
276         try {
277           ((RegionObserver)env.getInstance()).preClose(ctx, abortRequested);
278         } catch (Throwable e) {
279           handleCoprocessorThrowableNoRethrow(env, e);
280         }
281       }
282     }
283   }
284 
285   /**
286    * Invoked after a region is closed
287    * @param abortRequested true if the server is aborting
288    */
289   public void postClose(boolean abortRequested) {
290     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
291     for (RegionEnvironment env: coprocessors) {
292       if (env.getInstance() instanceof RegionObserver) {
293         ctx = ObserverContext.createAndPrepare(env, ctx);
294         try {
295           ((RegionObserver)env.getInstance()).postClose(ctx, abortRequested);
296         } catch (Throwable e) {
297           handleCoprocessorThrowableNoRethrow(env, e);
298         }
299 
300       }
301       shutdown(env);
302     }
303   }
304 
305   /**
306    * Called prior to selecting the {@link StoreFile}s for compaction from
307    * the list of currently available candidates.
308    * @param store The store where compaction is being requested
309    * @param candidates The currently available store files
310    * @return If {@code true}, skip the normal selection process and use the current list
311    */
312   public boolean preCompactSelection(Store store, List<StoreFile> candidates) {
313     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
314     boolean bypass = false;
315     for (RegionEnvironment env: coprocessors) {
316       if (env.getInstance() instanceof RegionObserver) {
317         ctx = ObserverContext.createAndPrepare(env, ctx);
318         ((RegionObserver)env.getInstance()).preCompactSelection(
319             ctx, store, candidates);
320         bypass |= ctx.shouldBypass();
321         if (ctx.shouldComplete()) {
322           break;
323         }
324       }
325     }
326     return bypass;
327   }
328 
329   /**
330    * Called after the {@link StoreFile}s to be compacted have been selected
331    * from the available candidates.
332    * @param store The store where compaction is being requested
333    * @param selected The store files selected to compact
334    */
335   public void postCompactSelection(Store store,
336       ImmutableList<StoreFile> selected) {
337     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
338     for (RegionEnvironment env: coprocessors) {
339       if (env.getInstance() instanceof RegionObserver) {
340         ctx = ObserverContext.createAndPrepare(env, ctx);
341         try {
342           ((RegionObserver)env.getInstance()).postCompactSelection(
343               ctx, store, selected);
344         } catch (Throwable e) {
345           handleCoprocessorThrowableNoRethrow(env,e);
346         }
347         if (ctx.shouldComplete()) {
348           break;
349         }
350       }
351     }
352   }
353 
354   /**
355    * Called prior to rewriting the store files selected for compaction
356    * @param store the store being compacted
357    * @param scanner the scanner used to read store data during compaction
358    */
359   public InternalScanner preCompact(Store store, InternalScanner scanner) {
360     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
361     boolean bypass = false;
362     for (RegionEnvironment env: coprocessors) {
363       if (env.getInstance() instanceof RegionObserver) {
364         ctx = ObserverContext.createAndPrepare(env, ctx);
365         try {
366           scanner = ((RegionObserver)env.getInstance()).preCompact(
367               ctx, store, scanner);
368         } catch (Throwable e) {
369           handleCoprocessorThrowableNoRethrow(env,e);
370         }
371         bypass |= ctx.shouldBypass();
372         if (ctx.shouldComplete()) {
373           break;
374         }
375       }
376     }
377     return bypass ? null : scanner;
378   }
379 
380   /**
381    * Called after the store compaction has completed.
382    * @param store the store being compacted
383    * @param resultFile the new store file written during compaction
384    */
385   public void postCompact(Store store, StoreFile resultFile) {
386     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
387     for (RegionEnvironment env: coprocessors) {
388       if (env.getInstance() instanceof RegionObserver) {
389         ctx = ObserverContext.createAndPrepare(env, ctx);
390         try {
391           ((RegionObserver)env.getInstance()).postCompact(ctx, store, resultFile);
392         } catch (Throwable e) {
393           handleCoprocessorThrowableNoRethrow(env, e);
394         }
395         if (ctx.shouldComplete()) {
396           break;
397         }
398       }
399     }
400   }
401 
402   /**
403    * Invoked before a memstore flush
404    */
405   public void preFlush() {
406     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
407     for (RegionEnvironment env: coprocessors) {
408       if (env.getInstance() instanceof RegionObserver) {
409         ctx = ObserverContext.createAndPrepare(env, ctx);
410         try {
411           ((RegionObserver)env.getInstance()).preFlush(ctx);
412         } catch (Throwable e) {
413           handleCoprocessorThrowableNoRethrow(env, e);
414         }
415         if (ctx.shouldComplete()) {
416           break;
417         }
418       }
419     }
420   }
421 
422   /**
423    * Invoked after a memstore flush
424    */
425   public void postFlush() {
426     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
427     for (RegionEnvironment env: coprocessors) {
428       if (env.getInstance() instanceof RegionObserver) {
429         ctx = ObserverContext.createAndPrepare(env, ctx);
430         try {
431           ((RegionObserver)env.getInstance()).postFlush(ctx);
432         } catch (Throwable e) {
433           handleCoprocessorThrowableNoRethrow(env, e);
434         }
435         if (ctx.shouldComplete()) {
436           break;
437         }
438       }
439     }
440   }
441 
442   /**
443    * Invoked just before a split
444    */
445   public void preSplit() {
446     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
447     for (RegionEnvironment env: coprocessors) {
448       if (env.getInstance() instanceof RegionObserver) {
449         ctx = ObserverContext.createAndPrepare(env, ctx);
450         try {
451           ((RegionObserver)env.getInstance()).preSplit(ctx);
452         } catch (Throwable e) {
453           handleCoprocessorThrowableNoRethrow(env, e);
454         }
455         if (ctx.shouldComplete()) {
456           break;
457         }
458       }
459     }
460   }
461 
462   /**
463    * Invoked just after a split
464    * @param l the new left-hand daughter region
465    * @param r the new right-hand daughter region
466    */
467   public void postSplit(HRegion l, HRegion r) {
468     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
469     for (RegionEnvironment env: coprocessors) {
470       if (env.getInstance() instanceof RegionObserver) {
471         ctx = ObserverContext.createAndPrepare(env, ctx);
472         try {
473           ((RegionObserver)env.getInstance()).postSplit(ctx, l, r);
474         } catch (Throwable e) {
475           handleCoprocessorThrowableNoRethrow(env, e);
476         }
477         if (ctx.shouldComplete()) {
478           break;
479         }
480       }
481     }
482   }
483 
484   // RegionObserver support
485 
486   /**
487    * @param row the row key
488    * @param family the family
489    * @param result the result set from the region
490    * @return true if default processing should be bypassed
491    * @exception IOException Exception
492    */
493   public boolean preGetClosestRowBefore(final byte[] row, final byte[] family,
494       final Result result) throws IOException {
495     boolean bypass = false;
496     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
497     for (RegionEnvironment env: coprocessors) {
498       if (env.getInstance() instanceof RegionObserver) {
499         ctx = ObserverContext.createAndPrepare(env, ctx);
500         try {
501           ((RegionObserver)env.getInstance()).preGetClosestRowBefore(ctx, row,
502               family, result);
503         } catch (Throwable e) {
504           handleCoprocessorThrowable(env, e);
505         }
506         bypass |= ctx.shouldBypass();
507         if (ctx.shouldComplete()) {
508           break;
509         }
510       }
511     }
512     return bypass;
513   }
514 
515   /**
516    * @param row the row key
517    * @param family the family
518    * @param result the result set from the region
519    * @exception IOException Exception
520    */
521   public void postGetClosestRowBefore(final byte[] row, final byte[] family,
522       final Result result) throws IOException {
523     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
524     for (RegionEnvironment env: coprocessors) {
525       if (env.getInstance() instanceof RegionObserver) {
526         ctx = ObserverContext.createAndPrepare(env, ctx);
527         try {
528           ((RegionObserver)env.getInstance()).postGetClosestRowBefore(ctx, row,
529               family, result);
530         } catch (Throwable e) {
531           handleCoprocessorThrowable(env, e);
532         }
533         if (ctx.shouldComplete()) {
534           break;
535         }
536       }
537     }
538   }
539 
540   /**
541    * @param get the Get request
542    * @return true if default processing should be bypassed
543    * @exception IOException Exception
544    */
545   public boolean preGet(final Get get, final List<KeyValue> results)
546       throws IOException {
547     boolean bypass = false;
548     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
549     for (RegionEnvironment env: coprocessors) {
550       if (env.getInstance() instanceof RegionObserver) {
551         ctx = ObserverContext.createAndPrepare(env, ctx);
552         try {
553           ((RegionObserver)env.getInstance()).preGet(ctx, get, results);
554         } catch (Throwable e) {
555           handleCoprocessorThrowable(env, e);
556         }
557         bypass |= ctx.shouldBypass();
558         if (ctx.shouldComplete()) {
559           break;
560         }
561       }
562     }
563     return bypass;
564   }
565 
566   /**
567    * @param get the Get request
568    * @param results the result set
569    * @exception IOException Exception
570    */
571   public void postGet(final Get get, final List<KeyValue> results)
572   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()).postGet(ctx, get, results);
579         } catch (Throwable e) {
580           handleCoprocessorThrowable(env, e);
581         }
582         if (ctx.shouldComplete()) {
583           break;
584         }
585       }
586     }
587   }
588 
589   /**
590    * @param get the Get request
591    * @return true or false to return to client if bypassing normal operation,
592    * or null otherwise
593    * @exception IOException Exception
594    */
595   public Boolean preExists(final Get get) throws IOException {
596     boolean bypass = false;
597     boolean exists = false;
598     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
599     for (RegionEnvironment env: coprocessors) {
600       if (env.getInstance() instanceof RegionObserver) {
601         ctx = ObserverContext.createAndPrepare(env, ctx);
602         try {
603           exists = ((RegionObserver)env.getInstance()).preExists(ctx, get, exists);
604         } catch (Throwable e) {
605           handleCoprocessorThrowable(env, e);
606         }
607         bypass |= ctx.shouldBypass();
608         if (ctx.shouldComplete()) {
609           break;
610         }
611       }
612     }
613     return bypass ? exists : null;
614   }
615 
616   /**
617    * @param get the Get request
618    * @param exists the result returned by the region server
619    * @return the result to return to the client
620    * @exception IOException Exception
621    */
622   public boolean postExists(final Get get, boolean exists)
623       throws IOException {
624     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
625     for (RegionEnvironment env: coprocessors) {
626       if (env.getInstance() instanceof RegionObserver) {
627         ctx = ObserverContext.createAndPrepare(env, ctx);
628         try {
629           exists = ((RegionObserver)env.getInstance()).postExists(ctx, get, exists);
630         } catch (Throwable e) {
631           handleCoprocessorThrowable(env, e);
632         }
633         if (ctx.shouldComplete()) {
634           break;
635         }
636       }
637     }
638     return exists;
639   }
640 
641   /**
642    * @param put The Put object
643    * @param edit The WALEdit object.
644    * @param writeToWAL true if the change should be written to the WAL
645    * @return true if default processing should be bypassed
646    * @exception IOException Exception
647    */
648   public boolean prePut(Put put, WALEdit edit,
649       final boolean writeToWAL) throws IOException {
650     boolean bypass = false;
651     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
652     for (RegionEnvironment env: coprocessors) {
653       if (env.getInstance() instanceof RegionObserver) {
654         ctx = ObserverContext.createAndPrepare(env, ctx);
655         try {
656           ((RegionObserver)env.getInstance()).prePut(ctx, put, edit, writeToWAL);
657         } catch (Throwable e) {
658           handleCoprocessorThrowable(env, e);
659         }
660         bypass |= ctx.shouldBypass();
661         if (ctx.shouldComplete()) {
662           break;
663         }
664       }
665     }
666     return bypass;
667   }
668 
669   /**
670    * @param put The Put object
671    * @param edit The WALEdit object.
672    * @param writeToWAL true if the change should be written to the WAL
673    * @exception IOException Exception
674    */
675   public void postPut(Put put, WALEdit edit,
676       final boolean writeToWAL) throws IOException {
677     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
678     for (RegionEnvironment env: coprocessors) {
679       if (env.getInstance() instanceof RegionObserver) {
680         ctx = ObserverContext.createAndPrepare(env, ctx);
681         try {
682           ((RegionObserver)env.getInstance()).postPut(ctx, put, edit, writeToWAL);
683         } catch (Throwable e) {
684           handleCoprocessorThrowable(env, e);
685         }
686         if (ctx.shouldComplete()) {
687           break;
688         }
689       }
690     }
691   }
692 
693   /**
694    * @param delete The Delete object
695    * @param edit The WALEdit object.
696    * @param writeToWAL true if the change should be written to the WAL
697    * @return true if default processing should be bypassed
698    * @exception IOException Exception
699    */
700   public boolean preDelete(Delete delete, WALEdit edit,
701       final boolean writeToWAL) throws IOException {
702     boolean bypass = false;
703     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
704     for (RegionEnvironment env: coprocessors) {
705       if (env.getInstance() instanceof RegionObserver) {
706         ctx = ObserverContext.createAndPrepare(env, ctx);
707         try {
708           ((RegionObserver)env.getInstance()).preDelete(ctx, delete, edit, writeToWAL);
709         } catch (Throwable e) {
710           handleCoprocessorThrowable(env, e);
711         }
712         bypass |= ctx.shouldBypass();
713         if (ctx.shouldComplete()) {
714           break;
715         }
716       }
717     }
718     return bypass;
719   }
720 
721   /**
722    * @param delete The Delete object
723    * @param edit The WALEdit object.
724    * @param writeToWAL true if the change should be written to the WAL
725    * @exception IOException Exception
726    */
727   public void postDelete(Delete delete, WALEdit edit,
728       final boolean writeToWAL) throws IOException {
729     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
730     for (RegionEnvironment env: coprocessors) {
731       if (env.getInstance() instanceof RegionObserver) {
732         ctx = ObserverContext.createAndPrepare(env, ctx);
733         try {
734           ((RegionObserver)env.getInstance()).postDelete(ctx, delete, edit, writeToWAL);
735         } catch (Throwable e) {
736           handleCoprocessorThrowable(env, e);
737         }
738         if (ctx.shouldComplete()) {
739           break;
740         }
741       }
742     }
743   }
744 
745   /**
746    * @param row row to check
747    * @param family column family
748    * @param qualifier column qualifier
749    * @param compareOp the comparison operation
750    * @param comparator the comparator
751    * @param put data to put if check succeeds
752    * @return true or false to return to client if default processing should
753    * be bypassed, or null otherwise
754    * @throws IOException e
755    */
756   public Boolean preCheckAndPut(final byte [] row, final byte [] family,
757       final byte [] qualifier, final CompareOp compareOp,
758       final WritableByteArrayComparable comparator, Put put)
759     throws IOException {
760     boolean bypass = false;
761     boolean result = false;
762     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
763     for (RegionEnvironment env: coprocessors) {
764       if (env.getInstance() instanceof RegionObserver) {
765         ctx = ObserverContext.createAndPrepare(env, ctx);
766         try {
767           result = ((RegionObserver)env.getInstance()).preCheckAndPut(ctx, row, family,
768             qualifier, compareOp, comparator, put, result);
769         } catch (Throwable e) {
770           handleCoprocessorThrowable(env, e);
771         }
772 
773 
774         bypass |= ctx.shouldBypass();
775         if (ctx.shouldComplete()) {
776           break;
777         }
778       }
779     }
780     return bypass ? result : null;
781   }
782 
783   /**
784    * @param row row to check
785    * @param family column family
786    * @param qualifier column qualifier
787    * @param compareOp the comparison operation
788    * @param comparator the comparator
789    * @param put data to put if check succeeds
790    * @throws IOException e
791    */
792   public boolean postCheckAndPut(final byte [] row, final byte [] family,
793       final byte [] qualifier, final CompareOp compareOp,
794       final WritableByteArrayComparable comparator, final Put put,
795       boolean result)
796     throws IOException {
797     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
798     for (RegionEnvironment env: coprocessors) {
799       if (env.getInstance() instanceof RegionObserver) {
800         ctx = ObserverContext.createAndPrepare(env, ctx);
801         try {
802           result = ((RegionObserver)env.getInstance()).postCheckAndPut(ctx, row,
803             family, qualifier, compareOp, comparator, put, result);
804         } catch (Throwable e) {
805           handleCoprocessorThrowable(env, e);
806         }
807         if (ctx.shouldComplete()) {
808           break;
809         }
810       }
811     }
812     return result;
813   }
814 
815   /**
816    * @param row row to check
817    * @param family column family
818    * @param qualifier column qualifier
819    * @param compareOp the comparison operation
820    * @param comparator the comparator
821    * @param delete delete to commit if check succeeds
822    * @return true or false to return to client if default processing should
823    * be bypassed, or null otherwise
824    * @throws IOException e
825    */
826   public Boolean preCheckAndDelete(final byte [] row, final byte [] family,
827       final byte [] qualifier, final CompareOp compareOp,
828       final WritableByteArrayComparable comparator, Delete delete)
829       throws IOException {
830     boolean bypass = false;
831     boolean result = false;
832     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
833     for (RegionEnvironment env: coprocessors) {
834       if (env.getInstance() instanceof RegionObserver) {
835         ctx = ObserverContext.createAndPrepare(env, ctx);
836         try {
837           result = ((RegionObserver)env.getInstance()).preCheckAndDelete(ctx, row,
838             family, qualifier, compareOp, comparator, delete, result);
839         } catch (Throwable e) {
840           handleCoprocessorThrowable(env, e);
841         }
842         bypass |= ctx.shouldBypass();
843         if (ctx.shouldComplete()) {
844           break;
845         }
846       }
847     }
848     return bypass ? result : null;
849   }
850 
851   /**
852    * @param row row to check
853    * @param family column family
854    * @param qualifier column qualifier
855    * @param compareOp the comparison operation
856    * @param comparator the comparator
857    * @param delete delete to commit if check succeeds
858    * @throws IOException e
859    */
860   public boolean postCheckAndDelete(final byte [] row, final byte [] family,
861       final byte [] qualifier, final CompareOp compareOp,
862       final WritableByteArrayComparable comparator, final Delete delete,
863       boolean result)
864     throws IOException {
865     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
866     for (RegionEnvironment env: coprocessors) {
867       if (env.getInstance() instanceof RegionObserver) {
868         ctx = ObserverContext.createAndPrepare(env, ctx);
869         try {
870           result = ((RegionObserver)env.getInstance())
871             .postCheckAndDelete(ctx, row, family, qualifier, compareOp,
872               comparator, delete, result);
873         } catch (Throwable e) {
874           handleCoprocessorThrowable(env, e);
875         }
876         if (ctx.shouldComplete()) {
877           break;
878         }
879       }
880     }
881     return result;
882   }
883 
884   /**
885    * @param row row to check
886    * @param family column family
887    * @param qualifier column qualifier
888    * @param amount long amount to increment
889    * @param writeToWAL true if the change should be written to the WAL
890    * @return return value for client if default operation should be bypassed,
891    * or null otherwise
892    * @throws IOException if an error occurred on the coprocessor
893    */
894   public Long preIncrementColumnValue(final byte [] row, final byte [] family,
895       final byte [] qualifier, long amount, final boolean writeToWAL)
896       throws IOException {
897     boolean bypass = false;
898     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
899     for (RegionEnvironment env: coprocessors) {
900       if (env.getInstance() instanceof RegionObserver) {
901         ctx = ObserverContext.createAndPrepare(env, ctx);
902         try {
903           amount = ((RegionObserver)env.getInstance()).preIncrementColumnValue(ctx,
904               row, family, qualifier, amount, writeToWAL);
905         } catch (Throwable e) {
906           handleCoprocessorThrowable(env, e);
907         }
908         bypass |= ctx.shouldBypass();
909         if (ctx.shouldComplete()) {
910           break;
911         }
912       }
913     }
914     return bypass ? amount : null;
915   }
916 
917   /**
918    * @param row row to check
919    * @param family column family
920    * @param qualifier column qualifier
921    * @param amount long amount to increment
922    * @param writeToWAL true if the change should be written to the WAL
923    * @param result the result returned by incrementColumnValue
924    * @return the result to return to the client
925    * @throws IOException if an error occurred on the coprocessor
926    */
927   public long postIncrementColumnValue(final byte [] row, final byte [] family,
928       final byte [] qualifier, final long amount, final boolean writeToWAL,
929       long result) throws IOException {
930     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
931     for (RegionEnvironment env: coprocessors) {
932       if (env.getInstance() instanceof RegionObserver) {
933         ctx = ObserverContext.createAndPrepare(env, ctx);
934         try {
935           result = ((RegionObserver)env.getInstance()).postIncrementColumnValue(ctx,
936               row, family, qualifier, amount, writeToWAL, result);
937         } catch (Throwable e) {
938           handleCoprocessorThrowable(env, e);
939         }
940         if (ctx.shouldComplete()) {
941           break;
942         }
943       }
944     }
945     return result;
946   }
947 
948   /**
949    * @param increment increment object
950    * @return result to return to client if default operation should be
951    * bypassed, null otherwise
952    * @throws IOException if an error occurred on the coprocessor
953    */
954   public Result preIncrement(Increment increment)
955       throws IOException {
956     boolean bypass = false;
957     Result result = new Result();
958     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
959     for (RegionEnvironment env: coprocessors) {
960       if (env.getInstance() instanceof RegionObserver) {
961         ctx = ObserverContext.createAndPrepare(env, ctx);
962         try {
963           ((RegionObserver)env.getInstance()).preIncrement(ctx, increment, result);
964         } catch (Throwable e) {
965           handleCoprocessorThrowable(env, e);
966         }
967         bypass |= ctx.shouldBypass();
968         if (ctx.shouldComplete()) {
969           break;
970         }
971       }
972     }
973     return bypass ? result : null;
974   }
975 
976   /**
977    * @param increment increment object
978    * @param result the result returned by postIncrement
979    * @throws IOException if an error occurred on the coprocessor
980    */
981   public void postIncrement(final Increment increment, Result result)
982       throws IOException {
983     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
984     for (RegionEnvironment env: coprocessors) {
985       if (env.getInstance() instanceof RegionObserver) {
986         ctx = ObserverContext.createAndPrepare(env, ctx);
987         try {
988           ((RegionObserver)env.getInstance()).postIncrement(ctx, increment, result);
989         } catch (Throwable e) {
990           handleCoprocessorThrowable(env, e);
991         }
992         if (ctx.shouldComplete()) {
993           break;
994         }
995       }
996     }
997   }
998 
999   /**
1000    * @param scan the Scan specification
1001    * @return scanner id to return to client if default operation should be
1002    * bypassed, false otherwise
1003    * @exception IOException Exception
1004    */
1005   public RegionScanner preScannerOpen(Scan scan) throws IOException {
1006     boolean bypass = false;
1007     RegionScanner s = null;
1008     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
1009     for (RegionEnvironment env: coprocessors) {
1010       if (env.getInstance() instanceof RegionObserver) {
1011         ctx = ObserverContext.createAndPrepare(env, ctx);
1012         try {
1013           s = ((RegionObserver)env.getInstance()).preScannerOpen(ctx, scan, s);
1014         } catch (Throwable e) {
1015           handleCoprocessorThrowable(env, e);
1016         }
1017         bypass |= ctx.shouldBypass();
1018         if (ctx.shouldComplete()) {
1019           break;
1020         }
1021       }
1022     }
1023     return bypass ? s : null;
1024   }
1025 
1026   /**
1027    * @param scan the Scan specification
1028    * @param s the scanner
1029    * @return the scanner instance to use
1030    * @exception IOException Exception
1031    */
1032   public RegionScanner postScannerOpen(final Scan scan, RegionScanner s)
1033       throws IOException {
1034     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
1035     for (RegionEnvironment env: coprocessors) {
1036       if (env.getInstance() instanceof RegionObserver) {
1037         ctx = ObserverContext.createAndPrepare(env, ctx);
1038         try {
1039           s = ((RegionObserver)env.getInstance()).postScannerOpen(ctx, scan, s);
1040         } catch (Throwable e) {
1041           handleCoprocessorThrowable(env, e);
1042         }
1043         if (ctx.shouldComplete()) {
1044           break;
1045         }
1046       }
1047     }
1048     return s;
1049   }
1050 
1051   /**
1052    * @param s the scanner
1053    * @param results the result set returned by the region server
1054    * @param limit the maximum number of results to return
1055    * @return 'has next' indication to client if bypassing default behavior, or
1056    * null otherwise
1057    * @exception IOException Exception
1058    */
1059   public Boolean preScannerNext(final InternalScanner s,
1060       final List<Result> results, int limit) throws IOException {
1061     boolean bypass = false;
1062     boolean hasNext = false;
1063     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
1064     for (RegionEnvironment env: coprocessors) {
1065       if (env.getInstance() instanceof RegionObserver) {
1066         ctx = ObserverContext.createAndPrepare(env, ctx);
1067         try {
1068           hasNext = ((RegionObserver)env.getInstance()).preScannerNext(ctx, s, results,
1069             limit, hasNext);
1070         } catch (Throwable e) {
1071           handleCoprocessorThrowable(env, e);
1072         }
1073         bypass |= ctx.shouldBypass();
1074         if (ctx.shouldComplete()) {
1075           break;
1076         }
1077       }
1078     }
1079     return bypass ? hasNext : null;
1080   }
1081 
1082   /**
1083    * @param s the scanner
1084    * @param results the result set returned by the region server
1085    * @param limit the maximum number of results to return
1086    * @param hasMore
1087    * @return 'has more' indication to give to client
1088    * @exception IOException Exception
1089    */
1090   public boolean postScannerNext(final InternalScanner s,
1091       final List<Result> results, final int limit, boolean hasMore)
1092       throws IOException {
1093     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
1094     for (RegionEnvironment env: coprocessors) {
1095       if (env.getInstance() instanceof RegionObserver) {
1096         ctx = ObserverContext.createAndPrepare(env, ctx);
1097         try {
1098           hasMore = ((RegionObserver)env.getInstance()).postScannerNext(ctx, s,
1099             results, limit, hasMore);
1100         } catch (Throwable e) {
1101           handleCoprocessorThrowable(env, e);
1102         }
1103         if (ctx.shouldComplete()) {
1104           break;
1105         }
1106       }
1107     }
1108     return hasMore;
1109   }
1110 
1111   /**
1112    * @param s the scanner
1113    * @return true if default behavior should be bypassed, false otherwise
1114    * @exception IOException Exception
1115    */
1116   public boolean preScannerClose(final InternalScanner s)
1117       throws IOException {
1118     boolean bypass = false;
1119     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
1120     for (RegionEnvironment env: coprocessors) {
1121       if (env.getInstance() instanceof RegionObserver) {
1122         ctx = ObserverContext.createAndPrepare(env, ctx);
1123         try {
1124           ((RegionObserver)env.getInstance()).preScannerClose(ctx, s);
1125         } catch (Throwable e) {
1126           handleCoprocessorThrowable(env, e);
1127         }
1128         bypass |= ctx.shouldBypass();
1129         if (ctx.shouldComplete()) {
1130           break;
1131         }
1132       }
1133     }
1134     return bypass;
1135   }
1136 
1137   /**
1138    * @param s the scanner
1139    * @exception IOException Exception
1140    */
1141   public void postScannerClose(final InternalScanner s)
1142       throws IOException {
1143     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
1144     for (RegionEnvironment env: coprocessors) {
1145       if (env.getInstance() instanceof RegionObserver) {
1146         ctx = ObserverContext.createAndPrepare(env, ctx);
1147         try {
1148           ((RegionObserver)env.getInstance()).postScannerClose(ctx, s);
1149         } catch (Throwable e) {
1150           handleCoprocessorThrowable(env, e);
1151         }
1152         if (ctx.shouldComplete()) {
1153           break;
1154         }
1155       }
1156     }
1157   }
1158 
1159   /**
1160    * @param info
1161    * @param logKey
1162    * @param logEdit
1163    * @return true if default behavior should be bypassed, false otherwise
1164    * @throws IOException
1165    */
1166   public boolean preWALRestore(HRegionInfo info, HLogKey logKey,
1167       WALEdit logEdit) throws IOException {
1168     boolean bypass = false;
1169     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
1170     for (RegionEnvironment env: coprocessors) {
1171       if (env.getInstance() instanceof RegionObserver) {
1172         ctx = ObserverContext.createAndPrepare(env, ctx);
1173         try {
1174           ((RegionObserver)env.getInstance()).preWALRestore(ctx, info, logKey,
1175               logEdit);
1176         } catch (Throwable e) {
1177           handleCoprocessorThrowable(env, e);
1178         }
1179         bypass |= ctx.shouldBypass();
1180         if (ctx.shouldComplete()) {
1181           break;
1182         }
1183       }
1184     }
1185     return bypass;
1186   }
1187 
1188   /**
1189    * @param info
1190    * @param logKey
1191    * @param logEdit
1192    * @throws IOException
1193    */
1194   public void postWALRestore(HRegionInfo info, HLogKey logKey,
1195       WALEdit logEdit) throws IOException {
1196     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
1197     for (RegionEnvironment env: coprocessors) {
1198       if (env.getInstance() instanceof RegionObserver) {
1199         ctx = ObserverContext.createAndPrepare(env, ctx);
1200         try {
1201           ((RegionObserver)env.getInstance()).postWALRestore(ctx, info,
1202               logKey, logEdit);
1203         } catch (Throwable e) {
1204           handleCoprocessorThrowable(env, e);
1205         }
1206         if (ctx.shouldComplete()) {
1207           break;
1208         }
1209       }
1210     }
1211   }
1212 }