Improve the "table stats"

Summary:
The primary motivation of the changes is to make it easier to figure out the inside of the tables.

* rename "table stats" to "table properties" since now we have more than "integers" to store in the property block.
* Add filter block size to the basic table properties.
* Whenever a table is built, we'll log the table properties (the sample output is in Test Plan).
* Make an api to expose deleted keys.

Test Plan:
Passed all existing test. and the sample output of table stats:

    ==================================================================
        Basic Properties
    ------------------------------------------------------------------
                  # data blocks: 1
                      # entries: 1

                   raw key size: 9
           raw average key size: 9
                 raw value size: 9
         raw average value size: 0

                data block size: 25
               index block size: 27
              filter block size: 18
         (estimated) table size: 70

                  filter policy: rocksdb.BuiltinBloomFilter
    ==================================================================
        User collected properties: InternalKeyPropertiesCollector
    ------------------------------------------------------------------
                    kDeletedKeys: 1
    ==================================================================

Reviewers: dhruba, haobo

Reviewed By: dhruba

CC: leveldb

Differential Revision: https://reviews.facebook.net/D14187
main
kailiu 11 years ago
parent f045871f1c
commit 1415f8820d
  1. 12
      Makefile
  2. 18
      db/db_impl.cc
  3. 8
      db/simple_table_db_test.cc
  4. 164
      db/table_properties_collector.cc
  5. 76
      db/table_properties_collector.h
  6. 58
      db/table_properties_collector_test.cc
  7. 55
      db/table_stats_collector.cc
  8. 58
      db/table_stats_collector.h
  9. 5
      include/rocksdb/options.h
  10. 4
      include/rocksdb/table.h
  11. 90
      include/rocksdb/table_properties.h
  12. 67
      include/rocksdb/table_stats.h
  13. 161
      table/block_based_table_builder.cc
  14. 100
      table/block_based_table_reader.cc
  15. 17
      table/block_based_table_reader.h
  16. 66
      table/table_test.cc
  17. 4
      util/options.cc

@ -36,7 +36,7 @@ VALGRIND_VER := $(join $(VALGRIND_VER),valgrind)
VALGRIND_OPTS = --error-exitcode=$(VALGRIND_ERROR) --leak-check=full
TESTS = \
table_stats_collector_test \
table_properties_collector_test \
arena_test \
auto_roll_logger_test \
block_test \
@ -46,7 +46,6 @@ TESTS = \
coding_test \
corruption_test \
crc32c_test \
db_test \
dbformat_test \
env_test \
blob_store_test \
@ -63,12 +62,13 @@ TESTS = \
simple_table_db_test \
skiplist_test \
stringappend_test \
table_test \
ttl_test \
version_edit_test \
version_set_test \
write_batch_test\
deletefile_test
deletefile_test \
table_test \
db_test
TOOLS = \
sst_dump \
@ -201,8 +201,8 @@ signal_test: util/signal_test.o $(LIBOBJECTS)
arena_test: util/arena_test.o $(LIBOBJECTS) $(TESTHARNESS)
$(CXX) util/arena_test.o $(LIBOBJECTS) $(TESTHARNESS) $(EXEC_LDFLAGS) -o $@ $(LDFLAGS) $(COVERAGEFLAGS)
table_stats_collector_test: db/table_stats_collector_test.o $(LIBOBJECTS) $(TESTHARNESS)
$(CXX) db/table_stats_collector_test.o $(LIBOBJECTS) $(TESTHARNESS) $(EXEC_LDFLAGS) -o $@ $(LDFLAGS) $(COVERAGEFLAGS)
table_properties_collector_test: db/table_properties_collector_test.o $(LIBOBJECTS) $(TESTHARNESS)
$(CXX) db/table_properties_collector_test.o $(LIBOBJECTS) $(TESTHARNESS) $(EXEC_LDFLAGS) -o $@ $(LDFLAGS) $(COVERAGEFLAGS)
bloom_test: util/bloom_test.o $(LIBOBJECTS) $(TESTHARNESS)
$(CXX) util/bloom_test.o $(LIBOBJECTS) $(TESTHARNESS) $(EXEC_LDFLAGS) -o $@ $(LDFLAGS) $(COVERAGEFLAGS)

@ -30,7 +30,7 @@
#include "db/merge_helper.h"
#include "db/prefix_filter_iterator.h"
#include "db/table_cache.h"
#include "db/table_stats_collector.h"
#include "db/table_properties_collector.h"
#include "db/transaction_log_impl.h"
#include "db/version_set.h"
#include "db/write_batch_internal.h"
@ -182,20 +182,20 @@ Options SanitizeOptions(const std::string& dbname,
result.wal_dir = dbname;
}
// -- Sanitize the table stats collector
// All user defined stats collectors will be wrapped by
// UserKeyTableStatsCollector since for them they only have the knowledge of
// the user keys; internal keys are invisible to them.
auto& collectors = result.table_stats_collectors;
for (size_t i = 0; i < result.table_stats_collectors.size(); ++i) {
// -- Sanitize the table properties collector
// All user defined properties collectors will be wrapped by
// UserKeyTablePropertiesCollector since for them they only have the
// knowledge of the user keys; internal keys are invisible to them.
auto& collectors = result.table_properties_collectors;
for (size_t i = 0; i < result.table_properties_collectors.size(); ++i) {
assert(collectors[i]);
collectors[i] =
std::make_shared<UserKeyTableStatsCollector>(collectors[i]);
std::make_shared<UserKeyTablePropertiesCollector>(collectors[i]);
}
// Add collector to collect internal key statistics
collectors.push_back(
std::make_shared<InternalKeyStatsCollector>()
std::make_shared<InternalKeyPropertiesCollector>()
);
if (!result.flush_block_policy_factory) {

@ -95,7 +95,7 @@ public:
void SetupForCompaction() override;
TableStats& GetTableStats() override;
TableProperties& GetTableProperties() override;
~SimpleTableReader();
@ -171,7 +171,7 @@ struct SimpleTableReader::Rep {
unique_ptr<RandomAccessFile> file;
uint64_t index_start_offset;
int num_entries;
TableStats table_stats;
TableProperties table_properties;
const static int user_key_size = 16;
const static int offset_length = 8;
@ -214,8 +214,8 @@ Status SimpleTableReader::Open(const Options& options,
void SimpleTableReader::SetupForCompaction() {
}
TableStats& SimpleTableReader::GetTableStats() {
return rep_->table_stats;
TableProperties& SimpleTableReader::GetTableProperties() {
return rep_->table_properties;
}
bool SimpleTableReader::PrefixMayMatch(const Slice& internal_prefix) {

@ -0,0 +1,164 @@
// Copyright (c) 2013, Facebook, Inc. All rights reserved.
// This source code is licensed under the BSD-style license found in the
// LICENSE file in the root directory of this source tree. An additional grant
// of patent rights can be found in the PATENTS file in the same directory.
#include "db/table_properties_collector.h"
#include "db/dbformat.h"
#include "util/coding.h"
namespace rocksdb {
namespace {
void AppendProperty(
std::string& props,
const std::string& key,
const std::string& value,
const std::string& prop_delim,
const std::string& kv_delim) {
props.append(key);
props.append(kv_delim);
props.append(value);
props.append(prop_delim);
}
template <class TValue>
void AppendProperty(
std::string& props,
const std::string& key,
const TValue& value,
const std::string& prop_delim,
const std::string& kv_delim) {
AppendProperty(
props, key, std::to_string(value), prop_delim, kv_delim
);
}
}
std::string TableProperties::ToString(
const std::string& prop_delim,
const std::string& kv_delim) const {
std::string result;
result.reserve(1024);
// Basic Info
AppendProperty(
result, "# data blocks", num_data_blocks, prop_delim, kv_delim
);
AppendProperty(result, "# entries", num_entries, prop_delim, kv_delim);
AppendProperty(result, "raw key size", raw_key_size, prop_delim, kv_delim);
AppendProperty(
result,
"raw average key size",
num_entries != 0 ? 1.0 * raw_key_size / num_entries : 0.0,
prop_delim,
kv_delim
);
AppendProperty(
result, "raw value size", raw_value_size, prop_delim, kv_delim
);
AppendProperty(
result,
"raw average value size",
num_entries != 0 ? 1.0 * raw_value_size / num_entries : 0.0,
prop_delim,
kv_delim
);
AppendProperty(result, "data block size", data_size, prop_delim, kv_delim);
AppendProperty(result, "index block size", index_size, prop_delim, kv_delim);
AppendProperty(
result, "filter block size", filter_size, prop_delim, kv_delim
);
AppendProperty(
result,
"(estimated) table size=",
data_size + index_size + filter_size,
prop_delim,
kv_delim
);
AppendProperty(
result,
"filter policy name",
filter_policy_name.empty() ? std::string("N/A") : filter_policy_name,
prop_delim,
kv_delim
);
return result;
}
Status InternalKeyPropertiesCollector::Add(
const Slice& key, const Slice& value) {
ParsedInternalKey ikey;
if (!ParseInternalKey(key, &ikey)) {
return Status::InvalidArgument("Invalid internal key");
}
if (ikey.type == ValueType::kTypeDeletion) {
++deleted_keys_;
}
return Status::OK();
}
Status InternalKeyPropertiesCollector::Finish(
TableProperties::UserCollectedProperties* properties) {
assert(properties);
assert(properties->find(
InternalKeyTablePropertiesNames::kDeletedKeys) == properties->end());
std::string val;
PutVarint64(&val, deleted_keys_);
properties->insert({ InternalKeyTablePropertiesNames::kDeletedKeys, val });
return Status::OK();
}
TableProperties::UserCollectedProperties
InternalKeyPropertiesCollector::GetReadableProperties() const {
return {
{ "kDeletedKeys", std::to_string(deleted_keys_) }
};
}
Status UserKeyTablePropertiesCollector::Add(
const Slice& key, const Slice& value) {
ParsedInternalKey ikey;
if (!ParseInternalKey(key, &ikey)) {
return Status::InvalidArgument("Invalid internal key");
}
return collector_->Add(ikey.user_key, value);
}
Status UserKeyTablePropertiesCollector::Finish(
TableProperties::UserCollectedProperties* properties) {
return collector_->Finish(properties);
}
TableProperties::UserCollectedProperties
UserKeyTablePropertiesCollector::GetReadableProperties() const {
return collector_->GetReadableProperties();
}
const std::string InternalKeyTablePropertiesNames::kDeletedKeys
= "rocksdb.deleted.keys";
uint64_t GetDeletedKeys(
const TableProperties::UserCollectedProperties& props) {
auto pos = props.find(InternalKeyTablePropertiesNames::kDeletedKeys);
if (pos == props.end()) {
return 0;
}
Slice raw = pos->second;
uint64_t val = 0;
return GetVarint64(&raw, &val) ? val : 0;
}
} // namespace rocksdb

@ -0,0 +1,76 @@
// Copyright (c) 2013, Facebook, Inc. All rights reserved.
// This source code is licensed under the BSD-style license found in the
// LICENSE file in the root directory of this source tree. An additional grant
// of patent rights can be found in the PATENTS file in the same directory.
//
// This file defines a collection of statistics collectors.
#pragma once
#include "rocksdb/table_properties.h"
#include <memory>
#include <string>
#include <vector>
namespace rocksdb {
struct InternalKeyTablePropertiesNames {
static const std::string kDeletedKeys;
};
// Collecting the statistics for internal keys. Visible only by internal
// rocksdb modules.
class InternalKeyPropertiesCollector : public TablePropertiesCollector {
public:
virtual Status Add(const Slice& key, const Slice& value) override;
virtual Status Finish(
TableProperties::UserCollectedProperties* properties) override;
virtual const char* Name() const override {
return "InternalKeyPropertiesCollector";
}
TableProperties::UserCollectedProperties
GetReadableProperties() const override;
private:
uint64_t deleted_keys_ = 0;
};
// When rocksdb creates a new table, it will encode all "user keys" into
// "internal keys", which contains meta information of a given entry.
//
// This class extracts user key from the encoded internal key when Add() is
// invoked.
class UserKeyTablePropertiesCollector : public TablePropertiesCollector {
public:
explicit UserKeyTablePropertiesCollector(
TablePropertiesCollector* collector) :
UserKeyTablePropertiesCollector(
std::shared_ptr<TablePropertiesCollector>(collector)
) {
}
explicit UserKeyTablePropertiesCollector(
std::shared_ptr<TablePropertiesCollector> collector) :
collector_(collector) {
}
virtual ~UserKeyTablePropertiesCollector() { }
virtual Status Add(const Slice& key, const Slice& value) override;
virtual Status Finish(
TableProperties::UserCollectedProperties* properties) override;
virtual const char* Name() const override { return collector_->Name(); }
TableProperties::UserCollectedProperties
GetReadableProperties() const override;
protected:
std::shared_ptr<TablePropertiesCollector> collector_;
};
} // namespace rocksdb

@ -9,8 +9,8 @@
#include "db/dbformat.h"
#include "db/db_impl.h"
#include "db/table_stats_collector.h"
#include "rocksdb/table_stats.h"
#include "db/table_properties_collector.h"
#include "rocksdb/table_properties.h"
#include "rocksdb/table.h"
#include "util/coding.h"
#include "util/testharness.h"
@ -18,7 +18,7 @@
namespace rocksdb {
class TableStatsTest {
class TablePropertiesTest {
private:
unique_ptr<TableReader> table_reader_;
};
@ -111,15 +111,15 @@ void OpenTable(
}
// Collects keys that starts with "A" in a table.
class RegularKeysStartWithA: public TableStatsCollector {
class RegularKeysStartWithA: public TablePropertiesCollector {
public:
const char* Name() const { return "RegularKeysStartWithA"; }
Status Finish(TableStats::UserCollectedStats* stats) {
Status Finish(TableProperties::UserCollectedProperties* properties) {
std::string encoded;
PutVarint32(&encoded, count_);
*stats = TableStats::UserCollectedStats {
{ "TableStatsTest", "Rocksdb" },
*properties = TableProperties::UserCollectedProperties {
{ "TablePropertiesTest", "Rocksdb" },
{ "Count", encoded }
};
return Status::OK();
@ -133,11 +133,17 @@ class RegularKeysStartWithA: public TableStatsCollector {
return Status::OK();
}
virtual TableProperties::UserCollectedProperties
GetReadableProperties() const {
return {};
}
private:
uint32_t count_ = 0;
};
TEST(TableStatsTest, CustomizedTableStatsCollector) {
TEST(TablePropertiesTest, CustomizedTablePropertiesCollector) {
Options options;
// make sure the entries will be inserted with order.
@ -151,17 +157,17 @@ TEST(TableStatsTest, CustomizedTableStatsCollector) {
{"Find", "val6"},
};
// Test stats collectors with internal keys or regular keys
// Test properties collectors with internal keys or regular keys
for (bool encode_as_internal : { true, false }) {
// -- Step 1: build table
auto collector = new RegularKeysStartWithA();
if (encode_as_internal) {
options.table_stats_collectors = {
std::make_shared<UserKeyTableStatsCollector>(collector)
options.table_properties_collectors = {
std::make_shared<UserKeyTablePropertiesCollector>(collector)
};
} else {
options.table_stats_collectors.resize(1);
options.table_stats_collectors[0].reset(collector);
options.table_properties_collectors.resize(1);
options.table_properties_collectors[0].reset(collector);
}
std::unique_ptr<TableBuilder> builder;
std::unique_ptr<FakeWritableFile> writable;
@ -180,18 +186,19 @@ TEST(TableStatsTest, CustomizedTableStatsCollector) {
// -- Step 2: Open table
std::unique_ptr<TableReader> table_reader;
OpenTable(options, writable->contents(), &table_reader);
const auto& stats = table_reader->GetTableStats().user_collected_stats;
const auto& properties =
table_reader->GetTableProperties().user_collected_properties;
ASSERT_EQ("Rocksdb", stats.at("TableStatsTest"));
ASSERT_EQ("Rocksdb", properties.at("TablePropertiesTest"));
uint32_t starts_with_A = 0;
Slice key(stats.at("Count"));
Slice key(properties.at("Count"));
ASSERT_TRUE(GetVarint32(&key, &starts_with_A));
ASSERT_EQ(3u, starts_with_A);
}
}
TEST(TableStatsTest, InternalKeyStatsCollector) {
TEST(TablePropertiesTest, InternalKeyPropertiesCollector) {
InternalKey keys[] = {
InternalKey("A", 0, ValueType::kTypeValue),
InternalKey("B", 0, ValueType::kTypeValue),
@ -207,10 +214,10 @@ TEST(TableStatsTest, InternalKeyStatsCollector) {
std::unique_ptr<FakeWritableFile> writable;
Options options;
if (sanitized) {
options.table_stats_collectors = {
options.table_properties_collectors = {
std::make_shared<RegularKeysStartWithA>()
};
// with sanitization, even regular stats collector will be able to
// with sanitization, even regular properties collector will be able to
// handle internal keys.
auto comparator = options.comparator;
// HACK: Set options.info_log to avoid writing log in
@ -224,8 +231,8 @@ TEST(TableStatsTest, InternalKeyStatsCollector) {
);
options.comparator = comparator;
} else {
options.table_stats_collectors = {
std::make_shared<InternalKeyStatsCollector>()
options.table_properties_collectors = {
std::make_shared<InternalKeyPropertiesCollector>()
};
}
@ -238,16 +245,15 @@ TEST(TableStatsTest, InternalKeyStatsCollector) {
std::unique_ptr<TableReader> table_reader;
OpenTable(options, writable->contents(), &table_reader);
const auto& stats = table_reader->GetTableStats().user_collected_stats;
const auto& properties =
table_reader->GetTableProperties().user_collected_properties;
uint64_t deleted = 0;
Slice key(stats.at(InternalKeyTableStatsNames::kDeletedKeys));
ASSERT_TRUE(GetVarint64(&key, &deleted));
uint64_t deleted = GetDeletedKeys(properties);
ASSERT_EQ(4u, deleted);
if (sanitized) {
uint32_t starts_with_A = 0;
Slice key(stats.at("Count"));
Slice key(properties.at("Count"));
ASSERT_TRUE(GetVarint32(&key, &starts_with_A));
ASSERT_EQ(1u, starts_with_A);
}

@ -1,55 +0,0 @@
// Copyright (c) 2013, Facebook, Inc. All rights reserved.
// This source code is licensed under the BSD-style license found in the
// LICENSE file in the root directory of this source tree. An additional grant
// of patent rights can be found in the PATENTS file in the same directory.
#include "db/table_stats_collector.h"
#include "db/dbformat.h"
#include "util/coding.h"
namespace rocksdb {
Status InternalKeyStatsCollector::Add(const Slice& key, const Slice& value) {
ParsedInternalKey ikey;
if (!ParseInternalKey(key, &ikey)) {
return Status::InvalidArgument("Invalid internal key");
}
if (ikey.type == ValueType::kTypeDeletion) {
++deleted_keys_;
}
return Status::OK();
}
Status InternalKeyStatsCollector::Finish(
TableStats::UserCollectedStats* stats) {
assert(stats);
assert(stats->find(InternalKeyTableStatsNames::kDeletedKeys) == stats->end());
std::string val;
PutVarint64(&val, deleted_keys_);
stats->insert(std::make_pair(InternalKeyTableStatsNames::kDeletedKeys, val));
return Status::OK();
}
Status UserKeyTableStatsCollector::Add(const Slice& key, const Slice& value) {
ParsedInternalKey ikey;
if (!ParseInternalKey(key, &ikey)) {
return Status::InvalidArgument("Invalid internal key");
}
return collector_->Add(ikey.user_key, value);
}
Status UserKeyTableStatsCollector::Finish(
TableStats::UserCollectedStats* stats) {
return collector_->Finish(stats);
}
const std::string InternalKeyTableStatsNames::kDeletedKeys
= "rocksdb.deleted.keys";
} // namespace rocksdb

@ -1,58 +0,0 @@
// Copyright (c) 2013, Facebook, Inc. All rights reserved.
// This source code is licensed under the BSD-style license found in the
// LICENSE file in the root directory of this source tree. An additional grant
// of patent rights can be found in the PATENTS file in the same directory.
//
// This file defines a collection of statistics collectors.
#pragma once
#include "rocksdb/table_stats.h"
#include <memory>
#include <string>
#include <vector>
namespace rocksdb {
struct InternalKeyTableStatsNames {
static const std::string kDeletedKeys;
};
// Collecting the statistics for internal keys. Visible only by internal
// rocksdb modules.
class InternalKeyStatsCollector : public TableStatsCollector {
public:
virtual Status Add(const Slice& key, const Slice& value);
virtual Status Finish(TableStats::UserCollectedStats* stats);
virtual const char* Name() const { return "InternalKeyStatsCollector"; }
private:
uint64_t deleted_keys_ = 0;
};
// When rocksdb creates a new table, it will encode all "user keys" into
// "internal keys", which contains meta information of a given entry.
//
// This class extracts user key from the encoded internal key when Add() is
// invoked.
class UserKeyTableStatsCollector : public TableStatsCollector {
public:
explicit UserKeyTableStatsCollector(TableStatsCollector* collector):
UserKeyTableStatsCollector(
std::shared_ptr<TableStatsCollector>(collector)
) {
}
explicit UserKeyTableStatsCollector(
std::shared_ptr<TableStatsCollector> collector) : collector_(collector) {
}
virtual ~UserKeyTableStatsCollector() { }
virtual Status Add(const Slice& key, const Slice& value);
virtual Status Finish(TableStats::UserCollectedStats* stats);
virtual const char* Name() const { return collector_->Name(); }
protected:
std::shared_ptr<TableStatsCollector> collector_;
};
} // namespace rocksdb

@ -21,7 +21,7 @@
#include "rocksdb/slice_transform.h"
#include "rocksdb/slice_transform.h"
#include "rocksdb/statistics.h"
#include "rocksdb/table_stats.h"
#include "rocksdb/table_properties.h"
#include "rocksdb/universal_compaction.h"
namespace rocksdb {
@ -619,7 +619,8 @@ struct Options {
// the tables.
// Default: emtpy vector -- no user-defined statistics collection will be
// performed.
std::vector<std::shared_ptr<TableStatsCollector>> table_stats_collectors;
std::vector<std::shared_ptr<TablePropertiesCollector>>
table_properties_collectors;
// Allows thread-safe inplace updates. Requires Updates iff
// * key exists in current memtable

@ -12,7 +12,7 @@
#include <stdint.h>
#include "rocksdb/env.h"
#include "rocksdb/iterator.h"
#include "rocksdb/table_stats.h"
#include "rocksdb/table_properties.h"
#include "rocksdb/options.h"
namespace rocksdb {
@ -98,7 +98,7 @@ class TableReader {
// posix_fadvise
virtual void SetupForCompaction() = 0;
virtual TableStats& GetTableStats() = 0;
virtual TableProperties& GetTableProperties() = 0;
// Calls (*result_handler)(handle_context, ...) repeatedly, starting with
// the entry found after a call to Seek(key), until result_handler returns

@ -0,0 +1,90 @@
// Copyright (c) 2013, Facebook, Inc. All rights reserved.
// This source code is licensed under the BSD-style license found in the
// LICENSE file in the root directory of this source tree. An additional grant
// of patent rights can be found in the PATENTS file in the same directory.
#pragma once
#include <string>
#include <unordered_map>
#include "rocksdb/status.h"
namespace rocksdb {
// TableProperties contains a bunch of read-only properties of its associated
// table.
struct TableProperties {
public:
// Other than basic table properties, each table may also have the user
// collected properties.
// The value of the user-collected properties are encoded as raw bytes --
// users have to interprete these values by themselves.
typedef
std::unordered_map<std::string, std::string>
UserCollectedProperties;
// the total size of all data blocks.
uint64_t data_size = 0;
// the size of index block.
uint64_t index_size = 0;
// the size of filter block.
uint64_t filter_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;
// The name of the filter policy used in this table.
// If no filter policy is used, `filter_policy_name` will be an empty string.
std::string filter_policy_name;
// user collected properties
UserCollectedProperties user_collected_properties;
// convert this object to a human readable form
// @prop_delim: delimiter for each property.
std::string ToString(
const std::string& prop_delim = "; ",
const std::string& kv_delim = "=") const;
};
// `TablePropertiesCollector` provides the mechanism for users to collect
// their own interested properties. This class is essentially a collection
// of callback functions that will be invoked during table building.
class TablePropertiesCollector {
public:
virtual ~TablePropertiesCollector() { }
// Add() will be called when a new key/value pair is inserted into the table.
// @params key the original key that is inserted into the table.
// @params value the original value that is inserted into the table.
virtual Status Add(const Slice& key, const Slice& value) = 0;
// Finish() will be called when a table has already been built and is ready
// for writing the properties block.
// @params properties User will add their collected statistics to
// `properties`.
virtual Status Finish(
TableProperties::UserCollectedProperties* properties) = 0;
// The name of the properties collector can be used for debugging purpose.
virtual const char* Name() const = 0;
// Return the human-readable properties, where the key is property name and
// the value is the human-readable form of value.
virtual TableProperties::UserCollectedProperties
GetReadableProperties() const = 0;
};
// Extra properties
// Below is a list of non-basic properties that are collected by database
// itself. Especially some properties regarding to the internal keys (which
// is unknown to `table`).
extern uint64_t GetDeletedKeys(
const TableProperties::UserCollectedProperties& props);
} // namespace rocksdb

@ -1,67 +0,0 @@
// Copyright (c) 2013, Facebook, Inc. All rights reserved.
// This source code is licensed under the BSD-style license found in the
// LICENSE file in the root directory of this source tree. An additional grant
// of patent rights can be found in the PATENTS file in the same directory.
#pragma once
#include <string>
#include <unordered_map>
#include "rocksdb/status.h"
namespace rocksdb {
// TableStats contains a bunch of read-only stats of its associated
// table.
struct TableStats {
public:
// 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;
// The name of the filter policy used in this table.
// If no filter policy is used, `filter_policy_name` will be an empty string.
std::string filter_policy_name;
// user collected stats
UserCollectedStats user_collected_stats;
};
// `TableStatsCollector` provides the mechanism for users to collect their own
// interested stats. This class is essentially a collection of callback
// functions that will be invoked during table building.
class TableStatsCollector {
public:
virtual ~TableStatsCollector() { }
// Add() will be called when a new key/value pair is inserted into the table.
// @params key the original key that is inserted into the table.
// @params value the original value that is inserted into the table.
virtual Status Add(const Slice& key, const Slice& value) = 0;
// Finish() will be called when a table has already been built and is ready
// for writing the stats block.
// @params stats User will add their collected statistics to `stats`.
virtual Status Finish(TableStats::UserCollectedStats* stats) = 0;
// The name of the stats collector can be used for debugging purpose.
virtual const char* Name() const = 0;
};
} // namespace rocksdb

@ -10,7 +10,9 @@
#include "table/block_based_table_builder.h"
#include <assert.h>
#include <inttypes.h>
#include <map>
#include <stdio.h>
#include "rocksdb/flush_block_policy.h"
#include "rocksdb/cache.h"
@ -45,13 +47,13 @@ struct BytewiseLessThan {
// 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());
void AddProperties(BytewiseSortedMap& props, std::string name, uint64_t val) {
assert(props.find(name) == props.end());
std::string dst;
PutVarint64(&dst, val);
stats.insert(
props.insert(
std::make_pair(name, dst)
);
}
@ -63,12 +65,12 @@ static bool GoodCompressionRatio(size_t compressed_size, size_t raw_size) {
// Were we encounter any error occurs during user-defined statistics collection,
// we'll write the warning message to info log.
void LogStatsCollectionError(
void LogPropertiesCollectionError(
Logger* info_log, const std::string& method, const std::string& name) {
assert(method == "Add" || method == "Finish");
std::string msg =
"[Warning] encountered error when calling TableStatsCollector::" +
"[Warning] encountered error when calling TablePropertiesCollector::" +
method + "() with collector name: " + name;
Log(info_log, "%s", msg.c_str());
}
@ -84,12 +86,7 @@ struct BlockBasedTableBuilder::Rep {
BlockBuilder index_block;
std::string last_key;
CompressionType compression_type;
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;
TableProperties props;
bool closed = false; // Either Finish() or Abandon() has been called.
FilterBlockBuilder* filter_block;
@ -142,7 +139,7 @@ void BlockBasedTableBuilder::Add(const Slice& key, const Slice& value) {
Rep* r = rep_;
assert(!r->closed);
if (!ok()) return;
if (r->num_entries > 0) {
if (r->props.num_entries > 0) {
assert(r->options.comparator->Compare(key, Slice(r->last_key)) > 0);
}
@ -173,14 +170,14 @@ void BlockBasedTableBuilder::Add(const Slice& key, const Slice& value) {
r->last_key.assign(key.data(), key.size());
r->data_block.Add(key, value);
r->num_entries++;
r->raw_key_size += key.size();
r->raw_value_size += value.size();
r->props.num_entries++;
r->props.raw_key_size += key.size();
r->props.raw_value_size += value.size();
for (auto collector : r->options.table_stats_collectors) {
for (auto collector : r->options.table_properties_collectors) {
Status s = collector->Add(key, value);
if (!s.ok()) {
LogStatsCollectionError(
LogPropertiesCollectionError(
r->options.info_log.get(),
"Add", /* method */
collector->Name()
@ -201,8 +198,8 @@ void BlockBasedTableBuilder::Flush() {
if (r->filter_block != nullptr) {
r->filter_block->StartBlock(r->offset);
}
r->data_size = r->offset;
++r->num_data_blocks;
r->props.data_size = r->offset;
++r->props.num_data_blocks;
}
void BlockBasedTableBuilder::WriteBlock(BlockBuilder* block,
@ -358,11 +355,12 @@ Status BlockBasedTableBuilder::Finish() {
// Write filter block
if (ok() && r->filter_block != nullptr) {
WriteRawBlock(r->filter_block->Finish(), kNoCompression,
&filter_block_handle);
auto filter_contents = r->filter_block->Finish();
r->props.filter_size = filter_contents.size();
WriteRawBlock(filter_contents, kNoCompression, &filter_block_handle);
}
// To make sure stats block is able to keep the accurate size of index
// To make sure properties 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() && !empty_data_block) {
@ -375,7 +373,7 @@ Status BlockBasedTableBuilder::Finish() {
// Write meta blocks and metaindex block with the following order.
// 1. [meta block: filter]
// 2. [meta block: stats]
// 2. [meta block: properties]
// 3. [metaindex block]
if (ok()) {
// We use `BytewiseComparator` as the comparator for meta block.
@ -400,67 +398,91 @@ Status BlockBasedTableBuilder::Finish() {
);
}
// Write stats block.
// Write properties block.
{
BlockBuilder stats_block(
BlockBuilder properties_block(
r->options.block_restart_interval,
BytewiseComparator()
);
BytewiseSortedMap stats;
// Add basic stats
AddStats(stats, BlockBasedTableStatsNames::kRawKeySize, r->raw_key_size);
AddStats(stats, BlockBasedTableStatsNames::kRawValueSize,
r->raw_value_size);
AddStats(stats, BlockBasedTableStatsNames::kDataSize, r->data_size);
AddStats(
stats,
BlockBasedTableStatsNames::kIndexSize,
r->index_block.CurrentSizeEstimate() + kBlockTrailerSize
BytewiseSortedMap properties;
// Add basic properties
AddProperties(
properties,
BlockBasedTablePropertiesNames::kRawKeySize,
r->props.raw_key_size
);
AddProperties(
properties,
BlockBasedTablePropertiesNames::kRawValueSize,
r->props.raw_value_size
);
AddProperties(
properties,
BlockBasedTablePropertiesNames::kDataSize,
r->props.data_size
);
AddStats(stats, BlockBasedTableStatsNames::kNumEntries, r->num_entries);
AddStats(stats, BlockBasedTableStatsNames::kNumDataBlocks,
r->num_data_blocks);
r->props.index_size =
r->index_block.CurrentSizeEstimate() + kBlockTrailerSize;
AddProperties(
properties,
BlockBasedTablePropertiesNames::kIndexSize,
r->props.index_size
);
AddProperties(
properties,
BlockBasedTablePropertiesNames::kNumEntries,
r->props.num_entries
);
AddProperties(
properties,
BlockBasedTablePropertiesNames::kNumDataBlocks,
r->props.num_data_blocks);
if (r->filter_block != nullptr) {
stats.insert(std::make_pair(
BlockBasedTableStatsNames::kFilterPolicy,
properties.insert({
BlockBasedTablePropertiesNames::kFilterPolicy,
r->options.filter_policy->Name()
));
});
}
AddProperties(
properties,
BlockBasedTablePropertiesNames::kFilterSize,
r->props.filter_size
);
for (auto collector : r->options.table_stats_collectors) {
TableStats::UserCollectedStats user_collected_stats;
for (auto collector : r->options.table_properties_collectors) {
TableProperties::UserCollectedProperties user_collected_properties;
Status s =
collector->Finish(&user_collected_stats);
collector->Finish(&user_collected_properties);
if (!s.ok()) {
LogStatsCollectionError(
LogPropertiesCollectionError(
r->options.info_log.get(),
"Finish", /* method */
collector->Name()
);
} else {
stats.insert(
user_collected_stats.begin(),
user_collected_stats.end()
properties.insert(
user_collected_properties.begin(),
user_collected_properties.end()
);
}
}
for (const auto& stat : stats) {
stats_block.Add(stat.first, stat.second);
for (const auto& stat : properties) {
properties_block.Add(stat.first, stat.second);
}
BlockHandle stats_block_handle;
WriteBlock(&stats_block, &stats_block_handle);
BlockHandle properties_block_handle;
WriteBlock(&properties_block, &properties_block_handle);
std::string handle_encoding;
stats_block_handle.EncodeTo(&handle_encoding);
properties_block_handle.EncodeTo(&handle_encoding);
meta_block_handles.insert(
std::make_pair(BlockBasedTable::kStatsBlock, handle_encoding)
{ BlockBasedTable::kPropertiesBlock, handle_encoding }
);
} // end of stats block writing
} // end of properties block writing
for (const auto& metablock : meta_block_handles) {
meta_index_block.Add(metablock.first, metablock.second);
@ -486,6 +508,31 @@ Status BlockBasedTableBuilder::Finish() {
r->offset += footer_encoding.size();
}
}
// Print out the table stats
if (ok()) {
// user collected properties
std::string user_collected;
user_collected.reserve(1024);
for (auto collector : r->options.table_properties_collectors) {
for (const auto& prop : collector->GetReadableProperties()) {
user_collected.append(prop.first);
user_collected.append("=");
user_collected.append(prop.second);
user_collected.append("; ");
}
}
Log(
r->options.info_log,
"Table was constructed:\n"
" basic properties: %s\n"
" user collected properties: %s",
r->props.ToString().c_str(),
user_collected.c_str()
);
}
return r->status;
}
@ -496,7 +543,7 @@ void BlockBasedTableBuilder::Abandon() {
}
uint64_t BlockBasedTableBuilder::NumEntries() const {
return rep_->num_entries;
return rep_->props.num_entries;
}
uint64_t BlockBasedTableBuilder::FileSize() const {

@ -58,7 +58,7 @@ struct BlockBasedTable::Rep {
unique_ptr<Block> index_block;
unique_ptr<FilterBlockReader> filter;
TableStats table_stats;
TableProperties table_properties;
};
BlockBasedTable::~BlockBasedTable() {
@ -261,18 +261,18 @@ Status BlockBasedTable::Open(const Options& options,
std::unique_ptr<Iterator> meta_iter;
s = ReadMetaBlock(rep, &meta, &meta_iter);
// Read the stats
meta_iter->Seek(kStatsBlock);
if (meta_iter->Valid() && meta_iter->key() == Slice(kStatsBlock)) {
// Read the properties
meta_iter->Seek(kPropertiesBlock);
if (meta_iter->Valid() && meta_iter->key() == Slice(kPropertiesBlock)) {
s = meta_iter->status();
if (s.ok()) {
s = ReadStats(meta_iter->value(), rep, &rep->table_stats);
s = ReadProperties(meta_iter->value(), rep, &rep->table_properties);
}
if (!s.ok()) {
auto err_msg =
"[Warning] Encountered error while reading data from stats block " +
s.ToString();
"[Warning] Encountered error while reading data from properties "
"block " + s.ToString();
Log(rep->options.info_log, "%s", err_msg.c_str());
}
}
@ -349,8 +349,8 @@ void BlockBasedTable::SetupForCompaction() {
compaction_optimized_ = true;
}
TableStats& BlockBasedTable::GetTableStats() {
return rep_->table_stats;
TableProperties& BlockBasedTable::GetTableProperties() {
return rep_->table_properties;
}
// Load the meta-block from the file. On success, return the loaded meta block
@ -372,8 +372,8 @@ Status BlockBasedTable::ReadMetaBlock(
if (!s.ok()) {
auto err_msg =
"[Warning] Encountered error while reading data from stats block " +
s.ToString();
"[Warning] Encountered error while reading data from properties"
"block " + s.ToString();
Log(rep->options.info_log, "%s", err_msg.c_str());
}
if (!s.ok()) {
@ -414,14 +414,14 @@ FilterBlockReader* BlockBasedTable::ReadFilter (
rep->options, block.data, block.heap_allocated);
}
Status BlockBasedTable::ReadStats(
const Slice& handle_value, Rep* rep, TableStats* table_stats) {
assert(table_stats);
Status BlockBasedTable::ReadProperties(
const Slice& handle_value, Rep* rep, TableProperties* table_properties) {
assert(table_properties);
Slice v = handle_value;
BlockHandle handle;
if (!handle.DecodeFrom(&v).ok()) {
return Status::InvalidArgument("Failed to decode stats block handle");
return Status::InvalidArgument("Failed to decode properties block handle");
}
BlockContents block_contents;
@ -438,20 +438,27 @@ Status BlockBasedTable::ReadStats(
return s;
}
Block stats_block(block_contents);
Block properties_block(block_contents);
std::unique_ptr<Iterator> iter(
stats_block.NewIterator(BytewiseComparator())
properties_block.NewIterator(BytewiseComparator())
);
// All pre-defined stats of type uint64_t
std::unordered_map<std::string, uint64_t*> predefined_uint64_stats = {
{ BlockBasedTableStatsNames::kDataSize, &table_stats->data_size },
{ BlockBasedTableStatsNames::kIndexSize, &table_stats->index_size },
{ BlockBasedTableStatsNames::kRawKeySize, &table_stats->raw_key_size },
{ BlockBasedTableStatsNames::kRawValueSize, &table_stats->raw_value_size },
{ BlockBasedTableStatsNames::kNumDataBlocks,
&table_stats->num_data_blocks },
{ BlockBasedTableStatsNames::kNumEntries, &table_stats->num_entries },
// All pre-defined properties of type uint64_t
std::unordered_map<std::string, uint64_t*> predefined_uint64_properties = {
{ BlockBasedTablePropertiesNames::kDataSize,
&table_properties->data_size },
{ BlockBasedTablePropertiesNames::kIndexSize,
&table_properties->index_size },
{ BlockBasedTablePropertiesNames::kFilterSize,
&table_properties->filter_size },
{ BlockBasedTablePropertiesNames::kRawKeySize,
&table_properties->raw_key_size },
{ BlockBasedTablePropertiesNames::kRawValueSize,
&table_properties->raw_value_size },
{ BlockBasedTablePropertiesNames::kNumDataBlocks,
&table_properties->num_data_blocks },
{ BlockBasedTablePropertiesNames::kNumEntries,
&table_properties->num_entries },
};
std::string last_key;
@ -462,7 +469,7 @@ Status BlockBasedTable::ReadStats(
}
auto key = iter->key().ToString();
// stats block is strictly sorted with no duplicate key.
// properties block is strictly sorted with no duplicate key.
assert(
last_key.empty() ||
BytewiseComparator()->Compare(key, last_key) > 0
@ -470,25 +477,25 @@ Status BlockBasedTable::ReadStats(
last_key = key;
auto raw_val = iter->value();
auto pos = predefined_uint64_stats.find(key);
auto pos = predefined_uint64_properties.find(key);
if (pos != predefined_uint64_stats.end()) {
// handle predefined rocksdb stats
if (pos != predefined_uint64_properties.end()) {
// handle predefined rocksdb properties
uint64_t val;
if (!GetVarint64(&raw_val, &val)) {
// skip malformed value
auto error_msg =
"[Warning] detect malformed value in stats meta-block:"
"[Warning] detect malformed value in properties meta-block:"
"\tkey: " + key + "\tval: " + raw_val.ToString();
Log(rep->options.info_log, "%s", error_msg.c_str());
continue;
}
*(pos->second) = val;
} else if (key == BlockBasedTableStatsNames::kFilterPolicy) {
table_stats->filter_policy_name = raw_val.ToString();
} else if (key == BlockBasedTablePropertiesNames::kFilterPolicy) {
table_properties->filter_policy_name = raw_val.ToString();
} else {
// handle user-collected
table_stats->user_collected_stats.insert(
table_properties->user_collected_properties.insert(
std::make_pair(key, raw_val.ToString())
);
}
@ -1066,20 +1073,25 @@ uint64_t BlockBasedTable::ApproximateOffsetOf(const Slice& key) {
return result;
}
const std::string BlockBasedTable::kFilterBlockPrefix = "filter.";
const std::string BlockBasedTable::kStatsBlock = "rocksdb.stats";
const std::string BlockBasedTableStatsNames::kDataSize = "rocksdb.data.size";
const std::string BlockBasedTableStatsNames::kIndexSize = "rocksdb.index.size";
const std::string BlockBasedTableStatsNames::kRawKeySize =
const std::string BlockBasedTable::kFilterBlockPrefix =
"filter.";
const std::string BlockBasedTable::kPropertiesBlock =
"rocksdb.properties";
const std::string BlockBasedTablePropertiesNames::kDataSize =
"rocksdb.data.size";
const std::string BlockBasedTablePropertiesNames::kIndexSize =
"rocksdb.index.size";
const std::string BlockBasedTablePropertiesNames::kFilterSize =
"rocksdb.filter.size";
const std::string BlockBasedTablePropertiesNames::kRawKeySize =
"rocksdb.raw.key.size";
const std::string BlockBasedTableStatsNames::kRawValueSize =
const std::string BlockBasedTablePropertiesNames::kRawValueSize =
"rocksdb.raw.value.size";
const std::string BlockBasedTableStatsNames::kNumDataBlocks =
const std::string BlockBasedTablePropertiesNames::kNumDataBlocks =
"rocksdb.num.data.blocks";
const std::string BlockBasedTableStatsNames::kNumEntries =
const std::string BlockBasedTablePropertiesNames::kNumEntries =
"rocksdb.num.entries";
const std::string BlockBasedTableStatsNames::kFilterPolicy =
const std::string BlockBasedTablePropertiesNames::kFilterPolicy =
"rocksdb.filter.policy";
} // namespace rocksdb

@ -14,7 +14,7 @@
#include "rocksdb/env.h"
#include "rocksdb/iterator.h"
#include "rocksdb/statistics.h"
#include "rocksdb/table_stats.h"
#include "rocksdb/table_properties.h"
#include "rocksdb/table.h"
#include "util/coding.h"
@ -38,7 +38,7 @@ using std::unique_ptr;
class BlockBasedTable : public TableReader {
public:
static const std::string kFilterBlockPrefix;
static const std::string kStatsBlock;
static const std::string kPropertiesBlock;
// Attempt to open the table that is stored in bytes [0..file_size)
// of "file", and read the metadata entries necessary to allow
@ -88,7 +88,7 @@ class BlockBasedTable : public TableReader {
// posix_fadvise
void SetupForCompaction() override;
TableStats& GetTableStats() override;
TableProperties& GetTableProperties() override;
~BlockBasedTable();
@ -142,7 +142,7 @@ class BlockBasedTable : public TableReader {
void ReadMeta(const Footer& footer);
void ReadFilter(const Slice& filter_handle_value);
static Status ReadStats(const Slice& handle_value, Rep* rep);
static Status ReadProperties(const Slice& handle_value, Rep* rep);
// Read the meta block from sst.
static Status ReadMetaBlock(
@ -156,9 +156,9 @@ class BlockBasedTable : public TableReader {
Rep* rep,
size_t* filter_size = nullptr);
// Read the table stats from stats block.
static Status ReadStats(
const Slice& handle_value, Rep* rep, TableStats* stats);
// Read the table properties from properties block.
static Status ReadProperties(
const Slice& handle_value, Rep* rep, TableProperties* properties);
static void SetupCacheKeyPrefix(Rep* rep);
@ -181,9 +181,10 @@ class BlockBasedTable : public TableReader {
void operator=(const TableReader&) = delete;
};
struct BlockBasedTableStatsNames {
struct BlockBasedTablePropertiesNames {
static const std::string kDataSize;
static const std::string kIndexSize;
static const std::string kFilterSize;
static const std::string kRawKeySize;
static const std::string kRawValueSize;
static const std::string kNumDataBlocks;

@ -799,7 +799,7 @@ 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) {
TEST(TableTest, BasicTableProperties) {
BlockBasedTableConstructor c(BytewiseComparator());
c.Add("a1", "val1");
@ -820,16 +820,16 @@ TEST(TableTest, BasicTableStats) {
c.Finish(options, &keys, &kvmap);
auto& stats = c.table_reader()->GetTableStats();
ASSERT_EQ(kvmap.size(), stats.num_entries);
auto& props = c.table_reader()->GetTableProperties();
ASSERT_EQ(kvmap.size(), props.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);
ASSERT_EQ("", stats.filter_policy_name); // no filter policy is used
ASSERT_EQ(raw_key_size, props.raw_key_size);
ASSERT_EQ(raw_value_size, props.raw_value_size);
ASSERT_EQ(1ul, props.num_data_blocks);
ASSERT_EQ("", props.filter_policy_name); // no filter policy is used
// Verify data size.
BlockBuilder block_builder(options);
@ -839,11 +839,11 @@ TEST(TableTest, BasicTableStats) {
Slice content = block_builder.Finish();
ASSERT_EQ(
content.size() + kBlockTrailerSize,
stats.data_size
props.data_size
);
}
TEST(TableTest, FilterPolicyNameStats) {
TEST(TableTest, FilterPolicyNameProperties) {
BlockBasedTableConstructor c(BytewiseComparator());
c.Add("a1", "val1");
std::vector<std::string> keys;
@ -855,8 +855,8 @@ TEST(TableTest, FilterPolicyNameStats) {
options.filter_policy = filter_policy.get();
c.Finish(options, &keys, &kvmap);
auto& stats = c.table_reader()->GetTableStats();
ASSERT_EQ("rocksdb.BuiltinBloomFilter", stats.filter_policy_name);
auto& props = c.table_reader()->GetTableProperties();
ASSERT_EQ("rocksdb.BuiltinBloomFilter", props.filter_policy_name);
}
static std::string RandomString(Random* rnd, int len) {
@ -897,7 +897,7 @@ TEST(TableTest, IndexSizeStat) {
c.Finish(options, &ks, &kvmap);
auto index_size =
c.table_reader()->GetTableStats().index_size;
c.table_reader()->GetTableProperties().index_size;
ASSERT_GT(index_size, last_index_size);
last_index_size = index_size;
}
@ -927,13 +927,13 @@ TEST(TableTest, NumBlockStat) {
c.Finish(options, &ks, &kvmap);
ASSERT_EQ(
kvmap.size(),
c.table_reader()->GetTableStats().num_data_blocks
c.table_reader()->GetTableProperties().num_data_blocks
);
}
class BlockCacheStats {
class BlockCacheProperties {
public:
explicit BlockCacheStats(std::shared_ptr<Statistics> statistics) {
explicit BlockCacheProperties(std::shared_ptr<Statistics> statistics) {
block_cache_miss =
statistics.get()->getTickerCount(BLOCK_CACHE_MISS);
block_cache_hit =
@ -948,7 +948,7 @@ class BlockCacheStats {
statistics.get()->getTickerCount(BLOCK_CACHE_DATA_HIT);
}
// Check if the fetched stats matches the expected ones.
// Check if the fetched props matches the expected ones.
void AssertEqual(
long index_block_cache_miss,
long index_block_cache_hit,
@ -996,9 +996,9 @@ TEST(TableTest, BlockCacheTest) {
// At first, no block will be accessed.
{
BlockCacheStats stats(options.statistics);
BlockCacheProperties props(options.statistics);
// index will be added to block cache.
stats.AssertEqual(
props.AssertEqual(
1, // index block miss
0,
0,
@ -1009,11 +1009,11 @@ TEST(TableTest, BlockCacheTest) {
// Only index block will be accessed
{
iter.reset(c.NewIterator());
BlockCacheStats stats(options.statistics);
BlockCacheProperties props(options.statistics);
// NOTE: to help better highlight the "detla" of each ticker, I use
// <last_value> + <added_value> to indicate the increment of changed
// value; other numbers remain the same.
stats.AssertEqual(
props.AssertEqual(
1,
0 + 1, // index block hit
0,
@ -1024,8 +1024,8 @@ TEST(TableTest, BlockCacheTest) {
// Only data block will be accessed
{
iter->SeekToFirst();
BlockCacheStats stats(options.statistics);
stats.AssertEqual(
BlockCacheProperties props(options.statistics);
props.AssertEqual(
1,
1,
0 + 1, // data block miss
@ -1037,8 +1037,8 @@ TEST(TableTest, BlockCacheTest) {
{
iter.reset(c.NewIterator());
iter->SeekToFirst();
BlockCacheStats stats(options.statistics);
stats.AssertEqual(
BlockCacheProperties props(options.statistics);
props.AssertEqual(
1,
1 + 1, // index block hit
1,
@ -1050,16 +1050,16 @@ TEST(TableTest, BlockCacheTest) {
// -- PART 2: Open without block cache
options.block_cache.reset();
options.statistics = CreateDBStatistics(); // reset the stats
options.statistics = CreateDBStatistics(); // reset the props
c.Reopen(options);
{
iter.reset(c.NewIterator());
iter->SeekToFirst();
ASSERT_EQ("key", iter->key().ToString());
BlockCacheStats stats(options.statistics);
BlockCacheProperties props(options.statistics);
// Nothing is affected at all
stats.AssertEqual(0, 0, 0, 0);
props.AssertEqual(0, 0, 0, 0);
}
// -- PART 3: Open with very small block cache
@ -1068,8 +1068,8 @@ TEST(TableTest, BlockCacheTest) {
options.block_cache = NewLRUCache(1);
c.Reopen(options);
{
BlockCacheStats stats(options.statistics);
stats.AssertEqual(
BlockCacheProperties props(options.statistics);
props.AssertEqual(
1, // index block miss
0,
0,
@ -1083,8 +1083,8 @@ TEST(TableTest, BlockCacheTest) {
// It first cache index block then data block. But since the cache size
// is only 1, index block will be purged after data block is inserted.
iter.reset(c.NewIterator());
BlockCacheStats stats(options.statistics);
stats.AssertEqual(
BlockCacheProperties props(options.statistics);
props.AssertEqual(
1 + 1, // index block miss
0,
0, // data block miss
@ -1096,8 +1096,8 @@ TEST(TableTest, BlockCacheTest) {
// SeekToFirst() accesses data block. With similar reason, we expect data
// block's cache miss.
iter->SeekToFirst();
BlockCacheStats stats(options.statistics);
stats.AssertEqual(
BlockCacheProperties props(options.statistics);
props.AssertEqual(
2,
0,
0 + 1, // data block miss

@ -280,11 +280,11 @@ Options::Dump(Logger* log) const
"max_size_amplification_percent: %u",
compaction_options_universal.max_size_amplification_percent);
std::string collector_names;
for (auto collector : table_stats_collectors) {
for (auto collector : table_properties_collectors) {
collector_names.append(collector->Name());
collector_names.append("; ");
}
Log(log, " Options.table_stats_collectors: %s",
Log(log, " Options.table_properties_collectors: %s",
collector_names.c_str());
Log(log, " Options.inplace_update_support: %d",
inplace_update_support);

Loading…
Cancel
Save