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.HFileContext;
47 import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
48 import org.apache.hadoop.hbase.io.hfile.TestHFileWriterV2;
49 import org.apache.hadoop.hbase.util.BloomFilterFactory;
50 import org.apache.hadoop.hbase.util.ByteBloomFilter;
51 import org.apache.hadoop.hbase.util.Bytes;
52 import org.apache.hadoop.hbase.util.CompoundBloomFilter;
53 import org.apache.hadoop.hbase.util.CompoundBloomFilterBase;
54 import org.apache.hadoop.hbase.util.CompoundBloomFilterWriter;
55 import org.junit.Before;
56 import org.junit.Test;
57 import org.junit.experimental.categories.Category;
58
59
60
61
62
63 @Category(MediumTests.class)
64 public class TestCompoundBloomFilter {
65
66 private static final HBaseTestingUtility TEST_UTIL =
67 new HBaseTestingUtility();
68
69 private static final Log LOG = LogFactory.getLog(
70 TestCompoundBloomFilter.class);
71
72 private static final int NUM_TESTS = 9;
73 private static final BloomType BLOOM_TYPES[] = { BloomType.ROW,
74 BloomType.ROW, BloomType.ROWCOL, BloomType.ROWCOL, BloomType.ROW,
75 BloomType.ROWCOL, BloomType.ROWCOL, BloomType.ROWCOL, BloomType.ROW };
76
77 private static final int NUM_KV[];
78 static {
79 final int N = 10000;
80 NUM_KV = new int[] { 21870, N, N, N, N, 1000, N, 7500, 7500};
81 assert NUM_KV.length == NUM_TESTS;
82 }
83
84 private static final int BLOCK_SIZES[];
85 static {
86 final int blkSize = 65536;
87 BLOCK_SIZES = new int[] { 512, 1000, blkSize, blkSize, blkSize, 128, 300,
88 blkSize, blkSize };
89 assert BLOCK_SIZES.length == NUM_TESTS;
90 }
91
92
93
94
95
96
97 private static final int BLOOM_BLOCK_SIZES[] = { 1000, 4096, 4096, 4096,
98 8192, 128, 1024, 600, 600 };
99 static { assert BLOOM_BLOCK_SIZES.length == NUM_TESTS; }
100
101 private static final double TARGET_ERROR_RATES[] = { 0.025, 0.01, 0.015,
102 0.01, 0.03, 0.01, 0.01, 0.07, 0.07 };
103 static { assert TARGET_ERROR_RATES.length == NUM_TESTS; }
104
105
106 private static final double TOO_HIGH_ERROR_RATE;
107 static {
108 double m = 0;
109 for (double errorRate : TARGET_ERROR_RATES)
110 m = Math.max(m, errorRate);
111 TOO_HIGH_ERROR_RATE = m + 0.03;
112 }
113
114 private static Configuration conf;
115 private static CacheConfig cacheConf;
116 private FileSystem fs;
117 private BlockCache blockCache;
118
119
120 private String testIdMsg;
121
122 private static final int GENERATION_SEED = 2319;
123 private static final int EVALUATION_SEED = 135;
124
125 @Before
126 public void setUp() throws IOException {
127 conf = TEST_UTIL.getConfiguration();
128
129
130 conf.setInt(HFile.FORMAT_VERSION_KEY, HFile.MAX_FORMAT_VERSION);
131
132 fs = FileSystem.get(conf);
133
134 cacheConf = new CacheConfig(conf);
135 blockCache = cacheConf.getBlockCache();
136 assertNotNull(blockCache);
137 }
138
139 private List<KeyValue> createSortedKeyValues(Random rand, int n) {
140 List<KeyValue> kvList = new ArrayList<KeyValue>(n);
141 for (int i = 0; i < n; ++i)
142 kvList.add(TestHFileWriterV2.randomKeyValue(rand));
143 Collections.sort(kvList, KeyValue.COMPARATOR);
144 return kvList;
145 }
146
147 @Test
148 public void testCompoundBloomFilter() throws IOException {
149 conf.setBoolean(BloomFilterFactory.IO_STOREFILE_BLOOM_ENABLED, true);
150 for (int t = 0; t < NUM_TESTS; ++t) {
151 conf.setFloat(BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE,
152 (float) TARGET_ERROR_RATES[t]);
153
154 testIdMsg = "in test #" + t + ":";
155 Random generationRand = new Random(GENERATION_SEED);
156 List<KeyValue> kvs = createSortedKeyValues(generationRand, NUM_KV[t]);
157 BloomType bt = BLOOM_TYPES[t];
158 Path sfPath = writeStoreFile(t, bt, kvs);
159 readStoreFile(t, bt, kvs, sfPath);
160 }
161 }
162
163
164
165
166
167
168
169
170
171
172
173
174
175 private void validateFalsePosRate(double falsePosRate, int nTrials,
176 double zValueBoundary, CompoundBloomFilter cbf, String additionalMsg) {
177 double p = BloomFilterFactory.getErrorRate(conf);
178 double zValue = (falsePosRate - p) / Math.sqrt(p * (1 - p) / nTrials);
179
180 String assortedStatsStr = " (targetErrorRate=" + p + ", falsePosRate="
181 + falsePosRate + ", nTrials=" + nTrials + ")";
182 LOG.info("z-value is " + zValue + assortedStatsStr);
183
184 boolean isUpperBound = zValueBoundary > 0;
185
186 if (isUpperBound && zValue > zValueBoundary ||
187 !isUpperBound && zValue < zValueBoundary) {
188 String errorMsg = "False positive rate z-value " + zValue + " is "
189 + (isUpperBound ? "higher" : "lower") + " than " + zValueBoundary
190 + assortedStatsStr + ". Per-chunk stats:\n"
191 + cbf.formatTestingStats();
192 fail(errorMsg + additionalMsg);
193 }
194 }
195
196 private void readStoreFile(int t, BloomType bt, List<KeyValue> kvs,
197 Path sfPath) throws IOException {
198 StoreFile sf = new StoreFile(fs, sfPath, conf, cacheConf, bt);
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 HFileContext meta = new HFileContextBuilder().withBlockSize(BLOCK_SIZES[t]).build();
297 StoreFile.Writer w = new StoreFile.WriterBuilder(conf, cacheConf, fs)
298 .withOutputDir(TEST_UTIL.getDataTestDir())
299 .withBloomType(bt)
300 .withFileContext(meta)
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