1   /*
2    * Copyright 2009 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
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.HBaseTestingUtility;
41  import org.apache.hadoop.hbase.KeyValue;
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.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  
57  /**
58   * Tests writing Bloom filter blocks in the same part of the file as data
59   * blocks.
60   */
61  public class TestCompoundBloomFilter {
62  
63    private static final HBaseTestingUtility TEST_UTIL =
64        new HBaseTestingUtility();
65  
66    private static final Log LOG = LogFactory.getLog(
67        TestCompoundBloomFilter.class);
68  
69    private static final int NUM_TESTS = 9;
70    private static final BloomType BLOOM_TYPES[] = { BloomType.ROW,
71        BloomType.ROW, BloomType.ROWCOL, BloomType.ROWCOL, BloomType.ROW,
72        BloomType.ROWCOL, BloomType.ROWCOL, BloomType.ROWCOL, BloomType.ROW };
73  
74    private static final int NUM_KV[];
75    static {
76      final int N = 10000; // Only used in initialization.
77      NUM_KV = new int[] { 21870, N, N, N, N, 1000, N, 7500, 7500};
78      assert NUM_KV.length == NUM_TESTS;
79    }
80  
81    private static final int BLOCK_SIZES[];
82    static {
83      final int blkSize = 65536;
84      BLOCK_SIZES = new int[] { 512, 1000, blkSize, blkSize, blkSize, 128, 300,
85          blkSize, blkSize };
86      assert BLOCK_SIZES.length == NUM_TESTS;
87    }
88  
89    /**
90     * Be careful not to specify too high a Bloom filter block size, otherwise
91     * there will only be one oversized chunk and the observed false positive
92     * rate will be too low.
93     */
94    private static final int BLOOM_BLOCK_SIZES[] = { 1000, 4096, 4096, 4096,
95        8192, 128, 1024, 600, 600 };
96    static { assert BLOOM_BLOCK_SIZES.length == NUM_TESTS; }
97  
98    private static final double TARGET_ERROR_RATES[] = { 0.025, 0.01, 0.015,
99        0.01, 0.03, 0.01, 0.01, 0.07, 0.07 };
100   static { assert TARGET_ERROR_RATES.length == NUM_TESTS; }
101 
102   /** A false positive rate that is obviously too high. */
103   private static final double TOO_HIGH_ERROR_RATE;
104   static {
105     double m = 0;
106     for (double errorRate : TARGET_ERROR_RATES)
107       m = Math.max(m, errorRate);
108     TOO_HIGH_ERROR_RATE = m + 0.03;
109   }
110 
111   private static Configuration conf;
112   private static CacheConfig cacheConf;
113   private FileSystem fs;
114   private BlockCache blockCache;
115 
116   /** A message of the form "in test#<number>:" to include in logging. */
117   private String testIdMsg;
118 
119   private static final int GENERATION_SEED = 2319;
120   private static final int EVALUATION_SEED = 135;
121 
122   @Before
123   public void setUp() throws IOException {
124     conf = TEST_UTIL.getConfiguration();
125 
126     // This test requires the most recent HFile format (i.e. v2).
127     conf.setInt(HFile.FORMAT_VERSION_KEY, HFile.MAX_FORMAT_VERSION);
128 
129     fs = FileSystem.get(conf);
130 
131     cacheConf = new CacheConfig(conf);
132     blockCache = cacheConf.getBlockCache();
133     assertNotNull(blockCache);
134   }
135 
136   private List<KeyValue> createSortedKeyValues(Random rand, int n) {
137     List<KeyValue> kvList = new ArrayList<KeyValue>(n);
138     for (int i = 0; i < n; ++i)
139       kvList.add(TestHFileWriterV2.randomKeyValue(rand));
140     Collections.sort(kvList, KeyValue.COMPARATOR);
141     return kvList;
142   }
143 
144   @Test
145   public void testCompoundBloomFilter() throws IOException {
146     conf.setBoolean(BloomFilterFactory.IO_STOREFILE_BLOOM_ENABLED, true);
147     for (int t = 0; t < NUM_TESTS; ++t) {
148       conf.setFloat(BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE,
149           (float) TARGET_ERROR_RATES[t]);
150 
151       testIdMsg = "in test #" + t + ":";
152       Random generationRand = new Random(GENERATION_SEED);
153       List<KeyValue> kvs = createSortedKeyValues(generationRand, NUM_KV[t]);
154       BloomType bt = BLOOM_TYPES[t];
155       Path sfPath = writeStoreFile(t, bt, kvs);
156       readStoreFile(t, bt, kvs, sfPath);
157     }
158   }
159 
160   /**
161    * Validates the false positive ratio by computing its z-value and comparing
162    * it to the provided threshold.
163    *
164    * @param falsePosRate experimental positive rate
165    * @param nTrials the number of calls to
166    *          {@link StoreFile.Reader#shouldSeek(Scan, java.util.SortedSet)}.
167    * @param zValueBoundary z-value boundary, positive for an upper bound and
168    *          negative for a lower bound
169    * @param cbf the compound Bloom filter we are using
170    * @param additionalMsg additional message to include in log output and
171    *          assertion failures
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; // does not really matter
199     StoreFileScanner scanner = r.getStoreFileScanner(true, pread);
200 
201     {
202       // Test for false negatives (not allowed).
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     // Test for false positives (some percentage allowed). We test in two modes:
215     // "fake lookup" which ignores the key distribution, and production mode.
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.getBloomFilter();
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         // Check for obvious Bloom filter crashes.
238         assertTrue("False positive is too high: " + falsePosRate + " (greater "
239             + "than " + TOO_HIGH_ERROR_RATE + ")" + fakeLookupModeStr,
240             falsePosRate < TOO_HIGH_ERROR_RATE);
241 
242         // Now a more precise check to see if the false positive rate is not
243         // too high. The reason we use a relaxed restriction for the real-world
244         // case as opposed to the "fake lookup" case is that our hash functions
245         // are not completely independent.
246 
247         double maxZValue = fakeLookupEnabled ? 1.96 : 2.5;
248         validateFalsePosRate(falsePosRate, nTrials, maxZValue, cbf,
249             fakeLookupModeStr);
250 
251         // For checking the lower bound we need to eliminate the last chunk,
252         // because it is frequently smaller and the false positive rate in it
253         // is too low. This does not help if there is only one under-sized
254         // chunk, though.
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); // end of test so evictOnClose
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.shouldSeek(scan, columns);
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 = StoreFile.createWriter(fs,
296         TEST_UTIL.getDataTestDir(), BLOCK_SIZES[t], null, null, conf,
297         cacheConf, bt, 0);
298 
299     assertTrue(w.hasBloom());
300     assertTrue(w.getBloomWriter() instanceof CompoundBloomFilterWriter);
301     CompoundBloomFilterWriter cbbf =
302         (CompoundBloomFilterWriter) w.getBloomWriter();
303 
304     int keyCount = 0;
305     KeyValue prev = null;
306     LOG.debug("Total keys/values to insert: " + kvs.size());
307     for (KeyValue kv : kvs) {
308       w.append(kv);
309 
310       // Validate the key count in the Bloom filter.
311       boolean newKey = true;
312       if (prev != null) {
313         newKey = !(bt == BloomType.ROW ? KeyValue.COMPARATOR.matchingRows(kv,
314             prev) : KeyValue.COMPARATOR.matchingRowColumn(kv, prev));
315       }
316       if (newKey)
317         ++keyCount;
318       assertEquals(keyCount, cbbf.getKeyCount());
319 
320       prev = kv;
321     }
322     w.close();
323 
324     return w.getPath();
325   }
326 
327   @Test
328   public void testCompoundBloomSizing() {
329     int bloomBlockByteSize = 4096;
330     int bloomBlockBitSize = bloomBlockByteSize * 8;
331     double targetErrorRate = 0.01;
332     long maxKeysPerChunk = ByteBloomFilter.idealMaxKeys(bloomBlockBitSize,
333         targetErrorRate);
334 
335     long bloomSize1 = bloomBlockByteSize * 8;
336     long bloomSize2 = ByteBloomFilter.computeBitSize(maxKeysPerChunk,
337         targetErrorRate);
338 
339     double bloomSizeRatio = (bloomSize2 * 1.0 / bloomSize1);
340     assertTrue(Math.abs(bloomSizeRatio - 0.9999) < 0.0001);
341   }
342 
343   @Test
344   public void testCreateKey() {
345     CompoundBloomFilterBase cbfb = new CompoundBloomFilterBase();
346     byte[] row = "myRow".getBytes();
347     byte[] qualifier = "myQualifier".getBytes();
348     byte[] rowKey = cbfb.createBloomKey(row, 0, row.length,
349         row, 0, 0);
350     byte[] rowColKey = cbfb.createBloomKey(row, 0, row.length,
351         qualifier, 0, qualifier.length);
352     KeyValue rowKV = KeyValue.createKeyValueFromKey(rowKey);
353     KeyValue rowColKV = KeyValue.createKeyValueFromKey(rowColKey);
354     assertEquals(rowKV.getTimestamp(), rowColKV.getTimestamp());
355     assertEquals(Bytes.toStringBinary(rowKV.getRow()),
356         Bytes.toStringBinary(rowColKV.getRow()));
357     assertEquals(0, rowKV.getQualifier().length);
358   }
359 
360 }