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, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.coprocessor;
019
020import static org.junit.Assert.assertEquals;
021import static org.junit.Assert.assertNull;
022
023import java.io.IOException;
024import java.util.Collections;
025import java.util.List;
026import java.util.Optional;
027import java.util.concurrent.CountDownLatch;
028import org.apache.hadoop.conf.Configuration;
029import org.apache.hadoop.fs.FileSystem;
030import org.apache.hadoop.fs.Path;
031import org.apache.hadoop.hbase.Cell;
032import org.apache.hadoop.hbase.Coprocessor;
033import org.apache.hadoop.hbase.HBaseClassTestRule;
034import org.apache.hadoop.hbase.HBaseTestingUtil;
035import org.apache.hadoop.hbase.HConstants;
036import org.apache.hadoop.hbase.TableName;
037import org.apache.hadoop.hbase.client.Admin;
038import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
039import org.apache.hadoop.hbase.client.CoprocessorDescriptorBuilder;
040import org.apache.hadoop.hbase.client.Get;
041import org.apache.hadoop.hbase.client.Put;
042import org.apache.hadoop.hbase.client.RegionInfo;
043import org.apache.hadoop.hbase.client.RegionInfoBuilder;
044import org.apache.hadoop.hbase.client.Result;
045import org.apache.hadoop.hbase.client.Scan;
046import org.apache.hadoop.hbase.client.Table;
047import org.apache.hadoop.hbase.client.TableDescriptor;
048import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
049import org.apache.hadoop.hbase.filter.FilterBase;
050import org.apache.hadoop.hbase.regionserver.ChunkCreator;
051import org.apache.hadoop.hbase.regionserver.FlushLifeCycleTracker;
052import org.apache.hadoop.hbase.regionserver.HRegion;
053import org.apache.hadoop.hbase.regionserver.HRegionServer;
054import org.apache.hadoop.hbase.regionserver.HStore;
055import org.apache.hadoop.hbase.regionserver.InternalScanner;
056import org.apache.hadoop.hbase.regionserver.MemStoreLAB;
057import org.apache.hadoop.hbase.regionserver.Region;
058import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
059import org.apache.hadoop.hbase.regionserver.RegionServerServices;
060import org.apache.hadoop.hbase.regionserver.ScanType;
061import org.apache.hadoop.hbase.regionserver.ScannerContext;
062import org.apache.hadoop.hbase.regionserver.Store;
063import org.apache.hadoop.hbase.regionserver.StoreScanner;
064import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
065import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
066import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
067import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
068import org.apache.hadoop.hbase.security.User;
069import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
070import org.apache.hadoop.hbase.testclassification.MediumTests;
071import org.apache.hadoop.hbase.util.Bytes;
072import org.apache.hadoop.hbase.wal.WAL;
073import org.junit.ClassRule;
074import org.junit.Rule;
075import org.junit.Test;
076import org.junit.experimental.categories.Category;
077import org.junit.rules.TestName;
078
079@Category({ CoprocessorTests.class, MediumTests.class })
080public class TestRegionObserverScannerOpenHook {
081
082  @ClassRule
083  public static final HBaseClassTestRule CLASS_RULE =
084    HBaseClassTestRule.forClass(TestRegionObserverScannerOpenHook.class);
085
086  private static HBaseTestingUtil UTIL = new HBaseTestingUtil();
087  static final Path DIR = UTIL.getDataTestDir();
088
089  @Rule
090  public TestName name = new TestName();
091
092  public static class NoDataFilter extends FilterBase {
093
094    @Override
095    public ReturnCode filterCell(final Cell ignored) {
096      return ReturnCode.SKIP;
097    }
098
099    @Override
100    public boolean filterAllRemaining() throws IOException {
101      return true;
102    }
103
104    @Override
105    public boolean filterRow() throws IOException {
106      return true;
107    }
108  }
109
110  /**
111   * Do the default logic in {@link RegionObserver} interface.
112   */
113  public static class EmptyRegionObsever implements RegionCoprocessor, RegionObserver {
114    @Override
115    public Optional<RegionObserver> getRegionObserver() {
116      return Optional.of(this);
117    }
118  }
119
120  /**
121   * Don't return any data from a scan by creating a custom {@link StoreScanner}.
122   */
123  public static class NoDataFromScan implements RegionCoprocessor, RegionObserver {
124    @Override
125    public Optional<RegionObserver> getRegionObserver() {
126      return Optional.of(this);
127    }
128
129    @Override
130    public void preGetOp(ObserverContext<RegionCoprocessorEnvironment> c, Get get,
131      List<Cell> result) throws IOException {
132      c.bypass();
133    }
134
135    @Override
136    public void preScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, Scan scan)
137      throws IOException {
138      scan.setFilter(new NoDataFilter());
139    }
140  }
141
142  private static final InternalScanner NO_DATA = new InternalScanner() {
143
144    @Override
145    public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {
146      return false;
147    }
148
149    @Override
150    public void close() throws IOException {
151    }
152  };
153
154  /**
155   * Don't allow any data in a flush by creating a custom {@link StoreScanner}.
156   */
157  public static class NoDataFromFlush implements RegionCoprocessor, RegionObserver {
158    @Override
159    public Optional<RegionObserver> getRegionObserver() {
160      return Optional.of(this);
161    }
162
163    @Override
164    public InternalScanner preFlush(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
165      InternalScanner scanner, FlushLifeCycleTracker tracker) throws IOException {
166      return NO_DATA;
167    }
168  }
169
170  /**
171   * Don't allow any data to be written out in the compaction by creating a custom
172   * {@link StoreScanner}.
173   */
174  public static class NoDataFromCompaction implements RegionCoprocessor, RegionObserver {
175    @Override
176    public Optional<RegionObserver> getRegionObserver() {
177      return Optional.of(this);
178    }
179
180    @Override
181    public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
182      InternalScanner scanner, ScanType scanType, CompactionLifeCycleTracker tracker,
183      CompactionRequest request) throws IOException {
184      return NO_DATA;
185    }
186  }
187
188  HRegion initHRegion(byte[] tableName, String callingMethod, Configuration conf,
189    byte[]... families) throws IOException {
190    TableDescriptorBuilder builder =
191      TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName));
192    for (byte[] family : families) {
193      builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(family));
194    }
195    TableDescriptor tableDescriptor = builder.build();
196    ChunkCreator.initialize(MemStoreLAB.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null,
197      MemStoreLAB.INDEX_CHUNK_SIZE_PERCENTAGE_DEFAULT);
198    RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
199    Path path = new Path(DIR + callingMethod);
200    WAL wal = HBaseTestingUtil.createWal(conf, path, info);
201    HRegion r = HRegion.createHRegion(info, path, conf, tableDescriptor, wal);
202    // this following piece is a hack. currently a coprocessorHost
203    // is secretly loaded at OpenRegionHandler. we don't really
204    // start a region server here, so just manually create cphost
205    // and set it to region.
206    RegionCoprocessorHost host = new RegionCoprocessorHost(r, null, conf);
207    r.setCoprocessorHost(host);
208    return r;
209  }
210
211  @Test
212  public void testRegionObserverScanTimeStacking() throws Exception {
213    byte[] ROW = Bytes.toBytes("testRow");
214    byte[] TABLE = Bytes.toBytes(getClass().getName());
215    byte[] A = Bytes.toBytes("A");
216    byte[][] FAMILIES = new byte[][] { A };
217
218    // Use new HTU to not overlap with the DFS cluster started in #CompactionStacking
219    Configuration conf = new HBaseTestingUtil().getConfiguration();
220    HRegion region = initHRegion(TABLE, getClass().getName(), conf, FAMILIES);
221    RegionCoprocessorHost h = region.getCoprocessorHost();
222    h.load(NoDataFromScan.class, Coprocessor.PRIORITY_HIGHEST, conf);
223    h.load(EmptyRegionObsever.class, Coprocessor.PRIORITY_USER, conf);
224
225    Put put = new Put(ROW);
226    put.addColumn(A, A, A);
227    region.put(put);
228
229    Get get = new Get(ROW);
230    Result r = region.get(get);
231    assertNull(
232      "Got an unexpected number of rows - "
233        + "no data should be returned with the NoDataFromScan coprocessor. Found: " + r,
234      r.listCells());
235    HBaseTestingUtil.closeRegionAndWAL(region);
236  }
237
238  @Test
239  public void testRegionObserverFlushTimeStacking() throws Exception {
240    byte[] ROW = Bytes.toBytes("testRow");
241    byte[] TABLE = Bytes.toBytes(getClass().getName());
242    byte[] A = Bytes.toBytes("A");
243    byte[][] FAMILIES = new byte[][] { A };
244
245    // Use new HTU to not overlap with the DFS cluster started in #CompactionStacking
246    Configuration conf = new HBaseTestingUtil().getConfiguration();
247    HRegion region = initHRegion(TABLE, getClass().getName(), conf, FAMILIES);
248    RegionCoprocessorHost h = region.getCoprocessorHost();
249    h.load(NoDataFromFlush.class, Coprocessor.PRIORITY_HIGHEST, conf);
250    h.load(EmptyRegionObsever.class, Coprocessor.PRIORITY_USER, conf);
251
252    // put a row and flush it to disk
253    Put put = new Put(ROW);
254    put.addColumn(A, A, A);
255    region.put(put);
256    region.flush(true);
257    Get get = new Get(ROW);
258    Result r = region.get(get);
259    assertNull(
260      "Got an unexpected number of rows - "
261        + "no data should be returned with the NoDataFromScan coprocessor. Found: " + r,
262      r.listCells());
263    HBaseTestingUtil.closeRegionAndWAL(region);
264  }
265
266  /*
267   * Custom HRegion which uses CountDownLatch to signal the completion of compaction
268   */
269  public static class CompactionCompletionNotifyingRegion extends HRegion {
270    private static volatile CountDownLatch compactionStateChangeLatch = null;
271
272    @SuppressWarnings("deprecation")
273    public CompactionCompletionNotifyingRegion(Path tableDir, WAL log, FileSystem fs,
274      Configuration confParam, RegionInfo info, TableDescriptor htd,
275      RegionServerServices rsServices) {
276      super(tableDir, log, fs, confParam, info, htd, rsServices);
277    }
278
279    public CountDownLatch getCompactionStateChangeLatch() {
280      if (compactionStateChangeLatch == null) {
281        compactionStateChangeLatch = new CountDownLatch(1);
282      }
283      return compactionStateChangeLatch;
284    }
285
286    @Override
287    public boolean compact(CompactionContext compaction, HStore store,
288      ThroughputController throughputController) throws IOException {
289      boolean ret = super.compact(compaction, store, throughputController);
290      if (ret) {
291        compactionStateChangeLatch.countDown();
292      }
293      return ret;
294    }
295
296    @Override
297    public boolean compact(CompactionContext compaction, HStore store,
298      ThroughputController throughputController, User user) throws IOException {
299      boolean ret = super.compact(compaction, store, throughputController, user);
300      if (ret) compactionStateChangeLatch.countDown();
301      return ret;
302    }
303  }
304
305  /**
306   * Unfortunately, the easiest way to test this is to spin up a mini-cluster since we want to do
307   * the usual compaction mechanism on the region, rather than going through the backdoor to the
308   * region
309   */
310  @Test
311  public void testRegionObserverCompactionTimeStacking() throws Exception {
312    // setup a mini cluster so we can do a real compaction on a region
313    Configuration conf = UTIL.getConfiguration();
314    conf.setClass(HConstants.REGION_IMPL, CompactionCompletionNotifyingRegion.class, HRegion.class);
315    conf.setInt("hbase.hstore.compaction.min", 2);
316    UTIL.startMiniCluster();
317    byte[] ROW = Bytes.toBytes("testRow");
318    byte[] A = Bytes.toBytes("A");
319    TableDescriptor tableDescriptor =
320      TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
321        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(A))
322        .setCoprocessor(CoprocessorDescriptorBuilder
323          .newBuilder(EmptyRegionObsever.class.getName()).setJarPath(null)
324          .setPriority(Coprocessor.PRIORITY_USER).setProperties(Collections.emptyMap()).build())
325        .setCoprocessor(CoprocessorDescriptorBuilder
326          .newBuilder(NoDataFromCompaction.class.getName()).setJarPath(null)
327          .setPriority(Coprocessor.PRIORITY_HIGHEST).setProperties(Collections.emptyMap()).build())
328        .build();
329
330    Admin admin = UTIL.getAdmin();
331    admin.createTable(tableDescriptor);
332
333    Table table = UTIL.getConnection().getTable(tableDescriptor.getTableName());
334
335    // put a row and flush it to disk
336    Put put = new Put(ROW);
337    put.addColumn(A, A, A);
338    table.put(put);
339
340    HRegionServer rs = UTIL.getRSForFirstRegionInTable(tableDescriptor.getTableName());
341    List<HRegion> regions = rs.getRegions(tableDescriptor.getTableName());
342    assertEquals("More than 1 region serving test table with 1 row", 1, regions.size());
343    Region region = regions.get(0);
344    admin.flushRegion(region.getRegionInfo().getRegionName());
345    CountDownLatch latch =
346      ((CompactionCompletionNotifyingRegion) region).getCompactionStateChangeLatch();
347
348    // put another row and flush that too
349    put = new Put(Bytes.toBytes("anotherrow"));
350    put.addColumn(A, A, A);
351    table.put(put);
352    admin.flushRegion(region.getRegionInfo().getRegionName());
353
354    // run a compaction, which normally would should get rid of the data
355    // wait for the compaction checker to complete
356    latch.await();
357    // check both rows to ensure that they aren't there
358    Get get = new Get(ROW);
359    Result r = table.get(get);
360    assertNull(
361      "Got an unexpected number of rows - "
362        + "no data should be returned with the NoDataFromScan coprocessor. Found: " + r,
363      r.listCells());
364
365    get = new Get(Bytes.toBytes("anotherrow"));
366    r = table.get(get);
367    assertNull(
368      "Got an unexpected number of rows - "
369        + "no data should be returned with the NoDataFromScan coprocessor Found: " + r,
370      r.listCells());
371
372    table.close();
373    UTIL.shutdownMiniCluster();
374  }
375}