提交 86ef6c3f 编写于 作者: K Kai Liu

Add statistics to sst file

Summary:
So far we only have key/value pairs as well as bloom filter stored in the
sst file.  It will be great if we are able to store more metadata about
this table itself, for example, the entry size, bloom filter name, etc.

This diff is the first step of this effort. It allows table to keep the
basic statistics mentioned in http://fburl.com/14995441, as well as
allowing writing user-collected stats to stats block.

After this diff, we will figure out the interface of how to allow user to collect their interested statistics.

Test Plan:
1. Added several unit tests.
2. Ran `make check` to ensure it doesn't break other tests.

Reviewers: dhruba, haobo

CC: leveldb

Differential Revision: https://reviews.facebook.net/D13419
上级 88f2f890
......@@ -1972,9 +1972,9 @@ Status DBImpl::DoCompactionWork(CompactionState* compact) {
compaction_filter_value.clear();
bool to_delete =
compaction_filter->Filter(compact->compaction->level(),
ikey.user_key, value,
&compaction_filter_value,
&value_changed);
ikey.user_key, value,
&compaction_filter_value,
&value_changed);
if (to_delete) {
// make a copy of the original key
delete_key.assign(key.data(), key.data() + key.size());
......
......@@ -30,11 +30,9 @@ namespace log { class Writer; }
class Compaction;
class Iterator;
class MemTable;
class TableBuilder;
class TableCache;
class Version;
class VersionSet;
class WritableFile;
// Return the smallest index i such that files[i]->largest >= key.
// Return files.size() if there is no such file.
......
// Copyright (c) 2013 Facebook
// 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 <string>
#include <unordered_map>
namespace rocksdb {
// TableStats contains a bunch of read-only stats of its associated
// table.
struct TableStats {
public:
// TODO(kailiu) we do not support user collected stats yet.
//
// Other than basic table stats, each table may also have the user
// collected stats.
// The value of the user-collected stats are encoded as raw bytes --
// users have to interprete these values by themselves.
typedef
std::unordered_map<std::string, std::string>
UserCollectedStats;
// the total size of all data blocks.
uint64_t data_size = 0;
// the total size of all index blocks.
uint64_t index_size = 0;
// total raw key size
uint64_t raw_key_size = 0;
// total raw value size
uint64_t raw_value_size = 0;
// the number of blocks in this table
uint64_t num_data_blocks = 0;
// the number of entries in this table
uint64_t num_entries = 0;
// user collected stats
UserCollectedStats user_collected_stats;
};
} // namespace rocksdb
......@@ -16,6 +16,7 @@
#include "table/block.h"
#include "table/filter_block.h"
#include "table/format.h"
#include "table/table.h"
#include "table/two_level_iterator.h"
#include "util/coding.h"
......@@ -50,6 +51,7 @@ struct Table::Rep {
BlockHandle metaindex_handle; // Handle to metaindex_block: saved from footer
Block* index_block;
TableStats table_stats;
};
// Helper function to setup the cache key's prefix for the Table.
......@@ -168,11 +170,11 @@ void Table::SetupForCompaction() {
compaction_optimized_ = true;
}
void Table::ReadMeta(const Footer& footer) {
if (rep_->options.filter_policy == nullptr) {
return; // Do not need any metadata
}
const TableStats& Table::GetTableStats() const {
return rep_->table_stats;
}
void Table::ReadMeta(const Footer& footer) {
// TODO(sanjay): Skip this if footer.metaindex_handle() size indicates
// it is an empty block.
// TODO: we never really verify check sum for meta index block
......@@ -184,12 +186,33 @@ void Table::ReadMeta(const Footer& footer) {
}
Iterator* iter = meta->NewIterator(BytewiseComparator());
std::string key = "filter.";
key.append(rep_->options.filter_policy->Name());
iter->Seek(key);
if (iter->Valid() && iter->key() == Slice(key)) {
ReadFilter(iter->value());
// read filter
if (rep_->options.filter_policy) {
std::string key = kFilterBlockPrefix;
key.append(rep_->options.filter_policy->Name());
iter->Seek(key);
if (iter->Valid() && iter->key() == Slice(key)) {
ReadFilter(iter->value());
}
}
// read stats
iter->Seek(kStatsBlock);
if (iter->Valid() && iter->key() == Slice(kStatsBlock)) {
auto s = iter->status();
if (s.ok()) {
s = ReadStats(iter->value(), rep_);
}
if (!s.ok()) {
auto err_msg =
"[Warning] Encountered error while reading data from stats block " +
s.ToString();
Log(rep_->options.info_log, err_msg.c_str());
}
}
delete iter;
delete meta;
}
......@@ -215,6 +238,82 @@ void Table::ReadFilter(const Slice& filter_handle_value) {
rep_->filter = new FilterBlockReader(rep_->options, block.data);
}
Status Table::ReadStats(const Slice& handle_value, Rep* rep) {
Slice v = handle_value;
BlockHandle handle;
if (!handle.DecodeFrom(&v).ok()) {
return Status::InvalidArgument("Failed to decode stats block handle");
}
BlockContents block_contents;
Status s = ReadBlockContents(
rep->file.get(),
ReadOptions(),
handle,
&block_contents,
rep->options.env
);
if (!s.ok()) {
return s;
}
Block stats_block(block_contents);
std::unique_ptr<Iterator> iter(
stats_block.NewIterator(BytewiseComparator())
);
auto& table_stats = rep->table_stats;
std::unordered_map<std::string, uint64_t*> predefined_name_stat_map = {
{ TableStatsNames::kDataSize, &table_stats.data_size },
{ TableStatsNames::kIndexSize, &table_stats.index_size },
{ TableStatsNames::kRawKeySize, &table_stats.raw_key_size },
{ TableStatsNames::kRawValueSize, &table_stats.raw_value_size },
{ TableStatsNames::kNumDataBlocks, &table_stats.num_data_blocks },
{ TableStatsNames::kNumEntries, &table_stats.num_entries },
};
std::string last_key;
for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {
s = iter->status();
if (!s.ok()) {
break;
}
auto key = iter->key().ToString();
// stats block is strictly sorted with no duplicate key.
assert(
last_key.empty() ||
BytewiseComparator()->Compare(key, last_key) > 0
);
last_key = key;
auto raw_val = iter->value();
auto pos = predefined_name_stat_map.find(key);
if (pos == predefined_name_stat_map.end()) {
// handle user-collected
table_stats.user_collected_stats.insert(
std::make_pair(iter->key().ToString(), raw_val.ToString())
);
} else {
// handle predefined rocksdb stats
uint64_t val;
if (!GetVarint64(&raw_val, &val)) {
// skip malformed value
auto error_msg =
"[Warning] detect malformed value in stats meta-block:"
"\tkey: " + key + "\tval: " + raw_val.ToString();
Log(rep->options.info_log, error_msg.c_str());
continue;
}
*(pos->second) = val;
}
}
return s;
}
Table::~Table() {
delete rep_;
}
......@@ -495,4 +594,14 @@ uint64_t Table::ApproximateOffsetOf(const Slice& key) const {
return result;
}
const std::string Table::kFilterBlockPrefix = "filter.";
const std::string Table::kStatsBlock = "rocksdb.stats";
const std::string TableStatsNames::kDataSize = "rocksdb.data.size";
const std::string TableStatsNames::kIndexSize = "rocksdb.index.size";
const std::string TableStatsNames::kRawKeySize = "rocksdb.raw.key.size";
const std::string TableStatsNames::kRawValueSize = "rocksdb.raw.value.size";
const std::string TableStatsNames::kNumDataBlocks = "rocksdb.num.data.blocks";
const std::string TableStatsNames::kNumEntries = "rocksdb.num.entries";
} // namespace rocksdb
......@@ -5,8 +5,9 @@
#pragma once
#include <memory>
#include <stdint.h>
#include "rocksdb/iterator.h"
#include "rocksdb/env.h"
#include "rocksdb/iterator.h"
#include "rocksdb/table_stats.h"
namespace rocksdb {
......@@ -25,6 +26,9 @@ using std::unique_ptr;
// multiple threads without external synchronization.
class Table {
public:
static const std::string kFilterBlockPrefix;
static const std::string kStatsBlock;
// Attempt to open the table that is stored in bytes [0..file_size)
// of "file", and read the metadata entries necessary to allow
// retrieving data from the table.
......@@ -68,10 +72,13 @@ class Table {
// posix_fadvise
void SetupForCompaction();
const TableStats& GetTableStats() const;
private:
struct Rep;
Rep* rep_;
bool compaction_optimized_;
explicit Table(Rep* rep) : compaction_optimized_(false) { rep_ = rep; }
static Iterator* BlockReader(void*, const ReadOptions&,
const EnvOptions& soptions, const Slice&,
......@@ -92,6 +99,7 @@ class Table {
void ReadMeta(const Footer& footer);
void ReadFilter(const Slice& filter_handle_value);
static Status ReadStats(const Slice& handle_value, Rep* rep);
static void SetupCacheKeyPrefix(Rep* rep);
......@@ -100,4 +108,13 @@ class Table {
void operator=(const Table&);
};
struct TableStatsNames {
static const std::string kDataSize;
static const std::string kIndexSize;
static const std::string kRawKeySize;
static const std::string kRawValueSize;
static const std::string kNumDataBlocks;
static const std::string kNumEntries;
};
} // namespace rocksdb
......@@ -5,6 +5,8 @@
#include "rocksdb/table_builder.h"
#include <assert.h>
#include <map>
#include "rocksdb/comparator.h"
#include "rocksdb/env.h"
#include "rocksdb/filter_policy.h"
......@@ -12,23 +14,63 @@
#include "table/block_builder.h"
#include "table/filter_block.h"
#include "table/format.h"
#include "table/table.h"
#include "util/coding.h"
#include "util/crc32c.h"
#include "util/stop_watch.h"
namespace rocksdb {
namespace {
struct BytewiseLessThan {
bool operator()(const std::string& key1, const std::string& key2) {
// smaller entries will be placed in front.
return comparator->Compare(key1, key2) <= 0;
}
const Comparator* comparator = BytewiseComparator();
};
// When writing to a block that requires entries to be sorted by
// `BytewiseComparator`, we can buffer the content to `BytewiseSortedMap`
// before writng to store.
typedef std::map<std::string, std::string, BytewiseLessThan> BytewiseSortedMap;
void AddStats(BytewiseSortedMap& stats, std::string name, uint64_t val) {
assert(stats.find(name) == stats.end());
std::string dst;
PutVarint64(&dst, val);
stats.insert(
std::make_pair(name, dst)
);
}
static bool GoodCompressionRatio(size_t compressed_size, size_t raw_size) {
// Check to see if compressed less than 12.5%
return compressed_size < raw_size - (raw_size / 8u);
}
} // anonymous namespace
struct TableBuilder::Rep {
Options options;
Options index_block_options;
WritableFile* file;
uint64_t offset;
uint64_t offset = 0;
Status status;
BlockBuilder data_block;
BlockBuilder index_block;
std::string last_key;
int64_t num_entries;
bool closed; // Either Finish() or Abandon() has been called.
uint64_t num_entries = 0;
uint64_t num_data_blocks = 0;
uint64_t raw_key_size = 0;
uint64_t raw_value_size = 0;
uint64_t data_size = 0;
bool closed = false; // Either Finish() or Abandon() has been called.
FilterBlockBuilder* filter_block;
// We do not emit the index entry for a block until we have seen the
......@@ -49,11 +91,8 @@ struct TableBuilder::Rep {
: options(opt),
index_block_options(opt),
file(f),
offset(0),
data_block(&options),
index_block(&index_block_options),
num_entries(0),
closed(false),
filter_block(opt.filter_policy == nullptr ? nullptr
: new FilterBlockBuilder(opt)),
pending_index_entry(false) {
......@@ -129,8 +168,10 @@ void TableBuilder::Add(const Slice& key, const Slice& value) {
}
r->last_key.assign(key.data(), key.size());
r->num_entries++;
r->data_block.Add(key, value);
r->num_entries++;
r->raw_key_size += key.size();
r->raw_value_size += value.size();
}
void TableBuilder::Flush() {
......@@ -147,11 +188,8 @@ void TableBuilder::Flush() {
if (r->filter_block != nullptr) {
r->filter_block->StartBlock(r->offset);
}
}
static bool GoodCompressionRatio(size_t compressed_size, size_t raw_size) {
// Check to see if compressed less than 12.5%
return compressed_size < raw_size - (raw_size / 8u);
r->data_size = r->offset;
++r->num_data_blocks;
}
void TableBuilder::WriteBlock(BlockBuilder* block, BlockHandle* handle) {
......@@ -267,35 +305,88 @@ Status TableBuilder::Finish() {
&filter_block_handle);
}
// Write metaindex block
// To make sure stats block is able to keep the accurate size of index
// block, we will finish writing all index entries here and flush them
// to storage after metaindex block is written.
if (ok() && (r->pending_index_entry)) {
r->options.comparator->FindShortSuccessor(&r->last_key);
std::string handle_encoding;
r->pending_handle.EncodeTo(&handle_encoding);
r->index_block.Add(r->last_key, Slice(handle_encoding));
r->pending_index_entry = false;
}
// Write meta blocks and metaindex block with the following order.
// 1. [meta block: filter]
// 2. [meta block: stats]
// 3. [metaindex block]
if (ok()) {
// We use `BytewiseComparator` as the comparator for meta block.
BlockBuilder meta_index_block(
r->options.block_restart_interval,
BytewiseComparator()
);
// Key: meta block name
// Value: block handle to that meta block
BytewiseSortedMap meta_block_handles;
// Write filter block.
if (r->filter_block != nullptr) {
// Add mapping from "filter.Name" to location of filter data
std::string key = "filter.";
// Add mapping from "<filter_block_prefix>.Name" to location
// of filter data.
std::string key = Table::kFilterBlockPrefix;
key.append(r->options.filter_policy->Name());
std::string handle_encoding;
filter_block_handle.EncodeTo(&handle_encoding);
meta_index_block.Add(key, handle_encoding);
meta_block_handles.insert(
std::make_pair(key, handle_encoding)
);
}
// Write stats block.
{
BlockBuilder stats_block(
r->options.block_restart_interval,
BytewiseComparator()
);
BytewiseSortedMap stats;
// Add basic stats
AddStats(stats, TableStatsNames::kRawKeySize, r->raw_key_size);
AddStats(stats, TableStatsNames::kRawValueSize, r->raw_value_size);
AddStats(stats, TableStatsNames::kDataSize, r->data_size);
AddStats(
stats,
TableStatsNames::kIndexSize,
r->index_block.CurrentSizeEstimate() + kBlockTrailerSize
);
AddStats(stats, TableStatsNames::kNumEntries, r->num_entries);
AddStats(stats, TableStatsNames::kNumDataBlocks, r->num_data_blocks);
for (const auto& stat : stats) {
stats_block.Add(stat.first, stat.second);
}
BlockHandle stats_block_handle;
WriteBlock(&stats_block, &stats_block_handle);
std::string handle_encoding;
stats_block_handle.EncodeTo(&handle_encoding);
meta_block_handles.insert(
std::make_pair(Table::kStatsBlock, handle_encoding)
);
} // end of stats block writing
for (const auto& metablock : meta_block_handles) {
meta_index_block.Add(metablock.first, metablock.second);
}
// TODO(postrelease): Add stats and other meta blocks
WriteBlock(&meta_index_block, &metaindex_block_handle);
}
} // meta blocks and metaindex block.
// Write index block
if (ok()) {
if (r->pending_index_entry) {
r->options.comparator->FindShortSuccessor(&r->last_key);
std::string handle_encoding;
r->pending_handle.EncodeTo(&handle_encoding);
r->index_block.Add(r->last_key, Slice(handle_encoding));
r->pending_index_entry = false;
}
WriteBlock(&r->index_block, &index_block_handle);
}
......
......@@ -4,6 +4,8 @@
#include <map>
#include <string>
#include <memory>
#include <vector>
#include "db/dbformat.h"
#include "db/memtable.h"
#include "db/write_batch_internal.h"
......@@ -232,7 +234,8 @@ class BlockConstructor: public Constructor {
class TableConstructor: public Constructor {
public:
explicit TableConstructor(const Comparator* cmp)
explicit TableConstructor(
const Comparator* cmp)
: Constructor(cmp) {
}
~TableConstructor() {
......@@ -846,6 +849,118 @@ static bool Between(uint64_t val, uint64_t low, uint64_t high) {
class TableTest { };
// This test include all the basic checks except those for index size and block
// size, which will be conducted in separated unit tests.
TEST(TableTest, BasicTableStats) {
TableConstructor c(BytewiseComparator());
c.Add("a1", "val1");
c.Add("b2", "val2");
c.Add("c3", "val3");
c.Add("d4", "val4");
c.Add("e5", "val5");
c.Add("f6", "val6");
c.Add("g7", "val7");
c.Add("h8", "val8");
c.Add("j9", "val9");
std::vector<std::string> keys;
KVMap kvmap;
Options options;
options.compression = kNoCompression;
options.block_restart_interval = 1;
c.Finish(options, &keys, &kvmap);
auto& stats = c.table()->GetTableStats();
ASSERT_EQ(kvmap.size(), stats.num_entries);
auto raw_key_size = kvmap.size() * 2ul;
auto raw_value_size = kvmap.size() * 4ul;
ASSERT_EQ(raw_key_size, stats.raw_key_size);
ASSERT_EQ(raw_value_size, stats.raw_value_size);
ASSERT_EQ(1ul, stats.num_data_blocks);
// Verify data size.
BlockBuilder block_builder(&options);
for (const auto& item : kvmap) {
block_builder.Add(item.first, item.second);
}
Slice content = block_builder.Finish();
ASSERT_EQ(
content.size() + kBlockTrailerSize,
stats.data_size
);
}
static std::string RandomString(Random* rnd, int len) {
std::string r;
test::RandomString(rnd, len, &r);
return r;
}
// It's very hard to figure out the index block size of a block accurately.
// To make sure we get the index size, we just make sure as key number
// grows, the filter block size also grows.
TEST(TableTest, IndexSizeStat) {
uint64_t last_index_size = 0;
// we need to use random keys since the pure human readable texts
// may be well compressed, resulting insignifcant change of index
// block size.
Random rnd(test::RandomSeed());
std::vector<std::string> keys;
for (int i = 0; i < 100; ++i) {
keys.push_back(RandomString(&rnd, 10000));
}
// Each time we load one more key to the table. the table index block
// size is expected to be larger than last time's.
for (size_t i = 1; i < keys.size(); ++i) {
TableConstructor c(BytewiseComparator());
for (size_t j = 0; j < i; ++j) {
c.Add(keys[j], "val");
}
std::vector<std::string> ks;
KVMap kvmap;
Options options;
options.compression = kNoCompression;
options.block_restart_interval = 1;
c.Finish(options, &ks, &kvmap);
auto index_size =
c.table()->GetTableStats().index_size;
ASSERT_GT(index_size, last_index_size);
last_index_size = index_size;
}
}
TEST(TableTest, NumBlockStat) {
Random rnd(test::RandomSeed());
TableConstructor c(BytewiseComparator());
Options options;
options.compression = kNoCompression;
options.block_restart_interval = 1;
options.block_size = 1000;
for (int i = 0; i < 10; ++i) {
// the key/val are slightly smaller than block size, so that each block
// holds roughly one key/value pair.
c.Add(RandomString(&rnd, 900), "val");
}
std::vector<std::string> ks;
KVMap kvmap;
c.Finish(options, &ks, &kvmap);
ASSERT_EQ(
kvmap.size(),
c.table()->GetTableStats().num_data_blocks
);
}
TEST(TableTest, ApproximateOffsetOfPlain) {
TableConstructor c(BytewiseComparator());
c.Add("k01", "hello");
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册