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