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