/**
   * Unfortunately, the easiest way to test this is to spin up a mini-cluster since we want to do
   * the usual compaction mechanism on the region, rather than going through the backdoor to the
   * region
   */
  @Test
  public void testRegionObserverCompactionTimeStacking() throws Exception {
    // setup a mini cluster so we can do a real compaction on a region
    Configuration conf = UTIL.getConfiguration();
    conf.setClass(HConstants.REGION_IMPL, CompactionCompletionNotifyingRegion.class, HRegion.class);
    conf.setInt("hbase.hstore.compaction.min", 2);
    UTIL.startMiniCluster();
    String tableName = "testRegionObserverCompactionTimeStacking";
    byte[] ROW = Bytes.toBytes("testRow");
    byte[] A = Bytes.toBytes("A");
    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
    desc.addFamily(new HColumnDescriptor(A));
    desc.addCoprocessor(EmptyRegionObsever.class.getName(), null, Coprocessor.PRIORITY_USER, null);
    desc.addCoprocessor(
        NoDataFromCompaction.class.getName(), null, Coprocessor.PRIORITY_HIGHEST, null);

    Admin admin = UTIL.getHBaseAdmin();
    admin.createTable(desc);

    Table table = UTIL.getConnection().getTable(desc.getTableName());

    // put a row and flush it to disk
    Put put = new Put(ROW);
    put.add(A, A, A);
    table.put(put);

    HRegionServer rs = UTIL.getRSForFirstRegionInTable(desc.getTableName());
    List<HRegion> regions = rs.getOnlineRegions(desc.getTableName());
    assertEquals("More than 1 region serving test table with 1 row", 1, regions.size());
    HRegion region = regions.get(0);
    admin.flushRegion(region.getRegionName());
    CountDownLatch latch =
        ((CompactionCompletionNotifyingRegion) region).getCompactionStateChangeLatch();

    // put another row and flush that too
    put = new Put(Bytes.toBytes("anotherrow"));
    put.add(A, A, A);
    table.put(put);
    admin.flushRegion(region.getRegionName());

    // run a compaction, which normally would should get rid of the data
    // wait for the compaction checker to complete
    latch.await();
    // check both rows to ensure that they aren't there
    Get get = new Get(ROW);
    Result r = table.get(get);
    assertNull(
        "Got an unexpected number of rows - no data should be returned with the NoDataFromScan coprocessor. Found: "
            + r,
        r.listCells());

    get = new Get(Bytes.toBytes("anotherrow"));
    r = table.get(get);
    assertNull(
        "Got an unexpected number of rows - no data should be returned with the NoDataFromScan coprocessor Found: "
            + r,
        r.listCells());

    table.close();
    UTIL.shutdownMiniCluster();
  }