1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  
21  package org.apache.hadoop.hbase.coprocessor;
22  
23  import java.io.IOException;
24  import java.lang.reflect.Method;
25  import java.util.ArrayList;
26  import java.util.Arrays;
27  import java.util.List;
28  
29  import org.apache.commons.logging.Log;
30  import org.apache.commons.logging.LogFactory;
31  import org.apache.hadoop.conf.Configuration;
32  import org.apache.hadoop.fs.FileSystem;
33  import org.apache.hadoop.fs.Path;
34  import org.apache.hadoop.hbase.*;
35  import org.apache.hadoop.hbase.client.*;
36  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
37  import org.apache.hadoop.hbase.io.hfile.HFile;
38  import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
39  import org.apache.hadoop.hbase.regionserver.HRegion;
40  import org.apache.hadoop.hbase.regionserver.InternalScanner;
41  import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
42  import org.apache.hadoop.hbase.regionserver.Store;
43  import org.apache.hadoop.hbase.regionserver.StoreFile;
44  import org.apache.hadoop.hbase.util.Bytes;
45  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
46  import org.apache.hadoop.hbase.util.JVMClusterUtil;
47  
48  import org.junit.AfterClass;
49  import org.junit.BeforeClass;
50  import org.junit.Test;
51  import org.junit.experimental.categories.Category;
52  
53  import static org.junit.Assert.*;
54  
55  @Category(MediumTests.class)
56  public class TestRegionObserverInterface {
57    static final Log LOG = LogFactory.getLog(TestRegionObserverInterface.class);
58    static final String DIR = "test/build/data/TestRegionObserver/";
59  
60    public static final byte[] TEST_TABLE = Bytes.toBytes("TestTable");
61    public final static byte[] A = Bytes.toBytes("a");
62    public final static byte[] B = Bytes.toBytes("b");
63    public final static byte[] C = Bytes.toBytes("c");
64    public final static byte[] ROW = Bytes.toBytes("testrow");
65  
66    private static HBaseTestingUtility util = new HBaseTestingUtility();
67    private static MiniHBaseCluster cluster = null;
68  
69    @BeforeClass
70    public static void setupBeforeClass() throws Exception {
71      // set configure to indicate which cp should be loaded
72      Configuration conf = util.getConfiguration();
73      conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
74          "org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver");
75  
76      util.startMiniCluster();
77      cluster = util.getMiniHBaseCluster();
78    }
79  
80    @AfterClass
81    public static void tearDownAfterClass() throws Exception {
82      util.shutdownMiniCluster();
83    }
84  
85    @Test
86    public void testRegionObserver() throws IOException {
87      byte[] tableName = TEST_TABLE;
88      // recreate table every time in order to reset the status of the
89      // coproccessor.
90      HTable table = util.createTable(tableName, new byte[][] {A, B, C});
91      verifyMethodResult(SimpleRegionObserver.class,
92          new String[] {"hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut",
93              "hadDelete"},
94          TEST_TABLE,
95          new Boolean[] {false, false, false, false, false});
96  
97      Put put = new Put(ROW);
98      put.add(A, A, A);
99      put.add(B, B, B);
100     put.add(C, C, C);
101     table.put(put);
102 
103     verifyMethodResult(SimpleRegionObserver.class,
104         new String[] {"hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut",
105             "hadPreBatchMutate", "hadPostBatchMutate", "hadDelete"},
106         TEST_TABLE,
107         new Boolean[] {false, false, true, true, true, true, false}
108     );
109 
110     Get get = new Get(ROW);
111     get.addColumn(A, A);
112     get.addColumn(B, B);
113     get.addColumn(C, C);
114     table.get(get);
115 
116     verifyMethodResult(SimpleRegionObserver.class,
117         new String[] {"hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut",
118             "hadDelete"},
119         TEST_TABLE,
120         new Boolean[] {true, true, true, true, false}
121     );
122 
123     Delete delete = new Delete(ROW);
124     delete.deleteColumn(A, A);
125     delete.deleteColumn(B, B);
126     delete.deleteColumn(C, C);
127     table.delete(delete);
128 
129     verifyMethodResult(SimpleRegionObserver.class,
130         new String[] {"hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut",
131              "hadPreBatchMutate", "hadPostBatchMutate", "hadDelete"},
132         TEST_TABLE,
133         new Boolean[] {true, true, true, true, true, true, true}
134     );
135     util.deleteTable(tableName);
136     table.close();
137   }
138 
139   @Test
140   public void testRowMutation() throws IOException {
141     byte[] tableName = TEST_TABLE;
142     HTable table = util.createTable(tableName, new byte[][] {A, B, C});
143     verifyMethodResult(SimpleRegionObserver.class,
144         new String[] {"hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut",
145             "hadDeleted"},
146         TEST_TABLE,
147         new Boolean[] {false, false, false, false, false});
148 
149     Put put = new Put(ROW);
150     put.add(A, A, A);
151     put.add(B, B, B);
152     put.add(C, C, C);
153 
154     Delete delete = new Delete(ROW);
155     delete.deleteColumn(A, A);
156     delete.deleteColumn(B, B);
157     delete.deleteColumn(C, C);
158 
159     RowMutations arm = new RowMutations(ROW);
160     arm.add(put);
161     arm.add(delete);
162     table.mutateRow(arm);
163 
164     verifyMethodResult(SimpleRegionObserver.class,
165         new String[] {"hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut",
166             "hadDeleted"},
167         TEST_TABLE,
168         new Boolean[] {false, false, true, true, true}
169     );
170     util.deleteTable(tableName);
171     table.close();
172   }
173 
174   @Test
175   public void testIncrementHook() throws IOException {
176     byte[] tableName = TEST_TABLE;
177 
178     HTable table = util.createTable(tableName, new byte[][] {A, B, C});
179     Increment inc = new Increment(Bytes.toBytes(0));
180     inc.addColumn(A, A, 1);
181 
182     verifyMethodResult(SimpleRegionObserver.class,
183         new String[] {"hadPreIncrement", "hadPostIncrement"},
184         tableName,
185         new Boolean[] {false, false}
186     );
187 
188     table.increment(inc);
189 
190     verifyMethodResult(SimpleRegionObserver.class,
191         new String[] {"hadPreIncrement", "hadPostIncrement"},
192         tableName,
193         new Boolean[] {true, true}
194     );
195     util.deleteTable(tableName);
196     table.close();
197   }
198 
199   @Test
200   // HBase-3583
201   public void testHBase3583() throws IOException {
202     byte[] tableName = Bytes.toBytes("testHBase3583");
203     util.createTable(tableName, new byte[][] {A, B, C});
204 
205     verifyMethodResult(SimpleRegionObserver.class,
206         new String[] {"hadPreGet", "hadPostGet", "wasScannerNextCalled",
207             "wasScannerCloseCalled"},
208         tableName,
209         new Boolean[] {false, false, false, false}
210     );
211 
212     HTable table = new HTable(util.getConfiguration(), tableName);
213     Put put = new Put(ROW);
214     put.add(A, A, A);
215     table.put(put);
216 
217     Get get = new Get(ROW);
218     get.addColumn(A, A);
219     table.get(get);
220 
221     // verify that scannerNext and scannerClose upcalls won't be invoked
222     // when we perform get().
223     verifyMethodResult(SimpleRegionObserver.class,
224         new String[] {"hadPreGet", "hadPostGet", "wasScannerNextCalled",
225             "wasScannerCloseCalled"},
226         tableName,
227         new Boolean[] {true, true, false, false}
228     );
229 
230     Scan s = new Scan();
231     ResultScanner scanner = table.getScanner(s);
232     try {
233       for (Result rr = scanner.next(); rr != null; rr = scanner.next()) {
234       }
235     } finally {
236       scanner.close();
237     }
238 
239     // now scanner hooks should be invoked.
240     verifyMethodResult(SimpleRegionObserver.class,
241         new String[] {"wasScannerNextCalled", "wasScannerCloseCalled"},
242         tableName,
243         new Boolean[] {true, true}
244     );
245     util.deleteTable(tableName);
246     table.close();
247   }
248 
249   @Test
250   // HBase-3758
251   public void testHBase3758() throws IOException {
252     byte[] tableName = Bytes.toBytes("testHBase3758");
253     util.createTable(tableName, new byte[][] {A, B, C});
254 
255     verifyMethodResult(SimpleRegionObserver.class,
256         new String[] {"hadDeleted", "wasScannerOpenCalled"},
257         tableName,
258         new Boolean[] {false, false}
259     );
260 
261     HTable table = new HTable(util.getConfiguration(), tableName);
262     Put put = new Put(ROW);
263     put.add(A, A, A);
264     table.put(put);
265 
266     Delete delete = new Delete(ROW);
267     table.delete(delete);
268 
269     verifyMethodResult(SimpleRegionObserver.class,
270         new String[] {"hadDeleted", "wasScannerOpenCalled"},
271         tableName,
272         new Boolean[] {true, false}
273     );
274 
275     Scan s = new Scan();
276     ResultScanner scanner = table.getScanner(s);
277     try {
278       for (Result rr = scanner.next(); rr != null; rr = scanner.next()) {
279       }
280     } finally {
281       scanner.close();
282     }
283 
284     // now scanner hooks should be invoked.
285     verifyMethodResult(SimpleRegionObserver.class,
286         new String[] {"wasScannerOpenCalled"},
287         tableName,
288         new Boolean[] {true}
289     );
290     util.deleteTable(tableName);
291     table.close();
292   }
293 
294   /* Overrides compaction to only output rows with keys that are even numbers */
295   public static class EvenOnlyCompactor extends BaseRegionObserver {
296     long lastCompaction;
297     long lastFlush;
298 
299     @Override
300     public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> e,
301         Store store, final InternalScanner scanner) {
302       return new InternalScanner() {
303         @Override
304         public boolean next(List<KeyValue> results) throws IOException {
305           return next(results, -1);
306         }
307         
308         @Override
309         public boolean next(List<KeyValue> results, String metric) 
310             throws IOException {
311           return next(results, -1, metric);
312         }
313 
314         @Override
315         public boolean next(List<KeyValue> results, int limit) 
316             throws IOException{
317           return next(results, limit, null);
318         }
319 
320         @Override
321         public boolean next(List<KeyValue> results, int limit, String metric) 
322             throws IOException {
323           List<KeyValue> internalResults = new ArrayList<KeyValue>();
324           boolean hasMore;
325           do {
326             hasMore = scanner.next(internalResults, limit, metric);
327             if (!internalResults.isEmpty()) {
328               long row = Bytes.toLong(internalResults.get(0).getRow());
329               if (row % 2 == 0) {
330                 // return this row
331                 break;
332               }
333               // clear and continue
334               internalResults.clear();
335             }
336           } while (hasMore);
337 
338           if (!internalResults.isEmpty()) {
339             results.addAll(internalResults);
340           }
341           return hasMore;
342         }
343 
344         @Override
345         public void close() throws IOException {
346           scanner.close();
347         }
348       };
349     }
350 
351     @Override
352     public void postCompact(ObserverContext<RegionCoprocessorEnvironment> e,
353         Store store, StoreFile resultFile) {
354       lastCompaction = EnvironmentEdgeManager.currentTimeMillis();
355     }
356 
357     @Override
358     public void postFlush(ObserverContext<RegionCoprocessorEnvironment> e) {
359       lastFlush = EnvironmentEdgeManager.currentTimeMillis();
360     }
361   }
362   /**
363    * Tests overriding compaction handling via coprocessor hooks
364    * @throws Exception
365    */
366   @Test
367   public void testCompactionOverride() throws Exception {
368     byte[] compactTable = Bytes.toBytes("TestCompactionOverride");
369     HBaseAdmin admin = util.getHBaseAdmin();
370     if (admin.tableExists(compactTable)) {
371       admin.disableTable(compactTable);
372       admin.deleteTable(compactTable);
373     }
374 
375     HTableDescriptor htd = new HTableDescriptor(compactTable);
376     htd.addFamily(new HColumnDescriptor(A));
377     htd.addCoprocessor(EvenOnlyCompactor.class.getName());
378     admin.createTable(htd);
379 
380     HTable table = new HTable(util.getConfiguration(), compactTable);
381     for (long i=1; i<=10; i++) {
382       byte[] iBytes = Bytes.toBytes(i);
383       Put put = new Put(iBytes);
384       put.setWriteToWAL(false);
385       put.add(A, A, iBytes);
386       table.put(put);
387     }
388 
389     HRegion firstRegion = cluster.getRegions(compactTable).get(0);
390     Coprocessor cp = firstRegion.getCoprocessorHost().findCoprocessor(
391         EvenOnlyCompactor.class.getName());
392     assertNotNull("EvenOnlyCompactor coprocessor should be loaded", cp);
393     EvenOnlyCompactor compactor = (EvenOnlyCompactor)cp;
394 
395     // force a compaction
396     long ts = System.currentTimeMillis();
397     admin.flush(compactTable);
398     // wait for flush
399     for (int i=0; i<10; i++) {
400       if (compactor.lastFlush >= ts) {
401         break;
402       }
403       Thread.sleep(1000);
404     }
405     assertTrue("Flush didn't complete", compactor.lastFlush >= ts);
406     LOG.debug("Flush complete");
407 
408     ts = compactor.lastFlush;
409     admin.majorCompact(compactTable);
410     // wait for compaction
411     for (int i=0; i<30; i++) {
412       if (compactor.lastCompaction >= ts) {
413         break;
414       }
415       Thread.sleep(1000);
416     }
417     LOG.debug("Last compaction was at "+compactor.lastCompaction);
418     assertTrue("Compaction didn't complete", compactor.lastCompaction >= ts);
419 
420     // only even rows should remain
421     ResultScanner scanner = table.getScanner(new Scan());
422     try {
423       for (long i=2; i<=10; i+=2) {
424         Result r = scanner.next();
425         assertNotNull(r);
426         assertFalse(r.isEmpty());
427         byte[] iBytes = Bytes.toBytes(i);
428         assertArrayEquals("Row should be "+i, r.getRow(), iBytes);
429         assertArrayEquals("Value should be "+i, r.getValue(A, A), iBytes);
430       }
431     } finally {
432       scanner.close();
433     }
434     table.close();
435   }
436 
437   @Test
438   public void bulkLoadHFileTest() throws Exception {
439     String testName = TestRegionObserverInterface.class.getName()+".bulkLoadHFileTest";
440     byte[] tableName = TEST_TABLE;
441     Configuration conf = util.getConfiguration();
442     HTable table = util.createTable(tableName, new byte[][] {A, B, C});
443 
444     verifyMethodResult(SimpleRegionObserver.class,
445         new String[] {"hadPreBulkLoadHFile", "hadPostBulkLoadHFile"},
446         tableName,
447         new Boolean[] {false, false}
448     );
449 
450     FileSystem fs = util.getTestFileSystem();
451     final Path dir = util.getDataTestDir(testName).makeQualified(fs);
452     Path familyDir = new Path(dir, Bytes.toString(A));
453 
454     createHFile(util.getConfiguration(), fs, new Path(familyDir,Bytes.toString(A)), A, A);
455 
456     //Bulk load
457     new LoadIncrementalHFiles(conf).doBulkLoad(dir, new HTable(conf, tableName));
458 
459     verifyMethodResult(SimpleRegionObserver.class,
460         new String[] {"hadPreBulkLoadHFile", "hadPostBulkLoadHFile"},
461         tableName,
462         new Boolean[] {true, true}
463     );
464     util.deleteTable(tableName);
465     table.close();
466   }
467 
468   // check each region whether the coprocessor upcalls are called or not.
469   private void verifyMethodResult(Class c, String methodName[], byte[] tableName,
470                                   Object value[]) throws IOException {
471     try {
472       for (JVMClusterUtil.RegionServerThread t : cluster.getRegionServerThreads()) {
473         for (HRegionInfo r : t.getRegionServer().getOnlineRegions()) {
474           if (!Arrays.equals(r.getTableName(), tableName)) {
475             continue;
476           }
477           RegionCoprocessorHost cph = t.getRegionServer().getOnlineRegion(r.getRegionName()).
478               getCoprocessorHost();
479 
480           Coprocessor cp = cph.findCoprocessor(c.getName());
481           assertNotNull(cp);
482           for (int i = 0; i < methodName.length; ++i) {
483             Method m = c.getMethod(methodName[i]);
484             Object o = m.invoke(cp);
485             assertTrue("Result of " + c.getName() + "." + methodName[i]
486                 + " is expected to be " + value[i].toString()
487                 + ", while we get " + o.toString(), o.equals(value[i]));
488           }
489         }
490       }
491     } catch (Exception e) {
492       throw new IOException(e.toString());
493     }
494   }
495 
496   private static void createHFile(
497       Configuration conf,
498       FileSystem fs, Path path,
499       byte[] family, byte[] qualifier) throws IOException {
500     HFile.Writer writer = HFile.getWriterFactory(conf, new CacheConfig(conf))
501         .withPath(fs, path)
502         .withComparator(KeyValue.KEY_COMPARATOR)
503         .create();
504     long now = System.currentTimeMillis();
505     try {
506       for (int i =1;i<=9;i++) {
507         KeyValue kv = new KeyValue(Bytes.toBytes(i+""), family, qualifier, now, Bytes.toBytes(i+""));
508         writer.append(kv);
509       }
510     } finally {
511       writer.close();
512     }
513   }
514 
515   private static byte [][] makeN(byte [] base, int n) {
516     byte [][] ret = new byte[n][];
517     for(int i=0;i<n;i++) {
518       ret[i] = Bytes.add(base, Bytes.toBytes(String.format("%02d", i)));
519     }
520     return ret;
521   }
522 
523   @org.junit.Rule
524   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
525     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
526 }
527 
528