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