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