1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21 package org.apache.hadoop.hbase.regionserver;
22
23 import static org.junit.Assert.assertEquals;
24 import static org.junit.Assert.assertNotNull;
25 import static org.junit.Assert.assertTrue;
26 import static org.junit.Assert.fail;
27
28 import java.io.IOException;
29 import java.util.ArrayList;
30 import java.util.Collections;
31 import java.util.List;
32 import java.util.Random;
33 import java.util.TreeSet;
34
35 import org.apache.commons.logging.Log;
36 import org.apache.commons.logging.LogFactory;
37 import org.apache.hadoop.conf.Configuration;
38 import org.apache.hadoop.fs.FileSystem;
39 import org.apache.hadoop.fs.Path;
40 import org.apache.hadoop.hbase.*;
41 import org.apache.hadoop.hbase.client.Scan;
42 import org.apache.hadoop.hbase.io.hfile.BlockCache;
43 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
44 import org.apache.hadoop.hbase.io.hfile.NoOpDataBlockEncoder;
45 import org.apache.hadoop.hbase.io.hfile.HFile;
46 import org.apache.hadoop.hbase.io.hfile.TestHFileWriterV2;
47 import org.apache.hadoop.hbase.regionserver.StoreFile.BloomType;
48 import org.apache.hadoop.hbase.util.BloomFilterFactory;
49 import org.apache.hadoop.hbase.util.ByteBloomFilter;
50 import org.apache.hadoop.hbase.util.Bytes;
51 import org.apache.hadoop.hbase.util.CompoundBloomFilter;
52 import org.apache.hadoop.hbase.util.CompoundBloomFilterBase;
53 import org.apache.hadoop.hbase.util.CompoundBloomFilterWriter;
54 import org.junit.Before;
55 import org.junit.Test;
56 import org.junit.experimental.categories.Category;
57
58
59
60
61
62 @Category(SmallTests.class)
63 public class TestCompoundBloomFilter {
64
65 private static final HBaseTestingUtility TEST_UTIL =
66 new HBaseTestingUtility();
67
68 private static final Log LOG = LogFactory.getLog(
69 TestCompoundBloomFilter.class);
70
71 private static final int NUM_TESTS = 9;
72 private static final BloomType BLOOM_TYPES[] = { BloomType.ROW,
73 BloomType.ROW, BloomType.ROWCOL, BloomType.ROWCOL, BloomType.ROW,
74 BloomType.ROWCOL, BloomType.ROWCOL, BloomType.ROWCOL, BloomType.ROW };
75
76 private static final int NUM_KV[];
77 static {
78 final int N = 10000;
79 NUM_KV = new int[] { 21870, N, N, N, N, 1000, N, 7500, 7500};
80 assert NUM_KV.length == NUM_TESTS;
81 }
82
83 private static final int BLOCK_SIZES[];
84 static {
85 final int blkSize = 65536;
86 BLOCK_SIZES = new int[] { 512, 1000, blkSize, blkSize, blkSize, 128, 300,
87 blkSize, blkSize };
88 assert BLOCK_SIZES.length == NUM_TESTS;
89 }
90
91
92
93
94
95
96 private static final int BLOOM_BLOCK_SIZES[] = { 1000, 4096, 4096, 4096,
97 8192, 128, 1024, 600, 600 };
98 static { assert BLOOM_BLOCK_SIZES.length == NUM_TESTS; }
99
100 private static final double TARGET_ERROR_RATES[] = { 0.025, 0.01, 0.015,
101 0.01, 0.03, 0.01, 0.01, 0.07, 0.07 };
102 static { assert TARGET_ERROR_RATES.length == NUM_TESTS; }
103
104
105 private static final double TOO_HIGH_ERROR_RATE;
106 static {
107 double m = 0;
108 for (double errorRate : TARGET_ERROR_RATES)
109 m = Math.max(m, errorRate);
110 TOO_HIGH_ERROR_RATE = m + 0.03;
111 }
112
113 private static Configuration conf;
114 private static CacheConfig cacheConf;
115 private FileSystem fs;
116 private BlockCache blockCache;
117
118
119 private String testIdMsg;
120
121 private static final int GENERATION_SEED = 2319;
122 private static final int EVALUATION_SEED = 135;
123
124 @Before
125 public void setUp() throws IOException {
126 conf = TEST_UTIL.getConfiguration();
127
128
129 conf.setInt(HFile.FORMAT_VERSION_KEY, HFile.MAX_FORMAT_VERSION);
130
131 fs = FileSystem.get(conf);
132
133 cacheConf = new CacheConfig(conf);
134 blockCache = cacheConf.getBlockCache();
135 assertNotNull(blockCache);
136 }
137
138 private List<KeyValue> createSortedKeyValues(Random rand, int n) {
139 List<KeyValue> kvList = new ArrayList<KeyValue>(n);
140 for (int i = 0; i < n; ++i)
141 kvList.add(TestHFileWriterV2.randomKeyValue(rand));
142 Collections.sort(kvList, KeyValue.COMPARATOR);
143 return kvList;
144 }
145
146 @Test
147 public void testCompoundBloomFilter() throws IOException {
148 conf.setBoolean(BloomFilterFactory.IO_STOREFILE_BLOOM_ENABLED, true);
149 for (int t = 0; t < NUM_TESTS; ++t) {
150 conf.setFloat(BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE,
151 (float) TARGET_ERROR_RATES[t]);
152
153 testIdMsg = "in test #" + t + ":";
154 Random generationRand = new Random(GENERATION_SEED);
155 List<KeyValue> kvs = createSortedKeyValues(generationRand, NUM_KV[t]);
156 BloomType bt = BLOOM_TYPES[t];
157 Path sfPath = writeStoreFile(t, bt, kvs);
158 readStoreFile(t, bt, kvs, sfPath);
159 }
160 }
161
162
163
164
165
166
167
168
169
170
171
172
173
174 private void validateFalsePosRate(double falsePosRate, int nTrials,
175 double zValueBoundary, CompoundBloomFilter cbf, String additionalMsg) {
176 double p = BloomFilterFactory.getErrorRate(conf);
177 double zValue = (falsePosRate - p) / Math.sqrt(p * (1 - p) / nTrials);
178
179 String assortedStatsStr = " (targetErrorRate=" + p + ", falsePosRate="
180 + falsePosRate + ", nTrials=" + nTrials + ")";
181 LOG.info("z-value is " + zValue + assortedStatsStr);
182
183 boolean isUpperBound = zValueBoundary > 0;
184
185 if (isUpperBound && zValue > zValueBoundary ||
186 !isUpperBound && zValue < zValueBoundary) {
187 String errorMsg = "False positive rate z-value " + zValue + " is "
188 + (isUpperBound ? "higher" : "lower") + " than " + zValueBoundary
189 + assortedStatsStr + ". Per-chunk stats:\n"
190 + cbf.formatTestingStats();
191 fail(errorMsg + additionalMsg);
192 }
193 }
194
195 private void readStoreFile(int t, BloomType bt, List<KeyValue> kvs,
196 Path sfPath) throws IOException {
197 StoreFile sf = new StoreFile(fs, sfPath, conf, cacheConf, bt,
198 NoOpDataBlockEncoder.INSTANCE);
199 StoreFile.Reader r = sf.createReader();
200 final boolean pread = true;
201 StoreFileScanner scanner = r.getStoreFileScanner(true, pread);
202
203 {
204
205 int numChecked = 0;
206 for (KeyValue kv : kvs) {
207 byte[] row = kv.getRow();
208 boolean present = isInBloom(scanner, row, kv.getQualifier());
209 assertTrue(testIdMsg + " Bloom filter false negative on row "
210 + Bytes.toStringBinary(row) + " after " + numChecked
211 + " successful checks", present);
212 ++numChecked;
213 }
214 }
215
216
217
218 for (boolean fakeLookupEnabled : new boolean[] { true, false }) {
219 ByteBloomFilter.setFakeLookupMode(fakeLookupEnabled);
220 try {
221 String fakeLookupModeStr = ", fake lookup is " + (fakeLookupEnabled ?
222 "enabled" : "disabled");
223 CompoundBloomFilter cbf = (CompoundBloomFilter) r.getGeneralBloomFilter();
224 cbf.enableTestingStats();
225 int numFalsePos = 0;
226 Random rand = new Random(EVALUATION_SEED);
227 int nTrials = NUM_KV[t] * 10;
228 for (int i = 0; i < nTrials; ++i) {
229 byte[] query = TestHFileWriterV2.randomRowOrQualifier(rand);
230 if (isInBloom(scanner, query, bt, rand)) {
231 numFalsePos += 1;
232 }
233 }
234 double falsePosRate = numFalsePos * 1.0 / nTrials;
235 LOG.debug(String.format(testIdMsg
236 + " False positives: %d out of %d (%f)",
237 numFalsePos, nTrials, falsePosRate) + fakeLookupModeStr);
238
239
240 assertTrue("False positive is too high: " + falsePosRate + " (greater "
241 + "than " + TOO_HIGH_ERROR_RATE + ")" + fakeLookupModeStr,
242 falsePosRate < TOO_HIGH_ERROR_RATE);
243
244
245
246
247
248
249 double maxZValue = fakeLookupEnabled ? 1.96 : 2.5;
250 validateFalsePosRate(falsePosRate, nTrials, maxZValue, cbf,
251 fakeLookupModeStr);
252
253
254
255
256
257 int nChunks = cbf.getNumChunks();
258 if (nChunks > 1) {
259 numFalsePos -= cbf.getNumPositivesForTesting(nChunks - 1);
260 nTrials -= cbf.getNumQueriesForTesting(nChunks - 1);
261 falsePosRate = numFalsePos * 1.0 / nTrials;
262 LOG.info(testIdMsg + " False positive rate without last chunk is " +
263 falsePosRate + fakeLookupModeStr);
264 }
265
266 validateFalsePosRate(falsePosRate, nTrials, -2.58, cbf,
267 fakeLookupModeStr);
268 } finally {
269 ByteBloomFilter.setFakeLookupMode(false);
270 }
271 }
272
273 r.close(true);
274 }
275
276 private boolean isInBloom(StoreFileScanner scanner, byte[] row, BloomType bt,
277 Random rand) {
278 return isInBloom(scanner, row,
279 TestHFileWriterV2.randomRowOrQualifier(rand));
280 }
281
282 private boolean isInBloom(StoreFileScanner scanner, byte[] row,
283 byte[] qualifier) {
284 Scan scan = new Scan(row, row);
285 TreeSet<byte[]> columns = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
286 columns.add(qualifier);
287 return scanner.shouldUseScanner(scan, columns, Long.MIN_VALUE);
288 }
289
290 private Path writeStoreFile(int t, BloomType bt, List<KeyValue> kvs)
291 throws IOException {
292 conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_BLOCK_SIZE,
293 BLOOM_BLOCK_SIZES[t]);
294 conf.setBoolean(CacheConfig.CACHE_BLOCKS_ON_WRITE_KEY, true);
295 cacheConf = new CacheConfig(conf);
296
297 StoreFile.Writer w = new StoreFile.WriterBuilder(conf, cacheConf, fs,
298 BLOCK_SIZES[t])
299 .withOutputDir(TEST_UTIL.getDataTestDir())
300 .withBloomType(bt)
301 .withChecksumType(HFile.DEFAULT_CHECKSUM_TYPE)
302 .withBytesPerChecksum(HFile.DEFAULT_BYTES_PER_CHECKSUM)
303 .build();
304
305 assertTrue(w.hasGeneralBloom());
306 assertTrue(w.getGeneralBloomWriter() instanceof CompoundBloomFilterWriter);
307 CompoundBloomFilterWriter cbbf =
308 (CompoundBloomFilterWriter) w.getGeneralBloomWriter();
309
310 int keyCount = 0;
311 KeyValue prev = null;
312 LOG.debug("Total keys/values to insert: " + kvs.size());
313 for (KeyValue kv : kvs) {
314 w.append(kv);
315
316
317 boolean newKey = true;
318 if (prev != null) {
319 newKey = !(bt == BloomType.ROW ? KeyValue.COMPARATOR.matchingRows(kv,
320 prev) : KeyValue.COMPARATOR.matchingRowColumn(kv, prev));
321 }
322 if (newKey)
323 ++keyCount;
324 assertEquals(keyCount, cbbf.getKeyCount());
325
326 prev = kv;
327 }
328 w.close();
329
330 return w.getPath();
331 }
332
333 @Test
334 public void testCompoundBloomSizing() {
335 int bloomBlockByteSize = 4096;
336 int bloomBlockBitSize = bloomBlockByteSize * 8;
337 double targetErrorRate = 0.01;
338 long maxKeysPerChunk = ByteBloomFilter.idealMaxKeys(bloomBlockBitSize,
339 targetErrorRate);
340
341 long bloomSize1 = bloomBlockByteSize * 8;
342 long bloomSize2 = ByteBloomFilter.computeBitSize(maxKeysPerChunk,
343 targetErrorRate);
344
345 double bloomSizeRatio = (bloomSize2 * 1.0 / bloomSize1);
346 assertTrue(Math.abs(bloomSizeRatio - 0.9999) < 0.0001);
347 }
348
349 @Test
350 public void testCreateKey() {
351 CompoundBloomFilterBase cbfb = new CompoundBloomFilterBase();
352 byte[] row = "myRow".getBytes();
353 byte[] qualifier = "myQualifier".getBytes();
354 byte[] rowKey = cbfb.createBloomKey(row, 0, row.length,
355 row, 0, 0);
356 byte[] rowColKey = cbfb.createBloomKey(row, 0, row.length,
357 qualifier, 0, qualifier.length);
358 KeyValue rowKV = KeyValue.createKeyValueFromKey(rowKey);
359 KeyValue rowColKV = KeyValue.createKeyValueFromKey(rowColKey);
360 assertEquals(rowKV.getTimestamp(), rowColKV.getTimestamp());
361 assertEquals(Bytes.toStringBinary(rowKV.getRow()),
362 Bytes.toStringBinary(rowColKV.getRow()));
363 assertEquals(0, rowKV.getQualifier().length);
364 }
365
366
367 @org.junit.Rule
368 public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
369 new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
370 }
371