View Javadoc

1   /*
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *   http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing,
13   * software distributed under the License is distributed on an
14   * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15   * KIND, either express or implied.  See the License for the
16   * specific language governing permissions and limitations
17   * under the License.
18   */
19  
20  package org.apache.hadoop.hbase.coprocessor;
21  
22  import java.io.IOException;
23  import java.util.List;
24  import java.util.NavigableSet;
25  
26  import org.apache.hadoop.fs.FileSystem;
27  import org.apache.hadoop.fs.Path;
28  import org.apache.hadoop.hbase.Cell;
29  import org.apache.hadoop.hbase.CoprocessorEnvironment;
30  import org.apache.hadoop.hbase.HBaseInterfaceAudience;
31  import org.apache.hadoop.hbase.HRegionInfo;
32  import org.apache.hadoop.hbase.classification.InterfaceAudience;
33  import org.apache.hadoop.hbase.classification.InterfaceStability;
34  import org.apache.hadoop.hbase.client.Append;
35  import org.apache.hadoop.hbase.client.Delete;
36  import org.apache.hadoop.hbase.client.Durability;
37  import org.apache.hadoop.hbase.client.Get;
38  import org.apache.hadoop.hbase.client.Increment;
39  import org.apache.hadoop.hbase.client.Mutation;
40  import org.apache.hadoop.hbase.client.Put;
41  import org.apache.hadoop.hbase.client.Result;
42  import org.apache.hadoop.hbase.client.Scan;
43  import org.apache.hadoop.hbase.filter.ByteArrayComparable;
44  import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
45  import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
46  import org.apache.hadoop.hbase.io.Reference;
47  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
48  import org.apache.hadoop.hbase.regionserver.DeleteTracker;
49  import org.apache.hadoop.hbase.regionserver.InternalScanner;
50  import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
51  import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
52  import org.apache.hadoop.hbase.regionserver.Region;
53  import org.apache.hadoop.hbase.regionserver.Region.Operation;
54  import org.apache.hadoop.hbase.regionserver.RegionScanner;
55  import org.apache.hadoop.hbase.regionserver.ScanType;
56  import org.apache.hadoop.hbase.regionserver.Store;
57  import org.apache.hadoop.hbase.regionserver.StoreFile;
58  import org.apache.hadoop.hbase.regionserver.StoreFile.Reader;
59  import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
60  import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
61  import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
62  import org.apache.hadoop.hbase.util.Pair;
63  import org.apache.hadoop.hbase.wal.WALKey;
64  
65  import com.google.common.collect.ImmutableList;
66  
67  /**
68   * An abstract class that implements RegionObserver.
69   * By extending it, you can create your own region observer without
70   * overriding all abstract methods of RegionObserver.
71   */
72  @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
73  @InterfaceStability.Evolving
74  public abstract class BaseRegionObserver implements RegionObserver {
75    @Override
76    public void start(CoprocessorEnvironment e) throws IOException { }
77  
78    @Override
79    public void stop(CoprocessorEnvironment e) throws IOException { }
80  
81    @Override
82    public void preOpen(ObserverContext<RegionCoprocessorEnvironment> e) throws IOException { }
83  
84    @Override
85    public void postOpen(ObserverContext<RegionCoprocessorEnvironment> e) { }
86  
87    @Override
88    public void postLogReplay(ObserverContext<RegionCoprocessorEnvironment> e) { }
89  
90    @Override
91    public void preClose(ObserverContext<RegionCoprocessorEnvironment> c, boolean abortRequested)
92        throws IOException { }
93  
94    @Override
95    public void postClose(ObserverContext<RegionCoprocessorEnvironment> e,
96        boolean abortRequested) { }
97  
98    @Override
99    public InternalScanner preFlushScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
100       final Store store, final KeyValueScanner memstoreScanner, final InternalScanner s)
101       throws IOException {
102     return s;
103   }
104 
105   @Override
106   public void preFlush(ObserverContext<RegionCoprocessorEnvironment> e) throws IOException {
107   }
108 
109   @Override
110   public void postFlush(ObserverContext<RegionCoprocessorEnvironment> e) throws IOException {
111   }
112 
113   @Override
114   public InternalScanner preFlush(ObserverContext<RegionCoprocessorEnvironment> e, Store store,
115       InternalScanner scanner) throws IOException {
116     return scanner;
117   }
118 
119   @Override
120   public void postFlush(ObserverContext<RegionCoprocessorEnvironment> e, Store store,
121       StoreFile resultFile) throws IOException {
122   }
123 
124   @Override
125   public void preSplit(ObserverContext<RegionCoprocessorEnvironment> e) throws IOException {
126   }
127 
128   @Override
129   public void preSplit(ObserverContext<RegionCoprocessorEnvironment> c,
130       byte[] splitRow) throws IOException {
131   }
132 
133   @Override
134   public void preSplitBeforePONR(ObserverContext<RegionCoprocessorEnvironment> ctx,
135       byte[] splitKey, List<Mutation> metaEntries) throws IOException {
136   }
137 
138   @Override
139   public void preSplitAfterPONR(
140       ObserverContext<RegionCoprocessorEnvironment> ctx) throws IOException {
141   }
142 
143   @Override
144   public void preRollBackSplit(ObserverContext<RegionCoprocessorEnvironment> ctx)
145       throws IOException {
146   }
147 
148   @Override
149   public void postRollBackSplit(
150       ObserverContext<RegionCoprocessorEnvironment> ctx) throws IOException {
151   }
152 
153   @Override
154   public void postCompleteSplit(
155       ObserverContext<RegionCoprocessorEnvironment> ctx) throws IOException {
156   }
157 
158   @Override
159   public void postSplit(ObserverContext<RegionCoprocessorEnvironment> e, Region l, Region r)
160       throws IOException {
161   }
162 
163   @Override
164   public void preCompactSelection(final ObserverContext<RegionCoprocessorEnvironment> c,
165       final Store store, final List<StoreFile> candidates) throws IOException { }
166 
167   @Override
168   public void preCompactSelection(final ObserverContext<RegionCoprocessorEnvironment> c,
169       final Store store, final List<StoreFile> candidates, final CompactionRequest request)
170       throws IOException {
171     preCompactSelection(c, store, candidates);
172   }
173 
174   @Override
175   public void postCompactSelection(final ObserverContext<RegionCoprocessorEnvironment> c,
176       final Store store, final ImmutableList<StoreFile> selected) { }
177 
178   @Override
179   public void postCompactSelection(final ObserverContext<RegionCoprocessorEnvironment> c,
180       final Store store, final ImmutableList<StoreFile> selected, CompactionRequest request) {
181     postCompactSelection(c, store, selected);
182   }
183 
184   @Override
185   public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> e,
186       final Store store, final InternalScanner scanner, final ScanType scanType)
187       throws IOException {
188     return scanner;
189   }
190 
191   @Override
192   public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> e,
193       final Store store, final InternalScanner scanner, final ScanType scanType,
194       CompactionRequest request) throws IOException {
195     return preCompact(e, store, scanner, scanType);
196   }
197 
198   @Override
199   public InternalScanner preCompactScannerOpen(
200       final ObserverContext<RegionCoprocessorEnvironment> c, final Store store,
201       List<? extends KeyValueScanner> scanners, final ScanType scanType, final long earliestPutTs,
202       final InternalScanner s) throws IOException {
203     return s;
204   }
205 
206   @Override
207   public InternalScanner preCompactScannerOpen(
208       final ObserverContext<RegionCoprocessorEnvironment> c, final Store store,
209       List<? extends KeyValueScanner> scanners, final ScanType scanType, final long earliestPutTs,
210       final InternalScanner s, CompactionRequest request) throws IOException {
211     return preCompactScannerOpen(c, store, scanners, scanType, earliestPutTs, s);
212   }
213 
214   @Override
215   public void postCompact(ObserverContext<RegionCoprocessorEnvironment> e, final Store store,
216       final StoreFile resultFile) throws IOException {
217   }
218 
219 @Override
220   public void postCompact(ObserverContext<RegionCoprocessorEnvironment> e, final Store store,
221       final StoreFile resultFile, CompactionRequest request) throws IOException {
222     postCompact(e, store, resultFile);
223   }
224 
225   @Override
226   public void preGetOp(final ObserverContext<RegionCoprocessorEnvironment> e,
227       final Get get, final List<Cell> results) throws IOException {
228   }
229 
230   @Override
231   public void postGetOp(final ObserverContext<RegionCoprocessorEnvironment> e,
232       final Get get, final List<Cell> results) throws IOException {
233   }
234 
235   @Override
236   public boolean preExists(final ObserverContext<RegionCoprocessorEnvironment> e,
237       final Get get, final boolean exists) throws IOException {
238     return exists;
239   }
240 
241   @Override
242   public boolean postExists(final ObserverContext<RegionCoprocessorEnvironment> e,
243       final Get get, boolean exists) throws IOException {
244     return exists;
245   }
246 
247   @Override
248   public void prePut(final ObserverContext<RegionCoprocessorEnvironment> e,
249       final Put put, final WALEdit edit, final Durability durability) throws IOException {
250   }
251 
252   @Override
253   public void postPut(final ObserverContext<RegionCoprocessorEnvironment> e,
254       final Put put, final WALEdit edit, final Durability durability) throws IOException {
255   }
256 
257   @Override
258   public void preDelete(final ObserverContext<RegionCoprocessorEnvironment> e, final Delete delete,
259       final WALEdit edit, final Durability durability) throws IOException {
260   }
261 
262   @Override
263   public void prePrepareTimeStampForDeleteVersion(
264       final ObserverContext<RegionCoprocessorEnvironment> e, final Mutation delete,
265       final Cell cell, final byte[] byteNow, final Get get) throws IOException {
266   }
267 
268   @Override
269   public void postDelete(final ObserverContext<RegionCoprocessorEnvironment> e,
270       final Delete delete, final WALEdit edit, final Durability durability)
271       throws IOException {
272   }
273 
274   @Override
275   public void preBatchMutate(final ObserverContext<RegionCoprocessorEnvironment> c,
276       final MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
277   }
278 
279   @Override
280   public void postBatchMutate(final ObserverContext<RegionCoprocessorEnvironment> c,
281       final MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
282   }
283 
284   @Override
285   public void postBatchMutateIndispensably(final ObserverContext<RegionCoprocessorEnvironment> ctx,
286       MiniBatchOperationInProgress<Mutation> miniBatchOp, final boolean success) throws IOException {
287   }
288 
289   @Override
290   public boolean preCheckAndPut(final ObserverContext<RegionCoprocessorEnvironment> e,
291       final byte [] row, final byte [] family, final byte [] qualifier,
292       final CompareOp compareOp, final ByteArrayComparable comparator,
293       final Put put, final boolean result) throws IOException {
294     return result;
295   }
296 
297   @Override
298   public boolean preCheckAndPutAfterRowLock(
299       final ObserverContext<RegionCoprocessorEnvironment> e,
300       final byte[] row, final byte[] family, final byte[] qualifier, final CompareOp compareOp,
301       final ByteArrayComparable comparator, final Put put,
302       final boolean result) throws IOException {
303     return result;
304   }
305 
306   @Override
307   public boolean postCheckAndPut(final ObserverContext<RegionCoprocessorEnvironment> e,
308       final byte [] row, final byte [] family, final byte [] qualifier,
309       final CompareOp compareOp, final ByteArrayComparable comparator,
310       final Put put, final boolean result) throws IOException {
311     return result;
312   }
313 
314   @Override
315   public boolean preCheckAndDelete(final ObserverContext<RegionCoprocessorEnvironment> e,
316       final byte [] row, final byte [] family, final byte [] qualifier,
317       final CompareOp compareOp, final ByteArrayComparable comparator,
318       final Delete delete, final boolean result) throws IOException {
319     return result;
320   }
321 
322   @Override
323   public boolean preCheckAndDeleteAfterRowLock(
324       final ObserverContext<RegionCoprocessorEnvironment> e,
325       final byte[] row, final byte[] family, final byte[] qualifier, final CompareOp compareOp,
326       final ByteArrayComparable comparator, final Delete delete,
327       final boolean result) throws IOException {
328     return result;
329   }
330 
331   @Override
332   public boolean postCheckAndDelete(final ObserverContext<RegionCoprocessorEnvironment> e,
333       final byte [] row, final byte [] family, final byte [] qualifier,
334       final CompareOp compareOp, final ByteArrayComparable comparator,
335       final Delete delete, final boolean result) throws IOException {
336     return result;
337   }
338 
339   @Override
340   public Result preAppend(final ObserverContext<RegionCoprocessorEnvironment> e,
341       final Append append) throws IOException {
342     return null;
343   }
344 
345   @Override
346   public Result preAppendAfterRowLock(final ObserverContext<RegionCoprocessorEnvironment> e,
347       final Append append) throws IOException {
348     return null;
349   }
350 
351   @Override
352   public Result postAppend(final ObserverContext<RegionCoprocessorEnvironment> e,
353       final Append append, final Result result) throws IOException {
354     return result;
355   }
356 
357   @Override
358   public long preIncrementColumnValue(final ObserverContext<RegionCoprocessorEnvironment> e,
359       final byte [] row, final byte [] family, final byte [] qualifier,
360       final long amount, final boolean writeToWAL) throws IOException {
361     return amount;
362   }
363 
364   @Override
365   public long postIncrementColumnValue(final ObserverContext<RegionCoprocessorEnvironment> e,
366       final byte [] row, final byte [] family, final byte [] qualifier,
367       final long amount, final boolean writeToWAL, long result)
368       throws IOException {
369     return result;
370   }
371 
372   @Override
373   public Result preIncrement(final ObserverContext<RegionCoprocessorEnvironment> e,
374       final Increment increment) throws IOException {
375     return null;
376   }
377 
378   @Override
379   public Result preIncrementAfterRowLock(final ObserverContext<RegionCoprocessorEnvironment> e,
380       final Increment increment) throws IOException {
381     return null;
382   }
383 
384   @Override
385   public Result postIncrement(final ObserverContext<RegionCoprocessorEnvironment> e,
386       final Increment increment, final Result result) throws IOException {
387     return result;
388   }
389 
390   @Override
391   public RegionScanner preScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> e,
392       final Scan scan, final RegionScanner s) throws IOException {
393     return s;
394   }
395 
396   @Override
397   public KeyValueScanner preStoreScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
398       final Store store, final Scan scan, final NavigableSet<byte[]> targetCols,
399       final KeyValueScanner s) throws IOException {
400     return s;
401   }
402 
403   @Override
404   public RegionScanner postScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> e,
405       final Scan scan, final RegionScanner s) throws IOException {
406     return s;
407   }
408 
409   @Override
410   public boolean preScannerNext(final ObserverContext<RegionCoprocessorEnvironment> e,
411       final InternalScanner s, final List<Result> results,
412       final int limit, final boolean hasMore) throws IOException {
413     return hasMore;
414   }
415 
416   @Override
417   public boolean postScannerNext(final ObserverContext<RegionCoprocessorEnvironment> e,
418       final InternalScanner s, final List<Result> results, final int limit,
419       final boolean hasMore) throws IOException {
420     return hasMore;
421   }
422 
423   @Override
424   @Deprecated
425   public boolean postScannerFilterRow(final ObserverContext<RegionCoprocessorEnvironment> e,
426       final InternalScanner s, final byte[] currentRow, final int offset, final short length,
427       final boolean hasMore) throws IOException {
428     return hasMore;
429   }
430 
431   @Override
432   public boolean postScannerFilterRow(final ObserverContext<RegionCoprocessorEnvironment> e,
433       final InternalScanner s, final Cell curRowCell, final boolean hasMore) throws IOException {
434     return postScannerFilterRow(e, s, curRowCell.getRowArray(), curRowCell.getRowOffset(),
435         curRowCell.getRowLength(), hasMore);
436   }
437 
438   @Override
439   public void preScannerClose(final ObserverContext<RegionCoprocessorEnvironment> e,
440       final InternalScanner s) throws IOException {
441   }
442 
443   @Override
444   public void postScannerClose(final ObserverContext<RegionCoprocessorEnvironment> e,
445       final InternalScanner s) throws IOException {
446   }
447 
448   /**
449    * Implementers should override this version of the method and leave the deprecated one as-is.
450    */
451   @Override
452   public void preWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> env,
453       HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
454   }
455 
456   @Override
457   public void preWALRestore(ObserverContext<RegionCoprocessorEnvironment> env, HRegionInfo info,
458       HLogKey logKey, WALEdit logEdit) throws IOException {
459     preWALRestore(env, info, (WALKey)logKey, logEdit);
460   }
461 
462   /**
463    * Implementers should override this version of the method and leave the deprecated one as-is.
464    */
465   @Override
466   public void postWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> env,
467       HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
468   }
469 
470   @Override
471   public void postWALRestore(ObserverContext<RegionCoprocessorEnvironment> env,
472       HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException {
473     postWALRestore(env, info, (WALKey)logKey, logEdit);
474   }
475 
476   @Override
477   public void preBulkLoadHFile(final ObserverContext<RegionCoprocessorEnvironment> ctx,
478     List<Pair<byte[], String>> familyPaths) throws IOException {
479   }
480 
481   @Override
482   public boolean postBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx,
483     List<Pair<byte[], String>> familyPaths, boolean hasLoaded) throws IOException {
484     return hasLoaded;
485   }
486 
487   @Override
488   public Reader preStoreFileReaderOpen(ObserverContext<RegionCoprocessorEnvironment> ctx,
489       FileSystem fs, Path p, FSDataInputStreamWrapper in, long size, CacheConfig cacheConf,
490       Reference r, Reader reader) throws IOException {
491     return reader;
492   }
493 
494   @Override
495   public Reader postStoreFileReaderOpen(ObserverContext<RegionCoprocessorEnvironment> ctx,
496       FileSystem fs, Path p, FSDataInputStreamWrapper in, long size, CacheConfig cacheConf,
497       Reference r, Reader reader) throws IOException {
498     return reader;
499   }
500 
501   @Override
502   public Cell postMutationBeforeWAL(ObserverContext<RegionCoprocessorEnvironment> ctx,
503       MutationType opType, Mutation mutation, Cell oldCell, Cell newCell) throws IOException {
504     return newCell;
505   }
506 
507   @Override
508   public void postStartRegionOperation(final ObserverContext<RegionCoprocessorEnvironment> ctx,
509       Operation op) throws IOException {
510   }
511 
512   @Override
513   public void postCloseRegionOperation(final ObserverContext<RegionCoprocessorEnvironment> ctx,
514       Operation op) throws IOException {
515   }
516 
517   @Override
518   public DeleteTracker postInstantiateDeleteTracker(
519       final ObserverContext<RegionCoprocessorEnvironment> ctx, DeleteTracker delTracker)
520       throws IOException {
521     return delTracker;
522   }
523 }