View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  
20  package org.apache.hadoop.hbase.coprocessor;
21  
22  import static org.junit.Assert.assertArrayEquals;
23  import static org.junit.Assert.assertFalse;
24  import static org.junit.Assert.assertNotNull;
25  import static org.junit.Assert.assertTrue;
26  
27  import java.io.IOException;
28  import java.lang.reflect.Method;
29  import java.util.ArrayList;
30  import java.util.List;
31  
32  import org.apache.commons.logging.Log;
33  import org.apache.commons.logging.LogFactory;
34  import org.apache.hadoop.conf.Configuration;
35  import org.apache.hadoop.fs.FileSystem;
36  import org.apache.hadoop.fs.Path;
37  import org.apache.hadoop.hbase.Cell;
38  import org.apache.hadoop.hbase.CellUtil;
39  import org.apache.hadoop.hbase.Coprocessor;
40  import org.apache.hadoop.hbase.HBaseTestingUtility;
41  import org.apache.hadoop.hbase.HColumnDescriptor;
42  import org.apache.hadoop.hbase.HRegionInfo;
43  import org.apache.hadoop.hbase.HTableDescriptor;
44  import org.apache.hadoop.hbase.KeyValue;
45  import org.apache.hadoop.hbase.testclassification.MediumTests;
46  import org.apache.hadoop.hbase.MiniHBaseCluster;
47  import org.apache.hadoop.hbase.ServerName;
48  import org.apache.hadoop.hbase.TableName;
49  import org.apache.hadoop.hbase.client.Admin;
50  import org.apache.hadoop.hbase.client.Append;
51  import org.apache.hadoop.hbase.client.Delete;
52  import org.apache.hadoop.hbase.client.Durability;
53  import org.apache.hadoop.hbase.client.Get;
54  import org.apache.hadoop.hbase.client.HTable;
55  import org.apache.hadoop.hbase.client.Increment;
56  import org.apache.hadoop.hbase.client.Put;
57  import org.apache.hadoop.hbase.client.Result;
58  import org.apache.hadoop.hbase.client.ResultScanner;
59  import org.apache.hadoop.hbase.client.RowMutations;
60  import org.apache.hadoop.hbase.client.Scan;
61  import org.apache.hadoop.hbase.client.Table;
62  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
63  import org.apache.hadoop.hbase.io.hfile.HFile;
64  import org.apache.hadoop.hbase.io.hfile.HFileContext;
65  import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
66  import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
67  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
68  import org.apache.hadoop.hbase.regionserver.HRegion;
69  import org.apache.hadoop.hbase.regionserver.InternalScanner;
70  import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
71  import org.apache.hadoop.hbase.regionserver.ScanType;
72  import org.apache.hadoop.hbase.regionserver.Store;
73  import org.apache.hadoop.hbase.regionserver.StoreFile;
74  import org.apache.hadoop.hbase.util.Bytes;
75  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
76  import org.apache.hadoop.hbase.util.JVMClusterUtil;
77  import org.apache.hadoop.hbase.util.Threads;
78  import org.junit.AfterClass;
79  import org.junit.BeforeClass;
80  import org.junit.Test;
81  import org.junit.experimental.categories.Category;
82  
83  @Category(MediumTests.class)
84  public class TestRegionObserverInterface {
85    static final Log LOG = LogFactory.getLog(TestRegionObserverInterface.class);
86  
87    public static final TableName TEST_TABLE = TableName.valueOf("TestTable");
88    public final static byte[] A = Bytes.toBytes("a");
89    public final static byte[] B = Bytes.toBytes("b");
90    public final static byte[] C = Bytes.toBytes("c");
91    public final static byte[] ROW = Bytes.toBytes("testrow");
92  
93    private static HBaseTestingUtility util = new HBaseTestingUtility();
94    private static MiniHBaseCluster cluster = null;
95  
96    @BeforeClass
97    public static void setupBeforeClass() throws Exception {
98      // set configure to indicate which cp should be loaded
99      Configuration conf = util.getConfiguration();
100     conf.setBoolean("hbase.master.distributed.log.replay", true);
101     conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
102         "org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver",
103         "org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver$Legacy");
104 
105     util.startMiniCluster();
106     cluster = util.getMiniHBaseCluster();
107   }
108 
109   @AfterClass
110   public static void tearDownAfterClass() throws Exception {
111     util.shutdownMiniCluster();
112   }
113 
114   @Test (timeout=300000)
115   public void testRegionObserver() throws IOException {
116     TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testRegionObserver");
117     // recreate table every time in order to reset the status of the
118     // coprocessor.
119     Table table = util.createTable(tableName, new byte[][] {A, B, C});
120     try {
121       verifyMethodResult(SimpleRegionObserver.class, new String[] { "hadPreGet", "hadPostGet",
122           "hadPrePut", "hadPostPut", "hadDelete", "hadPostStartRegionOperation",
123           "hadPostCloseRegionOperation", "hadPostBatchMutateIndispensably" }, tableName,
124         new Boolean[] { false, false, false, false, false, false, false, false });
125 
126       Put put = new Put(ROW);
127       put.add(A, A, A);
128       put.add(B, B, B);
129       put.add(C, C, C);
130       table.put(put);
131 
132       verifyMethodResult(SimpleRegionObserver.class, new String[] { "hadPreGet", "hadPostGet",
133           "hadPrePut", "hadPostPut", "hadPreBatchMutate", "hadPostBatchMutate", "hadDelete",
134           "hadPostStartRegionOperation", "hadPostCloseRegionOperation",
135           "hadPostBatchMutateIndispensably" }, TEST_TABLE, new Boolean[] { false, false, true,
136           true, true, true, false, true, true, true });
137 
138       verifyMethodResult(SimpleRegionObserver.class,
139           new String[] {"getCtPreOpen", "getCtPostOpen", "getCtPreClose", "getCtPostClose"},
140           tableName,
141           new Integer[] {1, 1, 0, 0});
142 
143       Get get = new Get(ROW);
144       get.addColumn(A, A);
145       get.addColumn(B, B);
146       get.addColumn(C, C);
147       table.get(get);
148 
149       verifyMethodResult(SimpleRegionObserver.class,
150           new String[] {"hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut",
151       "hadDelete", "hadPrePreparedDeleteTS"},
152       tableName,
153       new Boolean[] {true, true, true, true, false, false}
154           );
155 
156       Delete delete = new Delete(ROW);
157       delete.deleteColumn(A, A);
158       delete.deleteColumn(B, B);
159       delete.deleteColumn(C, C);
160       table.delete(delete);
161 
162       verifyMethodResult(SimpleRegionObserver.class,
163           new String[] {"hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut",
164         "hadPreBatchMutate", "hadPostBatchMutate", "hadDelete", "hadPrePreparedDeleteTS"},
165         tableName,
166         new Boolean[] {true, true, true, true, true, true, true, true}
167           );
168     } finally {
169       util.deleteTable(tableName);
170       table.close();
171     }
172     verifyMethodResult(SimpleRegionObserver.class,
173         new String[] {"getCtPreOpen", "getCtPostOpen", "getCtPreClose", "getCtPostClose"},
174         tableName,
175         new Integer[] {1, 1, 1, 1});
176   }
177 
178   @Test (timeout=300000)
179   public void testRowMutation() throws IOException {
180     TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testRowMutation");
181     Table table = util.createTable(tableName, new byte[][] {A, B, C});
182     try {
183       verifyMethodResult(SimpleRegionObserver.class,
184         new String[] {"hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut",
185             "hadDeleted"},
186         tableName,
187         new Boolean[] {false, false, false, false, false});
188       Put put = new Put(ROW);
189       put.add(A, A, A);
190       put.add(B, B, B);
191       put.add(C, C, C);
192 
193       Delete delete = new Delete(ROW);
194       delete.deleteColumn(A, A);
195       delete.deleteColumn(B, B);
196       delete.deleteColumn(C, C);
197 
198       RowMutations arm = new RowMutations(ROW);
199       arm.add(put);
200       arm.add(delete);
201       table.mutateRow(arm);
202 
203       verifyMethodResult(SimpleRegionObserver.class,
204           new String[] {"hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut",
205       "hadDeleted"},
206       tableName,
207       new Boolean[] {false, false, true, true, true}
208           );
209     } finally {
210       util.deleteTable(tableName);
211       table.close();
212     }
213   }
214 
215   @Test (timeout=300000)
216   public void testIncrementHook() throws IOException {
217     TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testIncrementHook");
218     Table table = util.createTable(tableName, new byte[][] {A, B, C});
219     try {
220       Increment inc = new Increment(Bytes.toBytes(0));
221       inc.addColumn(A, A, 1);
222 
223       verifyMethodResult(SimpleRegionObserver.class,
224           new String[] {"hadPreIncrement", "hadPostIncrement", "hadPreIncrementAfterRowLock"},
225           tableName,
226           new Boolean[] {false, false, false}
227           );
228 
229       table.increment(inc);
230 
231       verifyMethodResult(SimpleRegionObserver.class,
232           new String[] {"hadPreIncrement", "hadPostIncrement", "hadPreIncrementAfterRowLock"},
233           tableName,
234           new Boolean[] {true, true, true}
235           );
236     } finally {
237       util.deleteTable(tableName);
238       table.close();
239     }
240   }
241 
242   @Test (timeout=300000)
243   public void testCheckAndPutHooks() throws IOException {
244     TableName tableName =
245         TableName.valueOf(TEST_TABLE.getNameAsString() + ".testCheckAndPutHooks");
246     try (Table table = util.createTable(tableName, new byte[][] {A, B, C})) {
247       Put p = new Put(Bytes.toBytes(0));
248       p.add(A, A, A);
249       table.put(p);
250       p = new Put(Bytes.toBytes(0));
251       p.add(A, A, A);
252       verifyMethodResult(SimpleRegionObserver.class,
253           new String[] {"hadPreCheckAndPut",
254               "hadPreCheckAndPutAfterRowLock", "hadPostCheckAndPut"},
255           tableName,
256           new Boolean[] {false, false, false}
257           );
258       table.checkAndPut(Bytes.toBytes(0), A, A, A, p);
259       verifyMethodResult(SimpleRegionObserver.class,
260           new String[] {"hadPreCheckAndPut",
261               "hadPreCheckAndPutAfterRowLock", "hadPostCheckAndPut"},
262           tableName,
263           new Boolean[] {true, true, true}
264           );
265     } finally {
266       util.deleteTable(tableName);
267     }
268   }
269 
270   @Test (timeout=300000)
271   public void testCheckAndDeleteHooks() throws IOException {
272     TableName tableName =
273         TableName.valueOf(TEST_TABLE.getNameAsString() + ".testCheckAndDeleteHooks");
274     Table table = util.createTable(tableName, new byte[][] {A, B, C});
275     try {
276       Put p = new Put(Bytes.toBytes(0));
277       p.add(A, A, A);
278       table.put(p);
279       Delete d = new Delete(Bytes.toBytes(0));
280       table.delete(d);
281       verifyMethodResult(SimpleRegionObserver.class,
282           new String[] {"hadPreCheckAndDelete",
283               "hadPreCheckAndDeleteAfterRowLock", "hadPostCheckAndDelete"},
284           tableName,
285           new Boolean[] {false, false, false}
286           );
287       table.checkAndDelete(Bytes.toBytes(0), A, A, A, d);
288       verifyMethodResult(SimpleRegionObserver.class,
289           new String[] {"hadPreCheckAndDelete",
290               "hadPreCheckAndDeleteAfterRowLock", "hadPostCheckAndDelete"},
291           tableName,
292           new Boolean[] {true, true, true}
293           );
294     } finally {
295       util.deleteTable(tableName);
296       table.close();
297     }
298   }
299 
300   @Test (timeout=300000)
301   public void testAppendHook() throws IOException {
302     TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testAppendHook");
303     Table table = util.createTable(tableName, new byte[][] {A, B, C});
304     try {
305       Append app = new Append(Bytes.toBytes(0));
306       app.add(A, A, A);
307 
308       verifyMethodResult(SimpleRegionObserver.class,
309           new String[] {"hadPreAppend", "hadPostAppend", "hadPreAppendAfterRowLock"},
310           tableName,
311           new Boolean[] {false, false, false}
312           );
313 
314       table.append(app);
315 
316       verifyMethodResult(SimpleRegionObserver.class,
317           new String[] {"hadPreAppend", "hadPostAppend", "hadPreAppendAfterRowLock"},
318           tableName,
319           new Boolean[] {true, true, true}
320           );
321     } finally {
322       util.deleteTable(tableName);
323       table.close();
324     }
325   }
326 
327   @Test (timeout=300000)
328   // HBase-3583
329   public void testHBase3583() throws IOException {
330     TableName tableName =
331         TableName.valueOf("testHBase3583");
332     util.createTable(tableName, new byte[][] {A, B, C});
333     util.waitUntilAllRegionsAssigned(tableName);
334 
335     verifyMethodResult(SimpleRegionObserver.class,
336         new String[] {"hadPreGet", "hadPostGet", "wasScannerNextCalled",
337             "wasScannerCloseCalled"},
338         tableName,
339         new Boolean[] {false, false, false, false}
340     );
341 
342     Table table = new HTable(util.getConfiguration(), tableName);
343     Put put = new Put(ROW);
344     put.add(A, A, A);
345     table.put(put);
346 
347     Get get = new Get(ROW);
348     get.addColumn(A, A);
349     table.get(get);
350 
351     // verify that scannerNext and scannerClose upcalls won't be invoked
352     // when we perform get().
353     verifyMethodResult(SimpleRegionObserver.class,
354         new String[] {"hadPreGet", "hadPostGet", "wasScannerNextCalled",
355             "wasScannerCloseCalled"},
356         tableName,
357         new Boolean[] {true, true, false, false}
358     );
359 
360     Scan s = new Scan();
361     ResultScanner scanner = table.getScanner(s);
362     try {
363       for (Result rr = scanner.next(); rr != null; rr = scanner.next()) {
364       }
365     } finally {
366       scanner.close();
367     }
368 
369     // now scanner hooks should be invoked.
370     verifyMethodResult(SimpleRegionObserver.class,
371         new String[] {"wasScannerNextCalled", "wasScannerCloseCalled"},
372         tableName,
373         new Boolean[] {true, true}
374     );
375     util.deleteTable(tableName);
376     table.close();
377   }
378 
379   @Test (timeout=300000)
380   // HBase-3758
381   public void testHBase3758() throws IOException {
382     TableName tableName =
383         TableName.valueOf("testHBase3758");
384     util.createTable(tableName, new byte[][] {A, B, C});
385 
386     verifyMethodResult(SimpleRegionObserver.class,
387         new String[] {"hadDeleted", "wasScannerOpenCalled"},
388         tableName,
389         new Boolean[] {false, false}
390     );
391 
392     Table table = new HTable(util.getConfiguration(), tableName);
393     Put put = new Put(ROW);
394     put.add(A, A, A);
395     table.put(put);
396 
397     Delete delete = new Delete(ROW);
398     table.delete(delete);
399 
400     verifyMethodResult(SimpleRegionObserver.class,
401         new String[] {"hadDeleted", "wasScannerOpenCalled"},
402         tableName,
403         new Boolean[] {true, false}
404     );
405 
406     Scan s = new Scan();
407     ResultScanner scanner = table.getScanner(s);
408     try {
409       for (Result rr = scanner.next(); rr != null; rr = scanner.next()) {
410       }
411     } finally {
412       scanner.close();
413     }
414 
415     // now scanner hooks should be invoked.
416     verifyMethodResult(SimpleRegionObserver.class,
417         new String[] {"wasScannerOpenCalled"},
418         tableName,
419         new Boolean[] {true}
420     );
421     util.deleteTable(tableName);
422     table.close();
423   }
424 
425   /* Overrides compaction to only output rows with keys that are even numbers */
426   public static class EvenOnlyCompactor extends BaseRegionObserver {
427     long lastCompaction;
428     long lastFlush;
429 
430     @Override
431     public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> e,
432         Store store, final InternalScanner scanner, final ScanType scanType) {
433       return new InternalScanner() {
434         @Override
435         public boolean next(List<Cell> results) throws IOException {
436           return next(results, -1);
437         }
438 
439         @Override
440         public boolean next(List<Cell> results, int limit)
441             throws IOException{
442           List<Cell> internalResults = new ArrayList<Cell>();
443           boolean hasMore;
444           do {
445             hasMore = scanner.next(internalResults, limit);
446             if (!internalResults.isEmpty()) {
447               long row = Bytes.toLong(CellUtil.cloneValue(internalResults.get(0)));
448               if (row % 2 == 0) {
449                 // return this row
450                 break;
451               }
452               // clear and continue
453               internalResults.clear();
454             }
455           } while (hasMore);
456 
457           if (!internalResults.isEmpty()) {
458             results.addAll(internalResults);
459           }
460           return hasMore;
461         }
462 
463         @Override
464         public void close() throws IOException {
465           scanner.close();
466         }
467       };
468     }
469 
470     @Override
471     public void postCompact(ObserverContext<RegionCoprocessorEnvironment> e,
472         Store store, StoreFile resultFile) {
473       lastCompaction = EnvironmentEdgeManager.currentTime();
474     }
475 
476     @Override
477     public void postFlush(ObserverContext<RegionCoprocessorEnvironment> e) {
478       lastFlush = EnvironmentEdgeManager.currentTime();
479     }
480   }
481   /**
482    * Tests overriding compaction handling via coprocessor hooks
483    * @throws Exception
484    */
485   @Test (timeout=300000)
486   public void testCompactionOverride() throws Exception {
487     TableName compactTable = TableName.valueOf("TestCompactionOverride");
488     Admin admin = util.getHBaseAdmin();
489     if (admin.tableExists(compactTable)) {
490       admin.disableTable(compactTable);
491       admin.deleteTable(compactTable);
492     }
493 
494     HTableDescriptor htd = new HTableDescriptor(compactTable);
495     htd.addFamily(new HColumnDescriptor(A));
496     htd.addCoprocessor(EvenOnlyCompactor.class.getName());
497     admin.createTable(htd);
498 
499     Table table = new HTable(util.getConfiguration(), compactTable);
500     for (long i=1; i<=10; i++) {
501       byte[] iBytes = Bytes.toBytes(i);
502       Put put = new Put(iBytes);
503       put.setDurability(Durability.SKIP_WAL);
504       put.add(A, A, iBytes);
505       table.put(put);
506     }
507 
508     HRegion firstRegion = cluster.getRegions(compactTable).get(0);
509     Coprocessor cp = firstRegion.getCoprocessorHost().findCoprocessor(
510         EvenOnlyCompactor.class.getName());
511     assertNotNull("EvenOnlyCompactor coprocessor should be loaded", cp);
512     EvenOnlyCompactor compactor = (EvenOnlyCompactor)cp;
513 
514     // force a compaction
515     long ts = System.currentTimeMillis();
516     admin.flush(compactTable);
517     // wait for flush
518     for (int i=0; i<10; i++) {
519       if (compactor.lastFlush >= ts) {
520         break;
521       }
522       Thread.sleep(1000);
523     }
524     assertTrue("Flush didn't complete", compactor.lastFlush >= ts);
525     LOG.debug("Flush complete");
526 
527     ts = compactor.lastFlush;
528     admin.majorCompact(compactTable);
529     // wait for compaction
530     for (int i=0; i<30; i++) {
531       if (compactor.lastCompaction >= ts) {
532         break;
533       }
534       Thread.sleep(1000);
535     }
536     LOG.debug("Last compaction was at "+compactor.lastCompaction);
537     assertTrue("Compaction didn't complete", compactor.lastCompaction >= ts);
538 
539     // only even rows should remain
540     ResultScanner scanner = table.getScanner(new Scan());
541     try {
542       for (long i=2; i<=10; i+=2) {
543         Result r = scanner.next();
544         assertNotNull(r);
545         assertFalse(r.isEmpty());
546         byte[] iBytes = Bytes.toBytes(i);
547         assertArrayEquals("Row should be "+i, r.getRow(), iBytes);
548         assertArrayEquals("Value should be "+i, r.getValue(A, A), iBytes);
549       }
550     } finally {
551       scanner.close();
552     }
553     table.close();
554   }
555 
556   @Test (timeout=300000)
557   public void bulkLoadHFileTest() throws Exception {
558     String testName = TestRegionObserverInterface.class.getName()+".bulkLoadHFileTest";
559     TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".bulkLoadHFileTest");
560     Configuration conf = util.getConfiguration();
561     HTable table = util.createTable(tableName, new byte[][] {A, B, C});
562     try {
563       verifyMethodResult(SimpleRegionObserver.class,
564           new String[] {"hadPreBulkLoadHFile", "hadPostBulkLoadHFile"},
565           tableName,
566           new Boolean[] {false, false}
567           );
568 
569       FileSystem fs = util.getTestFileSystem();
570       final Path dir = util.getDataTestDirOnTestFS(testName).makeQualified(fs);
571       Path familyDir = new Path(dir, Bytes.toString(A));
572 
573       createHFile(util.getConfiguration(), fs, new Path(familyDir,Bytes.toString(A)), A, A);
574 
575       // Bulk load
576       new LoadIncrementalHFiles(conf).doBulkLoad(dir, table);
577 
578       verifyMethodResult(SimpleRegionObserver.class,
579           new String[] {"hadPreBulkLoadHFile", "hadPostBulkLoadHFile"},
580           tableName,
581           new Boolean[] {true, true}
582           );
583     } finally {
584       util.deleteTable(tableName);
585       table.close();
586     }
587   }
588 
589   @Test (timeout=300000)
590   public void testRecovery() throws Exception {
591     LOG.info(TestRegionObserverInterface.class.getName() +".testRecovery");
592     TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testRecovery");
593     HTable table = util.createTable(tableName, new byte[][] {A, B, C});
594     try {
595       JVMClusterUtil.RegionServerThread rs1 = cluster.startRegionServer();
596       ServerName sn2 = rs1.getRegionServer().getServerName();
597       String regEN = table.getRegionLocations().firstEntry().getKey().getEncodedName();
598 
599       util.getHBaseAdmin().move(regEN.getBytes(), sn2.getServerName().getBytes());
600       while (!sn2.equals(table.getRegionLocations().firstEntry().getValue() )){
601         Thread.sleep(100);
602       }
603 
604       Put put = new Put(ROW);
605       put.add(A, A, A);
606       put.add(B, B, B);
607       put.add(C, C, C);
608       table.put(put);
609 
610       verifyMethodResult(SimpleRegionObserver.class,
611           new String[] {"hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut",
612         "hadPreBatchMutate", "hadPostBatchMutate", "hadDelete"},
613         tableName,
614         new Boolean[] {false, false, true, true, true, true, false}
615           );
616 
617       verifyMethodResult(SimpleRegionObserver.class,
618           new String[] {"getCtPreWALRestore", "getCtPostWALRestore", "getCtPrePut", "getCtPostPut",
619               "getCtPreWALRestoreDeprecated", "getCtPostWALRestoreDeprecated"},
620           tableName,
621           new Integer[] {0, 0, 1, 1, 0, 0});
622 
623       cluster.killRegionServer(rs1.getRegionServer().getServerName());
624       Threads.sleep(1000); // Let the kill soak in.
625       util.waitUntilAllRegionsAssigned(tableName);
626       LOG.info("All regions assigned");
627 
628       verifyMethodResult(SimpleRegionObserver.class,
629           new String[] {"getCtPreWALRestore", "getCtPostWALRestore", "getCtPrePut", "getCtPostPut",
630               "getCtPreWALRestoreDeprecated", "getCtPostWALRestoreDeprecated"},
631           tableName,
632           new Integer[]{1, 1, 0, 0, 0, 0});
633     } finally {
634       util.deleteTable(tableName);
635       table.close();
636     }
637   }
638 
639   @Test (timeout=300000)
640   public void testLegacyRecovery() throws Exception {
641     LOG.info(TestRegionObserverInterface.class.getName() +".testLegacyRecovery");
642     TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testLegacyRecovery");
643     HTable table = util.createTable(tableName, new byte[][] {A, B, C});
644     try {
645       JVMClusterUtil.RegionServerThread rs1 = cluster.startRegionServer();
646       ServerName sn2 = rs1.getRegionServer().getServerName();
647       String regEN = table.getRegionLocations().firstEntry().getKey().getEncodedName();
648 
649       util.getHBaseAdmin().move(regEN.getBytes(), sn2.getServerName().getBytes());
650       while (!sn2.equals(table.getRegionLocations().firstEntry().getValue() )){
651         Thread.sleep(100);
652       }
653 
654       Put put = new Put(ROW);
655       put.add(A, A, A);
656       put.add(B, B, B);
657       put.add(C, C, C);
658       table.put(put);
659 
660       verifyMethodResult(SimpleRegionObserver.Legacy.class,
661           new String[] {"hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut",
662         "hadPreBatchMutate", "hadPostBatchMutate", "hadDelete"},
663         tableName,
664         new Boolean[] {false, false, true, true, true, true, false}
665           );
666 
667       verifyMethodResult(SimpleRegionObserver.Legacy.class,
668           new String[] {"getCtPreWALRestore", "getCtPostWALRestore", "getCtPrePut", "getCtPostPut",
669               "getCtPreWALRestoreDeprecated", "getCtPostWALRestoreDeprecated"},
670           tableName,
671           new Integer[] {0, 0, 1, 1, 0, 0});
672 
673       cluster.killRegionServer(rs1.getRegionServer().getServerName());
674       Threads.sleep(1000); // Let the kill soak in.
675       util.waitUntilAllRegionsAssigned(tableName);
676       LOG.info("All regions assigned");
677 
678       verifyMethodResult(SimpleRegionObserver.Legacy.class,
679           new String[] {"getCtPreWALRestore", "getCtPostWALRestore", "getCtPrePut", "getCtPostPut",
680               "getCtPreWALRestoreDeprecated", "getCtPostWALRestoreDeprecated"},
681           tableName,
682           new Integer[]{1, 1, 0, 0, 1, 1});
683     } finally {
684       util.deleteTable(tableName);
685       table.close();
686     }
687   }
688 
689   @Test (timeout=300000)
690   public void testPreWALRestoreSkip() throws Exception {
691     LOG.info(TestRegionObserverInterface.class.getName() + ".testPreWALRestoreSkip");
692     TableName tableName = TableName.valueOf(SimpleRegionObserver.TABLE_SKIPPED);
693     HTable table = util.createTable(tableName, new byte[][] { A, B, C });
694 
695     JVMClusterUtil.RegionServerThread rs1 = cluster.startRegionServer();
696     ServerName sn2 = rs1.getRegionServer().getServerName();
697     String regEN = table.getRegionLocations().firstEntry().getKey().getEncodedName();
698 
699     util.getHBaseAdmin().move(regEN.getBytes(), sn2.getServerName().getBytes());
700     while (!sn2.equals(table.getRegionLocations().firstEntry().getValue())) {
701       Thread.sleep(100);
702     }
703 
704     Put put = new Put(ROW);
705     put.add(A, A, A);
706     put.add(B, B, B);
707     put.add(C, C, C);
708     table.put(put);
709     table.flushCommits();
710 
711     cluster.killRegionServer(rs1.getRegionServer().getServerName());
712     Threads.sleep(20000); // just to be sure that the kill has fully started.
713     util.waitUntilAllRegionsAssigned(tableName);
714 
715     verifyMethodResult(SimpleRegionObserver.class, new String[] { "getCtPreWALRestore",
716         "getCtPostWALRestore", "getCtPreWALRestoreDeprecated", "getCtPostWALRestoreDeprecated"},
717         tableName,
718         new Integer[] {0, 0, 0, 0});
719 
720     util.deleteTable(tableName);
721     table.close();
722   }
723 
724   // check each region whether the coprocessor upcalls are called or not.
725   private void verifyMethodResult(Class<?> c, String methodName[], TableName tableName,
726                                   Object value[]) throws IOException {
727     try {
728       for (JVMClusterUtil.RegionServerThread t : cluster.getRegionServerThreads()) {
729         if (!t.isAlive() || t.getRegionServer().isAborted() || t.getRegionServer().isStopping()){
730           continue;
731         }
732         for (HRegionInfo r : ProtobufUtil.getOnlineRegions(t.getRegionServer().getRSRpcServices())) {
733           if (!r.getTable().equals(tableName)) {
734             continue;
735           }
736           RegionCoprocessorHost cph = t.getRegionServer().getOnlineRegion(r.getRegionName()).
737               getCoprocessorHost();
738 
739           Coprocessor cp = cph.findCoprocessor(c.getName());
740           assertNotNull(cp);
741           for (int i = 0; i < methodName.length; ++i) {
742             Method m = c.getMethod(methodName[i]);
743             Object o = m.invoke(cp);
744             assertTrue("Result of " + c.getName() + "." + methodName[i]
745                 + " is expected to be " + value[i].toString()
746                 + ", while we get " + o.toString(), o.equals(value[i]));
747           }
748         }
749       }
750     } catch (Exception e) {
751       throw new IOException(e.toString());
752     }
753   }
754 
755   private static void createHFile(
756       Configuration conf,
757       FileSystem fs, Path path,
758       byte[] family, byte[] qualifier) throws IOException {
759     HFileContext context = new HFileContextBuilder().build();
760     HFile.Writer writer = HFile.getWriterFactory(conf, new CacheConfig(conf))
761         .withPath(fs, path)
762         .withFileContext(context)
763         .create();
764     long now = System.currentTimeMillis();
765     try {
766       for (int i =1;i<=9;i++) {
767         KeyValue kv = new KeyValue(Bytes.toBytes(i+""), family, qualifier, now, Bytes.toBytes(i+""));
768         writer.append(kv);
769       }
770     } finally {
771       writer.close();
772     }
773   }
774 }