View Javadoc

1   /*
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *   http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing,
13   * software distributed under the License is distributed on an
14   * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15   * KIND, either express or implied.  See the License for the
16   * specific language governing permissions and limitations
17   * under the License.
18   */
19  
20  package org.apache.hadoop.hbase.coprocessor;
21  
22  import java.io.IOException;
23  import java.util.List;
24  import java.util.NavigableSet;
25  
26  import org.apache.hadoop.hbase.classification.InterfaceAudience;
27  import org.apache.hadoop.hbase.classification.InterfaceStability;
28  import org.apache.hadoop.fs.FileSystem;
29  import org.apache.hadoop.fs.Path;
30  import org.apache.hadoop.hbase.Cell;
31  import org.apache.hadoop.hbase.Coprocessor;
32  import org.apache.hadoop.hbase.HBaseInterfaceAudience;
33  import org.apache.hadoop.hbase.HRegionInfo;
34  import org.apache.hadoop.hbase.client.Append;
35  import org.apache.hadoop.hbase.client.Delete;
36  import org.apache.hadoop.hbase.client.Durability;
37  import org.apache.hadoop.hbase.client.Get;
38  import org.apache.hadoop.hbase.client.Increment;
39  import org.apache.hadoop.hbase.client.Mutation;
40  import org.apache.hadoop.hbase.client.Put;
41  import org.apache.hadoop.hbase.client.Result;
42  import org.apache.hadoop.hbase.client.Scan;
43  import org.apache.hadoop.hbase.filter.ByteArrayComparable;
44  import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
45  import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
46  import org.apache.hadoop.hbase.io.Reference;
47  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
48  import org.apache.hadoop.hbase.regionserver.DeleteTracker;
49  import org.apache.hadoop.hbase.regionserver.Region;
50  import org.apache.hadoop.hbase.regionserver.Region.Operation;
51  import org.apache.hadoop.hbase.regionserver.InternalScanner;
52  import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
53  import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
54  import org.apache.hadoop.hbase.regionserver.RegionScanner;
55  import org.apache.hadoop.hbase.regionserver.ScanType;
56  import org.apache.hadoop.hbase.regionserver.Store;
57  import org.apache.hadoop.hbase.regionserver.StoreFile;
58  import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
59  import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
60  import org.apache.hadoop.hbase.wal.WALKey;
61  import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
62  import org.apache.hadoop.hbase.util.Pair;
63  
64  import com.google.common.collect.ImmutableList;
65  
66  /**
67   * Coprocessors implement this interface to observe and mediate client actions
68   * on the region.
69   */
70  @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
71  @InterfaceStability.Evolving
72  // TODO as method signatures need to break, update to
73  // ObserverContext<? extends RegionCoprocessorEnvironment>
74  // so we can use additional environment state that isn't exposed to coprocessors.
75  public interface RegionObserver extends Coprocessor {
76  
77    /** Mutation type for postMutationBeforeWAL hook */
78    public enum MutationType {
79      APPEND, INCREMENT
80    }
81  
82    /**
83     * Called before the region is reported as open to the master.
84     * @param c the environment provided by the region server
85     * @throws IOException if an error occurred on the coprocessor
86     */
87    void preOpen(final ObserverContext<RegionCoprocessorEnvironment> c) throws IOException;
88  
89    /**
90     * Called after the region is reported as open to the master.
91     * @param c the environment provided by the region server
92     */
93    void postOpen(final ObserverContext<RegionCoprocessorEnvironment> c);
94  
95    /**
96     * Called after the log replay on the region is over.
97     * @param c the environment provided by the region server
98     */
99    void postLogReplay(final ObserverContext<RegionCoprocessorEnvironment> c);
100 
101   /**
102    * Called before a memstore is flushed to disk and prior to creating the scanner to read from
103    * the memstore.  To override or modify how a memstore is flushed,
104    * implementing classes can return a new scanner to provide the KeyValues to be
105    * stored into the new {@code StoreFile} or null to perform the default processing.
106    * Calling {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} has no
107    * effect in this hook.
108    * @param c the environment provided by the region server
109    * @param store the store being flushed
110    * @param memstoreScanner the scanner for the memstore that is flushed
111    * @param s the base scanner, if not {@code null}, from previous RegionObserver in the chain
112    * @return the scanner to use during the flush.  {@code null} if the default implementation
113    * is to be used.
114    * @throws IOException if an error occurred on the coprocessor
115    */
116   InternalScanner preFlushScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
117       final Store store, final KeyValueScanner memstoreScanner, final InternalScanner s)
118       throws IOException;
119 
120   /**
121    * Called before the memstore is flushed to disk.
122    * @param c the environment provided by the region server
123    * @throws IOException if an error occurred on the coprocessor
124    * @deprecated use {@link #preFlush(ObserverContext, Store, InternalScanner)} instead
125    */
126   void preFlush(final ObserverContext<RegionCoprocessorEnvironment> c) throws IOException;
127 
128   /**
129    * Called before a Store's memstore is flushed to disk.
130    * @param c the environment provided by the region server
131    * @param store the store where compaction is being requested
132    * @param scanner the scanner over existing data used in the store file
133    * @return the scanner to use during compaction.  Should not be {@code null}
134    * unless the implementation is writing new store files on its own.
135    * @throws IOException if an error occurred on the coprocessor
136    */
137   InternalScanner preFlush(final ObserverContext<RegionCoprocessorEnvironment> c, final Store store,
138       final InternalScanner scanner) throws IOException;
139 
140   /**
141    * Called after the memstore is flushed to disk.
142    * @param c the environment provided by the region server
143    * @throws IOException if an error occurred on the coprocessor
144    * @deprecated use {@link #preFlush(ObserverContext, Store, InternalScanner)} instead.
145    */
146   void postFlush(final ObserverContext<RegionCoprocessorEnvironment> c) throws IOException;
147 
148   /**
149    * Called after a Store's memstore is flushed to disk.
150    * @param c the environment provided by the region server
151    * @param store the store being flushed
152    * @param resultFile the new store file written out during compaction
153    * @throws IOException if an error occurred on the coprocessor
154    */
155   void postFlush(final ObserverContext<RegionCoprocessorEnvironment> c, final Store store,
156       final StoreFile resultFile) throws IOException;
157 
158   /**
159    * Called prior to selecting the {@link StoreFile StoreFiles} to compact from the list of
160    * available candidates. To alter the files used for compaction, you may mutate the passed in list
161    * of candidates.
162    * @param c the environment provided by the region server
163    * @param store the store where compaction is being requested
164    * @param candidates the store files currently available for compaction
165    * @param request custom compaction request
166    * @throws IOException if an error occurred on the coprocessor
167    */
168   void preCompactSelection(final ObserverContext<RegionCoprocessorEnvironment> c,
169       final Store store, final List<StoreFile> candidates, final CompactionRequest request)
170       throws IOException;
171 
172   /**
173    * Called prior to selecting the {@link StoreFile}s to compact from the list of available
174    * candidates. To alter the files used for compaction, you may mutate the passed in list of
175    * candidates.
176    * @param c the environment provided by the region server
177    * @param store the store where compaction is being requested
178    * @param candidates the store files currently available for compaction
179    * @throws IOException if an error occurred on the coprocessor
180    * @deprecated Use {@link #preCompactSelection(ObserverContext, Store, List, CompactionRequest)}
181    *             instead
182    */
183   @Deprecated
184   void preCompactSelection(final ObserverContext<RegionCoprocessorEnvironment> c,
185       final Store store, final List<StoreFile> candidates) throws IOException;
186 
187   /**
188    * Called after the {@link StoreFile}s to compact have been selected from the available
189    * candidates.
190    * @param c the environment provided by the region server
191    * @param store the store being compacted
192    * @param selected the store files selected to compact
193    * @param request custom compaction request
194    */
195   void postCompactSelection(final ObserverContext<RegionCoprocessorEnvironment> c,
196       final Store store, final ImmutableList<StoreFile> selected, CompactionRequest request);
197 
198   /**
199    * Called after the {@link StoreFile}s to compact have been selected from the available
200    * candidates.
201    * @param c the environment provided by the region server
202    * @param store the store being compacted
203    * @param selected the store files selected to compact
204    * @deprecated use {@link #postCompactSelection(ObserverContext, Store, ImmutableList,
205    *             CompactionRequest)} instead.
206    */
207   @Deprecated
208   void postCompactSelection(final ObserverContext<RegionCoprocessorEnvironment> c,
209       final Store store, final ImmutableList<StoreFile> selected);
210 
211   /**
212    * Called prior to writing the {@link StoreFile}s selected for compaction into a new
213    * {@code StoreFile}. To override or modify the compaction process, implementing classes have two
214    * options:
215    * <ul>
216    * <li>Wrap the provided {@link InternalScanner} with a custom implementation that is returned
217    * from this method. The custom scanner can then inspect 
218    * {@link org.apache.hadoop.hbase.KeyValue}s from the wrapped
219    * scanner, applying its own policy to what gets written.</li>
220    * <li>Call {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} and provide a
221    * custom implementation for writing of new {@link StoreFile}s. <strong>Note: any implementations
222    * bypassing core compaction using this approach must write out new store files themselves or the
223    * existing data will no longer be available after compaction.</strong></li>
224    * </ul>
225    * @param c the environment provided by the region server
226    * @param store the store being compacted
227    * @param scanner the scanner over existing data used in the store file rewriting
228    * @param scanType type of Scan
229    * @param request the requested compaction
230    * @return the scanner to use during compaction. Should not be {@code null} unless the
231    *         implementation is writing new store files on its own.
232    * @throws IOException if an error occurred on the coprocessor
233    */
234   InternalScanner preCompact(final ObserverContext<RegionCoprocessorEnvironment> c,
235       final Store store, final InternalScanner scanner, final ScanType scanType,
236       CompactionRequest request) throws IOException;
237 
238   /**
239    * Called prior to writing the {@link StoreFile}s selected for compaction into a new
240    * {@code StoreFile}. To override or modify the compaction process, implementing classes have two
241    * options:
242    * <ul>
243    * <li>Wrap the provided {@link InternalScanner} with a custom implementation that is returned
244    * from this method. The custom scanner can then inspect 
245    * {@link org.apache.hadoop.hbase.KeyValue}s from the wrapped
246    * scanner, applying its own policy to what gets written.</li>
247    * <li>Call {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} and provide a
248    * custom implementation for writing of new {@link StoreFile}s. <strong>Note: any implementations
249    * bypassing core compaction using this approach must write out new store files themselves or the
250    * existing data will no longer be available after compaction.</strong></li>
251    * </ul>
252    * @param c the environment provided by the region server
253    * @param store the store being compacted
254    * @param scanner the scanner over existing data used in the store file rewriting
255    * @param scanType type of Scan
256    * @return the scanner to use during compaction. Should not be {@code null} unless the
257    *         implementation is writing new store files on its own.
258    * @throws IOException if an error occurred on the coprocessor
259    * @deprecated use
260    *             {@link #preCompact(ObserverContext, Store, InternalScanner,
261    *             ScanType, CompactionRequest)} instead
262    */
263   @Deprecated
264   InternalScanner preCompact(final ObserverContext<RegionCoprocessorEnvironment> c,
265       final Store store, final InternalScanner scanner, final ScanType scanType) throws IOException;
266 
267   /**
268    * Called prior to writing the {@link StoreFile}s selected for compaction into a new
269    * {@code StoreFile} and prior to creating the scanner used to read the input files. To override
270    * or modify the compaction process, implementing classes can return a new scanner to provide the
271    * KeyValues to be stored into the new {@code StoreFile} or null to perform the default
272    * processing. Calling {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} has no
273    * effect in this hook.
274    * @param c the environment provided by the region server
275    * @param store the store being compacted
276    * @param scanners the list {@link org.apache.hadoop.hbase.regionserver.StoreFileScanner}s 
277    *        to be read from
278    * @param scanType the {@link ScanType} indicating whether this is a major or minor compaction
279    * @param earliestPutTs timestamp of the earliest put that was found in any of the involved store
280    *          files
281    * @param s the base scanner, if not {@code null}, from previous RegionObserver in the chain
282    * @param request the requested compaction
283    * @return the scanner to use during compaction. {@code null} if the default implementation is to
284    *         be used.
285    * @throws IOException if an error occurred on the coprocessor
286    */
287   InternalScanner preCompactScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
288       final Store store, List<? extends KeyValueScanner> scanners, final ScanType scanType,
289       final long earliestPutTs, final InternalScanner s, CompactionRequest request)
290       throws IOException;
291 
292   /**
293    * Called prior to writing the {@link StoreFile}s selected for compaction into a new
294    * {@code StoreFile} and prior to creating the scanner used to read the input files. To override
295    * or modify the compaction process, implementing classes can return a new scanner to provide the
296    * KeyValues to be stored into the new {@code StoreFile} or null to perform the default
297    * processing. Calling {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} has no
298    * effect in this hook.
299    * @param c the environment provided by the region server
300    * @param store the store being compacted
301    * @param scanners the list {@link org.apache.hadoop.hbase.regionserver.StoreFileScanner}s 
302    *        to be read from
303    * @param scanType the {@link ScanType} indicating whether this is a major or minor compaction
304    * @param earliestPutTs timestamp of the earliest put that was found in any of the involved store
305    *          files
306    * @param s the base scanner, if not {@code null}, from previous RegionObserver in the chain
307    * @return the scanner to use during compaction. {@code null} if the default implementation is to
308    *         be used.
309    * @throws IOException if an error occurred on the coprocessor
310    * @deprecated Use
311    *             {@link #preCompactScannerOpen(ObserverContext, Store, List, ScanType, long,
312    *             InternalScanner, CompactionRequest)} instead.
313    */
314   @Deprecated
315   InternalScanner preCompactScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
316       final Store store, List<? extends KeyValueScanner> scanners, final ScanType scanType,
317       final long earliestPutTs, final InternalScanner s) throws IOException;
318 
319   /**
320    * Called after compaction has completed and the new store file has been moved in to place.
321    * @param c the environment provided by the region server
322    * @param store the store being compacted
323    * @param resultFile the new store file written out during compaction
324    * @param request the requested compaction
325    * @throws IOException if an error occurred on the coprocessor
326    */
327   void postCompact(final ObserverContext<RegionCoprocessorEnvironment> c, final Store store,
328       StoreFile resultFile, CompactionRequest request) throws IOException;
329 
330   /**
331    * Called after compaction has completed and the new store file has been moved in to place.
332    * @param c the environment provided by the region server
333    * @param store the store being compacted
334    * @param resultFile the new store file written out during compaction
335    * @throws IOException if an error occurred on the coprocessor
336    * @deprecated Use {@link #postCompact(ObserverContext, Store, StoreFile, CompactionRequest)}
337    *             instead
338    */
339   @Deprecated
340   void postCompact(final ObserverContext<RegionCoprocessorEnvironment> c, final Store store,
341       StoreFile resultFile) throws IOException;
342 
343   /**
344    * Called before the region is split.
345    * @param c the environment provided by the region server
346    * (e.getRegion() returns the parent region)
347    * @throws IOException if an error occurred on the coprocessor
348    * @deprecated Use preSplit(
349    *    final ObserverContext&lt;RegionCoprocessorEnvironment&gt; c, byte[] splitRow)
350    */
351   void preSplit(final ObserverContext<RegionCoprocessorEnvironment> c) throws IOException;
352 
353   /**
354    * Called before the region is split.
355    * @param c the environment provided by the region server
356    * (e.getRegion() returns the parent region)
357    * @throws IOException if an error occurred on the coprocessor
358    */
359   void preSplit(final ObserverContext<RegionCoprocessorEnvironment> c, byte[] splitRow)
360       throws IOException;
361 
362   /**
363    * Called after the region is split.
364    * @param c the environment provided by the region server
365    * (e.getRegion() returns the parent region)
366    * @param l the left daughter region
367    * @param r the right daughter region
368    * @throws IOException if an error occurred on the coprocessor
369    * @deprecated Use postCompleteSplit() instead
370    */
371   void postSplit(final ObserverContext<RegionCoprocessorEnvironment> c, final Region l,
372       final Region r) throws IOException;
373 
374   /**
375    * This will be called before PONR step as part of split transaction. Calling
376    * {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} rollback the split
377    * @param ctx
378    * @param splitKey
379    * @param metaEntries
380    * @throws IOException
381    */
382   void preSplitBeforePONR(final ObserverContext<RegionCoprocessorEnvironment> ctx,
383       byte[] splitKey, List<Mutation> metaEntries) throws IOException;
384 
385   
386   /**
387    * This will be called after PONR step as part of split transaction
388    * Calling {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} has no
389    * effect in this hook.
390    * @param ctx
391    * @throws IOException
392    */
393   void preSplitAfterPONR(final ObserverContext<RegionCoprocessorEnvironment> ctx) throws IOException;
394   
395   /**
396    * This will be called before the roll back of the split region is completed 
397    * @param ctx
398    * @throws IOException
399    */
400   void preRollBackSplit(final ObserverContext<RegionCoprocessorEnvironment> ctx) throws IOException;
401 
402   /**
403    * This will be called after the roll back of the split region is completed
404    * @param ctx
405    * @throws IOException
406    */
407   void postRollBackSplit(final ObserverContext<RegionCoprocessorEnvironment> ctx)
408     throws IOException;
409 
410   /**
411    * Called after any split request is processed.  This will be called irrespective of success or
412    * failure of the split.
413    * @param ctx
414    * @throws IOException
415    */
416   void postCompleteSplit(final ObserverContext<RegionCoprocessorEnvironment> ctx)
417     throws IOException;
418   /**
419    * Called before the region is reported as closed to the master.
420    * @param c the environment provided by the region server
421    * @param abortRequested true if the region server is aborting
422    * @throws IOException 
423    */
424   void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
425       boolean abortRequested) throws IOException;
426 
427   /**
428    * Called after the region is reported as closed to the master.
429    * @param c the environment provided by the region server
430    * @param abortRequested true if the region server is aborting
431    */
432   void postClose(final ObserverContext<RegionCoprocessorEnvironment> c,
433       boolean abortRequested);
434 
435   /**
436    * Called before a client makes a GetClosestRowBefore request.
437    * <p>
438    * Call CoprocessorEnvironment#bypass to skip default actions
439    * <p>
440    * Call CoprocessorEnvironment#complete to skip any subsequent chained
441    * coprocessors
442    * @param c the environment provided by the region server
443    * @param row the row
444    * @param family the family
445    * @param result The result to return to the client if default processing
446    * is bypassed. Can be modified. Will not be used if default processing
447    * is not bypassed.
448    * @throws IOException if an error occurred on the coprocessor
449    */
450   void preGetClosestRowBefore(final ObserverContext<RegionCoprocessorEnvironment> c,
451       final byte [] row, final byte [] family, final Result result)
452     throws IOException;
453 
454   /**
455    * Called after a client makes a GetClosestRowBefore request.
456    * <p>
457    * Call CoprocessorEnvironment#complete to skip any subsequent chained
458    * coprocessors
459    * @param c the environment provided by the region server
460    * @param row the row
461    * @param family the desired family
462    * @param result the result to return to the client, modify as necessary
463    * @throws IOException if an error occurred on the coprocessor
464    */
465   void postGetClosestRowBefore(final ObserverContext<RegionCoprocessorEnvironment> c,
466       final byte [] row, final byte [] family, final Result result)
467     throws IOException;
468 
469   /**
470    * Called before the client performs a Get
471    * <p>
472    * Call CoprocessorEnvironment#bypass to skip default actions
473    * <p>
474    * Call CoprocessorEnvironment#complete to skip any subsequent chained
475    * coprocessors
476    * @param c the environment provided by the region server
477    * @param get the Get request
478    * @param result The result to return to the client if default processing
479    * is bypassed. Can be modified. Will not be used if default processing
480    * is not bypassed.
481    * @throws IOException if an error occurred on the coprocessor
482    */
483   void preGetOp(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
484       final List<Cell> result)
485     throws IOException;
486 
487   /**
488    * Called after the client performs a Get
489    * <p>
490    * Call CoprocessorEnvironment#complete to skip any subsequent chained
491    * coprocessors
492    * @param c the environment provided by the region server
493    * @param get the Get request
494    * @param result the result to return to the client, modify as necessary
495    * @throws IOException if an error occurred on the coprocessor
496    */
497   void postGetOp(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
498       final List<Cell> result)
499     throws IOException;
500 
501   /**
502    * Called before the client tests for existence using a Get.
503    * <p>
504    * Call CoprocessorEnvironment#bypass to skip default actions
505    * <p>
506    * Call CoprocessorEnvironment#complete to skip any subsequent chained
507    * coprocessors
508    * @param c the environment provided by the region server
509    * @param get the Get request
510    * @param exists
511    * @return the value to return to the client if bypassing default processing
512    * @throws IOException if an error occurred on the coprocessor
513    */
514   boolean preExists(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
515       final boolean exists)
516     throws IOException;
517 
518   /**
519    * Called after the client tests for existence using a Get.
520    * <p>
521    * Call CoprocessorEnvironment#complete to skip any subsequent chained
522    * coprocessors
523    * @param c the environment provided by the region server
524    * @param get the Get request
525    * @param exists the result returned by the region server
526    * @return the result to return to the client
527    * @throws IOException if an error occurred on the coprocessor
528    */
529   boolean postExists(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
530       final boolean exists)
531     throws IOException;
532 
533   /**
534    * Called before the client stores a value.
535    * <p>
536    * Call CoprocessorEnvironment#bypass to skip default actions
537    * <p>
538    * Call CoprocessorEnvironment#complete to skip any subsequent chained
539    * coprocessors
540    * @param c the environment provided by the region server
541    * @param put The Put object
542    * @param edit The WALEdit object that will be written to the wal
543    * @param durability Persistence guarantee for this Put
544    * @throws IOException if an error occurred on the coprocessor
545    */
546   void prePut(final ObserverContext<RegionCoprocessorEnvironment> c, 
547       final Put put, final WALEdit edit, final Durability durability)
548     throws IOException;
549 
550   /**
551    * Called after the client stores a value.
552    * <p>
553    * Call CoprocessorEnvironment#complete to skip any subsequent chained
554    * coprocessors
555    * @param c the environment provided by the region server
556    * @param put The Put object
557    * @param edit The WALEdit object for the wal
558    * @param durability Persistence guarantee for this Put
559    * @throws IOException if an error occurred on the coprocessor
560    */
561   void postPut(final ObserverContext<RegionCoprocessorEnvironment> c, 
562       final Put put, final WALEdit edit, final Durability durability)
563     throws IOException;
564 
565   /**
566    * Called before the client deletes a value.
567    * <p>
568    * Call CoprocessorEnvironment#bypass to skip default actions
569    * <p>
570    * Call CoprocessorEnvironment#complete to skip any subsequent chained
571    * coprocessors
572    * @param c the environment provided by the region server
573    * @param delete The Delete object
574    * @param edit The WALEdit object for the wal
575    * @param durability Persistence guarantee for this Delete
576    * @throws IOException if an error occurred on the coprocessor
577    */
578   void preDelete(final ObserverContext<RegionCoprocessorEnvironment> c, 
579       final Delete delete, final WALEdit edit, final Durability durability)
580     throws IOException;
581 /**
582  * Called before the server updates the timestamp for version delete with latest timestamp.
583  * <p>
584  * Call CoprocessorEnvironment#bypass to skip default actions
585  * <p>
586  * Call CoprocessorEnvironment#complete to skip any subsequent chained
587  * coprocessors
588  * @param c the environment provided by the region server
589  * @param mutation - the parent mutation associated with this delete cell
590  * @param cell - The deleteColumn with latest version cell
591  * @param byteNow - timestamp bytes
592  * @param get - the get formed using the current cell's row.
593  * Note that the get does not specify the family and qualifier
594  * @throws IOException
595  */
596   void prePrepareTimeStampForDeleteVersion(final ObserverContext<RegionCoprocessorEnvironment> c,
597       final Mutation mutation, final Cell cell, final byte[] byteNow,
598       final Get get) throws IOException;
599 
600   /**
601    * Called after the client deletes a value.
602    * <p>
603    * Call CoprocessorEnvironment#complete to skip any subsequent chained
604    * coprocessors
605    * @param c the environment provided by the region server
606    * @param delete The Delete object
607    * @param edit The WALEdit object for the wal
608    * @param durability Persistence guarantee for this Delete
609    * @throws IOException if an error occurred on the coprocessor
610    */
611   void postDelete(final ObserverContext<RegionCoprocessorEnvironment> c,
612       final Delete delete, final WALEdit edit, final Durability durability)
613     throws IOException;
614   
615   /**
616    * This will be called for every batch mutation operation happening at the server. This will be
617    * called after acquiring the locks on the mutating rows and after applying the proper timestamp
618    * for each Mutation at the server. The batch may contain Put/Delete. By setting OperationStatus
619    * of Mutations ({@link MiniBatchOperationInProgress#setOperationStatus(int, OperationStatus)}),
620    * {@link RegionObserver} can make Region to skip these Mutations.
621    * @param c the environment provided by the region server
622    * @param miniBatchOp batch of Mutations getting applied to region.
623    * @throws IOException if an error occurred on the coprocessor
624    */
625   void preBatchMutate(final ObserverContext<RegionCoprocessorEnvironment> c,
626       final MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException;
627 
628   /**
629    * This will be called after applying a batch of Mutations on a region. The Mutations are added to
630    * memstore and WAL.
631    * @param c the environment provided by the region server
632    * @param miniBatchOp batch of Mutations applied to region.
633    * @throws IOException if an error occurred on the coprocessor
634    */
635   void postBatchMutate(final ObserverContext<RegionCoprocessorEnvironment> c,
636       final MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException;
637 
638   /**
639    * This will be called for region operations where read lock is acquired in
640    * {@link Region#startRegionOperation()}.
641    * @param ctx
642    * @param operation The operation is about to be taken on the region
643    * @throws IOException
644    */
645   void postStartRegionOperation(final ObserverContext<RegionCoprocessorEnvironment> ctx,
646       Operation operation) throws IOException;
647 
648   /**
649    * Called after releasing read lock in {@link Region#closeRegionOperation()}.
650    * @param ctx
651    * @param operation
652    * @throws IOException
653    */
654   void postCloseRegionOperation(final ObserverContext<RegionCoprocessorEnvironment> ctx,
655       Operation operation) throws IOException;
656 
657   /**
658    * Called after the completion of batch put/delete and will be called even if the batch operation
659    * fails
660    * @param ctx
661    * @param miniBatchOp 
662    * @param success true if batch operation is successful otherwise false.
663    * @throws IOException
664    */
665   void postBatchMutateIndispensably(final ObserverContext<RegionCoprocessorEnvironment> ctx,
666       MiniBatchOperationInProgress<Mutation> miniBatchOp, final boolean success) throws IOException;
667 
668   /**
669    * Called before checkAndPut.
670    * <p>
671    * Call CoprocessorEnvironment#bypass to skip default actions
672    * <p>
673    * Call CoprocessorEnvironment#complete to skip any subsequent chained
674    * coprocessors
675    * @param c the environment provided by the region server
676    * @param row row to check
677    * @param family column family
678    * @param qualifier column qualifier
679    * @param compareOp the comparison operation
680    * @param comparator the comparator
681    * @param put data to put if check succeeds
682    * @param result 
683    * @return the return value to return to client if bypassing default
684    * processing
685    * @throws IOException if an error occurred on the coprocessor
686    */
687   boolean preCheckAndPut(final ObserverContext<RegionCoprocessorEnvironment> c,
688       final byte [] row, final byte [] family, final byte [] qualifier,
689       final CompareOp compareOp, final ByteArrayComparable comparator,
690       final Put put, final boolean result)
691     throws IOException;
692 
693   /**
694    * Called before checkAndPut but after acquiring rowlock.
695    * <p>
696    * <b>Note:</b> Caution to be taken for not doing any long time operation in this hook. 
697    * Row will be locked for longer time. Trying to acquire lock on another row, within this, 
698    * can lead to potential deadlock.
699    * <p>
700    * Call CoprocessorEnvironment#bypass to skip default actions
701    * <p>
702    * Call CoprocessorEnvironment#complete to skip any subsequent chained
703    * coprocessors
704    * @param c the environment provided by the region server
705    * @param row row to check
706    * @param family column family
707    * @param qualifier column qualifier
708    * @param compareOp the comparison operation
709    * @param comparator the comparator
710    * @param put data to put if check succeeds
711    * @param result 
712    * @return the return value to return to client if bypassing default
713    * processing
714    * @throws IOException if an error occurred on the coprocessor
715    */
716   boolean preCheckAndPutAfterRowLock(final ObserverContext<RegionCoprocessorEnvironment> c,
717       final byte[] row, final byte[] family, final byte[] qualifier, final CompareOp compareOp,
718       final ByteArrayComparable comparator, final Put put, 
719       final boolean result) throws IOException;
720 
721   /**
722    * Called after checkAndPut
723    * <p>
724    * Call CoprocessorEnvironment#complete to skip any subsequent chained
725    * coprocessors
726    * @param c the environment provided by the region server
727    * @param row row to check
728    * @param family column family
729    * @param qualifier column qualifier
730    * @param compareOp the comparison operation
731    * @param comparator the comparator
732    * @param put data to put if check succeeds
733    * @param result from the checkAndPut
734    * @return the possibly transformed return value to return to client
735    * @throws IOException if an error occurred on the coprocessor
736    */
737   boolean postCheckAndPut(final ObserverContext<RegionCoprocessorEnvironment> c,
738       final byte [] row, final byte [] family, final byte [] qualifier,
739       final CompareOp compareOp, final ByteArrayComparable comparator,
740       final Put put, final boolean result)
741     throws IOException;
742 
743   /**
744    * Called before checkAndDelete.
745    * <p>
746    * Call CoprocessorEnvironment#bypass to skip default actions
747    * <p>
748    * Call CoprocessorEnvironment#complete to skip any subsequent chained
749    * coprocessors
750    * @param c the environment provided by the region server
751    * @param row row to check
752    * @param family column family
753    * @param qualifier column qualifier
754    * @param compareOp the comparison operation
755    * @param comparator the comparator
756    * @param delete delete to commit if check succeeds
757    * @param result 
758    * @return the value to return to client if bypassing default processing
759    * @throws IOException if an error occurred on the coprocessor
760    */
761   boolean preCheckAndDelete(final ObserverContext<RegionCoprocessorEnvironment> c,
762       final byte [] row, final byte [] family, final byte [] qualifier,
763       final CompareOp compareOp, final ByteArrayComparable comparator,
764       final Delete delete, final boolean result)
765     throws IOException;
766 
767   /**
768    * Called before checkAndDelete but after acquiring rowock.
769    * <p>
770    * <b>Note:</b> Caution to be taken for not doing any long time operation in this hook. 
771    * Row will be locked for longer time. Trying to acquire lock on another row, within this, 
772    * can lead to potential deadlock.
773    * <p>
774    * Call CoprocessorEnvironment#bypass to skip default actions
775    * <p>
776    * Call CoprocessorEnvironment#complete to skip any subsequent chained
777    * coprocessors
778    * @param c the environment provided by the region server
779    * @param row row to check
780    * @param family column family
781    * @param qualifier column qualifier
782    * @param compareOp the comparison operation
783    * @param comparator the comparator
784    * @param delete delete to commit if check succeeds
785    * @param result 
786    * @return the value to return to client if bypassing default processing
787    * @throws IOException if an error occurred on the coprocessor
788    */
789   boolean preCheckAndDeleteAfterRowLock(final ObserverContext<RegionCoprocessorEnvironment> c,
790       final byte[] row, final byte[] family, final byte[] qualifier, final CompareOp compareOp,
791       final ByteArrayComparable comparator, final Delete delete,
792       final boolean result) throws IOException;
793 
794   /**
795    * Called after checkAndDelete
796    * <p>
797    * Call CoprocessorEnvironment#complete to skip any subsequent chained
798    * coprocessors
799    * @param c the environment provided by the region server
800    * @param row row to check
801    * @param family column family
802    * @param qualifier column qualifier
803    * @param compareOp the comparison operation
804    * @param comparator the comparator
805    * @param delete delete to commit if check succeeds
806    * @param result from the CheckAndDelete
807    * @return the possibly transformed returned value to return to client
808    * @throws IOException if an error occurred on the coprocessor
809    */
810   boolean postCheckAndDelete(final ObserverContext<RegionCoprocessorEnvironment> c,
811       final byte [] row, final byte [] family, final byte [] qualifier,
812       final CompareOp compareOp, final ByteArrayComparable comparator,
813       final Delete delete, final boolean result)
814     throws IOException;
815 
816   /**
817    * Called before incrementColumnValue
818    * <p>
819    * Call CoprocessorEnvironment#bypass to skip default actions
820    * <p>
821    * Call CoprocessorEnvironment#complete to skip any subsequent chained
822    * coprocessors
823    * @param c the environment provided by the region server
824    * @param row row to check
825    * @param family column family
826    * @param qualifier column qualifier
827    * @param amount long amount to increment
828    * @param writeToWAL true if the change should be written to the WAL
829    * @return value to return to the client if bypassing default processing
830    * @throws IOException if an error occurred on the coprocessor
831    * @deprecated This hook is no longer called by the RegionServer
832    */
833   @Deprecated
834   long preIncrementColumnValue(final ObserverContext<RegionCoprocessorEnvironment> c,
835       final byte [] row, final byte [] family, final byte [] qualifier,
836       final long amount, final boolean writeToWAL)
837     throws IOException;
838 
839   /**
840    * Called after incrementColumnValue
841    * <p>
842    * Call CoprocessorEnvironment#complete to skip any subsequent chained
843    * coprocessors
844    * @param c the environment provided by the region server
845    * @param row row to check
846    * @param family column family
847    * @param qualifier column qualifier
848    * @param amount long amount to increment
849    * @param writeToWAL true if the change should be written to the WAL
850    * @param result the result returned by incrementColumnValue
851    * @return the result to return to the client
852    * @throws IOException if an error occurred on the coprocessor
853    * @deprecated This hook is no longer called by the RegionServer
854    */
855   @Deprecated
856   long postIncrementColumnValue(final ObserverContext<RegionCoprocessorEnvironment> c,
857       final byte [] row, final byte [] family, final byte [] qualifier,
858       final long amount, final boolean writeToWAL, final long result)
859     throws IOException;
860 
861   /**
862    * Called before Append.
863    * <p>
864    * Call CoprocessorEnvironment#bypass to skip default actions
865    * <p>
866    * Call CoprocessorEnvironment#complete to skip any subsequent chained
867    * coprocessors
868    * @param c the environment provided by the region server
869    * @param append Append object
870    * @return result to return to the client if bypassing default processing
871    * @throws IOException if an error occurred on the coprocessor
872    */
873   Result preAppend(final ObserverContext<RegionCoprocessorEnvironment> c,
874       final Append append)
875     throws IOException;
876 
877   /**
878    * Called before Append but after acquiring rowlock.
879    * <p>
880    * <b>Note:</b> Caution to be taken for not doing any long time operation in this hook. 
881    * Row will be locked for longer time. Trying to acquire lock on another row, within this, 
882    * can lead to potential deadlock.
883    * <p>
884    * Call CoprocessorEnvironment#bypass to skip default actions
885    * <p>
886    * Call CoprocessorEnvironment#complete to skip any subsequent chained
887    * coprocessors
888    * @param c the environment provided by the region server
889    * @param append Append object
890    * @return result to return to the client if bypassing default processing
891    * @throws IOException if an error occurred on the coprocessor
892    */
893   Result preAppendAfterRowLock(final ObserverContext<RegionCoprocessorEnvironment> c,
894       final Append append) throws IOException;
895 
896   /**
897    * Called after Append
898    * <p>
899    * Call CoprocessorEnvironment#complete to skip any subsequent chained
900    * coprocessors
901    * @param c the environment provided by the region server
902    * @param append Append object
903    * @param result the result returned by increment
904    * @return the result to return to the client
905    * @throws IOException if an error occurred on the coprocessor
906    */
907   Result postAppend(final ObserverContext<RegionCoprocessorEnvironment> c,
908       final Append append, final Result result)
909     throws IOException;
910 
911   /**
912    * Called before Increment.
913    * <p>
914    * Call CoprocessorEnvironment#bypass to skip default actions
915    * <p>
916    * Call CoprocessorEnvironment#complete to skip any subsequent chained
917    * coprocessors
918    * @param c the environment provided by the region server
919    * @param increment increment object
920    * @return result to return to the client if bypassing default processing
921    * @throws IOException if an error occurred on the coprocessor
922    */
923   Result preIncrement(final ObserverContext<RegionCoprocessorEnvironment> c,
924       final Increment increment)
925     throws IOException;
926 
927   /**
928    * Called before Increment but after acquiring rowlock.
929    * <p>
930    * <b>Note:</b> Caution to be taken for not doing any long time operation in this hook. 
931    * Row will be locked for longer time. Trying to acquire lock on another row, within this, 
932    * can lead to potential deadlock.
933    * <p>
934    * Call CoprocessorEnvironment#bypass to skip default actions
935    * <p>
936    * Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
937    * 
938    * @param c
939    *          the environment provided by the region server
940    * @param increment
941    *          increment object
942    * @return result to return to the client if bypassing default processing
943    * @throws IOException
944    *           if an error occurred on the coprocessor
945    */
946   Result preIncrementAfterRowLock(final ObserverContext<RegionCoprocessorEnvironment> c,
947       final Increment increment) throws IOException;
948 
949   /**
950    * Called after increment
951    * <p>
952    * Call CoprocessorEnvironment#complete to skip any subsequent chained
953    * coprocessors
954    * @param c the environment provided by the region server
955    * @param increment increment object
956    * @param result the result returned by increment
957    * @return the result to return to the client
958    * @throws IOException if an error occurred on the coprocessor
959    */
960   Result postIncrement(final ObserverContext<RegionCoprocessorEnvironment> c,
961       final Increment increment, final Result result)
962     throws IOException;
963 
964   /**
965    * Called before the client opens a new scanner.
966    * <p>
967    * Call CoprocessorEnvironment#bypass to skip default actions
968    * <p>
969    * Call CoprocessorEnvironment#complete to skip any subsequent chained
970    * coprocessors
971    * @param c the environment provided by the region server
972    * @param scan the Scan specification
973    * @param s if not null, the base scanner
974    * @return an RegionScanner instance to use instead of the base scanner if
975    * overriding default behavior, null otherwise
976    * @throws IOException if an error occurred on the coprocessor
977    */
978   RegionScanner preScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
979       final Scan scan, final RegionScanner s)
980     throws IOException;
981 
982   /**
983    * Called before a store opens a new scanner.
984    * This hook is called when a "user" scanner is opened.
985    * <p>
986    * See {@link #preFlushScannerOpen(ObserverContext, Store, KeyValueScanner, InternalScanner)}
987    * and {@link #preCompactScannerOpen(ObserverContext,
988    *  Store, List, ScanType, long, InternalScanner)}
989    * to override scanners created for flushes or compactions, resp.
990    * <p>
991    * Call CoprocessorEnvironment#complete to skip any subsequent chained
992    * coprocessors.
993    * Calling {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} has no
994    * effect in this hook.
995    * @param c the environment provided by the region server
996    * @param store the store being scanned
997    * @param scan the Scan specification
998    * @param targetCols columns to be used in the scanner
999    * @param s the base scanner, if not {@code null}, from previous RegionObserver in the chain
1000    * @return a KeyValueScanner instance to use or {@code null} to use the default implementation
1001    * @throws IOException if an error occurred on the coprocessor
1002    */
1003   KeyValueScanner preStoreScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
1004       final Store store, final Scan scan, final NavigableSet<byte[]> targetCols,
1005       final KeyValueScanner s) throws IOException;
1006 
1007   /**
1008    * Called after the client opens a new scanner.
1009    * <p>
1010    * Call CoprocessorEnvironment#complete to skip any subsequent chained
1011    * coprocessors
1012    * @param c the environment provided by the region server
1013    * @param scan the Scan specification
1014    * @param s if not null, the base scanner
1015    * @return the scanner instance to use
1016    * @throws IOException if an error occurred on the coprocessor
1017    */
1018   RegionScanner postScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
1019       final Scan scan, final RegionScanner s)
1020     throws IOException;
1021 
1022   /**
1023    * Called before the client asks for the next row on a scanner.
1024    * <p>
1025    * Call CoprocessorEnvironment#bypass to skip default actions
1026    * <p>
1027    * Call CoprocessorEnvironment#complete to skip any subsequent chained
1028    * coprocessors
1029    * @param c the environment provided by the region server
1030    * @param s the scanner
1031    * @param result The result to return to the client if default processing
1032    * is bypassed. Can be modified. Will not be returned if default processing
1033    * is not bypassed.
1034    * @param limit the maximum number of results to return
1035    * @param hasNext the 'has more' indication
1036    * @return 'has more' indication that should be sent to client
1037    * @throws IOException if an error occurred on the coprocessor
1038    */
1039   boolean preScannerNext(final ObserverContext<RegionCoprocessorEnvironment> c,
1040       final InternalScanner s, final List<Result> result,
1041       final int limit, final boolean hasNext)
1042     throws IOException;
1043 
1044   /**
1045    * Called after the client asks for the next row on a scanner.
1046    * <p>
1047    * Call CoprocessorEnvironment#complete to skip any subsequent chained
1048    * coprocessors
1049    * @param c the environment provided by the region server
1050    * @param s the scanner
1051    * @param result the result to return to the client, can be modified
1052    * @param limit the maximum number of results to return
1053    * @param hasNext the 'has more' indication
1054    * @return 'has more' indication that should be sent to client
1055    * @throws IOException if an error occurred on the coprocessor
1056    */
1057   boolean postScannerNext(final ObserverContext<RegionCoprocessorEnvironment> c,
1058       final InternalScanner s, final List<Result> result, final int limit,
1059       final boolean hasNext)
1060     throws IOException;
1061 
1062   /**
1063    * This will be called by the scan flow when the current scanned row is being filtered out by the
1064    * filter. The filter may be filtering out the row via any of the below scenarios
1065    * <ol>
1066    * <li>
1067    * <code>boolean filterRowKey(byte [] buffer, int offset, int length)</code> returning true</li>
1068    * <li>
1069    * <code>boolean filterRow()</code> returning true</li>
1070    * <li>
1071    * <code>void filterRow(List&lt;KeyValue&gt; kvs)</code> removing all the kvs
1072    * from the passed List</li>
1073    * </ol>
1074    * @param c the environment provided by the region server
1075    * @param s the scanner
1076    * @param currentRow The current rowkey which got filtered out
1077    * @param offset offset to rowkey
1078    * @param length length of rowkey
1079    * @param hasMore the 'has more' indication
1080    * @return whether more rows are available for the scanner or not
1081    * @throws IOException
1082    */
1083   boolean postScannerFilterRow(final ObserverContext<RegionCoprocessorEnvironment> c,
1084       final InternalScanner s, final byte[] currentRow, final int offset, final short length,
1085       final boolean hasMore) throws IOException;
1086 
1087   /**
1088    * Called before the client closes a scanner.
1089    * <p>
1090    * Call CoprocessorEnvironment#bypass to skip default actions
1091    * <p>
1092    * Call CoprocessorEnvironment#complete to skip any subsequent chained
1093    * coprocessors
1094    * @param c the environment provided by the region server
1095    * @param s the scanner
1096    * @throws IOException if an error occurred on the coprocessor
1097    */
1098   void preScannerClose(final ObserverContext<RegionCoprocessorEnvironment> c,
1099       final InternalScanner s)
1100     throws IOException;
1101 
1102   /**
1103    * Called after the client closes a scanner.
1104    * <p>
1105    * Call CoprocessorEnvironment#complete to skip any subsequent chained
1106    * coprocessors
1107    * @param c the environment provided by the region server
1108    * @param s the scanner
1109    * @throws IOException if an error occurred on the coprocessor
1110    */
1111   void postScannerClose(final ObserverContext<RegionCoprocessorEnvironment> c,
1112       final InternalScanner s)
1113     throws IOException;
1114 
1115   /**
1116    * Called before a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit}
1117    * replayed for this region.
1118    */
1119   void preWALRestore(final ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
1120       HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException;
1121 
1122   /**
1123    * Called before a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit}
1124    * replayed for this region.
1125    *
1126    * This method is left in place to maintain binary compatibility with older
1127    * {@link RegionObserver}s. If an implementation directly overrides
1128    * {@link #preWALRestore(ObserverContext, HRegionInfo, WALKey, WALEdit)} then this version
1129    * won't be called at all, barring problems with the Security Manager. To work correctly
1130    * in the presence of a strict Security Manager, or in the case of an implementation that
1131    * relies on a parent class to implement preWALRestore, you should implement this method
1132    * as a call to the non-deprecated version.
1133    *
1134    * Users of this method will see all edits that can be treated as HLogKey. If there are
1135    * edits that can't be treated as HLogKey they won't be offered to coprocessors that rely
1136    * on this method. If a coprocessor gets skipped because of this mechanism, a log message
1137    * at ERROR will be generated per coprocessor on the logger for {@link CoprocessorHost} once per
1138    * classloader.
1139    *
1140    * @deprecated use {@link #preWALRestore(ObserverContext, HRegionInfo, WALKey, WALEdit)}
1141    */
1142   @Deprecated
1143   void preWALRestore(final ObserverContext<RegionCoprocessorEnvironment> ctx,
1144       HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException;
1145 
1146   /**
1147    * Called after a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit}
1148    * replayed for this region.
1149    */
1150   void postWALRestore(final ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
1151       HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException;
1152 
1153   /**
1154    * Called after a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit}
1155    * replayed for this region.
1156    *
1157    * This method is left in place to maintain binary compatibility with older
1158    * {@link RegionObserver}s. If an implementation directly overrides
1159    * {@link #postWALRestore(ObserverContext, HRegionInfo, WALKey, WALEdit)} then this version
1160    * won't be called at all, barring problems with the Security Manager. To work correctly
1161    * in the presence of a strict Security Manager, or in the case of an implementation that
1162    * relies on a parent class to implement preWALRestore, you should implement this method
1163    * as a call to the non-deprecated version.
1164    *
1165    * Users of this method will see all edits that can be treated as HLogKey. If there are
1166    * edits that can't be treated as HLogKey they won't be offered to coprocessors that rely
1167    * on this method. If a coprocessor gets skipped because of this mechanism, a log message
1168    * at ERROR will be generated per coprocessor on the logger for {@link CoprocessorHost} once per
1169    * classloader.
1170    *
1171    * @deprecated use {@link #postWALRestore(ObserverContext, HRegionInfo, WALKey, WALEdit)}
1172    */
1173   @Deprecated
1174   void postWALRestore(final ObserverContext<RegionCoprocessorEnvironment> ctx,
1175       HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException;
1176 
1177   /**
1178    * Called before bulkLoadHFile. Users can create a StoreFile instance to
1179    * access the contents of a HFile.
1180    *
1181    * @param ctx
1182    * @param familyPaths pairs of { CF, HFile path } submitted for bulk load. Adding
1183    * or removing from this list will add or remove HFiles to be bulk loaded.
1184    * @throws IOException
1185    */
1186   void preBulkLoadHFile(final ObserverContext<RegionCoprocessorEnvironment> ctx,
1187     List<Pair<byte[], String>> familyPaths) throws IOException;
1188 
1189   /**
1190    * Called after bulkLoadHFile.
1191    *
1192    * @param ctx
1193    * @param familyPaths pairs of { CF, HFile path } submitted for bulk load
1194    * @param hasLoaded whether the bulkLoad was successful
1195    * @return the new value of hasLoaded
1196    * @throws IOException
1197    */
1198   boolean postBulkLoadHFile(final ObserverContext<RegionCoprocessorEnvironment> ctx,
1199     List<Pair<byte[], String>> familyPaths, boolean hasLoaded) throws IOException;
1200 
1201   /**
1202    * Called before creation of Reader for a store file.
1203    * Calling {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} has no
1204    * effect in this hook.
1205    * 
1206    * @param ctx the environment provided by the region server
1207    * @param fs fileystem to read from
1208    * @param p path to the file
1209    * @param in {@link FSDataInputStreamWrapper}
1210    * @param size Full size of the file
1211    * @param cacheConf
1212    * @param r original reference file. This will be not null only when reading a split file.
1213    * @param reader the base reader, if not {@code null}, from previous RegionObserver in the chain
1214    * @return a Reader instance to use instead of the base reader if overriding
1215    * default behavior, null otherwise
1216    * @throws IOException
1217    */
1218   StoreFile.Reader preStoreFileReaderOpen(final ObserverContext<RegionCoprocessorEnvironment> ctx,
1219       final FileSystem fs, final Path p, final FSDataInputStreamWrapper in, long size,
1220       final CacheConfig cacheConf, final Reference r, StoreFile.Reader reader) throws IOException;
1221 
1222   /**
1223    * Called after the creation of Reader for a store file.
1224    * 
1225    * @param ctx the environment provided by the region server
1226    * @param fs fileystem to read from
1227    * @param p path to the file
1228    * @param in {@link FSDataInputStreamWrapper}
1229    * @param size Full size of the file
1230    * @param cacheConf
1231    * @param r original reference file. This will be not null only when reading a split file.
1232    * @param reader the base reader instance
1233    * @return The reader to use
1234    * @throws IOException
1235    */
1236   StoreFile.Reader postStoreFileReaderOpen(final ObserverContext<RegionCoprocessorEnvironment> ctx,
1237       final FileSystem fs, final Path p, final FSDataInputStreamWrapper in, long size,
1238       final CacheConfig cacheConf, final Reference r, StoreFile.Reader reader) throws IOException;
1239 
1240   /**
1241    * Called after a new cell has been created during an increment operation, but before
1242    * it is committed to the WAL or memstore.
1243    * Calling {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} has no
1244    * effect in this hook.
1245    * @param ctx the environment provided by the region server
1246    * @param opType the operation type
1247    * @param mutation the current mutation
1248    * @param oldCell old cell containing previous value
1249    * @param newCell the new cell containing the computed value
1250    * @return the new cell, possibly changed
1251    * @throws IOException
1252    */
1253   Cell postMutationBeforeWAL(ObserverContext<RegionCoprocessorEnvironment> ctx,
1254       MutationType opType, Mutation mutation, Cell oldCell, Cell newCell) throws IOException;
1255 
1256   /**
1257    * Called after the ScanQueryMatcher creates ScanDeleteTracker. Implementing
1258    * this hook would help in creating customised DeleteTracker and returning
1259    * the newly created DeleteTracker
1260    *
1261    * @param ctx the environment provided by the region server
1262    * @param delTracker the deleteTracker that is created by the QueryMatcher
1263    * @return the Delete Tracker
1264    * @throws IOException
1265    */
1266   DeleteTracker postInstantiateDeleteTracker(
1267       final ObserverContext<RegionCoprocessorEnvironment> ctx, DeleteTracker delTracker)
1268       throws IOException;
1269 }