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