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.assertNotNull; 022import static org.junit.Assert.assertTrue; 023import static org.junit.Assert.fail; 024import static org.mockito.Mockito.mock; 025import static org.mockito.Mockito.when; 026 027import java.io.IOException; 028import java.util.ArrayList; 029import java.util.Collections; 030import java.util.List; 031import java.util.Random; 032import org.apache.hadoop.conf.Configuration; 033import org.apache.hadoop.fs.FileSystem; 034import org.apache.hadoop.fs.Path; 035import org.apache.hadoop.hbase.CellComparatorImpl; 036import org.apache.hadoop.hbase.CellUtil; 037import org.apache.hadoop.hbase.HBaseClassTestRule; 038import org.apache.hadoop.hbase.HBaseTestingUtility; 039import org.apache.hadoop.hbase.KeyValue; 040import org.apache.hadoop.hbase.KeyValueUtil; 041import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; 042import org.apache.hadoop.hbase.client.Scan; 043import org.apache.hadoop.hbase.io.hfile.BlockCache; 044import org.apache.hadoop.hbase.io.hfile.CacheConfig; 045import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilter; 046import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilterWriter; 047import org.apache.hadoop.hbase.io.hfile.HFile; 048import org.apache.hadoop.hbase.io.hfile.HFileContext; 049import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; 050import org.apache.hadoop.hbase.io.hfile.RandomKeyValueUtil; 051import org.apache.hadoop.hbase.testclassification.MediumTests; 052import org.apache.hadoop.hbase.testclassification.RegionServerTests; 053import org.apache.hadoop.hbase.util.BloomFilterFactory; 054import org.apache.hadoop.hbase.util.BloomFilterUtil; 055import org.apache.hadoop.hbase.util.Bytes; 056import org.junit.Before; 057import org.junit.ClassRule; 058import org.junit.Test; 059import org.junit.experimental.categories.Category; 060import org.slf4j.Logger; 061import org.slf4j.LoggerFactory; 062 063/** 064 * Tests writing Bloom filter blocks in the same part of the file as data 065 * blocks. 066 */ 067@Category({RegionServerTests.class, MediumTests.class}) 068public class TestCompoundBloomFilter { 069 070 @ClassRule 071 public static final HBaseClassTestRule CLASS_RULE = 072 HBaseClassTestRule.forClass(TestCompoundBloomFilter.class); 073 074 private static final HBaseTestingUtility TEST_UTIL = 075 new HBaseTestingUtility(); 076 077 private static final Logger LOG = LoggerFactory.getLogger( 078 TestCompoundBloomFilter.class); 079 080 private static final int NUM_TESTS = 9; 081 private static final BloomType BLOOM_TYPES[] = { BloomType.ROW, 082 BloomType.ROW, BloomType.ROWCOL, BloomType.ROWCOL, BloomType.ROW, 083 BloomType.ROWCOL, BloomType.ROWCOL, BloomType.ROWCOL, BloomType.ROW }; 084 085 private static final int NUM_KV[]; 086 static { 087 final int N = 10000; // Only used in initialization. 088 NUM_KV = new int[] { 21870, N, N, N, N, 1000, N, 7500, 7500}; 089 assert NUM_KV.length == NUM_TESTS; 090 } 091 092 private static final int BLOCK_SIZES[]; 093 static { 094 final int blkSize = 65536; 095 BLOCK_SIZES = new int[] { 512, 1000, blkSize, blkSize, blkSize, 128, 300, 096 blkSize, blkSize }; 097 assert BLOCK_SIZES.length == NUM_TESTS; 098 } 099 100 /** 101 * Be careful not to specify too high a Bloom filter block size, otherwise 102 * there will only be one oversized chunk and the observed false positive 103 * rate will be too low. 104 */ 105 private static final int BLOOM_BLOCK_SIZES[] = { 1000, 4096, 4096, 4096, 106 8192, 128, 1024, 600, 600 }; 107 static { assert BLOOM_BLOCK_SIZES.length == NUM_TESTS; } 108 109 private static final double TARGET_ERROR_RATES[] = { 0.025, 0.01, 0.015, 110 0.01, 0.03, 0.01, 0.01, 0.07, 0.07 }; 111 static { assert TARGET_ERROR_RATES.length == NUM_TESTS; } 112 113 /** A false positive rate that is obviously too high. */ 114 private static final double TOO_HIGH_ERROR_RATE; 115 static { 116 double m = 0; 117 for (double errorRate : TARGET_ERROR_RATES) 118 m = Math.max(m, errorRate); 119 TOO_HIGH_ERROR_RATE = m + 0.03; 120 } 121 122 private static Configuration conf; 123 private static CacheConfig cacheConf; 124 private FileSystem fs; 125 private BlockCache blockCache; 126 127 /** A message of the form "in test#<number>:" to include in logging. */ 128 private String testIdMsg; 129 130 private static final int GENERATION_SEED = 2319; 131 private static final int EVALUATION_SEED = 135; 132 133 @Before 134 public void setUp() throws IOException { 135 conf = TEST_UTIL.getConfiguration(); 136 137 // This test requires the most recent HFile format (i.e. v2). 138 conf.setInt(HFile.FORMAT_VERSION_KEY, HFile.MAX_FORMAT_VERSION); 139 140 fs = FileSystem.get(conf); 141 142 CacheConfig.instantiateBlockCache(conf); 143 cacheConf = new CacheConfig(conf); 144 blockCache = cacheConf.getBlockCache(); 145 assertNotNull(blockCache); 146 } 147 148 private List<KeyValue> createSortedKeyValues(Random rand, int n) { 149 List<KeyValue> kvList = new ArrayList<>(n); 150 for (int i = 0; i < n; ++i) 151 kvList.add(RandomKeyValueUtil.randomKeyValue(rand)); 152 Collections.sort(kvList, CellComparatorImpl.COMPARATOR); 153 return kvList; 154 } 155 156 @Test 157 public void testCompoundBloomFilter() throws IOException { 158 conf.setBoolean(BloomFilterFactory.IO_STOREFILE_BLOOM_ENABLED, true); 159 for (int t = 0; t < NUM_TESTS; ++t) { 160 conf.setFloat(BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE, 161 (float) TARGET_ERROR_RATES[t]); 162 163 testIdMsg = "in test #" + t + ":"; 164 Random generationRand = new Random(GENERATION_SEED); 165 List<KeyValue> kvs = createSortedKeyValues(generationRand, NUM_KV[t]); 166 BloomType bt = BLOOM_TYPES[t]; 167 Path sfPath = writeStoreFile(t, bt, kvs); 168 readStoreFile(t, bt, kvs, sfPath); 169 } 170 } 171 172 /** 173 * Validates the false positive ratio by computing its z-value and comparing 174 * it to the provided threshold. 175 * 176 * @param falsePosRate experimental positive rate 177 * @param nTrials the number of Bloom filter checks 178 * @param zValueBoundary z-value boundary, positive for an upper bound and 179 * negative for a lower bound 180 * @param cbf the compound Bloom filter we are using 181 * @param additionalMsg additional message to include in log output and 182 * assertion failures 183 */ 184 private void validateFalsePosRate(double falsePosRate, int nTrials, 185 double zValueBoundary, CompoundBloomFilter cbf, String additionalMsg) { 186 double p = BloomFilterFactory.getErrorRate(conf); 187 double zValue = (falsePosRate - p) / Math.sqrt(p * (1 - p) / nTrials); 188 189 String assortedStatsStr = " (targetErrorRate=" + p + ", falsePosRate=" 190 + falsePosRate + ", nTrials=" + nTrials + ")"; 191 LOG.info("z-value is " + zValue + assortedStatsStr); 192 193 boolean isUpperBound = zValueBoundary > 0; 194 195 if (isUpperBound && zValue > zValueBoundary || 196 !isUpperBound && zValue < zValueBoundary) { 197 String errorMsg = "False positive rate z-value " + zValue + " is " 198 + (isUpperBound ? "higher" : "lower") + " than " + zValueBoundary 199 + assortedStatsStr + ". Per-chunk stats:\n" 200 + cbf.formatTestingStats(); 201 fail(errorMsg + additionalMsg); 202 } 203 } 204 205 private void readStoreFile(int t, BloomType bt, List<KeyValue> kvs, 206 Path sfPath) throws IOException { 207 HStoreFile sf = new HStoreFile(fs, sfPath, conf, cacheConf, bt, true); 208 sf.initReader(); 209 StoreFileReader r = sf.getReader(); 210 final boolean pread = true; // does not really matter 211 StoreFileScanner scanner = r.getStoreFileScanner(true, pread, false, 0, 0, false); 212 213 { 214 // Test for false negatives (not allowed). 215 int numChecked = 0; 216 for (KeyValue kv : kvs) { 217 byte[] row = CellUtil.cloneRow(kv); 218 boolean present = isInBloom(scanner, row, CellUtil.cloneQualifier(kv)); 219 assertTrue(testIdMsg + " Bloom filter false negative on row " 220 + Bytes.toStringBinary(row) + " after " + numChecked 221 + " successful checks", present); 222 ++numChecked; 223 } 224 } 225 226 // Test for false positives (some percentage allowed). We test in two modes: 227 // "fake lookup" which ignores the key distribution, and production mode. 228 for (boolean fakeLookupEnabled : new boolean[] { true, false }) { 229 if (fakeLookupEnabled) { 230 BloomFilterUtil.setRandomGeneratorForTest(new Random(283742987L)); 231 } 232 try { 233 String fakeLookupModeStr = ", fake lookup is " + (fakeLookupEnabled ? 234 "enabled" : "disabled"); 235 CompoundBloomFilter cbf = (CompoundBloomFilter) r.getGeneralBloomFilter(); 236 cbf.enableTestingStats(); 237 int numFalsePos = 0; 238 Random rand = new Random(EVALUATION_SEED); 239 int nTrials = NUM_KV[t] * 10; 240 for (int i = 0; i < nTrials; ++i) { 241 byte[] query = RandomKeyValueUtil.randomRowOrQualifier(rand); 242 if (isInBloom(scanner, query, bt, rand)) { 243 numFalsePos += 1; 244 } 245 } 246 double falsePosRate = numFalsePos * 1.0 / nTrials; 247 LOG.debug(String.format(testIdMsg 248 + " False positives: %d out of %d (%f)", 249 numFalsePos, nTrials, falsePosRate) + fakeLookupModeStr); 250 251 // Check for obvious Bloom filter crashes. 252 assertTrue("False positive is too high: " + falsePosRate + " (greater " 253 + "than " + TOO_HIGH_ERROR_RATE + ")" + fakeLookupModeStr, 254 falsePosRate < TOO_HIGH_ERROR_RATE); 255 256 // Now a more precise check to see if the false positive rate is not 257 // too high. The reason we use a relaxed restriction for the real-world 258 // case as opposed to the "fake lookup" case is that our hash functions 259 // are not completely independent. 260 261 double maxZValue = fakeLookupEnabled ? 1.96 : 2.5; 262 validateFalsePosRate(falsePosRate, nTrials, maxZValue, cbf, 263 fakeLookupModeStr); 264 265 // For checking the lower bound we need to eliminate the last chunk, 266 // because it is frequently smaller and the false positive rate in it 267 // is too low. This does not help if there is only one under-sized 268 // chunk, though. 269 int nChunks = cbf.getNumChunks(); 270 if (nChunks > 1) { 271 numFalsePos -= cbf.getNumPositivesForTesting(nChunks - 1); 272 nTrials -= cbf.getNumQueriesForTesting(nChunks - 1); 273 falsePosRate = numFalsePos * 1.0 / nTrials; 274 LOG.info(testIdMsg + " False positive rate without last chunk is " + 275 falsePosRate + fakeLookupModeStr); 276 } 277 278 validateFalsePosRate(falsePosRate, nTrials, -2.58, cbf, 279 fakeLookupModeStr); 280 } finally { 281 BloomFilterUtil.setRandomGeneratorForTest(null); 282 } 283 } 284 285 r.close(true); // end of test so evictOnClose 286 } 287 288 private boolean isInBloom(StoreFileScanner scanner, byte[] row, BloomType bt, 289 Random rand) { 290 return isInBloom(scanner, row, RandomKeyValueUtil.randomRowOrQualifier(rand)); 291 } 292 293 private boolean isInBloom(StoreFileScanner scanner, byte[] row, 294 byte[] qualifier) { 295 Scan scan = new Scan().withStartRow(row).withStopRow(row, true); 296 scan.addColumn(Bytes.toBytes(RandomKeyValueUtil.COLUMN_FAMILY_NAME), qualifier); 297 HStore store = mock(HStore.class); 298 when(store.getColumnFamilyDescriptor()) 299 .thenReturn(ColumnFamilyDescriptorBuilder.of(RandomKeyValueUtil.COLUMN_FAMILY_NAME)); 300 return scanner.shouldUseScanner(scan, store, Long.MIN_VALUE); 301 } 302 303 private Path writeStoreFile(int t, BloomType bt, List<KeyValue> kvs) 304 throws IOException { 305 conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_BLOCK_SIZE, 306 BLOOM_BLOCK_SIZES[t]); 307 conf.setBoolean(CacheConfig.CACHE_BLOCKS_ON_WRITE_KEY, true); 308 cacheConf = new CacheConfig(conf); 309 HFileContext meta = new HFileContextBuilder().withBlockSize(BLOCK_SIZES[t]).build(); 310 StoreFileWriter w = new StoreFileWriter.Builder(conf, cacheConf, fs) 311 .withOutputDir(TEST_UTIL.getDataTestDir()) 312 .withBloomType(bt) 313 .withFileContext(meta) 314 .build(); 315 316 assertTrue(w.hasGeneralBloom()); 317 assertTrue(w.getGeneralBloomWriter() instanceof CompoundBloomFilterWriter); 318 CompoundBloomFilterWriter cbbf = 319 (CompoundBloomFilterWriter) w.getGeneralBloomWriter(); 320 321 int keyCount = 0; 322 KeyValue prev = null; 323 LOG.debug("Total keys/values to insert: " + kvs.size()); 324 for (KeyValue kv : kvs) { 325 w.append(kv); 326 327 // Validate the key count in the Bloom filter. 328 boolean newKey = true; 329 if (prev != null) { 330 newKey = !(bt == BloomType.ROW ? CellUtil.matchingRows(kv, 331 prev) : CellUtil.matchingRowColumn(kv, prev)); 332 } 333 if (newKey) 334 ++keyCount; 335 assertEquals(keyCount, cbbf.getKeyCount()); 336 337 prev = kv; 338 } 339 w.close(); 340 341 return w.getPath(); 342 } 343 344 @Test 345 public void testCompoundBloomSizing() { 346 int bloomBlockByteSize = 4096; 347 int bloomBlockBitSize = bloomBlockByteSize * 8; 348 double targetErrorRate = 0.01; 349 long maxKeysPerChunk = BloomFilterUtil.idealMaxKeys(bloomBlockBitSize, 350 targetErrorRate); 351 352 long bloomSize1 = bloomBlockByteSize * 8; 353 long bloomSize2 = BloomFilterUtil.computeBitSize(maxKeysPerChunk, 354 targetErrorRate); 355 356 double bloomSizeRatio = (bloomSize2 * 1.0 / bloomSize1); 357 assertTrue(Math.abs(bloomSizeRatio - 0.9999) < 0.0001); 358 } 359 360 @Test 361 public void testCreateKey() { 362 byte[] row = "myRow".getBytes(); 363 byte[] qualifier = "myQualifier".getBytes(); 364 // Mimic what Storefile.createBloomKeyValue() does 365 byte[] rowKey = KeyValueUtil.createFirstOnRow(row, 0, row.length, new byte[0], 0, 0, row, 0, 0).getKey(); 366 byte[] rowColKey = KeyValueUtil.createFirstOnRow(row, 0, row.length, 367 new byte[0], 0, 0, qualifier, 0, qualifier.length).getKey(); 368 KeyValue rowKV = KeyValueUtil.createKeyValueFromKey(rowKey); 369 KeyValue rowColKV = KeyValueUtil.createKeyValueFromKey(rowColKey); 370 assertEquals(rowKV.getTimestamp(), rowColKV.getTimestamp()); 371 assertEquals(Bytes.toStringBinary(rowKV.getRowArray(), rowKV.getRowOffset(), 372 rowKV.getRowLength()), Bytes.toStringBinary(rowColKV.getRowArray(), rowColKV.getRowOffset(), 373 rowColKV.getRowLength())); 374 assertEquals(0, rowKV.getQualifierLength()); 375 } 376 377 378} 379