Details

    • Type: Task Task
    • Status: Resolved
    • Priority: Major Major
    • Resolution: Fixed
    • Affects Version/s: 0.20.5
    • Fix Version/s: 0.90.0
    • Component/s: None
    • Hadoop Flags:
      Reviewed

      Description

      Add bloomfiltering to hfile. Can be enabled on a family-level basis. Ability to configure a row vs row+col level bloom. We size the bloomfilter with the number of entries we are about to flush which seems like usually we'd be making a filter too big, so our implementation needs to take that into account.

      1. 1200-trunk.txt
        137 kB
        stack
      2. Bloom_Filters_in_HBase.pdf
        91 kB
        stack
      3. Bloom Filters in HBase.docx
        106 kB
        Nicolas Spiegelberg
      4. HBASE-1200-0.20.5.patch
        129 kB
        Nicolas Spiegelberg
      5. HBASE-1200-0.20.5-4.patch
        140 kB
        Nicolas Spiegelberg
      6. hbase-1200-0.20branch-v2.txt
        120 kB
        stack
      7. ryan_bloomfilter.patch
        20 kB
        stack
      8. Store.java.rej
        2 kB
        stack
      9. StoreScanner.java.rej
        2 kB
        stack

        Issue Links

          Activity

          Hide
          stack added a comment -

          Thing to do would be to run with them on for a while and then before release make a call.

          Here is from BloomFilterMapFile:

              private synchronized void initBloomFilter(Configuration conf) {
                numKeys = conf.getInt("io.mapfile.bloom.size", 1024 * 1024);
                // vector size should be <code>-kn / (ln(1 - c^(1/k)))</code> bits for
                // single key, where <code> is the number of hash functions,
                // <code>n</code> is the number of keys and <code>c</code> is the desired
                // max. error rate.
                // Our desired error rate is by default 0.005, i.e. 0.5%
                float errorRate = conf.getFloat("io.mapfile.bloom.error.rate", 0.005f);
                vectorSize = (int)Math.ceil((double)(-HASH_COUNT * numKeys) /
                    Math.log(1.0 - Math.pow(errorRate, 1.0/HASH_COUNT)));
                bloomFilter = new DynamicBloomFilter(vectorSize, HASH_COUNT,
                    Hash.getHashType(conf), numKeys);
              }
          
          Show
          stack added a comment - Thing to do would be to run with them on for a while and then before release make a call. Here is from BloomFilterMapFile: private synchronized void initBloomFilter(Configuration conf) { numKeys = conf.getInt( "io.mapfile.bloom.size" , 1024 * 1024); // vector size should be <code>-kn / (ln(1 - c^(1/k)))</code> bits for // single key, where <code> is the number of hash functions, // <code>n</code> is the number of keys and <code>c</code> is the desired // max. error rate. // Our desired error rate is by default 0.005, i.e. 0.5% float errorRate = conf.getFloat( "io.mapfile.bloom.error.rate" , 0.005f); vectorSize = ( int ) Math .ceil(( double )(-HASH_COUNT * numKeys) / Math .log(1.0 - Math .pow(errorRate, 1.0/HASH_COUNT))); bloomFilter = new DynamicBloomFilter(vectorSize, HASH_COUNT, Hash.getHashType(conf), numKeys); }
          Hide
          Erik Holstad added a comment -

          I think that the user should have an option to not use bloom filters, even though I can't really see
          why you wouldn't, but still have an option to do so. I also think that we should try to go towards
          row+column like BT. Using the Dynamic bloom filter seems like a reasonable way to go, the only
          thing I can see is that we are still going to have an overhead, even though it is smaller than now.
          So if possible wait until we know the exact number and then create the filter. Not sure what the time
          loss will be for the flush doing it this way, but that could be tested.

          Show
          Erik Holstad added a comment - I think that the user should have an option to not use bloom filters, even though I can't really see why you wouldn't, but still have an option to do so. I also think that we should try to go towards row+column like BT. Using the Dynamic bloom filter seems like a reasonable way to go, the only thing I can see is that we are still going to have an overhead, even though it is smaller than now. So if possible wait until we know the exact number and then create the filter. Not sure what the time loss will be for the flush doing it this way, but that could be tested.
          Hide
          stack added a comment -

          Let me know if you want me to put the hadoop 0.20.0 jars in TRUNK Ryan.

          Show
          stack added a comment - Let me know if you want me to put the hadoop 0.20.0 jars in TRUNK Ryan.
          Hide
          stack added a comment -

          Changed subject to be more general, more about adding bloomfilters, rather than prescriptive on how to do it.

          Show
          stack added a comment - Changed subject to be more general, more about adding bloomfilters, rather than prescriptive on how to do it.
          Hide
          stack added a comment -

          Latest state of RR's bloomfilter work copied from a patch posted to HBASE-1466.

          Show
          stack added a comment - Latest state of RR's bloomfilter work copied from a patch posted to HBASE-1466 .
          Hide
          Nicolas Spiegelberg added a comment -

          updating the title & description text. Note that I took out DynamicBloomFilter requirement. I will send out a document to compliment the code fix, talking about the implementation reasoning and possible future alternatives.

          Show
          Nicolas Spiegelberg added a comment - updating the title & description text. Note that I took out DynamicBloomFilter requirement. I will send out a document to compliment the code fix, talking about the implementation reasoning and possible future alternatives.
          Hide
          Nicolas Spiegelberg added a comment -

          Static bloom filter implementation for 0.20.5. See subsequent document for overview of config settings, implementation details, lessons learned, and future ideas. Has been through internal peer review, passing unit tests, and passed preliminary HBaseTest load test & random read test with expected results:

          1.8 mil rows, 1 col/row, 1 version/row, 51KB/entry
          = ~2 stores/region, ~2x read speedup, negligible load time difference

          PS - Could not submit this to review board for some reason. Said it couldn't find branches/0.20/src/test/org/apache/hadoop/hbase/HBaseTestingUtility.java. Todd?

          Show
          Nicolas Spiegelberg added a comment - Static bloom filter implementation for 0.20.5. See subsequent document for overview of config settings, implementation details, lessons learned, and future ideas. Has been through internal peer review, passing unit tests, and passed preliminary HBaseTest load test & random read test with expected results: 1.8 mil rows, 1 col/row, 1 version/row, 51KB/entry = ~2 stores/region, ~2x read speedup, negligible load time difference PS - Could not submit this to review board for some reason. Said it couldn't find branches/0.20/src/test/org/apache/hadoop/hbase/HBaseTestingUtility.java. Todd?
          Hide
          Nicolas Spiegelberg added a comment -

          Short technical layout + fodder for wiki

          Show
          Nicolas Spiegelberg added a comment - Short technical layout + fodder for wiki
          Hide
          stack added a comment -

          Doc as PDF.

          Here's some Nicolas answers to a few questions on doc:

          15:41 < St^Ack> So, what you do your hashing w/?
          15:42 < nspiegelberg> I do murmur hash with combinatoral generation
          15:43 < nspiegelberg> it's cache miss, but only need to compute the murmur twice, no matter the hashKey count
          15:44  * St^Ack excellent
          15:44 < St^Ack> So, its in the LRU cache.. whats that mean?
          15:45 < nspiegelberg> every call to bloom.contain calls getMetaBlock(BF_DATA), which is LRU cache
          15:45 < nspiegelberg> so CFs that aren't used don't have their blooms cached
          15:46 < St^Ack> excellent
          
          Show
          stack added a comment - Doc as PDF. Here's some Nicolas answers to a few questions on doc: 15:41 < St^Ack> So, what you do your hashing w/? 15:42 < nspiegelberg> I do murmur hash with combinatoral generation 15:43 < nspiegelberg> it's cache miss, but only need to compute the murmur twice, no matter the hashKey count 15:44 * St^Ack excellent 15:44 < St^Ack> So, its in the LRU cache.. whats that mean? 15:45 < nspiegelberg> every call to bloom.contain calls getMetaBlock(BF_DATA), which is LRU cache 15:45 < nspiegelberg> so CFs that aren't used don't have their blooms cached 15:46 < St^Ack> excellent
          Hide
          stack added a comment -

          Version that applies to HEAD of trunk but with two rejects that I can't really figure. Maybe you know whats up Nicolas?

          Show
          stack added a comment - Version that applies to HEAD of trunk but with two rejects that I can't really figure. Maybe you know whats up Nicolas?
          Hide
          stack added a comment -

          I added v2 to http://review.hbase.org/r/47/ so we can startup review (though it had the two rejects and doesn't yet compile)

          Show
          stack added a comment - I added v2 to http://review.hbase.org/r/47/ so we can startup review (though it had the two rejects and doesn't yet compile)
          Hide
          Nicolas Spiegelberg added a comment -

          Working version compiled against trunk at http://review.hbase.org/r/48/ . Note that I'm not our attachments here are pretty cluttered right now, so I'm holding off on adding a diff to jira until this code has been vetted a little more.

          Show
          Nicolas Spiegelberg added a comment - Working version compiled against trunk at http://review.hbase.org/r/48/ . Note that I'm not our attachments here are pretty cluttered right now, so I'm holding off on adding a diff to jira until this code has been vetted a little more.
          Hide
          Nicolas Spiegelberg added a comment -

          Code changes from Stack + Todd's peer review. The only unaddressed issue was the Array.add for Row+Col filters. Figure we can put that into a seperate JIRA if need be.

          Show
          Nicolas Spiegelberg added a comment - Code changes from Stack + Todd's peer review. The only unaddressed issue was the Array.add for Row+Col filters. Figure we can put that into a seperate JIRA if need be.
          Hide
          stack added a comment -

          Nicolas, I tried applying this last version of the patch but I get rejects that I'm not sure how to resolve still – the ones I attached above. Can you guide me on how to resolve these? Thanks. I'm trying to get this in. Yeah, lets open a new JIRA to address the Array.add issue Todd raises. Good stuff.

          Show
          stack added a comment - Nicolas, I tried applying this last version of the patch but I get rejects that I'm not sure how to resolve still – the ones I attached above. Can you guide me on how to resolve these? Thanks. I'm trying to get this in. Yeah, lets open a new JIRA to address the Array.add issue Todd raises. Good stuff.
          Hide
          Nicolas Spiegelberg added a comment -

          Sorry. Patch n00b. I did svn mv src/java/org/apache/hadoop/hbase/io/HalfHFileReader.java src/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java & apparently that doesn't reflect well in my svn diff output. This worked for me, but may not be the kosher way to do it:
          {{
          svn cp src/java/org/apache/hadoop/hbase/io/HalfHFileReader.java src/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java
          patch -p0 < HBASE-1200-0.20.5-4.patch
          svn rm src/java/org/apache/hadoop/hbase/io/HalfHFileReader.java
          }}

          Show
          Nicolas Spiegelberg added a comment - Sorry. Patch n00b. I did svn mv src/java/org/apache/hadoop/hbase/io/HalfHFileReader.java src/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java & apparently that doesn't reflect well in my svn diff output. This worked for me, but may not be the kosher way to do it: {{ svn cp src/java/org/apache/hadoop/hbase/io/HalfHFileReader.java src/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java patch -p0 < HBASE-1200 -0.20.5-4.patch svn rm src/java/org/apache/hadoop/hbase/io/HalfHFileReader.java }}
          Hide
          stack added a comment -

          Bulk move of 0.20.5 issues into 0.21.0 after vote that we merge branch into TRUNK up on list.

          Show
          stack added a comment - Bulk move of 0.20.5 issues into 0.21.0 after vote that we merge branch into TRUNK up on list.
          Hide
          stack added a comment -

          Here is a forward-port of HBASE-1200-0.20.5-4.patch. Missing is a story on HalfStoreFileReader. HBASE-1200-0.20.5-4.patch has diffs against HalfStoreFileReader but there is no file in TRUNK to apply the HalfStoreFileReader diffs against. You know whats up Nicolas?

          Show
          stack added a comment - Here is a forward-port of HBASE-1200 -0.20.5-4.patch. Missing is a story on HalfStoreFileReader. HBASE-1200 -0.20.5-4.patch has diffs against HalfStoreFileReader but there is no file in TRUNK to apply the HalfStoreFileReader diffs against. You know whats up Nicolas?
          Hide
          stack added a comment -

          Nicolas:

               // For each file, obtain a scanner:
          -    List<KeyValueScanner> scanners = StoreFileScanner.getScannersForStoreFiles(
          -        filesToCompact, false, false);
          +    List<StoreFileScanner> sfScanners = StoreFileScanner
          +    .getScannersForStoreFiles(filesToCompact, false, false);
          +    List<KeyValueScanner> scanners =
          +      new ArrayList<KeyValueScanner>(sfScanners.size()+1);
          
          

          Something wrong in the above. We make new scanners List but we don't add the sfScanners to it that I can see.

          Show
          stack added a comment - Nicolas: // For each file, obtain a scanner: - List<KeyValueScanner> scanners = StoreFileScanner.getScannersForStoreFiles( - filesToCompact, false , false ); + List<StoreFileScanner> sfScanners = StoreFileScanner + .getScannersForStoreFiles(filesToCompact, false , false ); + List<KeyValueScanner> scanners = + new ArrayList<KeyValueScanner>(sfScanners.size()+1); Something wrong in the above. We make new scanners List but we don't add the sfScanners to it that I can see.
          Hide
          stack added a comment -

          Oh, and Nicolas helped me w/ my half file prob. above. The half file was renamed from HalfHFileReader to HalfStoreFileReader.

          Show
          stack added a comment - Oh, and Nicolas helped me w/ my half file prob. above. The half file was renamed from HalfHFileReader to HalfStoreFileReader.
          Hide
          stack added a comment -

          Thanks for the patch Nicolas. Sweet. Thanks for reviews Todd.

          Show
          stack added a comment - Thanks for the patch Nicolas. Sweet. Thanks for reviews Todd.
          Hide
          stack added a comment -

          .bq The only unaddressed issue was the Array.add for Row+Col filters. Figure we can put that into a seperate JIRA if need be.

          Nicolas, what was this issue? I tried grepping the comments for Array.add. Is it this?

          branches/0.20/src/java/org/apache/hadoop/hbase/regionserver/StoreFile.java (Diff revision 1)
          663	
                          key = Bytes.add(row, col);
          this seems like it could get moderately expensive.
          
          since the bloom filter is just going to be calculating hashes of the bytes, could the API be modified a bit to take a List<ByteBuffer> or something else that allowed it to calculate the hash across a concatenation of byte arrays?
          Nicolas 2 weeks, 1 day ago (May 9th, 2010, 4:25 p.m.)
          this was a last-minute addition for the 2248 refactoring.  can revisit/optimize
          

          I'd like to make an issue for whatever it was. Thanks.

          Show
          stack added a comment - .bq The only unaddressed issue was the Array.add for Row+Col filters. Figure we can put that into a seperate JIRA if need be. Nicolas, what was this issue? I tried grepping the comments for Array.add. Is it this? branches/0.20/src/java/org/apache/hadoop/hbase/regionserver/StoreFile.java (Diff revision 1) 663 key = Bytes.add(row, col); this seems like it could get moderately expensive. since the bloom filter is just going to be calculating hashes of the bytes, could the API be modified a bit to take a List<ByteBuffer> or something else that allowed it to calculate the hash across a concatenation of byte arrays? Nicolas 2 weeks, 1 day ago (May 9th, 2010, 4:25 p.m.) this was a last-minute addition for the 2248 refactoring. can revisit/optimize I'd like to make an issue for whatever it was. Thanks.

            People

            • Assignee:
              Nicolas Spiegelberg
              Reporter:
              stack
            • Votes:
              1 Vote for this issue
              Watchers:
              6 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development