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