Revision 239d17a19c3cec16937aa4b6c56c90f4f217addf authored by Peter Dillinger on 18 December 2020, 22:29:48 UTC, committed by Facebook GitHub Bot on 18 December 2020, 22:31:03 UTC
Summary:
Primarily this change refactors the optimize_filters_for_memory
code for Bloom filters, based on malloc_usable_size, to also work for
Ribbon filters.

This change also replaces the somewhat slow but general
BuiltinFilterBitsBuilder::ApproximateNumEntries with
implementation-specific versions for Ribbon (new) and Legacy Bloom
(based on a recently deleted version). The reason is to emphasize
speed in ApproximateNumEntries rather than 100% accuracy.

Justification: ApproximateNumEntries (formerly CalculateNumEntry) is
only used by RocksDB for range-partitioned filters, called each time we
start to construct one. (In theory, it should be possible to reuse the
estimate, but the abstractions provided by FilterPolicy don't really
make that workable.) But this is only used as a heuristic estimate for
hitting a desired partitioned filter size because of alignment to data
blocks, which have various numbers of unique keys or prefixes. The two
factors lead us to prioritize reasonable speed over 100% accuracy.

optimize_filters_for_memory adds extra complication, because precisely
calculating num_entries for some allowed number of bytes depends on state
with optimize_filters_for_memory enabled. And the allocator-agnostic
implementation of optimize_filters_for_memory, using malloc_usable_size,
means we would have to actually allocate memory, many times, just to
precisely determine how many entries (keys) could be added and stay below
some size budget, for the current state. (In a draft, I got this
working, and then realized the balance of speed vs. accuracy was all
wrong.)

So related to that, I have made CalculateSpace, an internal-only API
only used for testing, non-authoritative also if
optimize_filters_for_memory is enabled. This simplifies some code.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/7774

Test Plan:
unit test updated, and for FilterSize test, range of tested
values is greatly expanded (still super fast)

Also tested `db_bench -benchmarks=fillrandom,stats -bloom_bits=10 -num=1000000 -partition_index_and_filters -format_version=5 [-optimize_filters_for_memory] [-use_ribbon_filter]` with temporary debug output of generated filter sizes.

Bloom+optimize_filters_for_memory:

      1 Filter size: 197 (224 in memory)
    134 Filter size: 3525 (3584 in memory)
    107 Filter size: 4037 (4096 in memory)
    Total on disk: 904,506
    Total in memory: 918,752

Ribbon+optimize_filters_for_memory:

      1 Filter size: 3061 (3072 in memory)
    110 Filter size: 3573 (3584 in memory)
     58 Filter size: 4085 (4096 in memory)
    Total on disk: 633,021 (-30.0%)
    Total in memory: 634,880 (-30.9%)

Bloom (no offm):

      1 Filter size: 261 (320 in memory)
      1 Filter size: 3333 (3584 in memory)
    240 Filter size: 3717 (4096 in memory)
    Total on disk: 895,674 (-1% on disk vs. +offm; known tolerable overhead of offm)
    Total in memory: 986,944 (+7.4% vs. +offm)

Ribbon (no offm):

      1 Filter size: 2949 (3072 in memory)
      1 Filter size: 3381 (3584 in memory)
    167 Filter size: 3701 (4096 in memory)
    Total on disk: 624,397 (-30.3% vs. Bloom)
    Total in memory: 690,688 (-30.0% vs. Bloom)

Note that optimize_filters_for_memory is even more effective for Ribbon filter than for cache-local Bloom, because it can close the unused memory gap even tighter than Bloom filter, because of 16 byte increments for Ribbon vs. 64 byte increments for Bloom.

Reviewed By: jay-zhuang

Differential Revision: D25592970

Pulled By: pdillinger

fbshipit-source-id: 606fdaa025bb790d7e9c21601e8ea86e10541912
1 parent 04b3524
Raw File
table_reader.h
//  Copyright (c) 2011-present, Facebook, Inc.  All rights reserved.
//  This source code is licensed under both the GPLv2 (found in the
//  COPYING file in the root directory) and Apache 2.0 License
//  (found in the LICENSE.Apache file in the root directory).
//
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
// Use of this source code is governed by a BSD-style license that can be
// found in the LICENSE file. See the AUTHORS file for names of contributors.

#pragma once
#include <memory>
#include "db/range_tombstone_fragmenter.h"
#include "rocksdb/slice_transform.h"
#include "table/get_context.h"
#include "table/internal_iterator.h"
#include "table/multiget_context.h"
#include "table/table_reader_caller.h"

namespace ROCKSDB_NAMESPACE {

class Iterator;
struct ParsedInternalKey;
class Slice;
class Arena;
struct ReadOptions;
struct TableProperties;
class GetContext;
class MultiGetContext;

// A Table (also referred to as SST) is a sorted map from strings to strings.
// Tables are immutable and persistent.  A Table may be safely accessed from
// multiple threads without external synchronization. Table readers are used
// for reading various types of table formats supported by rocksdb including
// BlockBasedTable, PlainTable and CuckooTable format.
class TableReader {
 public:
  virtual ~TableReader() {}

  // Returns a new iterator over the table contents.
  // The result of NewIterator() is initially invalid (caller must
  // call one of the Seek methods on the iterator before using it).
  //
  // read_options: Must outlive the returned iterator.
  // arena: If not null, the arena needs to be used to allocate the Iterator.
  //        When destroying the iterator, the caller will not call "delete"
  //        but Iterator::~Iterator() directly. The destructor needs to destroy
  //        all the states but those allocated in arena.
  // skip_filters: disables checking the bloom filters even if they exist. This
  //               option is effective only for block-based table format.
  // compaction_readahead_size: its value will only be used if caller =
  // kCompaction
  virtual InternalIterator* NewIterator(
      const ReadOptions& read_options, const SliceTransform* prefix_extractor,
      Arena* arena, bool skip_filters, TableReaderCaller caller,
      size_t compaction_readahead_size = 0,
      bool allow_unprepared_value = false) = 0;

  virtual FragmentedRangeTombstoneIterator* NewRangeTombstoneIterator(
      const ReadOptions& /*read_options*/) {
    return nullptr;
  }

  // Given a key, return an approximate byte offset in the file where
  // the data for that key begins (or would begin if the key were
  // present in the file).  The returned value is in terms of file
  // bytes, and so includes effects like compression of the underlying data.
  // E.g., the approximate offset of the last key in the table will
  // be close to the file length.
  // TODO(peterd): Since this function is only used for approximate size
  // from beginning of file, reduce code duplication by removing this
  // function and letting ApproximateSize take optional start and end, so
  // that absolute start and end can be specified and optimized without
  // key / index work.
  virtual uint64_t ApproximateOffsetOf(const Slice& key,
                                       TableReaderCaller caller) = 0;

  // Given start and end keys, return the approximate data size in the file
  // between the keys. The returned value is in terms of file bytes, and so
  // includes effects like compression of the underlying data and applicable
  // portions of metadata including filters and indexes. Nullptr for start or
  // end (or both) indicates absolute start or end of the table.
  virtual uint64_t ApproximateSize(const Slice& start, const Slice& end,
                                   TableReaderCaller caller) = 0;

  // Set up the table for Compaction. Might change some parameters with
  // posix_fadvise
  virtual void SetupForCompaction() = 0;

  virtual std::shared_ptr<const TableProperties> GetTableProperties() const = 0;

  // Prepare work that can be done before the real Get()
  virtual void Prepare(const Slice& /*target*/) {}

  // Report an approximation of how much memory has been used.
  virtual size_t ApproximateMemoryUsage() const = 0;

  // Calls get_context->SaveValue() repeatedly, starting with
  // the entry found after a call to Seek(key), until it returns false.
  // May not make such a call if filter policy says that key is not present.
  //
  // get_context->MarkKeyMayExist needs to be called when it is configured to be
  // memory only and the key is not found in the block cache.
  //
  // readOptions is the options for the read
  // key is the key to search for
  // skip_filters: disables checking the bloom filters even if they exist. This
  //               option is effective only for block-based table format.
  virtual Status Get(const ReadOptions& readOptions, const Slice& key,
                     GetContext* get_context,
                     const SliceTransform* prefix_extractor,
                     bool skip_filters = false) = 0;

  virtual void MultiGet(const ReadOptions& readOptions,
                        const MultiGetContext::Range* mget_range,
                        const SliceTransform* prefix_extractor,
                        bool skip_filters = false) {
    for (auto iter = mget_range->begin(); iter != mget_range->end(); ++iter) {
      *iter->s = Get(readOptions, iter->ikey, iter->get_context,
                     prefix_extractor, skip_filters);
    }
  }

  // Prefetch data corresponding to a give range of keys
  // Typically this functionality is required for table implementations that
  // persists the data on a non volatile storage medium like disk/SSD
  virtual Status Prefetch(const Slice* begin = nullptr,
                          const Slice* end = nullptr) {
    (void) begin;
    (void) end;
    // Default implementation is NOOP.
    // The child class should implement functionality when applicable
    return Status::OK();
  }

  // convert db file to a human readable form
  virtual Status DumpTable(WritableFile* /*out_file*/) {
    return Status::NotSupported("DumpTable() not supported");
  }

  // check whether there is corruption in this db file
  virtual Status VerifyChecksum(const ReadOptions& /*read_options*/,
                                TableReaderCaller /*caller*/) {
    return Status::NotSupported("VerifyChecksum() not supported");
  }
};

}  // namespace ROCKSDB_NAMESPACE
back to top