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