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