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.regionserver;
019
020import static org.junit.Assert.assertEquals;
021import static org.junit.Assert.assertFalse;
022import static org.junit.Assert.assertTrue;
023import static org.mockito.Mockito.mock;
024import static org.mockito.Mockito.when;
025
026import java.io.IOException;
027import java.util.concurrent.atomic.AtomicInteger;
028import org.apache.hadoop.conf.Configuration;
029import org.apache.hadoop.fs.FileSystem;
030import org.apache.hadoop.fs.Path;
031import org.apache.hadoop.hbase.HBaseClassTestRule;
032import org.apache.hadoop.hbase.HBaseTestingUtil;
033import org.apache.hadoop.hbase.KeyValue;
034import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
035import org.apache.hadoop.hbase.client.Get;
036import org.apache.hadoop.hbase.client.Scan;
037import org.apache.hadoop.hbase.io.hfile.CacheConfig;
038import org.apache.hadoop.hbase.io.hfile.HFileContext;
039import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
040import org.apache.hadoop.hbase.io.hfile.HFileInfo;
041import org.apache.hadoop.hbase.io.hfile.ReaderContext;
042import org.apache.hadoop.hbase.io.hfile.ReaderContextBuilder;
043import org.apache.hadoop.hbase.log.HBaseMarkers;
044import org.apache.hadoop.hbase.testclassification.RegionServerTests;
045import org.apache.hadoop.hbase.testclassification.SmallTests;
046import org.apache.hadoop.hbase.util.BloomFilterFactory;
047import org.apache.hadoop.hbase.util.BloomFilterUtil;
048import org.apache.hadoop.hbase.util.Bytes;
049import org.apache.hadoop.hbase.util.ChecksumType;
050import org.apache.hadoop.hbase.util.CommonFSUtils;
051import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
052import org.junit.After;
053import org.junit.Before;
054import org.junit.ClassRule;
055import org.junit.Rule;
056import org.junit.Test;
057import org.junit.experimental.categories.Category;
058import org.junit.rules.TestName;
059import org.slf4j.Logger;
060import org.slf4j.LoggerFactory;
061
062/**
063 * Test TestRowPrefixBloomFilter
064 */
065@Category({ RegionServerTests.class, SmallTests.class })
066public class TestRowPrefixBloomFilter {
067
068  @ClassRule
069  public static final HBaseClassTestRule CLASS_RULE =
070    HBaseClassTestRule.forClass(TestRowPrefixBloomFilter.class);
071
072  private static final Logger LOG = LoggerFactory.getLogger(TestRowPrefixBloomFilter.class);
073  private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil();
074  private CacheConfig cacheConf = new CacheConfig(TEST_UTIL.getConfiguration());
075  private static final ChecksumType CKTYPE = ChecksumType.CRC32C;
076  private static final int CKBYTES = 512;
077  private boolean localfs = false;
078  private static Configuration conf;
079  private static FileSystem fs;
080  private static Path testDir;
081  private static final int BLOCKSIZE_SMALL = 8192;
082  private static final float err = (float) 0.01;
083  private static final int prefixLength = 10;
084  private static final String invalidFormatter = "%08d";
085  private static final String prefixFormatter = "%010d";
086  private static final String suffixFormatter = "%010d";
087  private static final int prefixRowCount = 50;
088  private static final int suffixRowCount = 10;
089  private static final int fixedLengthExpKeys = prefixRowCount;
090  private static final BloomType bt = BloomType.ROWPREFIX_FIXED_LENGTH;
091
092  @Rule
093  public TestName name = new TestName();
094
095  @Before
096  public void setUp() throws Exception {
097    conf = TEST_UTIL.getConfiguration();
098    conf.setFloat(BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE, err);
099    conf.setBoolean(BloomFilterFactory.IO_STOREFILE_BLOOM_ENABLED, true);
100    conf.setInt(BloomFilterUtil.PREFIX_LENGTH_KEY, prefixLength);
101
102    localfs = (conf.get("fs.defaultFS", "file:///").compareTo("file:///") == 0);
103
104    if (fs == null) {
105      fs = FileSystem.get(conf);
106    }
107    try {
108      if (localfs) {
109        testDir = TEST_UTIL.getDataTestDir("TestRowPrefixBloomFilter");
110        if (fs.exists(testDir)) {
111          fs.delete(testDir, true);
112        }
113      } else {
114        testDir = CommonFSUtils.getRootDir(conf);
115      }
116    } catch (Exception e) {
117      LOG.error(HBaseMarkers.FATAL, "error during setup", e);
118      throw e;
119    }
120  }
121
122  @After
123  public void tearDown() throws Exception {
124    try {
125      if (localfs) {
126        if (fs.exists(testDir)) {
127          fs.delete(testDir, true);
128        }
129      }
130    } catch (Exception e) {
131      LOG.error(HBaseMarkers.FATAL, "error during tear down", e);
132    }
133  }
134
135  private static StoreFileScanner getStoreFileScanner(StoreFileReader reader) {
136    return reader.getStoreFileScanner(false, false, false, 0, 0, false);
137  }
138
139  private void writeStoreFile(final Path f, BloomType bt, int expKeys) throws IOException {
140    HFileContext meta = new HFileContextBuilder().withBlockSize(BLOCKSIZE_SMALL)
141      .withChecksumType(CKTYPE).withBytesPerCheckSum(CKBYTES).build();
142    // Make a store file and write data to it.
143    StoreFileWriter writer = new StoreFileWriter.Builder(conf, cacheConf, fs).withFilePath(f)
144      .withBloomType(bt).withMaxKeyCount(expKeys).withFileContext(meta).build();
145    long now = EnvironmentEdgeManager.currentTime();
146    try {
147      // Put with valid row style
148      for (int i = 0; i < prefixRowCount; i += 2) { // prefix rows
149        String prefixRow = String.format(prefixFormatter, i);
150        for (int j = 0; j < suffixRowCount; j++) { // suffix rows
151          String row = generateRowWithSuffix(prefixRow, j);
152          KeyValue kv = new KeyValue(Bytes.toBytes(row), Bytes.toBytes("family"),
153            Bytes.toBytes("col"), now, Bytes.toBytes("value"));
154          writer.append(kv);
155        }
156      }
157
158      // Put with invalid row style
159      for (int i = prefixRowCount; i < prefixRowCount * 2; i += 2) { // prefix rows
160        String row = String.format(invalidFormatter, i);
161        KeyValue kv = new KeyValue(Bytes.toBytes(row), Bytes.toBytes("family"),
162          Bytes.toBytes("col"), now, Bytes.toBytes("value"));
163        writer.append(kv);
164      }
165    } finally {
166      writer.close();
167    }
168  }
169
170  private String generateRowWithSuffix(String prefixRow, int suffix) {
171    StringBuilder row = new StringBuilder(prefixRow);
172    row.append("#");
173    row.append(String.format(suffixFormatter, suffix));
174    return row.toString();
175  }
176
177  @Test
178  public void testRowPrefixBloomFilter() throws Exception {
179    FileSystem fs = FileSystem.getLocal(conf);
180    float expErr = 2 * prefixRowCount * suffixRowCount * err;
181    int expKeys = fixedLengthExpKeys;
182    // write the file
183    Path f = new Path(testDir, name.getMethodName());
184    writeStoreFile(f, bt, expKeys);
185
186    // read the file
187    ReaderContext context = new ReaderContextBuilder().withFileSystemAndPath(fs, f).build();
188    HFileInfo fileInfo = new HFileInfo(context, conf);
189    StoreFileReader reader =
190      new StoreFileReader(context, fileInfo, cacheConf, new AtomicInteger(0), conf);
191    fileInfo.initMetaAndIndex(reader.getHFileReader());
192    reader.loadFileInfo();
193    reader.loadBloomfilter();
194
195    // check basic param
196    assertEquals(bt, reader.getBloomFilterType());
197    assertEquals(prefixLength, reader.getPrefixLength());
198    assertEquals(expKeys, reader.getGeneralBloomFilter().getKeyCount());
199    StoreFileScanner scanner = getStoreFileScanner(reader);
200    HStore store = mock(HStore.class);
201    when(store.getColumnFamilyDescriptor()).thenReturn(ColumnFamilyDescriptorBuilder.of("family"));
202    // check false positives rate
203    int falsePos = 0;
204    int falseNeg = 0;
205    for (int i = 0; i < prefixRowCount; i++) { // prefix rows
206      String prefixRow = String.format(prefixFormatter, i);
207      for (int j = 0; j < suffixRowCount; j++) { // suffix rows
208        String startRow = generateRowWithSuffix(prefixRow, j);
209        String stopRow = generateRowWithSuffix(prefixRow, j + 1);
210        Scan scan =
211          new Scan().withStartRow(Bytes.toBytes(startRow)).withStopRow(Bytes.toBytes(stopRow));
212        boolean exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
213        boolean shouldPrefixRowExist = i % 2 == 0;
214        if (shouldPrefixRowExist) {
215          if (!exists) {
216            falseNeg++;
217          }
218        } else {
219          if (exists) {
220            falsePos++;
221          }
222        }
223      }
224    }
225
226    for (int i = prefixRowCount; i < prefixRowCount * 2; i++) { // prefix rows
227      String row = String.format(invalidFormatter, i);
228      Scan scan = new Scan(new Get(Bytes.toBytes(row)));
229      boolean exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
230      boolean shouldPrefixRowExist = i % 2 == 0;
231      if (shouldPrefixRowExist) {
232        if (!exists) {
233          falseNeg++;
234        }
235      } else {
236        if (exists) {
237          falsePos++;
238        }
239      }
240    }
241    reader.close(true); // evict because we are about to delete the file
242    fs.delete(f, true);
243    assertEquals("False negatives: " + falseNeg, 0, falseNeg);
244    int maxFalsePos = (int) (2 * expErr);
245    assertTrue("Too many false positives: " + falsePos + " (err=" + err + ", expected no more than "
246      + maxFalsePos + ")", falsePos <= maxFalsePos);
247  }
248
249  @Test
250  public void testRowPrefixBloomFilterWithGet() throws Exception {
251    FileSystem fs = FileSystem.getLocal(conf);
252    int expKeys = fixedLengthExpKeys;
253    // write the file
254    Path f = new Path(testDir, name.getMethodName());
255    writeStoreFile(f, bt, expKeys);
256
257    ReaderContext context = new ReaderContextBuilder().withFileSystemAndPath(fs, f).build();
258    HFileInfo fileInfo = new HFileInfo(context, conf);
259    StoreFileReader reader =
260      new StoreFileReader(context, fileInfo, cacheConf, new AtomicInteger(0), conf);
261    fileInfo.initMetaAndIndex(reader.getHFileReader());
262    reader.loadFileInfo();
263    reader.loadBloomfilter();
264
265    StoreFileScanner scanner = getStoreFileScanner(reader);
266    HStore store = mock(HStore.class);
267    when(store.getColumnFamilyDescriptor()).thenReturn(ColumnFamilyDescriptorBuilder.of("family"));
268
269    // Get with valid row style
270    // prefix row in bloom
271    String prefixRow = String.format(prefixFormatter, prefixRowCount - 2);
272    String row = generateRowWithSuffix(prefixRow, 0);
273    Scan scan = new Scan(new Get(Bytes.toBytes(row)));
274    boolean exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
275    assertTrue(exists);
276
277    // prefix row not in bloom
278    prefixRow = String.format(prefixFormatter, prefixRowCount - 1);
279    row = generateRowWithSuffix(prefixRow, 0);
280    scan = new Scan(new Get(Bytes.toBytes(row)));
281    exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
282    assertFalse(exists);
283
284    // Get with invalid row style
285    // ROWPREFIX: the length of row is less than prefixLength
286    // row in bloom
287    row = String.format(invalidFormatter, prefixRowCount + 2);
288    scan = new Scan(new Get(Bytes.toBytes(row)));
289    exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
290    assertTrue(exists);
291
292    // row not in bloom
293    row = String.format(invalidFormatter, prefixRowCount + 1);
294    scan = new Scan(new Get(Bytes.toBytes(row)));
295    exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
296    assertFalse(exists);
297
298    reader.close(true); // evict because we are about to delete the file
299    fs.delete(f, true);
300  }
301
302  @Test
303  public void testRowPrefixBloomFilterWithScan() throws Exception {
304    FileSystem fs = FileSystem.getLocal(conf);
305    int expKeys = fixedLengthExpKeys;
306    // write the file
307    Path f = new Path(testDir, name.getMethodName());
308    writeStoreFile(f, bt, expKeys);
309
310    ReaderContext context = new ReaderContextBuilder().withFileSystemAndPath(fs, f).build();
311    HFileInfo fileInfo = new HFileInfo(context, conf);
312    StoreFileReader reader =
313      new StoreFileReader(context, fileInfo, cacheConf, new AtomicInteger(0), conf);
314    fileInfo.initMetaAndIndex(reader.getHFileReader());
315    reader.loadFileInfo();
316    reader.loadBloomfilter();
317
318    StoreFileScanner scanner = getStoreFileScanner(reader);
319    HStore store = mock(HStore.class);
320    when(store.getColumnFamilyDescriptor()).thenReturn(ColumnFamilyDescriptorBuilder.of("family"));
321
322    // Scan with valid row style. startRow and stopRow have a common prefix.
323    // And the length of the common prefix is no less than prefixLength.
324    // prefix row in bloom
325    String prefixRow = String.format(prefixFormatter, prefixRowCount - 2);
326    String startRow = generateRowWithSuffix(prefixRow, 0);
327    String stopRow = generateRowWithSuffix(prefixRow, 1);
328    Scan scan =
329      new Scan().withStartRow(Bytes.toBytes(startRow)).withStopRow(Bytes.toBytes(stopRow));
330    boolean exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
331    assertTrue(exists);
332
333    // prefix row not in bloom
334    prefixRow = String.format(prefixFormatter, prefixRowCount - 1);
335    startRow = generateRowWithSuffix(prefixRow, 0);
336    stopRow = generateRowWithSuffix(prefixRow, 1);
337    scan = new Scan().withStartRow(Bytes.toBytes(startRow)).withStopRow(Bytes.toBytes(stopRow));
338    exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
339    assertFalse(exists);
340
341    // There is no common prefix between startRow and stopRow.
342    prefixRow = String.format(prefixFormatter, prefixRowCount - 2);
343    startRow = generateRowWithSuffix(prefixRow, 0);
344    scan = new Scan().withStartRow(Bytes.toBytes(startRow));
345    exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
346    assertTrue(exists);
347
348    // startRow and stopRow have a common prefix.
349    // But the length of the common prefix is less than prefixLength.
350    String prefixStartRow = String.format(prefixFormatter, prefixRowCount - 2);
351    String prefixStopRow = String.format(prefixFormatter, prefixRowCount - 1);
352    startRow = generateRowWithSuffix(prefixStartRow, 0);
353    stopRow = generateRowWithSuffix(prefixStopRow, 0);
354    scan = new Scan().withStartRow(Bytes.toBytes(startRow)).withStopRow(Bytes.toBytes(stopRow));
355    exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
356    assertTrue(exists);
357
358    reader.close(true); // evict because we are about to delete the file
359    fs.delete(f, true);
360  }
361}