For ApproximateSizes, pro-rate table metadata size over data blocks (#6784)

Summary:
The implementation of GetApproximateSizes was inconsistent in
its treatment of the size of non-data blocks of SST files, sometimes
including and sometimes now. This was at its worst with large portion
of table file used by filters and querying a small range that crossed
a table boundary: the size estimate would include large filter size.

It's conceivable that someone might want only to know the size in terms
of data blocks, but I believe that's unlikely enough to ignore for now.
Similarly, there's no evidence the internal function AppoximateOffsetOf
is used for anything other than a one-sided ApproximateSize, so I intend
to refactor to remove redundancy in a follow-up commit.

So to fix this, GetApproximateSizes (and implementation details
ApproximateSize and ApproximateOffsetOf) now consistently include in
their returned sizes a portion of table file metadata (incl filters
and indexes) based on the size portion of the data blocks in range. In
other words, if a key range covers data blocks that are X% by size of all
the table's data blocks, returned approximate size is X% of the total
file size. It would technically be more accurate to attribute metadata
based on number of keys, but that's not computationally efficient with
data available and rarely a meaningful difference.

Also includes miscellaneous comment improvements / clarifications.

Also included is a new approximatesizerandom benchmark for db_bench.
No significant performance difference seen with this change, whether ~700 ops/sec with cache_index_and_filter_blocks and small cache or ~150k ops/sec without cache_index_and_filter_blocks.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/6784

Test Plan:
Test added to DBTest.ApproximateSizesFilesWithErrorMargin.
Old code running new test...

    [ RUN      ] DBTest.ApproximateSizesFilesWithErrorMargin
    db/db_test.cc:1562: Failure
    Expected: (size) <= (11 * 100), actual: 9478 vs 1100

Other tests updated to reflect consistent accounting of metadata.

Reviewed By: siying

Differential Revision: D21334706

Pulled By: pdillinger

fbshipit-source-id: 6f86870e45213334fedbe9c73b4ebb1d8d611185
main
Peter Dillinger 4 years ago committed by Facebook GitHub Bot
parent 298b00a396
commit 14eca6bf04
  1. 1
      HISTORY.md
  2. 79
      db/db_test.cc
  3. 3
      db/version_set.cc
  4. 13
      include/rocksdb/db.h
  5. 68
      table/block_based/block_based_table_reader.cc
  6. 16
      table/block_based/block_based_table_reader.h
  7. 2
      table/block_based/mock_block_based_table.h
  8. 4
      table/block_based/partitioned_filter_block_test.cc
  9. 9
      table/table_reader.h
  10. 12
      table/table_test.cc
  11. 52
      tools/db_bench_tool.cc

@ -10,6 +10,7 @@
* Fix a use-after-free bug in best-efforts recovery. column_family_memtables_ needs to point to valid ColumnFamilySet.
* Let best-efforts recovery ignore corrupted files during table loading.
* Fix corrupt key read from ingested file when iterator direction switches from reverse to forward at a key that is a prefix of another key in the same file. It is only possible in files with a non-zero global seqno.
* Fix abnormally large estimate from GetApproximateSizes when a range starts near the end of one SST file and near the beginning of another. Now GetApproximateSizes consistently and fairly includes the size of SST metadata in addition to data blocks, attributing metadata proportionally among the data blocks based on their size.
### Public API Change
* Flush(..., column_family) may return Status::ColumnFamilyDropped() instead of Status::InvalidArgument() if column_family is dropped while processing the flush request.

@ -1489,18 +1489,24 @@ TEST_F(DBTest, ApproximateSizesMemTable) {
}
TEST_F(DBTest, ApproximateSizesFilesWithErrorMargin) {
// Roughly 4 keys per data block, 1000 keys per file,
// with filter substantially larger than a data block
BlockBasedTableOptions table_options;
table_options.filter_policy.reset(NewBloomFilterPolicy(16));
table_options.block_size = 100;
Options options = CurrentOptions();
options.write_buffer_size = 1024 * 1024;
options.table_factory.reset(NewBlockBasedTableFactory(table_options));
options.write_buffer_size = 24 * 1024;
options.compression = kNoCompression;
options.create_if_missing = true;
options.target_file_size_base = 1024 * 1024;
options.target_file_size_base = 24 * 1024;
DestroyAndReopen(options);
const auto default_cf = db_->DefaultColumnFamily();
const int N = 64000;
Random rnd(301);
for (int i = 0; i < N; i++) {
ASSERT_OK(Put(Key(i), RandomString(&rnd, 1024)));
ASSERT_OK(Put(Key(i), RandomString(&rnd, 24)));
}
// Flush everything to files
Flush();
@ -1509,7 +1515,7 @@ TEST_F(DBTest, ApproximateSizesFilesWithErrorMargin) {
// Write more keys
for (int i = N; i < (N + N / 4); i++) {
ASSERT_OK(Put(Key(i), RandomString(&rnd, 1024)));
ASSERT_OK(Put(Key(i), RandomString(&rnd, 24)));
}
// Flush everything to files again
Flush();
@ -1517,27 +1523,45 @@ TEST_F(DBTest, ApproximateSizesFilesWithErrorMargin) {
// Wait for compaction to finish
ASSERT_OK(dbfull()->TEST_WaitForCompact());
const std::string start = Key(0);
const std::string end = Key(2 * N);
const Range r(start, end);
{
const std::string start = Key(0);
const std::string end = Key(2 * N);
const Range r(start, end);
SizeApproximationOptions size_approx_options;
size_approx_options.include_memtabtles = false;
size_approx_options.include_files = true;
size_approx_options.files_size_error_margin = -1.0; // disabled
SizeApproximationOptions size_approx_options;
size_approx_options.include_memtabtles = false;
size_approx_options.include_files = true;
size_approx_options.files_size_error_margin = -1.0; // disabled
// Get the precise size without any approximation heuristic
uint64_t size;
db_->GetApproximateSizes(size_approx_options, default_cf, &r, 1, &size);
ASSERT_NE(size, 0);
// Get the precise size without any approximation heuristic
uint64_t size;
db_->GetApproximateSizes(size_approx_options, default_cf, &r, 1, &size);
ASSERT_NE(size, 0);
// Get the size with an approximation heuristic
uint64_t size2;
const double error_margin = 0.2;
size_approx_options.files_size_error_margin = error_margin;
db_->GetApproximateSizes(size_approx_options, default_cf, &r, 1, &size2);
ASSERT_LT(size2, size * (1 + error_margin));
ASSERT_GT(size2, size * (1 - error_margin));
// Get the size with an approximation heuristic
uint64_t size2;
const double error_margin = 0.2;
size_approx_options.files_size_error_margin = error_margin;
db_->GetApproximateSizes(size_approx_options, default_cf, &r, 1, &size2);
ASSERT_LT(size2, size * (1 + error_margin));
ASSERT_GT(size2, size * (1 - error_margin));
}
{
// Ensure that metadata is not falsely attributed only to the last data in
// the file. (In some applications, filters can be large portion of data
// size.)
// Perform many queries over small range, enough to ensure crossing file
// boundary, and make sure we never see a spike for large filter.
for (int i = 0; i < 3000; i += 10) {
const std::string start = Key(i);
const std::string end = Key(i + 11); // overlap by 1 key
const Range r(start, end);
uint64_t size;
db_->GetApproximateSizes(&r, 1, &size);
ASSERT_LE(size, 11 * 100);
}
}
}
TEST_F(DBTest, GetApproximateMemTableStats) {
@ -1675,12 +1699,13 @@ TEST_F(DBTest, ApproximateSizes_MixOfSmallAndLarge) {
ASSERT_TRUE(Between(Size("", Key(2), 1), 20000, 21000));
ASSERT_TRUE(Between(Size("", Key(3), 1), 120000, 121000));
ASSERT_TRUE(Between(Size("", Key(4), 1), 130000, 131000));
ASSERT_TRUE(Between(Size("", Key(5), 1), 230000, 231000));
ASSERT_TRUE(Between(Size("", Key(6), 1), 240000, 241000));
ASSERT_TRUE(Between(Size("", Key(7), 1), 540000, 541000));
ASSERT_TRUE(Between(Size("", Key(8), 1), 550000, 560000));
ASSERT_TRUE(Between(Size("", Key(5), 1), 230000, 232000));
ASSERT_TRUE(Between(Size("", Key(6), 1), 240000, 242000));
// Ensure some overhead is accounted for, even without including all
ASSERT_TRUE(Between(Size("", Key(7), 1), 540500, 545000));
ASSERT_TRUE(Between(Size("", Key(8), 1), 550500, 555000));
ASSERT_TRUE(Between(Size(Key(3), Key(5), 1), 110000, 111000));
ASSERT_TRUE(Between(Size(Key(3), Key(5), 1), 110100, 111000));
dbfull()->TEST_CompactRange(0, nullptr, nullptr, handles_[1]);
}

@ -5489,7 +5489,8 @@ uint64_t VersionSet::ApproximateSize(const SizeApproximationOptions& options,
static_cast<uint64_t>(total_full_size * margin)) {
total_full_size += total_intersecting_size / 2;
} else {
// Estimate for all the first files, at each level
// Estimate for all the first files (might also be last files), at each
// level
for (const auto file_ptr : first_files) {
total_full_size += ApproximateSize(v, *file_ptr, start, end, caller);
}

@ -1004,32 +1004,33 @@ class DB {
};
// For each i in [0,n-1], store in "sizes[i]", the approximate
// file system space used by keys in "[range[i].start .. range[i].limit)".
// file system space used by keys in "[range[i].start .. range[i].limit)"
// in a single column family.
//
// Note that the returned sizes measure file system space usage, so
// if the user data compresses by a factor of ten, the returned
// sizes will be one-tenth the size of the corresponding user data size.
virtual Status GetApproximateSizes(const SizeApproximationOptions& options,
ColumnFamilyHandle* column_family,
const Range* range, int n,
const Range* ranges, int n,
uint64_t* sizes) = 0;
// Simpler versions of the GetApproximateSizes() method above.
// The include_flags argumenbt must of type DB::SizeApproximationFlags
// and can not be NONE.
virtual void GetApproximateSizes(ColumnFamilyHandle* column_family,
const Range* range, int n, uint64_t* sizes,
const Range* ranges, int n, uint64_t* sizes,
uint8_t include_flags = INCLUDE_FILES) {
SizeApproximationOptions options;
options.include_memtabtles =
(include_flags & SizeApproximationFlags::INCLUDE_MEMTABLES) != 0;
options.include_files =
(include_flags & SizeApproximationFlags::INCLUDE_FILES) != 0;
GetApproximateSizes(options, column_family, range, n, sizes);
GetApproximateSizes(options, column_family, ranges, n, sizes);
}
virtual void GetApproximateSizes(const Range* range, int n, uint64_t* sizes,
virtual void GetApproximateSizes(const Range* ranges, int n, uint64_t* sizes,
uint8_t include_flags = INCLUDE_FILES) {
GetApproximateSizes(DefaultColumnFamily(), range, n, sizes, include_flags);
GetApproximateSizes(DefaultColumnFamily(), ranges, n, sizes, include_flags);
}
// The method is similar to GetApproximateSizes, except it

@ -656,8 +656,8 @@ Status BlockBasedTable::Open(
// access a dangling pointer.
BlockCacheLookupContext lookup_context{TableReaderCaller::kPrefetch};
Rep* rep = new BlockBasedTable::Rep(ioptions, env_options, table_options,
internal_comparator, skip_filters, level,
immortal_table);
internal_comparator, skip_filters,
file_size, level, immortal_table);
rep->file = std::move(file);
rep->footer = footer;
rep->hash_index_allow_collision = table_options.hash_index_allow_collision;
@ -3012,30 +3012,37 @@ Status BlockBasedTable::CreateIndexReader(
}
}
uint64_t BlockBasedTable::ApproximateOffsetOf(
const InternalIteratorBase<IndexValue>& index_iter) const {
uint64_t result = 0;
uint64_t BlockBasedTable::ApproximateDataOffsetOf(
const InternalIteratorBase<IndexValue>& index_iter,
uint64_t data_size) const {
if (index_iter.Valid()) {
BlockHandle handle = index_iter.value().handle;
result = handle.offset();
return handle.offset();
} else {
// The iterator is past the last key in the file. If table_properties is not
// available, approximate the offset by returning the offset of the
// metaindex block (which is right near the end of the file).
if (rep_->table_properties) {
result = rep_->table_properties->data_size;
}
// table_properties is not present in the table.
if (result == 0) {
result = rep_->footer.metaindex_handle().offset();
}
// The iterator is past the last key in the file.
return data_size;
}
}
return result;
uint64_t BlockBasedTable::GetApproximateDataSize() {
// Should be in table properties unless super old version
if (rep_->table_properties) {
return rep_->table_properties->data_size;
}
// Fall back to rough estimate from footer
return rep_->footer.metaindex_handle().offset();
}
uint64_t BlockBasedTable::ApproximateOffsetOf(const Slice& key,
TableReaderCaller caller) {
uint64_t data_size = GetApproximateDataSize();
if (UNLIKELY(data_size == 0)) {
// Hmm. Let's just split in half to avoid skewing one way or another,
// since we don't know whether we're operating on lower bound or
// upper bound.
return rep_->file_size / 2;
}
BlockCacheLookupContext context(caller);
IndexBlockIter iiter_on_stack;
ReadOptions ro;
@ -3050,13 +3057,27 @@ uint64_t BlockBasedTable::ApproximateOffsetOf(const Slice& key,
}
index_iter->Seek(key);
return ApproximateOffsetOf(*index_iter);
uint64_t offset = ApproximateDataOffsetOf(*index_iter, data_size);
// Pro-rate file metadata (incl filters) size-proportionally across data
// blocks.
double size_ratio =
static_cast<double>(offset) / static_cast<double>(data_size);
return static_cast<uint64_t>(size_ratio *
static_cast<double>(rep_->file_size));
}
uint64_t BlockBasedTable::ApproximateSize(const Slice& start, const Slice& end,
TableReaderCaller caller) {
assert(rep_->internal_comparator.Compare(start, end) <= 0);
uint64_t data_size = GetApproximateDataSize();
if (UNLIKELY(data_size == 0)) {
// Hmm. Assume whole file is involved, since we have lower and upper
// bound.
return rep_->file_size;
}
BlockCacheLookupContext context(caller);
IndexBlockIter iiter_on_stack;
ReadOptions ro;
@ -3071,12 +3092,17 @@ uint64_t BlockBasedTable::ApproximateSize(const Slice& start, const Slice& end,
}
index_iter->Seek(start);
uint64_t start_offset = ApproximateOffsetOf(*index_iter);
uint64_t start_offset = ApproximateDataOffsetOf(*index_iter, data_size);
index_iter->Seek(end);
uint64_t end_offset = ApproximateOffsetOf(*index_iter);
uint64_t end_offset = ApproximateDataOffsetOf(*index_iter, data_size);
assert(end_offset >= start_offset);
return end_offset - start_offset;
// Pro-rate file metadata (incl filters) size-proportionally across data
// blocks.
double size_ratio = static_cast<double>(end_offset - start_offset) /
static_cast<double>(data_size);
return static_cast<uint64_t>(size_ratio *
static_cast<double>(rep_->file_size));
}
bool BlockBasedTable::TEST_FilterBlockInCache() const {

@ -441,9 +441,13 @@ class BlockBasedTable : public TableReader {
static void GenerateCachePrefix(Cache* cc, FSWritableFile* file, char* buffer,
size_t* size);
// Given an iterator return its offset in file.
uint64_t ApproximateOffsetOf(
const InternalIteratorBase<IndexValue>& index_iter) const;
// Size of all data blocks, maybe approximate
uint64_t GetApproximateDataSize();
// Given an iterator return its offset in data block section of file.
uint64_t ApproximateDataOffsetOf(
const InternalIteratorBase<IndexValue>& index_iter,
uint64_t data_size) const;
// Helper functions for DumpTable()
Status DumpIndexBlock(WritableFile* out_file);
@ -482,7 +486,7 @@ struct BlockBasedTable::Rep {
Rep(const ImmutableCFOptions& _ioptions, const EnvOptions& _env_options,
const BlockBasedTableOptions& _table_opt,
const InternalKeyComparator& _internal_comparator, bool skip_filters,
int _level, const bool _immortal_table)
uint64_t _file_size, int _level, const bool _immortal_table)
: ioptions(_ioptions),
env_options(_env_options),
table_options(_table_opt),
@ -494,6 +498,7 @@ struct BlockBasedTable::Rep {
whole_key_filtering(_table_opt.whole_key_filtering),
prefix_filtering(true),
global_seqno(kDisableGlobalSequenceNumber),
file_size(_file_size),
level(_level),
immortal_table(_immortal_table) {}
@ -551,6 +556,9 @@ struct BlockBasedTable::Rep {
// and every key have it's own seqno.
SequenceNumber global_seqno;
// Size of the table file on disk
uint64_t file_size;
// the level when the table is opened, could potentially change when trivial
// move is involved
int level;

@ -39,7 +39,7 @@ class MockBlockBasedTableTester {
constexpr bool immortal_table = false;
table_.reset(new MockBlockBasedTable(new BlockBasedTable::Rep(
ioptions_, env_options_, table_options_, icomp_, skip_filters,
kMockLevel, immortal_table)));
12345 /*file_size*/, kMockLevel, immortal_table)));
}
FilterBitsBuilder* GetBuilder() const {

@ -143,11 +143,13 @@ class PartitionedFilterBlockTest
} while (status.IsIncomplete());
constexpr bool skip_filters = false;
constexpr uint64_t file_size = 12345;
constexpr int level = 0;
constexpr bool immortal_table = false;
table_.reset(new MockedBlockBasedTable(
new BlockBasedTable::Rep(ioptions_, env_options_, table_options_,
icomp_, skip_filters, level, immortal_table),
icomp_, skip_filters, file_size, level,
immortal_table),
pib));
BlockContents contents(slice);
CachableEntry<Block> block(

@ -64,12 +64,19 @@ class TableReader {
// 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.
// 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;

@ -3557,12 +3557,12 @@ static void DoCompressionTest(CompressionType comp) {
const MutableCFOptions moptions(options);
c.Finish(options, ioptions, moptions, table_options, ikc, &keys, &kvmap);
ASSERT_TRUE(Between(c.ApproximateOffsetOf("abc"), 0, 0));
ASSERT_TRUE(Between(c.ApproximateOffsetOf("k01"), 0, 0));
ASSERT_TRUE(Between(c.ApproximateOffsetOf("k02"), 0, 0));
ASSERT_TRUE(Between(c.ApproximateOffsetOf("k03"), 2000, 3500));
ASSERT_TRUE(Between(c.ApproximateOffsetOf("k04"), 2000, 3500));
ASSERT_TRUE(Between(c.ApproximateOffsetOf("xyz"), 4000, 6500));
ASSERT_TRUE(Between(c.ApproximateOffsetOf("abc"), 0, 0));
ASSERT_TRUE(Between(c.ApproximateOffsetOf("k01"), 0, 0));
ASSERT_TRUE(Between(c.ApproximateOffsetOf("k02"), 0, 0));
ASSERT_TRUE(Between(c.ApproximateOffsetOf("k03"), 2000, 3500));
ASSERT_TRUE(Between(c.ApproximateOffsetOf("k04"), 2000, 3500));
ASSERT_TRUE(Between(c.ApproximateOffsetOf("xyz"), 4000, 7000));
c.ResetTableReader();
}

@ -115,6 +115,7 @@ DEFINE_string(
"readrandomwriterandom,"
"updaterandom,"
"xorupdaterandom,"
"approximatesizerandom,"
"randomwithverify,"
"fill100K,"
"crc32c,"
@ -3021,6 +3022,10 @@ class Benchmark {
fprintf(stderr, "entries_per_batch = %" PRIi64 "\n",
entries_per_batch_);
method = &Benchmark::MultiReadRandom;
} else if (name == "approximatesizerandom") {
fprintf(stderr, "entries_per_batch = %" PRIi64 "\n",
entries_per_batch_);
method = &Benchmark::ApproximateSizeRandom;
} else if (name == "mixgraph") {
method = &Benchmark::MixGraph;
} else if (name == "readmissing") {
@ -5211,6 +5216,53 @@ class Benchmark {
thread->stats.AddMessage(msg);
}
// Calls ApproximateSize over random key ranges.
void ApproximateSizeRandom(ThreadState* thread) {
int64_t size_sum = 0;
int64_t num_sizes = 0;
const size_t batch_size = entries_per_batch_;
std::vector<Range> ranges;
std::vector<Slice> lkeys;
std::vector<std::unique_ptr<const char[]>> lkey_guards;
std::vector<Slice> rkeys;
std::vector<std::unique_ptr<const char[]>> rkey_guards;
std::vector<uint64_t> sizes;
while (ranges.size() < batch_size) {
// Ugly without C++17 return from emplace_back
lkey_guards.emplace_back();
rkey_guards.emplace_back();
lkeys.emplace_back(AllocateKey(&lkey_guards.back()));
rkeys.emplace_back(AllocateKey(&rkey_guards.back()));
ranges.emplace_back(lkeys.back(), rkeys.back());
sizes.push_back(0);
}
Duration duration(FLAGS_duration, reads_);
while (!duration.Done(1)) {
DB* db = SelectDB(thread);
for (size_t i = 0; i < batch_size; ++i) {
int64_t lkey = GetRandomKey(&thread->rand);
int64_t rkey = GetRandomKey(&thread->rand);
if (lkey > rkey) {
std::swap(lkey, rkey);
}
GenerateKeyFromInt(lkey, FLAGS_num, &lkeys[i]);
GenerateKeyFromInt(rkey, FLAGS_num, &rkeys[i]);
}
db->GetApproximateSizes(&ranges[0], static_cast<int>(entries_per_batch_),
&sizes[0]);
num_sizes += entries_per_batch_;
for (int64_t size : sizes) {
size_sum += size;
}
thread->stats.FinishedOps(nullptr, db, entries_per_batch_, kOthers);
}
char msg[100];
snprintf(msg, sizeof(msg), "(Avg approx size=%g)",
static_cast<double>(size_sum) / static_cast<double>(num_sizes));
thread->stats.AddMessage(msg);
}
// The inverse function of Pareto distribution
int64_t ParetoCdfInversion(double u, double theta, double k, double sigma) {
double ret;

Loading…
Cancel
Save