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,
013 * software distributed under the License is distributed on an
014 * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
015 * KIND, either express or implied.  See the License for the
016 * specific language governing permissions and limitations
017 * under the License.
018 */
019
020package org.apache.hadoop.hbase.coprocessor;
021
022import java.io.IOException;
023import java.util.List;
024import java.util.Map;
025
026import org.apache.hadoop.fs.FileSystem;
027import org.apache.hadoop.fs.Path;
028import org.apache.hadoop.hbase.Cell;
029import org.apache.hadoop.hbase.CompareOperator;
030import org.apache.hadoop.hbase.HBaseInterfaceAudience;
031import org.apache.hadoop.hbase.client.Append;
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.ByteArrayComparable;
042import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
043import org.apache.hadoop.hbase.io.Reference;
044import org.apache.hadoop.hbase.io.hfile.CacheConfig;
045import org.apache.hadoop.hbase.regionserver.FlushLifeCycleTracker;
046import org.apache.hadoop.hbase.regionserver.InternalScanner;
047import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
048import org.apache.hadoop.hbase.regionserver.OperationStatus;
049import org.apache.hadoop.hbase.regionserver.Region;
050import org.apache.hadoop.hbase.regionserver.Region.Operation;
051import org.apache.hadoop.hbase.regionserver.RegionScanner;
052import org.apache.hadoop.hbase.regionserver.ScanOptions;
053import org.apache.hadoop.hbase.regionserver.ScanType;
054import org.apache.hadoop.hbase.regionserver.Store;
055import org.apache.hadoop.hbase.regionserver.StoreFile;
056import org.apache.hadoop.hbase.regionserver.StoreFileReader;
057import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
058import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
059import org.apache.hadoop.hbase.regionserver.querymatcher.DeleteTracker;
060import org.apache.hadoop.hbase.util.Pair;
061import org.apache.hadoop.hbase.wal.WALEdit;
062import org.apache.hadoop.hbase.wal.WALKey;
063import org.apache.yetus.audience.InterfaceAudience;
064import org.apache.yetus.audience.InterfaceStability;
065
066/**
067 * Coprocessors implement this interface to observe and mediate client actions on the region.
068 * <p>
069 * Since most implementations will be interested in only a subset of hooks, this class uses
070 * 'default' functions to avoid having to add unnecessary overrides. When the functions are
071 * non-empty, it's simply to satisfy the compiler by returning value of expected (non-void) type. It
072 * is done in a way that these default definitions act as no-op. So our suggestion to implementation
073 * would be to not call these 'default' methods from overrides.
074 * <p>
075 * <h3>Exception Handling</h3><br>
076 * For all functions, exception handling is done as follows:
077 * <ul>
078 * <li>Exceptions of type {@link IOException} are reported back to client.</li>
079 * <li>For any other kind of exception:
080 * <ul>
081 * <li>If the configuration {@link CoprocessorHost#ABORT_ON_ERROR_KEY} is set to true, then the
082 * server aborts.</li>
083 * <li>Otherwise, coprocessor is removed from the server and
084 * {@link org.apache.hadoop.hbase.DoNotRetryIOException} is returned to the client.</li>
085 * </ul>
086 * </li>
087 * </ul>
088 * <p>
089 * <h3>For Split Related Hooks</h3> <br>
090 * In hbase2/AMv2, master runs splits, so the split related hooks are moved to
091 * {@link MasterObserver}.
092 * <p>
093 * <h3>Increment Column Value</h3><br>
094 * We do not call this hook anymore.
095 */
096@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
097@InterfaceStability.Evolving
098// TODO as method signatures need to break, update to
099// ObserverContext<? extends RegionCoprocessorEnvironment>
100// so we can use additional environment state that isn't exposed to coprocessors.
101public interface RegionObserver {
102  /** Mutation type for postMutationBeforeWAL hook */
103  enum MutationType {
104    APPEND, INCREMENT
105  }
106
107  /**
108   * Called before the region is reported as open to the master.
109   * @param c the environment provided by the region server
110   */
111  default void preOpen(ObserverContext<RegionCoprocessorEnvironment> c) throws IOException {}
112
113  /**
114   * Called after the region is reported as open to the master.
115   * @param c the environment provided by the region server
116   */
117  default void postOpen(ObserverContext<RegionCoprocessorEnvironment> c) {}
118
119  /**
120   * Called before the memstore is flushed to disk.
121   * @param c the environment provided by the region server
122   * @param tracker tracker used to track the life cycle of a flush
123   */
124  default void preFlush(final ObserverContext<RegionCoprocessorEnvironment> c,
125      FlushLifeCycleTracker tracker) throws IOException {}
126
127  /**
128   * Called before we open store scanner for flush. You can use the {@code options} to change max
129   * versions and TTL for the scanner being opened.
130   * @param c the environment provided by the region server
131   * @param store the store where flush is being requested
132   * @param options used to change max versions and TTL for the scanner being opened
133   */
134  default void preFlushScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
135      ScanOptions options,FlushLifeCycleTracker tracker) throws IOException {}
136
137  /**
138   * Called before a Store's memstore is flushed to disk.
139   * @param c the environment provided by the region server
140   * @param store the store where flush is being requested
141   * @param scanner the scanner over existing data used in the memstore
142   * @param tracker tracker used to track the life cycle of a flush
143   * @return the scanner to use during flush. Should not be {@code null} unless the implementation
144   *         is writing new store files on its own.
145   */
146  default InternalScanner preFlush(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
147      InternalScanner scanner, FlushLifeCycleTracker tracker) throws IOException {
148    return scanner;
149  }
150
151  /**
152   * Called after the memstore is flushed to disk.
153   * @param c the environment provided by the region server
154   * @param tracker tracker used to track the life cycle of a flush
155   * @throws IOException if an error occurred on the coprocessor
156   */
157  default void postFlush(ObserverContext<RegionCoprocessorEnvironment> c,
158      FlushLifeCycleTracker tracker) throws IOException {}
159
160  /**
161   * Called after a Store's memstore is flushed to disk.
162   * @param c the environment provided by the region server
163   * @param store the store being flushed
164   * @param resultFile the new store file written out during compaction
165   * @param tracker tracker used to track the life cycle of a flush
166   */
167  default void postFlush(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
168      StoreFile resultFile, FlushLifeCycleTracker tracker) throws IOException {}
169
170  /**
171   * Called before in memory compaction started.
172   * @param c the environment provided by the region server
173   * @param store the store where in memory compaction is being requested
174   */
175  default void preMemStoreCompaction(ObserverContext<RegionCoprocessorEnvironment> c, Store store)
176      throws IOException {}
177
178  /**
179   * Called before we open store scanner for in memory compaction. You can use the {@code options}
180   * to change max versions and TTL for the scanner being opened. Notice that this method will only
181   * be called when you use {@code eager} mode. For {@code basic} mode we will not drop any cells
182   * thus we do not open a store scanner.
183   * @param c the environment provided by the region server
184   * @param store the store where in memory compaction is being requested
185   * @param options used to change max versions and TTL for the scanner being opened
186   */
187  default void preMemStoreCompactionCompactScannerOpen(
188      ObserverContext<RegionCoprocessorEnvironment> c, Store store, ScanOptions options)
189      throws IOException {}
190
191  /**
192   * Called before we do in memory compaction. Notice that this method will only be called when you
193   * use {@code eager} mode. For {@code basic} mode we will not drop any cells thus there is no
194   * {@link InternalScanner}.
195   * @param c the environment provided by the region server
196   * @param store the store where in memory compaction is being executed
197   * @param scanner the scanner over existing data used in the memstore segments being compact
198   * @return the scanner to use during in memory compaction. Must be non-null.
199   */
200  default InternalScanner preMemStoreCompactionCompact(
201      ObserverContext<RegionCoprocessorEnvironment> c, Store store, InternalScanner scanner)
202      throws IOException {
203    return scanner;
204  }
205
206  /**
207   * Called after the in memory compaction is finished.
208   * @param c the environment provided by the region server
209   * @param store the store where in memory compaction is being executed
210   */
211  default void postMemStoreCompaction(ObserverContext<RegionCoprocessorEnvironment> c, Store store)
212      throws IOException {}
213
214  /**
215   * Called prior to selecting the {@link StoreFile StoreFiles} to compact from the list of
216   * available candidates. To alter the files used for compaction, you may mutate the passed in list
217   * of candidates. If you remove all the candidates then the compaction will be canceled.
218   * <p>Supports Coprocessor 'bypass' -- 'bypass' is how this method indicates that it changed
219   * the passed in <code>candidates</code>.
220   * If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
221   * @param c the environment provided by the region server
222   * @param store the store where compaction is being requested
223   * @param candidates the store files currently available for compaction
224   * @param tracker tracker used to track the life cycle of a compaction
225   */
226  default void preCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
227      List<? extends StoreFile> candidates, CompactionLifeCycleTracker tracker)
228      throws IOException {}
229
230  /**
231   * Called after the {@link StoreFile}s to compact have been selected from the available
232   * candidates.
233   * @param c the environment provided by the region server
234   * @param store the store being compacted
235   * @param selected the store files selected to compact
236   * @param tracker tracker used to track the life cycle of a compaction
237   * @param request the requested compaction
238   */
239  default void postCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
240      List<? extends StoreFile> selected, CompactionLifeCycleTracker tracker,
241      CompactionRequest request) {}
242
243  /**
244   * Called before we open store scanner for compaction. You can use the {@code options} to change max
245   * versions and TTL for the scanner being opened.
246   * @param c the environment provided by the region server
247   * @param store the store being compacted
248   * @param scanType type of Scan
249   * @param options used to change max versions and TTL for the scanner being opened
250   * @param tracker tracker used to track the life cycle of a compaction
251   * @param request the requested compaction
252   */
253  default void preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
254      ScanType scanType, ScanOptions options, CompactionLifeCycleTracker tracker,
255      CompactionRequest request) throws IOException {}
256
257  /**
258   * Called prior to writing the {@link StoreFile}s selected for compaction into a new
259   * {@code StoreFile}.
260   * <p>
261   * To override or modify the compaction process, implementing classes can wrap the provided
262   * {@link InternalScanner} with a custom implementation that is returned from this method. The
263   * custom scanner can then inspect {@link org.apache.hadoop.hbase.Cell}s from the wrapped scanner,
264   * applying its own policy to what gets written.
265   * @param c the environment provided by the region server
266   * @param store the store being compacted
267   * @param scanner the scanner over existing data used in the store file rewriting
268   * @param scanType type of Scan
269   * @param tracker tracker used to track the life cycle of a compaction
270   * @param request the requested compaction
271   * @return the scanner to use during compaction. Should not be {@code null} unless the
272   *         implementation is writing new store files on its own.
273   */
274  default InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
275      InternalScanner scanner, ScanType scanType, CompactionLifeCycleTracker tracker,
276      CompactionRequest request) throws IOException {
277    return scanner;
278  }
279
280  /**
281   * Called after compaction has completed and the new store file has been moved in to place.
282   * @param c the environment provided by the region server
283   * @param store the store being compacted
284   * @param resultFile the new store file written out during compaction
285   * @param tracker used to track the life cycle of a compaction
286   * @param request the requested compaction
287   */
288  default void postCompact(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
289      StoreFile resultFile, CompactionLifeCycleTracker tracker, CompactionRequest request)
290      throws IOException {}
291
292  /**
293   * Called before the region is reported as closed to the master.
294   * @param c the environment provided by the region server
295   * @param abortRequested true if the region server is aborting
296   */
297  default void preClose(ObserverContext<RegionCoprocessorEnvironment> c, boolean abortRequested)
298      throws IOException {}
299
300  /**
301   * Called after the region is reported as closed to the master.
302   * @param c the environment provided by the region server
303   * @param abortRequested true if the region server is aborting
304   */
305  default void postClose(ObserverContext<RegionCoprocessorEnvironment> c, boolean abortRequested) {}
306
307  /**
308   * Called before the client performs a Get
309   * <p>
310   * Call CoprocessorEnvironment#bypass to skip default actions.
311   * If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
312   * @param c the environment provided by the region server
313   * @param get the Get request
314   * @param result The result to return to the client if default processing
315   * is bypassed. Can be modified. Will not be used if default processing
316   * is not bypassed.
317   */
318  default void preGetOp(ObserverContext<RegionCoprocessorEnvironment> c, Get get, List<Cell> result)
319      throws IOException {}
320
321  /**
322   * Called after the client performs a Get
323   * <p>
324   * Note: Do not retain references to any Cells in 'result' beyond the life of this invocation.
325   * If need a Cell reference for later use, copy the cell and use that.
326   * @param c the environment provided by the region server
327   * @param get the Get request
328   * @param result the result to return to the client, modify as necessary
329   */
330  default void postGetOp(ObserverContext<RegionCoprocessorEnvironment> c, Get get,
331      List<Cell> result) throws IOException {}
332
333  /**
334   * Called before the client tests for existence using a Get.
335   * <p>
336   * Call CoprocessorEnvironment#bypass to skip default actions.
337   * If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
338   * @param c the environment provided by the region server
339   * @param get the Get request
340   * @param exists the result returned by the region server
341   * @return the value to return to the client if bypassing default processing
342   */
343  default boolean preExists(ObserverContext<RegionCoprocessorEnvironment> c, Get get,
344      boolean exists) throws IOException {
345    return exists;
346  }
347
348  /**
349   * Called after the client tests for existence using a Get.
350   * @param c the environment provided by the region server
351   * @param get the Get request
352   * @param exists the result returned by the region server
353   * @return the result to return to the client
354   */
355  default boolean postExists(ObserverContext<RegionCoprocessorEnvironment> c, Get get,
356      boolean exists) throws IOException {
357    return exists;
358  }
359
360  /**
361   * Called before the client stores a value.
362   * <p>
363   * Call CoprocessorEnvironment#bypass to skip default actions.
364   * If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
365   * <p>
366   * Note: Do not retain references to any Cells in 'put' beyond the life of this invocation.
367   * If need a Cell reference for later use, copy the cell and use that.
368   * @param c the environment provided by the region server
369   * @param put The Put object
370   * @param edit The WALEdit object that will be written to the wal
371   * @param durability Persistence guarantee for this Put
372   */
373  default void prePut(ObserverContext<RegionCoprocessorEnvironment> c, Put put, WALEdit edit,
374      Durability durability) throws IOException {}
375
376  /**
377   * Called after the client stores a value.
378   * <p>
379   * Note: Do not retain references to any Cells in 'put' beyond the life of this invocation.
380   * If need a Cell reference for later use, copy the cell and use that.
381   * @param c the environment provided by the region server
382   * @param put The Put object
383   * @param edit The WALEdit object for the wal
384   * @param durability Persistence guarantee for this Put
385   */
386  default void postPut(ObserverContext<RegionCoprocessorEnvironment> c, Put put, WALEdit edit,
387      Durability durability) throws IOException {}
388
389  /**
390   * Called before the client deletes a value.
391   * <p>
392   * Call CoprocessorEnvironment#bypass to skip default actions.
393   * If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
394   * <p>
395   * Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation.
396   * If 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 delete The Delete object
399   * @param edit The WALEdit object for the wal
400   * @param durability Persistence guarantee for this Delete
401   */
402  default void preDelete(ObserverContext<RegionCoprocessorEnvironment> c, Delete delete,
403      WALEdit edit, Durability durability) throws IOException {}
404
405  /**
406   * Called before the server updates the timestamp for version delete with latest timestamp.
407   * <p>
408   * Call CoprocessorEnvironment#bypass to skip default actions.
409   * If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
410   * @param c the environment provided by the region server
411   * @param mutation - the parent mutation associated with this delete cell
412   * @param cell - The deleteColumn with latest version cell
413   * @param byteNow - timestamp bytes
414   * @param get - the get formed using the current cell's row. Note that the get does not specify
415   *          the family and qualifier
416   * @deprecated Since hbase-2.0.0. No replacement. To be removed in hbase-3.0.0 and replaced
417   * with something that doesn't expose IntefaceAudience.Private classes.
418   */
419  @Deprecated
420  default void prePrepareTimeStampForDeleteVersion(ObserverContext<RegionCoprocessorEnvironment> c,
421      Mutation mutation, Cell cell, byte[] byteNow, Get get) throws IOException {}
422
423  /**
424   * Called after the client deletes a value.
425   * <p>
426   * Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation.
427   * If need a Cell reference for later use, copy the cell and use that.
428   * @param c the environment provided by the region server
429   * @param delete The Delete object
430   * @param edit The WALEdit object for the wal
431   * @param durability Persistence guarantee for this Delete
432   */
433  default void postDelete(ObserverContext<RegionCoprocessorEnvironment> c, Delete delete,
434      WALEdit edit, Durability durability) throws IOException {}
435
436  /**
437   * This will be called for every batch mutation operation happening at the server. This will be
438   * called after acquiring the locks on the mutating rows and after applying the proper timestamp
439   * for each Mutation at the server. The batch may contain Put/Delete. By setting OperationStatus
440   * of Mutations ({@link MiniBatchOperationInProgress#setOperationStatus(int, OperationStatus)}),
441   * {@link RegionObserver} can make Region to skip these Mutations.
442   * <p>
443   * Note: Do not retain references to any Cells in Mutations beyond the life of this invocation.
444   * If need a Cell reference for later use, copy the cell and use that.
445   * @param c the environment provided by the region server
446   * @param miniBatchOp batch of Mutations getting applied to region.
447   */
448  default void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
449      MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {}
450
451  /**
452   * This will be called after applying a batch of Mutations on a region. The Mutations are added to
453   * memstore and WAL. The difference of this one with
454   * {@link #postPut(ObserverContext, Put, WALEdit, Durability) }
455   * and {@link #postDelete(ObserverContext, Delete, WALEdit, Durability) } is
456   * this hook will be executed before the mvcc transaction completion.
457   * <p>
458   * Note: Do not retain references to any Cells in Mutations beyond the life of this invocation.
459   * If need a Cell reference for later use, copy the cell and use that.
460   * @param c the environment provided by the region server
461   * @param miniBatchOp batch of Mutations applied to region. Coprocessors are discouraged from
462   *                    manipulating its state.
463   */
464  // Coprocessors can do a form of bypass by changing state in miniBatchOp.
465  default void postBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
466      MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {}
467
468  /**
469   * This will be called for region operations where read lock is acquired in
470   * {@link Region#startRegionOperation()}.
471   * @param ctx
472   * @param operation The operation is about to be taken on the region
473   */
474  default void postStartRegionOperation(ObserverContext<RegionCoprocessorEnvironment> ctx,
475      Operation operation) throws IOException {}
476
477  /**
478   * Called after releasing read lock in {@link Region#closeRegionOperation()}.
479   * @param ctx
480   * @param operation
481   */
482  default void postCloseRegionOperation(ObserverContext<RegionCoprocessorEnvironment> ctx,
483      Operation operation) throws IOException {}
484
485  /**
486   * Called after the completion of batch put/delete and will be called even if the batch operation
487   * fails.
488   * <p>
489   * Note: Do not retain references to any Cells in Mutations beyond the life of this invocation.
490   * If need a Cell reference for later use, copy the cell and use that.
491   * @param ctx
492   * @param miniBatchOp
493   * @param success true if batch operation is successful otherwise false.
494   */
495  default void postBatchMutateIndispensably(ObserverContext<RegionCoprocessorEnvironment> ctx,
496      MiniBatchOperationInProgress<Mutation> miniBatchOp, boolean success) throws IOException {}
497
498  /**
499   * Called before checkAndPut.
500   * <p>
501   * Call CoprocessorEnvironment#bypass to skip default actions.
502   * If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
503   * <p>
504   * Note: Do not retain references to any Cells in 'put' beyond the life of this invocation.
505   * If need a Cell reference for later use, copy the cell and use that.
506   * @param c the environment provided by the region server
507   * @param row row to check
508   * @param family column family
509   * @param qualifier column qualifier
510   * @param op the comparison operation
511   * @param comparator the comparator
512   * @param put data to put if check succeeds
513   * @param result
514   * @return the return value to return to client if bypassing default
515   * processing
516   */
517  default boolean preCheckAndPut(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row,
518      byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Put put,
519      boolean result) throws IOException {
520    return result;
521  }
522
523  /**
524   * Called before checkAndPut but after acquiring rowlock.
525   * <p>
526   * <b>Note:</b> Caution to be taken for not doing any long time operation in this hook.
527   * Row will be locked for longer time. Trying to acquire lock on another row, within this,
528   * can lead to potential deadlock.
529   * <p>
530   * Call CoprocessorEnvironment#bypass to skip default actions.
531   * If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
532   * <p>
533   * Note: Do not retain references to any Cells in 'put' beyond the life of this invocation.
534   * If need a Cell reference for later use, copy the cell and use that.
535   * @param c the environment provided by the region server
536   * @param row row to check
537   * @param family column family
538   * @param qualifier column qualifier
539   * @param op the comparison operation
540   * @param comparator the comparator
541   * @param put data to put if check succeeds
542   * @param result
543   * @return the return value to return to client if bypassing default
544   * processing
545   */
546  default boolean preCheckAndPutAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c,
547      byte[] row, byte[] family, byte[] qualifier, CompareOperator op,
548      ByteArrayComparable comparator, Put put, boolean result) throws IOException {
549    return result;
550  }
551
552  /**
553   * Called after checkAndPut
554   * <p>
555   * Note: Do not retain references to any Cells in 'put' beyond the life of this invocation.
556   * If need a Cell reference for later use, copy the cell and use that.
557   * @param c the environment provided by the region server
558   * @param row row to check
559   * @param family column family
560   * @param qualifier column qualifier
561   * @param op the comparison operation
562   * @param comparator the comparator
563   * @param put data to put if check succeeds
564   * @param result from the checkAndPut
565   * @return the possibly transformed return value to return to client
566   */
567  default boolean postCheckAndPut(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row,
568      byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Put put,
569      boolean result) throws IOException {
570    return result;
571  }
572
573  /**
574   * Called before checkAndDelete.
575   * <p>
576   * Call CoprocessorEnvironment#bypass to skip default actions.
577   * If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
578   * <p>
579   * Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation.
580   * If need a Cell reference for later use, copy the cell and use that.
581   * @param c the environment provided by the region server
582   * @param row row to check
583   * @param family column family
584   * @param qualifier column qualifier
585   * @param op the comparison operation
586   * @param comparator the comparator
587   * @param delete delete to commit if check succeeds
588   * @param result
589   * @return the value to return to client if bypassing default processing
590   */
591  default boolean preCheckAndDelete(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row,
592      byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator,
593      Delete delete, boolean result) throws IOException {
594    return result;
595  }
596
597  /**
598   * Called before checkAndDelete but after acquiring rowock.
599   * <p>
600   * <b>Note:</b> Caution to be taken for not doing any long time operation in this hook.
601   * Row will be locked for longer time. Trying to acquire lock on another row, within this,
602   * can lead to potential deadlock.
603   * <p>
604   * Call CoprocessorEnvironment#bypass to skip default actions.
605   * If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
606   * <p>
607   * Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation.
608   * If need a Cell reference for later use, copy the cell and use that.
609   * @param c the environment provided by the region server
610   * @param row row to check
611   * @param family column family
612   * @param qualifier column qualifier
613   * @param op the comparison operation
614   * @param comparator the comparator
615   * @param delete delete to commit if check succeeds
616   * @param result
617   * @return the value to return to client if bypassing default processing
618   */
619  default boolean preCheckAndDeleteAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c,
620      byte[] row, byte[] family, byte[] qualifier, CompareOperator op,
621      ByteArrayComparable comparator, Delete delete, boolean result) throws IOException {
622    return result;
623  }
624
625  /**
626   * Called after checkAndDelete
627   * <p>
628   * Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation.
629   * If need a Cell reference for later use, copy the cell and use that.
630   * @param c the environment provided by the region server
631   * @param row row to check
632   * @param family column family
633   * @param qualifier column qualifier
634   * @param op the comparison operation
635   * @param comparator the comparator
636   * @param delete delete to commit if check succeeds
637   * @param result from the CheckAndDelete
638   * @return the possibly transformed returned value to return to client
639   */
640  default boolean postCheckAndDelete(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row,
641      byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator,
642      Delete delete, boolean result) throws IOException {
643    return result;
644  }
645
646  /**
647   * Called before Append.
648   * <p>
649   * Call CoprocessorEnvironment#bypass to skip default actions.
650   * If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
651   * <p>
652   * Note: Do not retain references to any Cells in 'append' beyond the life of this invocation.
653   * If need a Cell reference for later use, copy the cell and use that.
654   * @param c the environment provided by the region server
655   * @param append Append object
656   * @return result to return to the client if bypassing default processing
657   */
658  default Result preAppend(ObserverContext<RegionCoprocessorEnvironment> c, Append append)
659      throws IOException {
660    return null;
661  }
662
663  /**
664   * Called before Append but after acquiring rowlock.
665   * <p>
666   * <b>Note:</b> Caution to be taken for not doing any long time operation in this hook.
667   * Row will be locked for longer time. Trying to acquire lock on another row, within this,
668   * can lead to potential deadlock.
669   * <p>
670   * Call CoprocessorEnvironment#bypass to skip default actions.
671   * If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
672   * <p>
673   * Note: Do not retain references to any Cells in 'append' beyond the life of this invocation.
674   * If need a Cell reference for later use, copy the cell and use that.
675   * @param c the environment provided by the region server
676   * @param append Append object
677   * @return result to return to the client if bypassing default processing
678   */
679  default Result preAppendAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c,
680      Append append) throws IOException {
681    return null;
682  }
683
684  /**
685   * Called after Append
686   * <p>
687   * Note: Do not retain references to any Cells in 'append' beyond the life of this invocation.
688   * If need a Cell reference for later use, copy the cell and use that.
689   * @param c the environment provided by the region server
690   * @param append Append object
691   * @param result the result returned by increment
692   * @return the result to return to the client
693   */
694  default Result postAppend(ObserverContext<RegionCoprocessorEnvironment> c, Append append,
695      Result result) throws IOException {
696    return result;
697  }
698
699  /**
700   * Called before Increment.
701   * <p>
702   * Call CoprocessorEnvironment#bypass to skip default actions.
703   * If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
704   * <p>
705   * Note: Do not retain references to any Cells in 'increment' beyond the life of this invocation.
706   * If need a Cell reference for later use, copy the cell and use that.
707   * @param c the environment provided by the region server
708   * @param increment increment object
709   * @return result to return to the client if bypassing default processing
710   */
711  default Result preIncrement(ObserverContext<RegionCoprocessorEnvironment> c, Increment increment)
712      throws IOException {
713    return null;
714  }
715
716  /**
717   * Called before Increment but after acquiring rowlock.
718   * <p>
719   * <b>Note:</b> Caution to be taken for not doing any long time operation in this hook.
720   * Row will be locked for longer time. Trying to acquire lock on another row, within this,
721   * can lead to potential deadlock.
722   * <p>
723   * Call CoprocessorEnvironment#bypass to skip default actions.
724   * If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
725   * <p>
726   * Note: Do not retain references to any Cells in 'increment' beyond the life of this invocation.
727   * If need a Cell reference for later use, copy the cell and use that.
728   *
729   * @param c
730   *          the environment provided by the region server
731   * @param increment
732   *          increment object
733   * @return result to return to the client if bypassing default processing
734   *           if an error occurred on the coprocessor
735   */
736  default Result preIncrementAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c,
737      Increment increment) throws IOException {
738    return null;
739  }
740
741  /**
742   * Called after increment
743   * <p>
744   * Note: Do not retain references to any Cells in 'increment' beyond the life of this invocation.
745   * If need a Cell reference for later use, copy the cell and use that.
746   * @param c the environment provided by the region server
747   * @param increment increment object
748   * @param result the result returned by increment
749   * @return the result to return to the client
750   */
751  default Result postIncrement(ObserverContext<RegionCoprocessorEnvironment> c, Increment increment,
752      Result result) throws IOException {
753    return result;
754  }
755
756  /**
757   * Called before the client opens a new scanner.
758   * <p>
759   * Note: Do not retain references to any Cells returned by scanner, beyond the life of this
760   * invocation. If need a Cell reference for later use, copy the cell and use that.
761   * @param c the environment provided by the region server
762   * @param scan the Scan specification
763   */
764  default void preScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, Scan scan)
765      throws IOException {
766  }
767
768  /**
769   * Called after the client opens a new scanner.
770   * <p>
771   * Note: Do not retain references to any Cells returned by scanner, beyond the life of this
772   * invocation. If need a Cell reference for later use, copy the cell and use that.
773   * @param c the environment provided by the region server
774   * @param scan the Scan specification
775   * @param s if not null, the base scanner
776   * @return the scanner instance to use
777   */
778  default RegionScanner postScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, Scan scan,
779      RegionScanner s) throws IOException {
780    return s;
781  }
782
783  /**
784   * Called before the client asks for the next row on a scanner.
785   * <p>
786   * Call CoprocessorEnvironment#bypass to skip default actions.
787   * If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
788   * <p>
789   * Note: Do not retain references to any Cells returned by scanner, beyond the life of this
790   * invocation. If need a Cell reference for later use, copy the cell and use that.
791   * @param c the environment provided by the region server
792   * @param s the scanner
793   * @param result The result to return to the client if default processing
794   * is bypassed. Can be modified. Will not be returned if default processing
795   * is not bypassed.
796   * @param limit the maximum number of results to return
797   * @param hasNext the 'has more' indication
798   * @return 'has more' indication that should be sent to client
799   */
800  default boolean preScannerNext(ObserverContext<RegionCoprocessorEnvironment> c, InternalScanner s,
801      List<Result> result, int limit, boolean hasNext) throws IOException {
802    return hasNext;
803  }
804
805  /**
806   * Called after the client asks for the next row on a scanner.
807   * <p>
808   * Note: Do not retain references to any Cells returned by scanner, beyond the life of this
809   * invocation. If need a Cell reference for later use, copy the cell and use that.
810   * @param c the environment provided by the region server
811   * @param s the scanner
812   * @param result the result to return to the client, can be modified
813   * @param limit the maximum number of results to return
814   * @param hasNext the 'has more' indication
815   * @return 'has more' indication that should be sent to client
816   */
817  default boolean postScannerNext(ObserverContext<RegionCoprocessorEnvironment> c,
818      InternalScanner s, List<Result> result, int limit, boolean hasNext) throws IOException {
819    return hasNext;
820  }
821
822  /**
823   * This will be called by the scan flow when the current scanned row is being filtered out by the
824   * filter. The filter may be filtering out the row via any of the below scenarios
825   * <ol>
826   * <li>
827   * <code>boolean filterRowKey(byte [] buffer, int offset, int length)</code> returning true</li>
828   * <li>
829   * <code>boolean filterRow()</code> returning true</li>
830   * <li>
831   * <code>default void filterRow(List&lt;KeyValue&gt; kvs)</code> removing all the kvs from
832   * the passed List</li>
833   * </ol>
834   * <p>
835   * Note: Do not retain references to any Cells returned by scanner, beyond the life of this
836   * invocation. If need a Cell reference for later use, copy the cell and use that.
837   * @param c the environment provided by the region server
838   * @param s the scanner
839   * @param curRowCell The cell in the current row which got filtered out
840   * @param hasMore the 'has more' indication
841   * @return whether more rows are available for the scanner or not
842   */
843  default boolean postScannerFilterRow(ObserverContext<RegionCoprocessorEnvironment> c,
844      InternalScanner s, Cell curRowCell, boolean hasMore) throws IOException {
845    return hasMore;
846  }
847
848  /**
849   * Called before the client closes a scanner.
850   * <p>
851   * Call CoprocessorEnvironment#bypass to skip default actions.
852   * If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
853   * @param c the environment provided by the region server
854   * @param s the scanner
855   */
856  default void preScannerClose(ObserverContext<RegionCoprocessorEnvironment> c, InternalScanner s)
857      throws IOException {}
858
859  /**
860   * Called after the client closes a scanner.
861   * @param ctx the environment provided by the region server
862   * @param s the scanner
863   */
864  default void postScannerClose(ObserverContext<RegionCoprocessorEnvironment> ctx,
865      InternalScanner s) throws IOException {}
866
867  /**
868   * Called before a store opens a new scanner.
869   * <p>
870   * This hook is called when a "user" scanner is opened. Use {@code preFlushScannerOpen} and
871   * {@code preCompactScannerOpen} to inject flush/compaction.
872   * <p>
873   * Notice that, this method is used to change the inherent max versions and TTL for a Store. For
874   * example, you can change the max versions option for a {@link Scan} object to 10 in
875   * {@code preScannerOpen}, but if the max versions config on the Store is 1, then you still can
876   * only read 1 version. You need also to inject here to change the max versions to 10 if you want
877   * to get more versions.
878   * @param ctx the environment provided by the region server
879   * @param store the store which we want to get scanner from
880   * @param options used to change max versions and TTL for the scanner being opened
881   * @see #preFlushScannerOpen(ObserverContext, Store, ScanOptions, FlushLifeCycleTracker)
882   * @see #preCompactScannerOpen(ObserverContext, Store, ScanType, ScanOptions,
883   *      CompactionLifeCycleTracker, CompactionRequest)
884   */
885  default void preStoreScannerOpen(ObserverContext<RegionCoprocessorEnvironment> ctx, Store store,
886      ScanOptions options) throws IOException {}
887
888  /**
889   * Called before replaying WALs for this region.
890   * Calling {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} has no
891   * effect in this hook.
892   * @param ctx the environment provided by the region server
893   * @param info the RegionInfo for this region
894   * @param edits the file of recovered edits
895   */
896  // todo: what about these?
897  default void preReplayWALs(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
898    RegionInfo info, Path edits) throws IOException {}
899
900  /**
901   * Called after replaying WALs for this region.
902   * @param ctx the environment provided by the region server
903   * @param info the RegionInfo for this region
904   * @param edits the file of recovered edits
905   */
906  default void postReplayWALs(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
907    RegionInfo info, Path edits) throws IOException {}
908
909  /**
910   * Called before a {@link WALEdit}
911   * replayed for this region.
912   * @param ctx the environment provided by the region server
913   */
914  default void preWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
915    RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {}
916
917  /**
918   * Called after a {@link WALEdit}
919   * replayed for this region.
920   * @param ctx the environment provided by the region server
921   */
922  default void postWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
923    RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {}
924
925  /**
926   * Called before bulkLoadHFile. Users can create a StoreFile instance to
927   * access the contents of a HFile.
928   *
929   * @param ctx the environment provided by the region server
930   * @param familyPaths pairs of { CF, HFile path } submitted for bulk load. Adding
931   * or removing from this list will add or remove HFiles to be bulk loaded.
932   */
933  default void preBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx,
934    List<Pair<byte[], String>> familyPaths) throws IOException {}
935
936  /**
937   * Called before moving bulk loaded hfile to region directory.
938   *
939   * @param ctx the environment provided by the region server
940   * @param family column family
941   * @param pairs List of pairs of { HFile location in staging dir, HFile path in region dir }
942   * Each pair are for the same hfile.
943   */
944  default void preCommitStoreFile(ObserverContext<RegionCoprocessorEnvironment> ctx, byte[] family,
945      List<Pair<Path, Path>> pairs) throws IOException {}
946
947  /**
948   * Called after moving bulk loaded hfile to region directory.
949   *
950   * @param ctx the environment provided by the region server
951   * @param family column family
952   * @param srcPath Path to file before the move
953   * @param dstPath Path to file after the move
954   */
955  default void postCommitStoreFile(ObserverContext<RegionCoprocessorEnvironment> ctx, byte[] family,
956      Path srcPath, Path dstPath) throws IOException {}
957
958  /**
959   * Called after bulkLoadHFile.
960   *
961   * @param ctx the environment provided by the region server
962   * @param stagingFamilyPaths pairs of { CF, HFile path } submitted for bulk load
963   * @param finalPaths Map of CF to List of file paths for the loaded files
964   *   if the Map is not null, the bulkLoad was successful. Otherwise the bulk load failed.
965   *   bulkload is done by the time this hook is called.
966   */
967  default void postBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx,
968      List<Pair<byte[], String>> stagingFamilyPaths, Map<byte[], List<Path>> finalPaths)
969          throws IOException {
970  }
971
972  /**
973   * Called before creation of Reader for a store file.
974   * Calling {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} has no
975   * effect in this hook.
976   *
977   * @param ctx the environment provided by the region server
978   * @param fs fileystem to read from
979   * @param p path to the file
980   * @param in {@link FSDataInputStreamWrapper}
981   * @param size Full size of the file
982   * @param cacheConf
983   * @param r original reference file. This will be not null only when reading a split file.
984   * @param reader the base reader, if not {@code null}, from previous RegionObserver in the chain
985   * @return a Reader instance to use instead of the base reader if overriding
986   * default behavior, null otherwise
987   * @deprecated For Phoenix only, StoreFileReader is not a stable interface.
988   */
989  @Deprecated
990  // Passing InterfaceAudience.Private args FSDataInputStreamWrapper, CacheConfig and Reference.
991  // This is fine as the hook is deprecated any way.
992  default StoreFileReader preStoreFileReaderOpen(ObserverContext<RegionCoprocessorEnvironment> ctx,
993      FileSystem fs, Path p, FSDataInputStreamWrapper in, long size, CacheConfig cacheConf,
994      Reference r, StoreFileReader reader) throws IOException {
995    return reader;
996  }
997
998  /**
999   * Called after the creation of Reader for a store file.
1000   *
1001   * @param ctx the environment provided by the region server
1002   * @param fs fileystem to read from
1003   * @param p path to the file
1004   * @param in {@link FSDataInputStreamWrapper}
1005   * @param size Full size of the file
1006   * @param cacheConf
1007   * @param r original reference file. This will be not null only when reading a split file.
1008   * @param reader the base reader instance
1009   * @return The reader to use
1010   * @deprecated For Phoenix only, StoreFileReader is not a stable interface.
1011   */
1012  @Deprecated
1013  // Passing InterfaceAudience.Private args FSDataInputStreamWrapper, CacheConfig and Reference.
1014  // This is fine as the hook is deprecated any way.
1015  default StoreFileReader postStoreFileReaderOpen(ObserverContext<RegionCoprocessorEnvironment> ctx,
1016      FileSystem fs, Path p, FSDataInputStreamWrapper in, long size, CacheConfig cacheConf,
1017      Reference r, StoreFileReader reader) throws IOException {
1018    return reader;
1019  }
1020
1021  /**
1022   * Called after a new cell has been created during an increment operation, but before
1023   * it is committed to the WAL or memstore.
1024   * Calling {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} has no
1025   * effect in this hook.
1026   * @param ctx the environment provided by the region server
1027   * @param opType the operation type
1028   * @param mutation the current mutation
1029   * @param oldCell old cell containing previous value
1030   * @param newCell the new cell containing the computed value
1031   * @return the new cell, possibly changed
1032   */
1033  default Cell postMutationBeforeWAL(ObserverContext<RegionCoprocessorEnvironment> ctx,
1034      MutationType opType, Mutation mutation, Cell oldCell, Cell newCell) throws IOException {
1035    return newCell;
1036  }
1037
1038  /**
1039   * Called after the ScanQueryMatcher creates ScanDeleteTracker. Implementing
1040   * this hook would help in creating customised DeleteTracker and returning
1041   * the newly created DeleteTracker
1042   * <p>
1043   * Warn: This is used by internal coprocessors. Should not be implemented by user coprocessors
1044   * @param ctx the environment provided by the region server
1045   * @param delTracker the deleteTracker that is created by the QueryMatcher
1046   * @return the Delete Tracker
1047   * @deprecated Since 2.0 with out any replacement and will be removed in 3.0
1048   */
1049  @Deprecated
1050  default DeleteTracker postInstantiateDeleteTracker(
1051      ObserverContext<RegionCoprocessorEnvironment> ctx, DeleteTracker delTracker)
1052      throws IOException {
1053    return delTracker;
1054  }
1055}