001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.coprocessor;
019
020import java.io.IOException;
021import java.util.ArrayList;
022import java.util.List;
023import java.util.Map;
024import org.apache.hadoop.fs.FileSystem;
025import org.apache.hadoop.fs.Path;
026import org.apache.hadoop.hbase.Cell;
027import org.apache.hadoop.hbase.CompareOperator;
028import org.apache.hadoop.hbase.HBaseInterfaceAudience;
029import org.apache.hadoop.hbase.client.Append;
030import org.apache.hadoop.hbase.client.CheckAndMutate;
031import org.apache.hadoop.hbase.client.CheckAndMutateResult;
032import org.apache.hadoop.hbase.client.Delete;
033import org.apache.hadoop.hbase.client.Durability;
034import org.apache.hadoop.hbase.client.Get;
035import org.apache.hadoop.hbase.client.Increment;
036import org.apache.hadoop.hbase.client.Mutation;
037import org.apache.hadoop.hbase.client.Put;
038import org.apache.hadoop.hbase.client.RegionInfo;
039import org.apache.hadoop.hbase.client.Result;
040import org.apache.hadoop.hbase.client.Scan;
041import org.apache.hadoop.hbase.filter.BinaryComparator;
042import org.apache.hadoop.hbase.filter.ByteArrayComparable;
043import org.apache.hadoop.hbase.filter.Filter;
044import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
045import org.apache.hadoop.hbase.io.Reference;
046import org.apache.hadoop.hbase.io.hfile.CacheConfig;
047import org.apache.hadoop.hbase.regionserver.FlushLifeCycleTracker;
048import org.apache.hadoop.hbase.regionserver.InternalScanner;
049import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
050import org.apache.hadoop.hbase.regionserver.OperationStatus;
051import org.apache.hadoop.hbase.regionserver.Region;
052import org.apache.hadoop.hbase.regionserver.Region.Operation;
053import org.apache.hadoop.hbase.regionserver.RegionScanner;
054import org.apache.hadoop.hbase.regionserver.ScanOptions;
055import org.apache.hadoop.hbase.regionserver.ScanType;
056import org.apache.hadoop.hbase.regionserver.Shipper;
057import org.apache.hadoop.hbase.regionserver.Store;
058import org.apache.hadoop.hbase.regionserver.StoreFile;
059import org.apache.hadoop.hbase.regionserver.StoreFileReader;
060import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
061import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
062import org.apache.hadoop.hbase.regionserver.querymatcher.DeleteTracker;
063import org.apache.hadoop.hbase.util.Pair;
064import org.apache.hadoop.hbase.wal.WALEdit;
065import org.apache.hadoop.hbase.wal.WALKey;
066import org.apache.yetus.audience.InterfaceAudience;
067import org.apache.yetus.audience.InterfaceStability;
068
069/**
070 * Coprocessors implement this interface to observe and mediate client actions on the region.
071 * <p>
072 * Since most implementations will be interested in only a subset of hooks, this class uses
073 * 'default' functions to avoid having to add unnecessary overrides. When the functions are
074 * non-empty, it's simply to satisfy the compiler by returning value of expected (non-void) type. It
075 * is done in a way that these default definitions act as no-op. So our suggestion to implementation
076 * would be to not call these 'default' methods from overrides.
077 * <p>
078 * <h3>Exception Handling</h3><br>
079 * For all functions, exception handling is done as follows:
080 * <ul>
081 * <li>Exceptions of type {@link IOException} are reported back to client.</li>
082 * <li>For any other kind of exception:
083 * <ul>
084 * <li>If the configuration {@link CoprocessorHost#ABORT_ON_ERROR_KEY} is set to true, then the
085 * server aborts.</li>
086 * <li>Otherwise, coprocessor is removed from the server and
087 * {@link org.apache.hadoop.hbase.DoNotRetryIOException} is returned to the client.</li>
088 * </ul>
089 * </li>
090 * </ul>
091 * <p>
092 * <h3>For Split Related Hooks</h3> <br>
093 * In hbase2/AMv2, master runs splits, so the split related hooks are moved to
094 * {@link MasterObserver}.
095 * <p>
096 * <h3>Increment Column Value</h3><br>
097 * We do not call this hook anymore.
098 */
099@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
100@InterfaceStability.Evolving
101public interface RegionObserver {
102  /** Mutation type for postMutationBeforeWAL hook */
103  enum MutationType {
104    APPEND,
105    INCREMENT
106  }
107
108  /**
109   * Called before the region is reported as open to the master.
110   * @param c the environment provided by the region server
111   */
112  default void preOpen(ObserverContext<? extends RegionCoprocessorEnvironment> c)
113    throws IOException {
114  }
115
116  /**
117   * Called after the region is reported as open to the master.
118   * @param c the environment provided by the region server
119   */
120  default void postOpen(ObserverContext<? extends RegionCoprocessorEnvironment> c) {
121  }
122
123  /**
124   * Called before the memstore is flushed to disk.
125   * @param c       the environment provided by the region server
126   * @param tracker tracker used to track the life cycle of a flush
127   */
128  default void preFlush(final ObserverContext<? extends RegionCoprocessorEnvironment> c,
129    FlushLifeCycleTracker tracker) throws IOException {
130  }
131
132  /**
133   * Called before we open store scanner for flush. You can use the {@code options} to change max
134   * versions and TTL for the scanner being opened.
135   * @param c       the environment provided by the region server
136   * @param store   the store where flush is being requested
137   * @param options used to change max versions and TTL for the scanner being opened
138   */
139  default void preFlushScannerOpen(ObserverContext<? extends RegionCoprocessorEnvironment> c,
140    Store store, ScanOptions options, FlushLifeCycleTracker tracker) throws IOException {
141  }
142
143  /**
144   * Called before a Store's memstore is flushed to disk.
145   * @param c       the environment provided by the region server
146   * @param store   the store where flush is being requested
147   * @param scanner the scanner over existing data used in the memstore
148   * @param tracker tracker used to track the life cycle of a flush
149   * @return the scanner to use during flush. Should not be {@code null} unless the implementation
150   *         is writing new store files on its own.
151   */
152  default InternalScanner preFlush(ObserverContext<? extends RegionCoprocessorEnvironment> c,
153    Store store, InternalScanner scanner, FlushLifeCycleTracker tracker) throws IOException {
154    return scanner;
155  }
156
157  /**
158   * Called after the memstore is flushed to disk.
159   * @param c       the environment provided by the region server
160   * @param tracker tracker used to track the life cycle of a flush
161   * @throws IOException if an error occurred on the coprocessor
162   */
163  default void postFlush(ObserverContext<? extends RegionCoprocessorEnvironment> c,
164    FlushLifeCycleTracker tracker) throws IOException {
165  }
166
167  /**
168   * Called after a Store's memstore is flushed to disk.
169   * @param c          the environment provided by the region server
170   * @param store      the store being flushed
171   * @param resultFile the new store file written out during compaction
172   * @param tracker    tracker used to track the life cycle of a flush
173   */
174  default void postFlush(ObserverContext<? extends RegionCoprocessorEnvironment> c, Store store,
175    StoreFile resultFile, FlushLifeCycleTracker tracker) throws IOException {
176  }
177
178  /**
179   * Called before in memory compaction started.
180   * @param c     the environment provided by the region server
181   * @param store the store where in memory compaction is being requested
182   */
183  default void preMemStoreCompaction(ObserverContext<? extends RegionCoprocessorEnvironment> c,
184    Store store) throws IOException {
185  }
186
187  /**
188   * Called before we open store scanner for in memory compaction. You can use the {@code options}
189   * to change max versions and TTL for the scanner being opened. Notice that this method will only
190   * be called when you use {@code eager} mode. For {@code basic} mode we will not drop any cells
191   * thus we do not open a store scanner.
192   * @param c       the environment provided by the region server
193   * @param store   the store where in memory compaction is being requested
194   * @param options used to change max versions and TTL for the scanner being opened
195   */
196  default void preMemStoreCompactionCompactScannerOpen(
197    ObserverContext<? extends RegionCoprocessorEnvironment> c, Store store, ScanOptions options)
198    throws IOException {
199  }
200
201  /**
202   * Called before we do in memory compaction. Notice that this method will only be called when you
203   * use {@code eager} mode. For {@code basic} mode we will not drop any cells thus there is no
204   * {@link InternalScanner}.
205   * @param c       the environment provided by the region server
206   * @param store   the store where in memory compaction is being executed
207   * @param scanner the scanner over existing data used in the memstore segments being compact
208   * @return the scanner to use during in memory compaction. Must be non-null.
209   */
210  default InternalScanner preMemStoreCompactionCompact(
211    ObserverContext<? extends RegionCoprocessorEnvironment> c, Store store, InternalScanner scanner)
212    throws IOException {
213    return scanner;
214  }
215
216  /**
217   * Called after the in memory compaction is finished.
218   * @param c     the environment provided by the region server
219   * @param store the store where in memory compaction is being executed
220   */
221  default void postMemStoreCompaction(ObserverContext<? extends RegionCoprocessorEnvironment> c,
222    Store store) throws IOException {
223  }
224
225  /**
226   * Called prior to selecting the {@link StoreFile StoreFiles} to compact from the list of
227   * available candidates. To alter the files used for compaction, you may mutate the passed in list
228   * of candidates. If you remove all the candidates then the compaction will be canceled.
229   * <p>
230   * Supports Coprocessor 'bypass' -- 'bypass' is how this method indicates that it changed the
231   * passed in <code>candidates</code>. If 'bypass' is set, we skip out on calling any subsequent
232   * chained coprocessors.
233   * @param c          the environment provided by the region server
234   * @param store      the store where compaction is being requested
235   * @param candidates the store files currently available for compaction
236   * @param tracker    tracker used to track the life cycle of a compaction
237   */
238  default void preCompactSelection(ObserverContext<? extends RegionCoprocessorEnvironment> c,
239    Store store, List<? extends StoreFile> candidates, CompactionLifeCycleTracker tracker)
240    throws IOException {
241  }
242
243  /**
244   * Called after the {@link StoreFile}s to compact have been selected from the available
245   * candidates.
246   * @param c        the environment provided by the region server
247   * @param store    the store being compacted
248   * @param selected the store files selected to compact
249   * @param tracker  tracker used to track the life cycle of a compaction
250   * @param request  the requested compaction
251   */
252  default void postCompactSelection(ObserverContext<? extends RegionCoprocessorEnvironment> c,
253    Store store, List<? extends StoreFile> selected, CompactionLifeCycleTracker tracker,
254    CompactionRequest request) {
255  }
256
257  /**
258   * Called before we open store scanner for compaction. You can use the {@code options} to change
259   * max versions and TTL for the scanner being opened.
260   * @param c        the environment provided by the region server
261   * @param store    the store being compacted
262   * @param scanType type of Scan
263   * @param options  used to change max versions and TTL for the scanner being opened
264   * @param tracker  tracker used to track the life cycle of a compaction
265   * @param request  the requested compaction
266   */
267  default void preCompactScannerOpen(ObserverContext<? extends RegionCoprocessorEnvironment> c,
268    Store store, ScanType scanType, ScanOptions options, CompactionLifeCycleTracker tracker,
269    CompactionRequest request) throws IOException {
270  }
271
272  /**
273   * Called prior to writing the {@link StoreFile}s selected for compaction into a new
274   * {@code StoreFile}.
275   * <p>
276   * To override or modify the compaction process, implementing classes can wrap the provided
277   * {@link InternalScanner} with a custom implementation that is returned from this method. The
278   * custom scanner can then inspect {@link org.apache.hadoop.hbase.Cell}s from the wrapped scanner,
279   * applying its own policy to what gets written.
280   * <p>
281   * If implementations are wrapping the passed in {@link InternalScanner}, they can also have their
282   * implementation implement {@link Shipper} and delegate to the original scanner. This will cause
283   * compactions to free up memory as they progress, which is especially important for people using
284   * off-heap memory pools.
285   * <p>
286   * Keep in mind that when {@link Shipper#shipped()} is called, any cell references you maintain in
287   * your implementation may get corrupted. As such you should make sure to deep clone any cells
288   * that you need to keep reference to across invocations of shipped.
289   * @param c        the environment provided by the region server
290   * @param store    the store being compacted
291   * @param scanner  the scanner over existing data used in the store file rewriting
292   * @param scanType type of Scan
293   * @param tracker  tracker used to track the life cycle of a compaction
294   * @param request  the requested compaction
295   * @return the scanner to use during compaction. Should not be {@code null} unless the
296   *         implementation is writing new store files on its own.
297   */
298  default InternalScanner preCompact(ObserverContext<? extends RegionCoprocessorEnvironment> c,
299    Store store, InternalScanner scanner, ScanType scanType, CompactionLifeCycleTracker tracker,
300    CompactionRequest request) throws IOException {
301    return scanner;
302  }
303
304  /**
305   * Called after compaction has completed and the new store file has been moved in to place.
306   * @param c          the environment provided by the region server
307   * @param store      the store being compacted
308   * @param resultFile the new store file written out during compaction
309   * @param tracker    used to track the life cycle of a compaction
310   * @param request    the requested compaction
311   */
312  default void postCompact(ObserverContext<? extends RegionCoprocessorEnvironment> c, Store store,
313    StoreFile resultFile, CompactionLifeCycleTracker tracker, CompactionRequest request)
314    throws IOException {
315  }
316
317  /**
318   * Called before the region is reported as closed to the master.
319   * @param c              the environment provided by the region server
320   * @param abortRequested true if the region server is aborting
321   */
322  default void preClose(ObserverContext<? extends RegionCoprocessorEnvironment> c,
323    boolean abortRequested) throws IOException {
324  }
325
326  /**
327   * Called after the region is reported as closed to the master.
328   * @param c              the environment provided by the region server
329   * @param abortRequested true if the region server is aborting
330   */
331  default void postClose(ObserverContext<? extends RegionCoprocessorEnvironment> c,
332    boolean abortRequested) {
333  }
334
335  /**
336   * Called before the client performs a Get
337   * <p>
338   * Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on
339   * calling any subsequent chained coprocessors.
340   * @param c      the environment provided by the region server
341   * @param get    the Get request
342   * @param result The result to return to the client if default processing is bypassed. Can be
343   *               modified. Will not be used if default processing is not bypassed.
344   */
345  default void preGetOp(ObserverContext<? extends RegionCoprocessorEnvironment> c, Get get,
346    List<Cell> result) throws IOException {
347  }
348
349  /**
350   * Called after the client performs a Get
351   * <p>
352   * Note: Do not retain references to any Cells in 'result' beyond the life of this invocation. If
353   * need a Cell reference for later use, copy the cell and use that.
354   * @param c      the environment provided by the region server
355   * @param get    the Get request
356   * @param result the result to return to the client, modify as necessary
357   */
358  default void postGetOp(ObserverContext<? extends RegionCoprocessorEnvironment> c, Get get,
359    List<Cell> result) throws IOException {
360  }
361
362  /**
363   * Called before the client tests for existence using a Get.
364   * <p>
365   * Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on
366   * calling any subsequent chained coprocessors.
367   * @param c      the environment provided by the region server
368   * @param get    the Get request
369   * @param exists the result returned by the region server
370   * @return the value to return to the client if bypassing default processing
371   */
372  default boolean preExists(ObserverContext<? extends RegionCoprocessorEnvironment> c, Get get,
373    boolean exists) throws IOException {
374    return exists;
375  }
376
377  /**
378   * Called after the client tests for existence using a Get.
379   * @param c      the environment provided by the region server
380   * @param get    the Get request
381   * @param exists the result returned by the region server
382   * @return the result to return to the client
383   */
384  default boolean postExists(ObserverContext<? extends RegionCoprocessorEnvironment> c, Get get,
385    boolean exists) throws IOException {
386    return exists;
387  }
388
389  /**
390   * Called before the client stores a value.
391   * <p>
392   * Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on
393   * calling any subsequent chained coprocessors.
394   * <p>
395   * Note: Do not retain references to any Cells in 'put' beyond the life of this invocation. If
396   * need a Cell reference for later use, copy the cell and use that.
397   * @param c          the environment provided by the region server
398   * @param put        The Put object
399   * @param edit       The WALEdit object that will be written to the wal
400   * @param durability Persistence guarantee for this Put
401   * @deprecated since 3.0.0 and will be removed in 4.0.0. Use
402   *             {@link #prePut(ObserverContext, Put, WALEdit)} instead.
403   */
404  @Deprecated
405  default void prePut(ObserverContext<? extends RegionCoprocessorEnvironment> c, Put put,
406    WALEdit edit, Durability durability) throws IOException {
407  }
408
409  /**
410   * Called before the client stores a value.
411   * <p>
412   * Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on
413   * calling any subsequent chained coprocessors.
414   * <p>
415   * Note: Do not retain references to any Cells in 'put' beyond the life of this invocation. If
416   * need a Cell reference for later use, copy the cell and use that.
417   * @param c    the environment provided by the region server
418   * @param put  The Put object
419   * @param edit The WALEdit object that will be written to the wal
420   */
421  default void prePut(ObserverContext<? extends RegionCoprocessorEnvironment> c, Put put,
422    WALEdit edit) throws IOException {
423    prePut(c, put, edit, put.getDurability());
424  }
425
426  /**
427   * Called after the client stores a value.
428   * <p>
429   * Note: Do not retain references to any Cells in 'put' beyond the life of this invocation. If
430   * need a Cell reference for later use, copy the cell and use that.
431   * @param c          the environment provided by the region server
432   * @param put        The Put object
433   * @param edit       The WALEdit object for the wal
434   * @param durability Persistence guarantee for this Put
435   * @deprecated since 3.0.0 and will be removed in 4.0.0. Use
436   *             {@link #postPut(ObserverContext, Put, WALEdit)} instead.
437   */
438  @Deprecated
439  default void postPut(ObserverContext<? extends RegionCoprocessorEnvironment> c, Put put,
440    WALEdit edit, Durability durability) throws IOException {
441  }
442
443  /**
444   * Called after the client stores a value.
445   * <p>
446   * Note: Do not retain references to any Cells in 'put' beyond the life of this invocation. If
447   * need a Cell reference for later use, copy the cell and use that.
448   * @param c    the environment provided by the region server
449   * @param put  The Put object
450   * @param edit The WALEdit object for the wal
451   */
452  default void postPut(ObserverContext<? extends RegionCoprocessorEnvironment> c, Put put,
453    WALEdit edit) throws IOException {
454    postPut(c, put, edit, put.getDurability());
455  }
456
457  /**
458   * Called before the client deletes a value.
459   * <p>
460   * Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on
461   * calling any subsequent chained coprocessors.
462   * <p>
463   * Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation. If
464   * need a Cell reference for later use, copy the cell and use that.
465   * @param c          the environment provided by the region server
466   * @param delete     The Delete object
467   * @param edit       The WALEdit object for the wal
468   * @param durability Persistence guarantee for this Delete
469   * @deprecated since 3.0.0 and will be removed in 4.0.0. Use
470   *             {@link #preDelete(ObserverContext, Delete, WALEdit)} instead.
471   */
472  @Deprecated
473  default void preDelete(ObserverContext<? extends RegionCoprocessorEnvironment> c, Delete delete,
474    WALEdit edit, Durability durability) throws IOException {
475  }
476
477  /**
478   * Called before the client deletes a value.
479   * <p>
480   * Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on
481   * calling any subsequent chained coprocessors.
482   * <p>
483   * Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation. If
484   * need a Cell reference for later use, copy the cell and use that.
485   * @param c      the environment provided by the region server
486   * @param delete The Delete object
487   * @param edit   The WALEdit object for the wal
488   */
489  default void preDelete(ObserverContext<? extends RegionCoprocessorEnvironment> c, Delete delete,
490    WALEdit edit) throws IOException {
491    preDelete(c, delete, edit, delete.getDurability());
492  }
493
494  /**
495   * Called before the server updates the timestamp for version delete with latest timestamp.
496   * <p>
497   * Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on
498   * calling any subsequent chained coprocessors.
499   * @param c        the environment provided by the region server
500   * @param mutation - the parent mutation associated with this delete cell
501   * @param cell     - The deleteColumn with latest version cell
502   * @param byteNow  - timestamp bytes
503   * @param get      - the get formed using the current cell's row. Note that the get does not
504   *                 specify the family and qualifier
505   * @deprecated Since hbase-2.0.0. No replacement. To be removed in hbase-4.0.0 and replaced with
506   *             something that doesn't expose IntefaceAudience.Private classes.
507   *             VisibilityController still needs this, need to change the logic there first.
508   */
509  @Deprecated
510  default void prePrepareTimeStampForDeleteVersion(
511    ObserverContext<? extends RegionCoprocessorEnvironment> c, Mutation mutation, Cell cell,
512    byte[] byteNow, Get get) throws IOException {
513  }
514
515  /**
516   * Called after the client deletes a value.
517   * <p>
518   * Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation. If
519   * need a Cell reference for later use, copy the cell and use that.
520   * @param c          the environment provided by the region server
521   * @param delete     The Delete object
522   * @param edit       The WALEdit object for the wal
523   * @param durability Persistence guarantee for this Delete
524   * @deprecated since 3.0.0 and will be removed in 4.0.0. Use
525   *             {@link #postDelete(ObserverContext, Delete, WALEdit)} instead.
526   */
527  @Deprecated
528  default void postDelete(ObserverContext<? extends RegionCoprocessorEnvironment> c, Delete delete,
529    WALEdit edit, Durability durability) throws IOException {
530  }
531
532  /**
533   * Called after the client deletes a value.
534   * <p>
535   * Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation. If
536   * need a Cell reference for later use, copy the cell and use that.
537   * @param c      the environment provided by the region server
538   * @param delete The Delete object
539   * @param edit   The WALEdit object for the wal
540   */
541  default void postDelete(ObserverContext<? extends RegionCoprocessorEnvironment> c, Delete delete,
542    WALEdit edit) throws IOException {
543    postDelete(c, delete, edit, delete.getDurability());
544  }
545
546  /**
547   * This will be called for every batch mutation operation happening at the server. This will be
548   * called after acquiring the locks on the mutating rows and after applying the proper timestamp
549   * for each Mutation at the server. The batch may contain Put/Delete/Increment/Append. By setting
550   * OperationStatus of Mutations
551   * ({@link MiniBatchOperationInProgress#setOperationStatus(int, OperationStatus)}),
552   * {@link RegionObserver} can make Region to skip these Mutations.
553   * <p>
554   * Note: Do not retain references to any Cells in Mutations beyond the life of this invocation. If
555   * need a Cell reference for later use, copy the cell and use that.
556   * @param c           the environment provided by the region server
557   * @param miniBatchOp batch of Mutations getting applied to region.
558   */
559  default void preBatchMutate(ObserverContext<? extends RegionCoprocessorEnvironment> c,
560    MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
561  }
562
563  /**
564   * This will be called after applying a batch of Mutations on a region. The Mutations are added to
565   * memstore and WAL. The difference of this one with
566   * {@link #postPut(ObserverContext, Put, WALEdit)} and
567   * {@link #postDelete(ObserverContext, Delete, WALEdit)} and
568   * {@link #postIncrement(ObserverContext, Increment, Result, WALEdit)} and
569   * {@link #postAppend(ObserverContext, Append, Result, WALEdit)} is this hook will be executed
570   * before the mvcc transaction completion.
571   * <p>
572   * Note: Do not retain references to any Cells in Mutations beyond the life of this invocation. If
573   * need a Cell reference for later use, copy the cell and use that.
574   * @param c           the environment provided by the region server
575   * @param miniBatchOp batch of Mutations applied to region. Coprocessors are discouraged from
576   *                    manipulating its state.
577   */
578  // Coprocessors can do a form of bypass by changing state in miniBatchOp.
579  default void postBatchMutate(ObserverContext<? extends RegionCoprocessorEnvironment> c,
580    MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
581  }
582
583  /**
584   * This will be called for region operations where read lock is acquired in
585   * {@link Region#startRegionOperation()}.
586   * @param operation The operation is about to be taken on the region
587   */
588  default void postStartRegionOperation(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
589    Operation operation) throws IOException {
590  }
591
592  /**
593   * Called after releasing read lock in {@link Region#closeRegionOperation()}.
594   */
595  default void postCloseRegionOperation(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
596    Operation operation) throws IOException {
597  }
598
599  /**
600   * Called after the completion of batch put/delete/increment/append and will be called even if the
601   * batch operation fails.
602   * <p>
603   * Note: Do not retain references to any Cells in Mutations beyond the life of this invocation. If
604   * need a Cell reference for later use, copy the cell and use that.
605   * @param success true if batch operation is successful otherwise false.
606   */
607  default void postBatchMutateIndispensably(
608    ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
609    MiniBatchOperationInProgress<Mutation> miniBatchOp, boolean success) throws IOException {
610  }
611
612  /**
613   * Called before checkAndPut.
614   * <p>
615   * Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on
616   * calling any subsequent chained coprocessors.
617   * <p>
618   * Note: Do not retain references to any Cells in 'put' beyond the life of this invocation. If
619   * need a Cell reference for later use, copy the cell and use that.
620   * @param c          the environment provided by the region server
621   * @param row        row to check
622   * @param family     column family
623   * @param qualifier  column qualifier
624   * @param op         the comparison operation
625   * @param comparator the comparator
626   * @param put        data to put if check succeeds
627   * @param result     the default value of the result
628   * @return the return value to return to client if bypassing default processing
629   * @deprecated since 3.0.0 and will be removed in 4.0.0. Use
630   *             {@link #preCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult)}
631   *             instead.
632   */
633  @Deprecated
634  default boolean preCheckAndPut(ObserverContext<? extends RegionCoprocessorEnvironment> c,
635    byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator,
636    Put put, boolean result) throws IOException {
637    return result;
638  }
639
640  /**
641   * Called before checkAndPut.
642   * <p>
643   * Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on
644   * calling any subsequent chained coprocessors.
645   * <p>
646   * Note: Do not retain references to any Cells in 'put' beyond the life of this invocation. If
647   * need a Cell reference for later use, copy the cell and use that.
648   * @param c      the environment provided by the region server
649   * @param row    row to check
650   * @param filter filter
651   * @param put    data to put if check succeeds
652   * @param result the default value of the result
653   * @return the return value to return to client if bypassing default processing
654   * @deprecated since 3.0.0 and will be removed in 4.0.0. Use
655   *             {@link #preCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult)}
656   *             instead.
657   */
658  @Deprecated
659  default boolean preCheckAndPut(ObserverContext<? extends RegionCoprocessorEnvironment> c,
660    byte[] row, Filter filter, Put put, boolean result) throws IOException {
661    return result;
662  }
663
664  /**
665   * Called before checkAndPut but after acquiring rowlock.
666   * <p>
667   * <b>Note:</b> Caution to be taken for not doing any long time operation in this hook. Row will
668   * be locked for longer time. Trying to acquire lock on another row, within this, can lead to
669   * potential deadlock.
670   * <p>
671   * Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on
672   * calling any subsequent chained coprocessors.
673   * <p>
674   * Note: Do not retain references to any Cells in 'put' beyond the life of this invocation. If
675   * need a Cell reference for later use, copy the cell and use that.
676   * @param c          the environment provided by the region server
677   * @param row        row to check
678   * @param family     column family
679   * @param qualifier  column qualifier
680   * @param op         the comparison operation
681   * @param comparator the comparator
682   * @param put        data to put if check succeeds
683   * @param result     the default value of the result
684   * @return the return value to return to client if bypassing default processing
685   * @deprecated since 3.0.0 and will be removed in 4.0.0. Use
686   *             {@link #preCheckAndMutateAfterRowLock(ObserverContext, CheckAndMutate,CheckAndMutateResult)}
687   *             instead.
688   */
689  @Deprecated
690  default boolean preCheckAndPutAfterRowLock(
691    ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, byte[] family,
692    byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Put put, boolean result)
693    throws IOException {
694    return result;
695  }
696
697  /**
698   * Called before checkAndPut but after acquiring rowlock.
699   * <p>
700   * <b>Note:</b> Caution to be taken for not doing any long time operation in this hook. Row will
701   * be locked for longer time. Trying to acquire lock on another row, within this, can lead to
702   * potential deadlock.
703   * <p>
704   * Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on
705   * calling any subsequent chained coprocessors.
706   * <p>
707   * Note: Do not retain references to any Cells in 'put' beyond the life of this invocation. If
708   * need a Cell reference for later use, copy the cell and use that.
709   * @param c      the environment provided by the region server
710   * @param row    row to check
711   * @param filter filter
712   * @param put    data to put if check succeeds
713   * @param result the default value of the result
714   * @return the return value to return to client if bypassing default processing
715   * @deprecated since 3.0.0 and will be removed in 4.0.0. Use
716   *             {@link #preCheckAndMutateAfterRowLock(ObserverContext, CheckAndMutate,CheckAndMutateResult)}
717   *             instead.
718   */
719  @Deprecated
720  default boolean preCheckAndPutAfterRowLock(
721    ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, Filter filter, Put put,
722    boolean result) throws IOException {
723    return result;
724  }
725
726  /**
727   * Called after checkAndPut
728   * <p>
729   * Note: Do not retain references to any Cells in 'put' beyond the life of this invocation. If
730   * need a Cell reference for later use, copy the cell and use that.
731   * @param c          the environment provided by the region server
732   * @param row        row to check
733   * @param family     column family
734   * @param qualifier  column qualifier
735   * @param op         the comparison operation
736   * @param comparator the comparator
737   * @param put        data to put if check succeeds
738   * @param result     from the checkAndPut
739   * @return the possibly transformed return value to return to client
740   * @deprecated since 3.0.0 and will be removed in 4.0.0. Use
741   *             {@link #postCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult)}
742   *             instead.
743   */
744  @Deprecated
745  default boolean postCheckAndPut(ObserverContext<? extends RegionCoprocessorEnvironment> c,
746    byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator,
747    Put put, boolean result) throws IOException {
748    return result;
749  }
750
751  /**
752   * Called after checkAndPut
753   * <p>
754   * Note: Do not retain references to any Cells in 'put' beyond the life of this invocation. If
755   * need a Cell reference for later use, copy the cell and use that.
756   * @param c      the environment provided by the region server
757   * @param row    row to check
758   * @param filter filter
759   * @param put    data to put if check succeeds
760   * @param result from the checkAndPut
761   * @return the possibly transformed return value to return to client
762   * @deprecated since 3.0.0 and will be removed in 4.0.0. Use
763   *             {@link #postCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult)}
764   *             instead.
765   */
766  @Deprecated
767  default boolean postCheckAndPut(ObserverContext<? extends RegionCoprocessorEnvironment> c,
768    byte[] row, Filter filter, Put put, boolean result) throws IOException {
769    return result;
770  }
771
772  /**
773   * Called before checkAndDelete.
774   * <p>
775   * Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on
776   * calling any subsequent chained coprocessors.
777   * <p>
778   * Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation. If
779   * need a Cell reference for later use, copy the cell and use that.
780   * @param c          the environment provided by the region server
781   * @param row        row to check
782   * @param family     column family
783   * @param qualifier  column qualifier
784   * @param op         the comparison operation
785   * @param comparator the comparator
786   * @param delete     delete to commit if check succeeds
787   * @param result     the default value of the result
788   * @return the value to return to client if bypassing default processing
789   * @deprecated since 3.0.0 and will be removed in 4.0.0. Use
790   *             {@link #preCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult)}
791   *             instead.
792   */
793  @Deprecated
794  default boolean preCheckAndDelete(ObserverContext<? extends RegionCoprocessorEnvironment> c,
795    byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator,
796    Delete delete, boolean result) throws IOException {
797    return result;
798  }
799
800  /**
801   * Called before checkAndDelete.
802   * <p>
803   * Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on
804   * calling any subsequent chained coprocessors.
805   * <p>
806   * Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation. If
807   * need a Cell reference for later use, copy the cell and use that.
808   * @param c      the environment provided by the region server
809   * @param row    row to check
810   * @param filter column family
811   * @param delete delete to commit if check succeeds
812   * @param result the default value of the result
813   * @return the value to return to client if bypassing default processing
814   * @deprecated since 3.0.0 and will be removed in 4.0.0. Use
815   *             {@link #preCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult)}
816   *             instead.
817   */
818  @Deprecated
819  default boolean preCheckAndDelete(ObserverContext<? extends RegionCoprocessorEnvironment> c,
820    byte[] row, Filter filter, Delete delete, boolean result) throws IOException {
821    return result;
822  }
823
824  /**
825   * Called before checkAndDelete but after acquiring rowock.
826   * <p>
827   * <b>Note:</b> Caution to be taken for not doing any long time operation in this hook. Row will
828   * be locked for longer time. Trying to acquire lock on another row, within this, can lead to
829   * potential deadlock.
830   * <p>
831   * Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on
832   * calling any subsequent chained coprocessors.
833   * <p>
834   * Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation. If
835   * need a Cell reference for later use, copy the cell and use that.
836   * @param c          the environment provided by the region server
837   * @param row        row to check
838   * @param family     column family
839   * @param qualifier  column qualifier
840   * @param op         the comparison operation
841   * @param comparator the comparator
842   * @param delete     delete to commit if check succeeds
843   * @param result     the default value of the result
844   * @return the value to return to client if bypassing default processing
845   * @deprecated since 3.0.0 and will be removed in 4.0.0. Use
846   *             {@link #preCheckAndMutateAfterRowLock(ObserverContext, CheckAndMutate,CheckAndMutateResult)}
847   *             instead.
848   */
849  @Deprecated
850  default boolean preCheckAndDeleteAfterRowLock(
851    ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, byte[] family,
852    byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Delete delete,
853    boolean result) throws IOException {
854    return result;
855  }
856
857  /**
858   * Called before checkAndDelete but after acquiring rowock.
859   * <p>
860   * <b>Note:</b> Caution to be taken for not doing any long time operation in this hook. Row will
861   * be locked for longer time. Trying to acquire lock on another row, within this, can lead to
862   * potential deadlock.
863   * <p>
864   * Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on
865   * calling any subsequent chained coprocessors.
866   * <p>
867   * Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation. If
868   * need a Cell reference for later use, copy the cell and use that.
869   * @param c      the environment provided by the region server
870   * @param row    row to check
871   * @param filter filter
872   * @param delete delete to commit if check succeeds
873   * @param result the default value of the result
874   * @return the value to return to client if bypassing default processing
875   * @deprecated since 3.0.0 and will be removed in 4.0.0. Use
876   *             {@link #preCheckAndMutateAfterRowLock(ObserverContext, CheckAndMutate,CheckAndMutateResult)}
877   *             instead.
878   */
879  @Deprecated
880  default boolean preCheckAndDeleteAfterRowLock(
881    ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, Filter filter,
882    Delete delete, boolean result) throws IOException {
883    return result;
884  }
885
886  /**
887   * Called after checkAndDelete
888   * <p>
889   * Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation. If
890   * need a Cell reference for later use, copy the cell and use that.
891   * @param c          the environment provided by the region server
892   * @param row        row to check
893   * @param family     column family
894   * @param qualifier  column qualifier
895   * @param op         the comparison operation
896   * @param comparator the comparator
897   * @param delete     delete to commit if check succeeds
898   * @param result     from the CheckAndDelete
899   * @return the possibly transformed returned value to return to client
900   * @deprecated since 3.0.0 and will be removed in 4.0.0. Use
901   *             {@link #postCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult)}
902   *             instead.
903   */
904  @Deprecated
905  default boolean postCheckAndDelete(ObserverContext<? extends RegionCoprocessorEnvironment> c,
906    byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator,
907    Delete delete, boolean result) throws IOException {
908    return result;
909  }
910
911  /**
912   * Called after checkAndDelete
913   * <p>
914   * Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation. If
915   * need a Cell reference for later use, copy the cell and use that.
916   * @param c      the environment provided by the region server
917   * @param row    row to check
918   * @param filter filter
919   * @param delete delete to commit if check succeeds
920   * @param result from the CheckAndDelete
921   * @return the possibly transformed returned value to return to client
922   * @deprecated since 3.0.0 and will be removed in 4.0.0. Use
923   *             {@link #postCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult)}
924   *             instead.
925   */
926  @Deprecated
927  default boolean postCheckAndDelete(ObserverContext<? extends RegionCoprocessorEnvironment> c,
928    byte[] row, Filter filter, Delete delete, boolean result) throws IOException {
929    return result;
930  }
931
932  /**
933   * Called before checkAndMutate
934   * <p>
935   * Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on
936   * calling any subsequent chained coprocessors.
937   * <p>
938   * Note: Do not retain references to any Cells in actions beyond the life of this invocation. If
939   * need a Cell reference for later use, copy the cell and use that.
940   * @param c              the environment provided by the region server
941   * @param checkAndMutate the CheckAndMutate object
942   * @param result         the default value of the result
943   * @return the return value to return to client if bypassing default processing
944   * @throws IOException if an error occurred on the coprocessor
945   */
946  default CheckAndMutateResult preCheckAndMutate(
947    ObserverContext<? extends RegionCoprocessorEnvironment> c, CheckAndMutate checkAndMutate,
948    CheckAndMutateResult result) throws IOException {
949    if (checkAndMutate.getAction() instanceof Put) {
950      boolean success;
951      if (checkAndMutate.hasFilter()) {
952        success = preCheckAndPut(c, checkAndMutate.getRow(), checkAndMutate.getFilter(),
953          (Put) checkAndMutate.getAction(), result.isSuccess());
954      } else {
955        success = preCheckAndPut(c, checkAndMutate.getRow(), checkAndMutate.getFamily(),
956          checkAndMutate.getQualifier(), checkAndMutate.getCompareOp(),
957          new BinaryComparator(checkAndMutate.getValue()), (Put) checkAndMutate.getAction(),
958          result.isSuccess());
959      }
960      return new CheckAndMutateResult(success, null);
961    } else if (checkAndMutate.getAction() instanceof Delete) {
962      boolean success;
963      if (checkAndMutate.hasFilter()) {
964        success = preCheckAndDelete(c, checkAndMutate.getRow(), checkAndMutate.getFilter(),
965          (Delete) checkAndMutate.getAction(), result.isSuccess());
966      } else {
967        success = preCheckAndDelete(c, checkAndMutate.getRow(), checkAndMutate.getFamily(),
968          checkAndMutate.getQualifier(), checkAndMutate.getCompareOp(),
969          new BinaryComparator(checkAndMutate.getValue()), (Delete) checkAndMutate.getAction(),
970          result.isSuccess());
971      }
972      return new CheckAndMutateResult(success, null);
973    }
974    return result;
975  }
976
977  /**
978   * Called before checkAndDelete but after acquiring rowlock.
979   * <p>
980   * <b>Note:</b> Caution to be taken for not doing any long time operation in this hook. Row will
981   * be locked for longer time. Trying to acquire lock on another row, within this, can lead to
982   * potential deadlock.
983   * <p>
984   * Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on
985   * calling any subsequent chained coprocessors.
986   * <p>
987   * Note: Do not retain references to any Cells in actions beyond the life of this invocation. If
988   * need a Cell reference for later use, copy the cell and use that.
989   * @param c              the environment provided by the region server
990   * @param checkAndMutate the CheckAndMutate object
991   * @param result         the default value of the result
992   * @return the value to return to client if bypassing default processing
993   * @throws IOException if an error occurred on the coprocessor
994   */
995  default CheckAndMutateResult preCheckAndMutateAfterRowLock(
996    ObserverContext<? extends RegionCoprocessorEnvironment> c, CheckAndMutate checkAndMutate,
997    CheckAndMutateResult result) throws IOException {
998    if (checkAndMutate.getAction() instanceof Put) {
999      boolean success;
1000      if (checkAndMutate.hasFilter()) {
1001        success = preCheckAndPutAfterRowLock(c, checkAndMutate.getRow(), checkAndMutate.getFilter(),
1002          (Put) checkAndMutate.getAction(), result.isSuccess());
1003      } else {
1004        success = preCheckAndPutAfterRowLock(c, checkAndMutate.getRow(), checkAndMutate.getFamily(),
1005          checkAndMutate.getQualifier(), checkAndMutate.getCompareOp(),
1006          new BinaryComparator(checkAndMutate.getValue()), (Put) checkAndMutate.getAction(),
1007          result.isSuccess());
1008      }
1009      return new CheckAndMutateResult(success, null);
1010    } else if (checkAndMutate.getAction() instanceof Delete) {
1011      boolean success;
1012      if (checkAndMutate.hasFilter()) {
1013        success = preCheckAndDeleteAfterRowLock(c, checkAndMutate.getRow(),
1014          checkAndMutate.getFilter(), (Delete) checkAndMutate.getAction(), result.isSuccess());
1015      } else {
1016        success = preCheckAndDeleteAfterRowLock(c, checkAndMutate.getRow(),
1017          checkAndMutate.getFamily(), checkAndMutate.getQualifier(), checkAndMutate.getCompareOp(),
1018          new BinaryComparator(checkAndMutate.getValue()), (Delete) checkAndMutate.getAction(),
1019          result.isSuccess());
1020      }
1021      return new CheckAndMutateResult(success, null);
1022    }
1023    return result;
1024  }
1025
1026  /**
1027   * Called after checkAndMutate
1028   * <p>
1029   * Note: Do not retain references to any Cells in actions beyond the life of this invocation. If
1030   * need a Cell reference for later use, copy the cell and use that.
1031   * @param c              the environment provided by the region server
1032   * @param checkAndMutate the CheckAndMutate object
1033   * @param result         from the checkAndMutate
1034   * @return the possibly transformed returned value to return to client
1035   * @throws IOException if an error occurred on the coprocessor
1036   */
1037  default CheckAndMutateResult postCheckAndMutate(
1038    ObserverContext<? extends RegionCoprocessorEnvironment> c, CheckAndMutate checkAndMutate,
1039    CheckAndMutateResult result) throws IOException {
1040    if (checkAndMutate.getAction() instanceof Put) {
1041      boolean success;
1042      if (checkAndMutate.hasFilter()) {
1043        success = postCheckAndPut(c, checkAndMutate.getRow(), checkAndMutate.getFilter(),
1044          (Put) checkAndMutate.getAction(), result.isSuccess());
1045      } else {
1046        success = postCheckAndPut(c, checkAndMutate.getRow(), checkAndMutate.getFamily(),
1047          checkAndMutate.getQualifier(), checkAndMutate.getCompareOp(),
1048          new BinaryComparator(checkAndMutate.getValue()), (Put) checkAndMutate.getAction(),
1049          result.isSuccess());
1050      }
1051      return new CheckAndMutateResult(success, null);
1052    } else if (checkAndMutate.getAction() instanceof Delete) {
1053      boolean success;
1054      if (checkAndMutate.hasFilter()) {
1055        success = postCheckAndDelete(c, checkAndMutate.getRow(), checkAndMutate.getFilter(),
1056          (Delete) checkAndMutate.getAction(), result.isSuccess());
1057      } else {
1058        success = postCheckAndDelete(c, checkAndMutate.getRow(), checkAndMutate.getFamily(),
1059          checkAndMutate.getQualifier(), checkAndMutate.getCompareOp(),
1060          new BinaryComparator(checkAndMutate.getValue()), (Delete) checkAndMutate.getAction(),
1061          result.isSuccess());
1062      }
1063      return new CheckAndMutateResult(success, null);
1064    }
1065    return result;
1066  }
1067
1068  /**
1069   * Called before Append.
1070   * <p>
1071   * Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on
1072   * calling any subsequent chained coprocessors.
1073   * <p>
1074   * Note: Do not retain references to any Cells in 'append' beyond the life of this invocation. If
1075   * need a Cell reference for later use, copy the cell and use that.
1076   * @param c      the environment provided by the region server
1077   * @param append Append object
1078   * @return result to return to the client if bypassing default processing
1079   * @deprecated since 3.0.0 and will be removed in 4.0.0. Use
1080   *             {@link #preAppend(ObserverContext, Append, WALEdit)} instead.
1081   */
1082  @Deprecated
1083  default Result preAppend(ObserverContext<? extends RegionCoprocessorEnvironment> c, Append append)
1084    throws IOException {
1085    return null;
1086  }
1087
1088  /**
1089   * Called before Append.
1090   * <p>
1091   * Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on
1092   * calling any subsequent chained coprocessors.
1093   * <p>
1094   * Note: Do not retain references to any Cells in 'append' beyond the life of this invocation. If
1095   * need a Cell reference for later use, copy the cell and use that.
1096   * @param c      the environment provided by the region server
1097   * @param append Append object
1098   * @param edit   The WALEdit object that will be written to the wal
1099   * @return result to return to the client if bypassing default processing
1100   */
1101  default Result preAppend(ObserverContext<? extends RegionCoprocessorEnvironment> c, Append append,
1102    WALEdit edit) throws IOException {
1103    return preAppend(c, append);
1104  }
1105
1106  /**
1107   * Called before Append but after acquiring rowlock.
1108   * <p>
1109   * <b>Note:</b> Caution to be taken for not doing any long time operation in this hook. Row will
1110   * be locked for longer time. Trying to acquire lock on another row, within this, can lead to
1111   * potential deadlock.
1112   * <p>
1113   * Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on
1114   * calling any subsequent chained coprocessors.
1115   * <p>
1116   * Note: Do not retain references to any Cells in 'append' beyond the life of this invocation. If
1117   * need a Cell reference for later use, copy the cell and use that.
1118   * @param c      the environment provided by the region server
1119   * @param append Append object
1120   * @return result to return to the client if bypassing default processing
1121   * @deprecated since 3.0.0 and will be removed in 4.0.0. Use
1122   *             {@link #preBatchMutate(ObserverContext, MiniBatchOperationInProgress)} instead.
1123   */
1124  @Deprecated
1125  default Result preAppendAfterRowLock(ObserverContext<? extends RegionCoprocessorEnvironment> c,
1126    Append append) throws IOException {
1127    return null;
1128  }
1129
1130  /**
1131   * Called after Append
1132   * <p>
1133   * Note: Do not retain references to any Cells in 'append' beyond the life of this invocation. If
1134   * need a Cell reference for later use, copy the cell and use that.
1135   * @param c      the environment provided by the region server
1136   * @param append Append object
1137   * @param result the result returned by increment
1138   * @return the result to return to the client
1139   * @deprecated since 3.0.0 and will be removed in 4.0.0. Use
1140   *             {@link #postAppend(ObserverContext, Append, Result, WALEdit)} instead.
1141   */
1142  @Deprecated
1143  default Result postAppend(ObserverContext<? extends RegionCoprocessorEnvironment> c,
1144    Append append, Result result) throws IOException {
1145    return result;
1146  }
1147
1148  /**
1149   * Called after Append
1150   * <p>
1151   * Note: Do not retain references to any Cells in 'append' beyond the life of this invocation. If
1152   * need a Cell reference for later use, copy the cell and use that.
1153   * @param c      the environment provided by the region server
1154   * @param append Append object
1155   * @param result the result returned by increment
1156   * @param edit   The WALEdit object for the wal
1157   * @return the result to return to the client
1158   */
1159  default Result postAppend(ObserverContext<? extends RegionCoprocessorEnvironment> c,
1160    Append append, Result result, WALEdit edit) throws IOException {
1161    return postAppend(c, append, result);
1162  }
1163
1164  /**
1165   * Called before Increment.
1166   * <p>
1167   * Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on
1168   * calling any subsequent chained coprocessors.
1169   * <p>
1170   * Note: Do not retain references to any Cells in 'increment' beyond the life of this invocation.
1171   * If need a Cell reference for later use, copy the cell and use that.
1172   * @param c         the environment provided by the region server
1173   * @param increment increment object
1174   * @return result to return to the client if bypassing default processing
1175   * @deprecated since 3.0.0 and will be removed in 4.0.0. Use
1176   *             {@link #preIncrement(ObserverContext, Increment, WALEdit)} instead.
1177   */
1178  @Deprecated
1179  default Result preIncrement(ObserverContext<? extends RegionCoprocessorEnvironment> c,
1180    Increment increment) throws IOException {
1181    return null;
1182  }
1183
1184  /**
1185   * Called before Increment.
1186   * <p>
1187   * Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on
1188   * calling any subsequent chained coprocessors.
1189   * <p>
1190   * Note: Do not retain references to any Cells in 'increment' beyond the life of this invocation.
1191   * If need a Cell reference for later use, copy the cell and use that.
1192   * @param c         the environment provided by the region server
1193   * @param increment increment object
1194   * @param edit      The WALEdit object that will be written to the wal
1195   * @return result to return to the client if bypassing default processing
1196   */
1197  default Result preIncrement(ObserverContext<? extends RegionCoprocessorEnvironment> c,
1198    Increment increment, WALEdit edit) throws IOException {
1199    return preIncrement(c, increment);
1200  }
1201
1202  /**
1203   * Called before Increment but after acquiring rowlock.
1204   * <p>
1205   * <b>Note:</b> Caution to be taken for not doing any long time operation in this hook. Row will
1206   * be locked for longer time. Trying to acquire lock on another row, within this, can lead to
1207   * potential deadlock.
1208   * <p>
1209   * Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on
1210   * calling any subsequent chained coprocessors.
1211   * <p>
1212   * Note: Do not retain references to any Cells in 'increment' beyond the life of this invocation.
1213   * If need a Cell reference for later use, copy the cell and use that.
1214   * @param c         the environment provided by the region server
1215   * @param increment increment object
1216   * @return result to return to the client if bypassing default processing
1217   * @deprecated since 3.0.0 and will be removed in 4.0.0. Use
1218   *             {@link #preBatchMutate(ObserverContext, MiniBatchOperationInProgress)} instead.
1219   */
1220  @Deprecated
1221  default Result preIncrementAfterRowLock(ObserverContext<? extends RegionCoprocessorEnvironment> c,
1222    Increment increment) throws IOException {
1223    return null;
1224  }
1225
1226  /**
1227   * Called after increment
1228   * <p>
1229   * Note: Do not retain references to any Cells in 'increment' beyond the life of this invocation.
1230   * If need a Cell reference for later use, copy the cell and use that.
1231   * @param c         the environment provided by the region server
1232   * @param increment increment object
1233   * @param result    the result returned by increment
1234   * @return the result to return to the client
1235   * @deprecated since 3.0.0 and will be removed in 4.0.0. Use
1236   *             {@link #postIncrement(ObserverContext, Increment, Result, WALEdit)} instead.
1237   */
1238  @Deprecated
1239  default Result postIncrement(ObserverContext<? extends RegionCoprocessorEnvironment> c,
1240    Increment increment, Result result) throws IOException {
1241    return result;
1242  }
1243
1244  /**
1245   * Called after increment
1246   * <p>
1247   * Note: Do not retain references to any Cells in 'increment' beyond the life of this invocation.
1248   * If need a Cell reference for later use, copy the cell and use that.
1249   * @param c         the environment provided by the region server
1250   * @param increment increment object
1251   * @param result    the result returned by increment
1252   * @param edit      The WALEdit object for the wal
1253   * @return the result to return to the client
1254   */
1255  default Result postIncrement(ObserverContext<? extends RegionCoprocessorEnvironment> c,
1256    Increment increment, Result result, WALEdit edit) throws IOException {
1257    return postIncrement(c, increment, result);
1258  }
1259
1260  /**
1261   * Called before the client opens a new scanner.
1262   * <p>
1263   * Note: Do not retain references to any Cells returned by scanner, beyond the life of this
1264   * invocation. If need a Cell reference for later use, copy the cell and use that.
1265   * @param c    the environment provided by the region server
1266   * @param scan the Scan specification
1267   */
1268  default void preScannerOpen(ObserverContext<? extends RegionCoprocessorEnvironment> c, Scan scan)
1269    throws IOException {
1270  }
1271
1272  /**
1273   * Called after the client opens a new scanner.
1274   * <p>
1275   * Note: Do not retain references to any Cells returned by scanner, beyond the life of this
1276   * invocation. If need a Cell reference for later use, copy the cell and use that.
1277   * @param c    the environment provided by the region server
1278   * @param scan the Scan specification
1279   * @param s    if not null, the base scanner
1280   * @return the scanner instance to use
1281   */
1282  default RegionScanner postScannerOpen(ObserverContext<? extends RegionCoprocessorEnvironment> c,
1283    Scan scan, RegionScanner s) throws IOException {
1284    return s;
1285  }
1286
1287  /**
1288   * Called before the client asks for the next row on a scanner.
1289   * <p>
1290   * Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on
1291   * calling any subsequent chained coprocessors.
1292   * <p>
1293   * Note: Do not retain references to any Cells returned by scanner, beyond the life of this
1294   * invocation. If need a Cell reference for later use, copy the cell and use that.
1295   * @param c       the environment provided by the region server
1296   * @param s       the scanner
1297   * @param result  The result to return to the client if default processing is bypassed. Can be
1298   *                modified. Will not be returned if default processing is not bypassed.
1299   * @param limit   the maximum number of results to return
1300   * @param hasNext the 'has more' indication
1301   * @return 'has more' indication that should be sent to client
1302   */
1303  default boolean preScannerNext(ObserverContext<? extends RegionCoprocessorEnvironment> c,
1304    InternalScanner s, List<Result> result, int limit, boolean hasNext) throws IOException {
1305    return hasNext;
1306  }
1307
1308  /**
1309   * Called after the client asks for the next row on a scanner.
1310   * <p>
1311   * Note: Do not retain references to any Cells returned by scanner, beyond the life of this
1312   * invocation. If need a Cell reference for later use, copy the cell and use that.
1313   * @param c       the environment provided by the region server
1314   * @param s       the scanner
1315   * @param result  the result to return to the client, can be modified
1316   * @param limit   the maximum number of results to return
1317   * @param hasNext the 'has more' indication
1318   * @return 'has more' indication that should be sent to client
1319   */
1320  default boolean postScannerNext(ObserverContext<? extends RegionCoprocessorEnvironment> c,
1321    InternalScanner s, List<Result> result, int limit, boolean hasNext) throws IOException {
1322    return hasNext;
1323  }
1324
1325  /**
1326   * This will be called by the scan flow when the current scanned row is being filtered out by the
1327   * filter. The filter may be filtering out the row via any of the below scenarios
1328   * <ol>
1329   * <li><code>boolean filterRowKey(byte [] buffer, int offset, int length)</code> returning
1330   * true</li>
1331   * <li><code>boolean filterRow()</code> returning true</li>
1332   * <li><code>default void filterRow(List&lt;KeyValue&gt; kvs)</code> removing all the kvs from the
1333   * passed List</li>
1334   * </ol>
1335   * <p>
1336   * Note: Do not retain references to any Cells returned by scanner, beyond the life of this
1337   * invocation. If need a Cell reference for later use, copy the cell and use that.
1338   * @param c          the environment provided by the region server
1339   * @param s          the scanner
1340   * @param curRowCell The cell in the current row which got filtered out
1341   * @param hasMore    the 'has more' indication
1342   * @return whether more rows are available for the scanner or not
1343   */
1344  default boolean postScannerFilterRow(ObserverContext<? extends RegionCoprocessorEnvironment> c,
1345    InternalScanner s, Cell curRowCell, boolean hasMore) throws IOException {
1346    return hasMore;
1347  }
1348
1349  /**
1350   * Called before the client closes a scanner.
1351   * <p>
1352   * Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on
1353   * calling any subsequent chained coprocessors.
1354   * @param c the environment provided by the region server
1355   * @param s the scanner
1356   */
1357  default void preScannerClose(ObserverContext<? extends RegionCoprocessorEnvironment> c,
1358    InternalScanner s) throws IOException {
1359  }
1360
1361  /**
1362   * Called after the client closes a scanner.
1363   * @param ctx the environment provided by the region server
1364   * @param s   the scanner
1365   */
1366  default void postScannerClose(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
1367    InternalScanner s) throws IOException {
1368  }
1369
1370  /**
1371   * Called before a store opens a new scanner.
1372   * <p>
1373   * This hook is called when a "user" scanner is opened. Use {@code preFlushScannerOpen} and
1374   * {@code preCompactScannerOpen} to inject flush/compaction.
1375   * <p>
1376   * Notice that, this method is used to change the inherent max versions and TTL for a Store. For
1377   * example, you can change the max versions option for a {@link Scan} object to 10 in
1378   * {@code preScannerOpen}, but if the max versions config on the Store is 1, then you still can
1379   * only read 1 version. You need also to inject here to change the max versions to 10 if you want
1380   * to get more versions.
1381   * @param ctx     the environment provided by the region server
1382   * @param store   the store which we want to get scanner from
1383   * @param options used to change max versions and TTL for the scanner being opened
1384   * @see #preFlushScannerOpen(ObserverContext, Store, ScanOptions, FlushLifeCycleTracker)
1385   * @see #preCompactScannerOpen(ObserverContext, Store, ScanType, ScanOptions,
1386   *      CompactionLifeCycleTracker, CompactionRequest)
1387   */
1388  default void preStoreScannerOpen(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
1389    Store store, ScanOptions options) throws IOException {
1390  }
1391
1392  /**
1393   * Called before replaying WALs for this region. Calling
1394   * {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} has no effect in this
1395   * hook.
1396   * @param ctx   the environment provided by the region server
1397   * @param info  the RegionInfo for this region
1398   * @param edits the file of recovered edits
1399   */
1400  // todo: what about these?
1401  default void preReplayWALs(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
1402    RegionInfo info, Path edits) throws IOException {
1403  }
1404
1405  /**
1406   * Called after replaying WALs for this region.
1407   * @param ctx   the environment provided by the region server
1408   * @param info  the RegionInfo for this region
1409   * @param edits the file of recovered edits
1410   */
1411  default void postReplayWALs(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
1412    RegionInfo info, Path edits) throws IOException {
1413  }
1414
1415  /**
1416   * Called before bulkLoadHFile. Users can create a StoreFile instance to access the contents of a
1417   * HFile.
1418   * @param ctx         the environment provided by the region server
1419   * @param familyPaths pairs of { CF, HFile path } submitted for bulk load. Adding or removing from
1420   *                    this list will add or remove HFiles to be bulk loaded.
1421   */
1422  default void preBulkLoadHFile(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
1423    List<Pair<byte[], String>> familyPaths) throws IOException {
1424  }
1425
1426  /**
1427   * Called before moving bulk loaded hfile to region directory.
1428   * @param ctx    the environment provided by the region server
1429   * @param family column family
1430   * @param pairs  List of pairs of { HFile location in staging dir, HFile path in region dir } Each
1431   *               pair are for the same hfile.
1432   */
1433  default void preCommitStoreFile(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
1434    byte[] family, List<Pair<Path, Path>> pairs) throws IOException {
1435  }
1436
1437  /**
1438   * Called after moving bulk loaded hfile to region directory.
1439   * @param ctx     the environment provided by the region server
1440   * @param family  column family
1441   * @param srcPath Path to file before the move
1442   * @param dstPath Path to file after the move
1443   */
1444  default void postCommitStoreFile(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
1445    byte[] family, Path srcPath, Path dstPath) throws IOException {
1446  }
1447
1448  /**
1449   * Called after bulkLoadHFile.
1450   * @param ctx                the environment provided by the region server
1451   * @param stagingFamilyPaths pairs of { CF, HFile path } submitted for bulk load
1452   * @param finalPaths         Map of CF to List of file paths for the loaded files if the Map is
1453   *                           not null, the bulkLoad was successful. Otherwise the bulk load
1454   *                           failed. bulkload is done by the time this hook is called.
1455   */
1456  default void postBulkLoadHFile(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
1457    List<Pair<byte[], String>> stagingFamilyPaths, Map<byte[], List<Path>> finalPaths)
1458    throws IOException {
1459  }
1460
1461  /**
1462   * Called before creation of Reader for a store file. Calling
1463   * {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} has no effect in this
1464   * hook.
1465   * @param ctx    the environment provided by the region server
1466   * @param fs     fileystem to read from
1467   * @param p      path to the file
1468   * @param in     {@link FSDataInputStreamWrapper}
1469   * @param size   Full size of the file
1470   * @param r      original reference file. This will be not null only when reading a split file.
1471   * @param reader the base reader, if not {@code null}, from previous RegionObserver in the chain
1472   * @return a Reader instance to use instead of the base reader if overriding default behavior,
1473   *         null otherwise
1474   * @deprecated For Phoenix only, StoreFileReader is not a stable interface.
1475   */
1476  @Deprecated
1477  // Passing InterfaceAudience.Private args FSDataInputStreamWrapper, CacheConfig and Reference.
1478  // This is fine as the hook is deprecated any way.
1479  default StoreFileReader preStoreFileReaderOpen(
1480    ObserverContext<? extends RegionCoprocessorEnvironment> ctx, FileSystem fs, Path p,
1481    FSDataInputStreamWrapper in, long size, CacheConfig cacheConf, Reference r,
1482    StoreFileReader reader) throws IOException {
1483    return reader;
1484  }
1485
1486  /**
1487   * Called after the creation of Reader for a store file.
1488   * @param ctx    the environment provided by the region server
1489   * @param fs     fileystem to read from
1490   * @param p      path to the file
1491   * @param in     {@link FSDataInputStreamWrapper}
1492   * @param size   Full size of the file
1493   * @param r      original reference file. This will be not null only when reading a split file.
1494   * @param reader the base reader instance
1495   * @return The reader to use
1496   * @deprecated For Phoenix only, StoreFileReader is not a stable interface.
1497   */
1498  @Deprecated
1499  // Passing InterfaceAudience.Private args FSDataInputStreamWrapper, CacheConfig and Reference.
1500  // This is fine as the hook is deprecated any way.
1501  default StoreFileReader postStoreFileReaderOpen(
1502    ObserverContext<? extends RegionCoprocessorEnvironment> ctx, FileSystem fs, Path p,
1503    FSDataInputStreamWrapper in, long size, CacheConfig cacheConf, Reference r,
1504    StoreFileReader reader) throws IOException {
1505    return reader;
1506  }
1507
1508  /**
1509   * Called after a new cell has been created during an increment operation, but before it is
1510   * committed to the WAL or memstore. Calling
1511   * {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} has no effect in this
1512   * hook.
1513   * @param ctx      the environment provided by the region server
1514   * @param opType   the operation type
1515   * @param mutation the current mutation
1516   * @param oldCell  old cell containing previous value
1517   * @param newCell  the new cell containing the computed value
1518   * @return the new cell, possibly changed
1519   * @deprecated since 2.2.0 and will be removedin 4.0.0. Use
1520   *             {@link #postIncrementBeforeWAL(ObserverContext, Mutation, List)} or
1521   *             {@link #postAppendBeforeWAL(ObserverContext, Mutation, List)} instead.
1522   * @see #postIncrementBeforeWAL(ObserverContext, Mutation, List)
1523   * @see #postAppendBeforeWAL(ObserverContext, Mutation, List)
1524   * @see <a href="https://issues.apache.org/jira/browse/HBASE-21643">HBASE-21643</a>
1525   */
1526  @Deprecated
1527  default Cell postMutationBeforeWAL(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
1528    MutationType opType, Mutation mutation, Cell oldCell, Cell newCell) throws IOException {
1529    return newCell;
1530  }
1531
1532  /**
1533   * Called after a list of new cells has been created during an increment operation, but before
1534   * they are committed to the WAL or memstore.
1535   * @param ctx       the environment provided by the region server
1536   * @param mutation  the current mutation
1537   * @param cellPairs a list of cell pair. The first cell is old cell which may be null. And the
1538   *                  second cell is the new cell.
1539   * @return a list of cell pair, possibly changed.
1540   */
1541  default List<Pair<Cell, Cell>> postIncrementBeforeWAL(
1542    ObserverContext<? extends RegionCoprocessorEnvironment> ctx, Mutation mutation,
1543    List<Pair<Cell, Cell>> cellPairs) throws IOException {
1544    List<Pair<Cell, Cell>> resultPairs = new ArrayList<>(cellPairs.size());
1545    for (Pair<Cell, Cell> pair : cellPairs) {
1546      resultPairs.add(new Pair<>(pair.getFirst(), postMutationBeforeWAL(ctx, MutationType.INCREMENT,
1547        mutation, pair.getFirst(), pair.getSecond())));
1548    }
1549    return resultPairs;
1550  }
1551
1552  /**
1553   * Called after a list of new cells has been created during an append operation, but before they
1554   * are committed to the WAL or memstore.
1555   * @param ctx       the environment provided by the region server
1556   * @param mutation  the current mutation
1557   * @param cellPairs a list of cell pair. The first cell is old cell which may be null. And the
1558   *                  second cell is the new cell.
1559   * @return a list of cell pair, possibly changed.
1560   */
1561  default List<Pair<Cell, Cell>> postAppendBeforeWAL(
1562    ObserverContext<? extends RegionCoprocessorEnvironment> ctx, Mutation mutation,
1563    List<Pair<Cell, Cell>> cellPairs) throws IOException {
1564    List<Pair<Cell, Cell>> resultPairs = new ArrayList<>(cellPairs.size());
1565    for (Pair<Cell, Cell> pair : cellPairs) {
1566      resultPairs.add(new Pair<>(pair.getFirst(), postMutationBeforeWAL(ctx, MutationType.APPEND,
1567        mutation, pair.getFirst(), pair.getSecond())));
1568    }
1569    return resultPairs;
1570  }
1571
1572  /**
1573   * Called after the ScanQueryMatcher creates ScanDeleteTracker. Implementing this hook would help
1574   * in creating customised DeleteTracker and returning the newly created DeleteTracker
1575   * <p>
1576   * Warn: This is used by internal coprocessors. Should not be implemented by user coprocessors
1577   * @param ctx        the environment provided by the region server
1578   * @param delTracker the deleteTracker that is created by the QueryMatcher
1579   * @return the Delete Tracker
1580   * @deprecated Since 2.0.0, will be removed in 4.0.0. Visibility label feature still use this
1581   *             method, so it can not be removed in 3.0.0
1582   */
1583  @Deprecated
1584  default DeleteTracker postInstantiateDeleteTracker(
1585    ObserverContext<? extends RegionCoprocessorEnvironment> ctx, DeleteTracker delTracker)
1586    throws IOException {
1587    return delTracker;
1588  }
1589
1590  /**
1591   * Called just before the WAL Entry is appended to the WAL. Implementing this hook allows
1592   * coprocessors to add extended attributes to the WALKey that then get persisted to the WAL, and
1593   * are available to replication endpoints to use in processing WAL Entries.
1594   * @param ctx the environment provided by the region server
1595   * @param key the WALKey associated with a particular append to a WAL
1596   */
1597  default void preWALAppend(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, WALKey key,
1598    WALEdit edit) throws IOException {
1599  }
1600}