Add more LSM info to FilterBuildingContext (#8246)

Summary:
Add `num_levels`, `is_bottommost`, and table file creation
`reason` to `FilterBuildingContext`, in anticipation of more powerful
Bloom-like filter support.

To support this, added `is_bottommost` and `reason` to
`TableBuilderOptions`, which allowed removing `reason` parameter from
`rocksdb::BuildTable`.

I attempted to remove `skip_filters` from `TableBuilderOptions`, because
filter construction decisions should arise from options, not one-off
parameters. I could not completely remove it because the public API for
SstFileWriter takes a `skip_filters` parameter, and translating this
into an option change would mean awkwardly replacing the table_factory
if it is BlockBasedTableFactory with new filter_policy=nullptr option.
I marked this public skip_filters option as deprecated because of this
oddity. (skip_filters on the read side probably makes sense.)

At least `skip_filters` is now largely hidden for users of
`TableBuilderOptions` and is no longer used for implementing the
optimize_filters_for_hits option. Bringing the logic for that option
closer to handling of FilterBuildingContext makes it more obvious that
hese two are using the same notion of "bottommost." (Planned:
configuration options for Bloom-like filters that generalize
`optimize_filters_for_hits`)

Recommended follow-up: Try to get away from "bottommost level" naming of
things, which is inaccurate (see
VersionStorageInfo::RangeMightExistAfterSortedRun), and move to
"bottommost run" or just "bottommost."

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

Test Plan:
extended an existing unit test to exercise and check various
filter building contexts. Also, existing tests for
optimize_filters_for_hits validate some of the "bottommost" handling,
which is now closely connected to FilterBuildingContext::is_bottommost
through TableBuilderOptions::is_bottommost

Reviewed By: mrambacher

Differential Revision: D28099346

Pulled By: pdillinger

fbshipit-source-id: 2c1072e29c24d4ac404c761a7b7663292372600a
main
Peter Dillinger 3 years ago committed by Facebook GitHub Bot
parent 85becd94c1
commit d2ca04e3ed
  1. 2
      HISTORY.md
  2. 16
      db/builder.cc
  3. 4
      db/builder.h
  4. 11
      db/compaction/compaction_job.cc
  5. 49
      db/db_bloom_filter_test.cc
  6. 12
      db/db_impl/db_impl_open.cc
  7. 13
      db/flush_job.cc
  8. 12
      db/repair.cc
  9. 4
      db/table_properties_collector_test.cc
  10. 1
      db/version_set_test.cc
  11. 28
      include/rocksdb/filter_policy.h
  12. 7
      include/rocksdb/listener.h
  13. 3
      include/rocksdb/sst_file_writer.h
  14. 7
      include/rocksdb/types.h
  15. 29
      table/block_based/block_based_table_builder.cc
  16. 4
      table/block_based/block_based_table_reader_test.cc
  17. 2
      table/block_based/data_block_hash_index_test.cc
  18. 4
      table/block_fetcher_test.cc
  19. 2
      table/sst_file_dumper.cc
  20. 11
      table/sst_file_writer.cc
  21. 34
      table/table_builder.h
  22. 3
      table/table_reader_bench.cc
  23. 45
      table/table_test.cc
  24. 1
      tools/sst_dump_test.cc

@ -11,12 +11,14 @@
### New Features
* Add new option allow_stall passed during instance creation of WriteBufferManager. When allow_stall is set, WriteBufferManager will stall all writers shared across multiple DBs and columns if memory usage goes beyond specified WriteBufferManager::buffer_size (soft limit). Stall will be cleared when memory is freed after flush and memory usage goes down below buffer_size.
* Added more fields to FilterBuildingContext with LSM details, for custom filter policies that vary behavior based on where they are in the LSM-tree.
### Performace Improvements
* BlockPrefetcher is used by iterators to prefetch data if they anticipate more data to be used in future. It is enabled implicitly by rocksdb. Added change to take in account read pattern if reads are sequential. This would disable prefetching for random reads in MultiGet and iterators as readahead_size is increased exponential doing large prefetches.
### Public API change
* Removed a parameter from TableFactory::NewTableBuilder, which should not be called by user code because TableBuilder is not a public API.
* The `skip_filters` parameter to SstFileWriter is now considered deprecated. Use `BlockBasedTableOptions::filter_policy` to control generation of filters.
## 6.20.0 (04/16/2021)
### Behavior Changes

@ -64,9 +64,9 @@ Status BuildTable(
std::vector<SequenceNumber> snapshots,
SequenceNumber earliest_write_conflict_snapshot,
SnapshotChecker* snapshot_checker, bool paranoid_file_checks,
InternalStats* internal_stats, TableFileCreationReason reason,
IOStatus* io_status, const std::shared_ptr<IOTracer>& io_tracer,
EventLogger* event_logger, int job_id, const Env::IOPriority io_priority,
InternalStats* internal_stats, IOStatus* io_status,
const std::shared_ptr<IOTracer>& io_tracer, EventLogger* event_logger,
int job_id, const Env::IOPriority io_priority,
TableProperties* table_properties, Env::WriteLifeTimeHint write_hint,
const std::string* full_history_ts_low,
BlobFileCompletionCallback* blob_callback) {
@ -100,7 +100,7 @@ Status BuildTable(
#ifndef ROCKSDB_LITE
EventHelpers::NotifyTableFileCreationStarted(ioptions.listeners, dbname,
tboptions.column_family_name,
fname, job_id, reason);
fname, job_id, tboptions.reason);
#endif // !ROCKSDB_LITE
Env* env = db_options.env;
assert(env);
@ -127,7 +127,7 @@ Status BuildTable(
EventHelpers::LogAndNotifyTableFileCreationFinished(
event_logger, ioptions.listeners, dbname,
tboptions.column_family_name, fname, job_id, meta->fd,
kInvalidBlobFileNumber, tp, reason, s, file_checksum,
kInvalidBlobFileNumber, tp, tboptions.reason, s, file_checksum,
file_checksum_func_name);
return s;
}
@ -280,7 +280,7 @@ Status BuildTable(
(internal_stats == nullptr) ? nullptr
: internal_stats->GetFileReadHist(0),
TableReaderCaller::kFlush, /*arena=*/nullptr,
/*skip_filter=*/false, tboptions.level,
/*skip_filter=*/false, tboptions.level_at_creation,
MaxFileSizeForL0MetaPin(mutable_cf_options),
/*smallest_compaction_key=*/nullptr,
/*largest_compaction_key*/ nullptr,
@ -333,8 +333,8 @@ Status BuildTable(
// Output to event logger and fire events.
EventHelpers::LogAndNotifyTableFileCreationFinished(
event_logger, ioptions.listeners, dbname, tboptions.column_family_name,
fname, job_id, meta->fd, meta->oldest_blob_file_number, tp, reason, s,
file_checksum, file_checksum_func_name);
fname, job_id, meta->fd, meta->oldest_blob_file_number, tp,
tboptions.reason, s, file_checksum, file_checksum_func_name);
return s;
}

@ -58,8 +58,8 @@ extern Status BuildTable(
std::vector<SequenceNumber> snapshots,
SequenceNumber earliest_write_conflict_snapshot,
SnapshotChecker* snapshot_checker, bool paranoid_file_checks,
InternalStats* internal_stats, TableFileCreationReason reason,
IOStatus* io_status, const std::shared_ptr<IOTracer>& io_tracer,
InternalStats* internal_stats, IOStatus* io_status,
const std::shared_ptr<IOTracer>& io_tracer,
EventLogger* event_logger = nullptr, int job_id = 0,
const Env::IOPriority io_priority = Env::IO_HIGH,
TableProperties* table_properties = nullptr,

@ -1755,18 +1755,13 @@ Status CompactionJob::OpenCompactionOutputFile(
db_options_.file_checksum_gen_factory.get(),
tmp_set.Contains(FileType::kTableFile)));
// If the Column family flag is to only optimize filters for hits,
// we can skip creating filters if this is the bottommost_level where
// data is going to be found
bool skip_filters =
cfd->ioptions()->optimize_filters_for_hits && bottommost_level_;
TableBuilderOptions tboptions(
*cfd->ioptions(), *(sub_compact->compaction->mutable_cf_options()),
cfd->internal_comparator(), cfd->int_tbl_prop_collector_factories(),
sub_compact->compaction->output_compression(),
sub_compact->compaction->output_compression_opts(), skip_filters,
cfd->GetID(), cfd->GetName(), sub_compact->compaction->output_level(),
sub_compact->compaction->output_compression_opts(), cfd->GetID(),
cfd->GetName(), sub_compact->compaction->output_level(),
bottommost_level_, TableFileCreationReason::kCompaction,
oldest_ancester_time, 0 /* oldest_key_time */, current_time, db_id_,
db_session_id_, sub_compact->compaction->max_output_file_size());
sub_compact->builder.reset(

@ -774,6 +774,14 @@ class LevelAndStyleCustomFilterPolicy : public FilterPolicy {
const std::unique_ptr<const FilterPolicy> policy_otherwise_;
};
static std::map<TableFileCreationReason, std::string>
table_file_creation_reason_to_string{
{TableFileCreationReason::kCompaction, "kCompaction"},
{TableFileCreationReason::kFlush, "kFlush"},
{TableFileCreationReason::kMisc, "kMisc"},
{TableFileCreationReason::kRecovery, "kRecovery"},
};
class TestingContextCustomFilterPolicy
: public LevelAndStyleCustomFilterPolicy {
public:
@ -786,11 +794,17 @@ class TestingContextCustomFilterPolicy
const FilterBuildingContext& context) const override {
test_report_ += "cf=";
test_report_ += context.column_family_name;
test_report_ += ",cs=";
test_report_ += ",s=";
test_report_ +=
OptionsHelper::compaction_style_to_string[context.compaction_style];
test_report_ += ",lv=";
test_report_ += std::to_string(context.level_at_creation);
test_report_ += ",n=";
test_report_ += ToString(context.num_levels);
test_report_ += ",l=";
test_report_ += ToString(context.level_at_creation);
test_report_ += ",b=";
test_report_ += ToString(int{context.is_bottommost});
test_report_ += ",r=";
test_report_ += table_file_creation_reason_to_string[context.reason];
test_report_ += "\n";
return LevelAndStyleCustomFilterPolicy::GetBuilderWithContext(context);
@ -808,15 +822,16 @@ class TestingContextCustomFilterPolicy
} // namespace
TEST_F(DBBloomFilterTest, ContextCustomFilterPolicy) {
auto policy = std::make_shared<TestingContextCustomFilterPolicy>(15, 8, 5);
Options options;
for (bool fifo : {true, false}) {
Options options = CurrentOptions();
options = CurrentOptions();
options.max_open_files = fifo ? -1 : options.max_open_files;
options.statistics = ROCKSDB_NAMESPACE::CreateDBStatistics();
options.compaction_style =
fifo ? kCompactionStyleFIFO : kCompactionStyleLevel;
BlockBasedTableOptions table_options;
auto policy = std::make_shared<TestingContextCustomFilterPolicy>(15, 8, 5);
table_options.filter_policy = policy;
table_options.format_version = 5;
options.table_factory.reset(NewBlockBasedTableFactory(table_options));
@ -832,16 +847,16 @@ TEST_F(DBBloomFilterTest, ContextCustomFilterPolicy) {
ASSERT_OK(Put(1, Key(maxKey + 55555), Key(maxKey + 55555)));
Flush(1);
EXPECT_EQ(policy->DumpTestReport(),
fifo ? "cf=abe,cs=kCompactionStyleFIFO,lv=0\n"
: "cf=bob,cs=kCompactionStyleLevel,lv=0\n");
fifo ? "cf=abe,s=kCompactionStyleFIFO,n=1,l=0,b=0,r=kFlush\n"
: "cf=bob,s=kCompactionStyleLevel,n=7,l=0,b=0,r=kFlush\n");
for (int i = maxKey / 2; i < maxKey; i++) {
ASSERT_OK(Put(1, Key(i), Key(i)));
}
Flush(1);
EXPECT_EQ(policy->DumpTestReport(),
fifo ? "cf=abe,cs=kCompactionStyleFIFO,lv=0\n"
: "cf=bob,cs=kCompactionStyleLevel,lv=0\n");
fifo ? "cf=abe,s=kCompactionStyleFIFO,n=1,l=0,b=0,r=kFlush\n"
: "cf=bob,s=kCompactionStyleLevel,n=7,l=0,b=0,r=kFlush\n");
// Check that they can be found
for (int i = 0; i < maxKey; i++) {
@ -869,7 +884,7 @@ TEST_F(DBBloomFilterTest, ContextCustomFilterPolicy) {
ASSERT_OK(db_->CompactRange(CompactRangeOptions(), handles_[1], nullptr,
nullptr));
EXPECT_EQ(policy->DumpTestReport(),
"cf=bob,cs=kCompactionStyleLevel,lv=1\n");
"cf=bob,s=kCompactionStyleLevel,n=7,l=1,b=1,r=kCompaction\n");
// Check that we now have one filter, about 9.2% FP rate (5 bits per key)
for (int i = 0; i < maxKey; i++) {
@ -881,6 +896,20 @@ TEST_F(DBBloomFilterTest, ContextCustomFilterPolicy) {
EXPECT_GE(useful_count, maxKey * 0.90);
EXPECT_LE(useful_count, maxKey * 0.91);
}
} else {
#ifndef ROCKSDB_LITE
// Also try external SST file
{
std::string file_path = dbname_ + "/external.sst";
SstFileWriter sst_file_writer(EnvOptions(), options, handles_[1]);
ASSERT_OK(sst_file_writer.Open(file_path));
ASSERT_OK(sst_file_writer.Put("key", "value"));
ASSERT_OK(sst_file_writer.Finish());
}
// Note: kCompactionStyleLevel is default, ignored if num_levels == -1
EXPECT_EQ(policy->DumpTestReport(),
"cf=abe,s=kCompactionStyleLevel,n=-1,l=-1,b=0,r=kMisc\n");
#endif
}
// Destroy

@ -1397,8 +1397,9 @@ Status DBImpl::WriteLevel0TableForRecovery(int job_id, ColumnFamilyData* cfd,
*cfd->ioptions(), mutable_cf_options, cfd->internal_comparator(),
cfd->int_tbl_prop_collector_factories(),
GetCompressionFlush(*cfd->ioptions(), mutable_cf_options),
mutable_cf_options.compression_opts, false /* skip_filters */,
cfd->GetID(), cfd->GetName(), 0 /* level */, current_time,
mutable_cf_options.compression_opts, cfd->GetID(), cfd->GetName(),
0 /* level */, false /* is_bottommost */,
TableFileCreationReason::kRecovery, current_time,
0 /* oldest_key_time */, 0 /* file_creation_time */, db_id_,
db_session_id_, 0 /* target_file_size */);
s = BuildTable(
@ -1406,10 +1407,9 @@ Status DBImpl::WriteLevel0TableForRecovery(int job_id, ColumnFamilyData* cfd,
file_options_for_compaction_, cfd->table_cache(), iter.get(),
std::move(range_del_iters), &meta, &blob_file_additions,
snapshot_seqs, earliest_write_conflict_snapshot, snapshot_checker,
paranoid_file_checks, cfd->internal_stats(),
TableFileCreationReason::kRecovery, &io_s, io_tracer_, &event_logger_,
job_id, Env::IO_HIGH, nullptr /* table_properties */, write_hint,
nullptr /*full_history_ts_low*/, &blob_callback_);
paranoid_file_checks, cfd->internal_stats(), &io_s, io_tracer_,
&event_logger_, job_id, Env::IO_HIGH, nullptr /* table_properties */,
write_hint, nullptr /*full_history_ts_low*/, &blob_callback_);
LogFlush(immutable_db_options_.info_log);
ROCKS_LOG_DEBUG(immutable_db_options_.info_log,
"[%s] [WriteLevel0TableForRecovery]"

@ -409,19 +409,18 @@ Status FlushJob::WriteLevel0Table() {
TableBuilderOptions tboptions(
*cfd_->ioptions(), mutable_cf_options_, cfd_->internal_comparator(),
cfd_->int_tbl_prop_collector_factories(), output_compression_,
mutable_cf_options_.compression_opts, false /* skip_filters */,
cfd_->GetID(), cfd_->GetName(), 0 /* level */, creation_time,
oldest_key_time, current_time, db_id_, db_session_id_,
0 /* target_file_size */);
mutable_cf_options_.compression_opts, cfd_->GetID(), cfd_->GetName(),
0 /* level */, false /* is_bottommost */,
TableFileCreationReason::kFlush, creation_time, oldest_key_time,
current_time, db_id_, db_session_id_, 0 /* target_file_size */);
s = BuildTable(
dbname_, versions_, db_options_, tboptions, file_options_,
cfd_->table_cache(), iter.get(), std::move(range_del_iters), &meta_,
&blob_file_additions, existing_snapshots_,
earliest_write_conflict_snapshot_, snapshot_checker_,
mutable_cf_options_.paranoid_file_checks, cfd_->internal_stats(),
TableFileCreationReason::kFlush, &io_s, io_tracer_, event_logger_,
job_context_->job_id, Env::IO_HIGH, &table_properties_, write_hint,
full_history_ts_low, blob_callback_);
&io_s, io_tracer_, event_logger_, job_context_->job_id, Env::IO_HIGH,
&table_properties_, write_hint, full_history_ts_low, blob_callback_);
if (!io_s.ok()) {
io_status_ = io_s;
}

@ -444,8 +444,9 @@ class Repairer {
TableBuilderOptions tboptions(
*cfd->ioptions(), *cfd->GetLatestMutableCFOptions(),
cfd->internal_comparator(), cfd->int_tbl_prop_collector_factories(),
kNoCompression, default_compression, false /* skip_filters */,
cfd->GetID(), cfd->GetName(), -1 /* level */, current_time,
kNoCompression, default_compression, cfd->GetID(), cfd->GetName(),
-1 /* level */, false /* is_bottommost */,
TableFileCreationReason::kRecovery, current_time,
0 /* oldest_key_time */, 0 /* file_creation_time */,
"DB Repairer" /* db_id */, db_session_id_, 0 /*target_file_size*/);
status = BuildTable(
@ -453,10 +454,9 @@ class Repairer {
env_options_, table_cache_.get(), iter.get(),
std::move(range_del_iters), &meta, nullptr /* blob_file_additions */,
{}, kMaxSequenceNumber, snapshot_checker,
false /* paranoid_file_checks*/, nullptr /* internal_stats */,
TableFileCreationReason::kRecovery, &io_s, nullptr /*IOTracer*/,
nullptr /* event_logger */, 0 /* job_id */, Env::IO_HIGH,
nullptr /* table_properties */, write_hint);
false /* paranoid_file_checks*/, nullptr /* internal_stats */, &io_s,
nullptr /*IOTracer*/, nullptr /* event_logger */, 0 /* job_id */,
Env::IO_HIGH, nullptr /* table_properties */, write_hint);
ROCKS_LOG_INFO(db_options_.info_log,
"Log #%" PRIu64 ": %d ops saved to Table #%" PRIu64 " %s",
log, counter, meta.fd.GetNumber(),

@ -54,8 +54,8 @@ void MakeBuilder(const Options& options, const ImmutableCFOptions& ioptions,
int unknown_level = -1;
TableBuilderOptions tboptions(
ioptions, moptions, internal_comparator, int_tbl_prop_collector_factories,
options.compression, options.compression_opts, false /*skip_filters*/,
kTestColumnFamilyId, kTestColumnFamilyName, unknown_level);
options.compression, options.compression_opts, kTestColumnFamilyId,
kTestColumnFamilyName, unknown_level);
builder->reset(NewTableBuilder(tboptions, writable->get()));
}
} // namespace

@ -2782,7 +2782,6 @@ class VersionSetTestMissingFiles : public VersionSetTestBase,
immutable_cf_options_, mutable_cf_options_, *internal_comparator_,
&int_tbl_prop_collector_factories, kNoCompression,
CompressionOptions(),
/*_skip_filters=*/false,
TablePropertiesCollectorFactory::Context::kUnknownColumnFamily,
info.column_family, info.level),
fwriter.get()));

@ -29,6 +29,7 @@
#include "rocksdb/advanced_options.h"
#include "rocksdb/status.h"
#include "rocksdb/types.h"
namespace ROCKSDB_NAMESPACE {
@ -99,19 +100,32 @@ struct FilterBuildingContext {
// Options for the table being built
const BlockBasedTableOptions& table_options;
// BEGIN from (DB|ColumnFamily)Options in effect at table creation time
CompactionStyle compaction_style = kCompactionStyleLevel;
// Number of LSM levels, or -1 if unknown
int num_levels = -1;
// An optional logger for reporting errors, warnings, etc.
Logger* info_log = nullptr;
// END from (DB|ColumnFamily)Options
// Name of the column family for the table (or empty string if unknown)
// TODO: consider changing to Slice
std::string column_family_name;
// The compactions style in effect for the table
CompactionStyle compaction_style = kCompactionStyleLevel;
// The table level at time of constructing the SST file, or -1 if unknown.
// (The table file could later be used at a different level.)
// The table level at time of constructing the SST file, or -1 if unknown
// or N/A as in SstFileWriter. (The table file could later be used at a
// different level.)
int level_at_creation = -1;
// An optional logger for reporting errors, warnings, etc.
Logger* info_log = nullptr;
// True if known to be going into bottommost sorted run for applicable
// key range (which might not even be last level with data). False
// otherwise.
bool is_bottommost = false;
// Reason for creating the file with the filter
TableFileCreationReason reason = TableFileCreationReason::kMisc;
};
// We add a new format of filter block called full filter block

@ -27,13 +27,6 @@ class ColumnFamilyHandle;
class Status;
struct CompactionJobStats;
enum class TableFileCreationReason {
kFlush,
kCompaction,
kRecovery,
kMisc,
};
struct TableFileCreationBriefInfo {
// the name of the database where the file was created
std::string db_name;

@ -86,6 +86,9 @@ class SstFileWriter {
// hint that this file pages is not needed every time we write 1MB to the
// file. To use the rate limiter an io_priority smaller than IO_TOTAL can be
// passed.
// The `skip_filters` option is DEPRECATED and could be removed in the
// future. Use `BlockBasedTableOptions::filter_policy` to control filter
// generation.
SstFileWriter(const EnvOptions& env_options, const Options& options,
ColumnFamilyHandle* column_family = nullptr,
bool invalidate_page_cache = true,

@ -19,6 +19,13 @@ typedef uint64_t SequenceNumber;
const SequenceNumber kMinUnCommittedSeq = 1; // 0 is always committed
enum class TableFileCreationReason {
kFlush,
kCompaction,
kRecovery,
kMisc,
};
// The types of files RocksDB uses in a DB directory. (Available for
// advanced options.)
enum FileType {

@ -66,7 +66,7 @@ FilterBlockBuilder* CreateFilterBlockBuilder(
const bool use_delta_encoding_for_index_values,
PartitionedIndexBuilder* const p_index_builder) {
const BlockBasedTableOptions& table_opt = context.table_options;
if (table_opt.filter_policy == nullptr) return nullptr;
assert(table_opt.filter_policy); // precondition
FilterBitsBuilder* filter_bits_builder =
BloomFilterPolicy::GetBuilderFromContext(context);
@ -473,15 +473,32 @@ struct BlockBasedTableBuilder::Rep {
&this->internal_prefix_transform, use_delta_encoding_for_index_values,
table_options));
}
if (tbo.skip_filters) {
filter_builder = nullptr;
if (ioptions.optimize_filters_for_hits && tbo.is_bottommost) {
// Apply optimize_filters_for_hits setting here when applicable by
// skipping filter generation
filter_builder.reset();
} else if (tbo.skip_filters) {
// For SstFileWriter skip_filters
filter_builder.reset();
} else if (!table_options.filter_policy) {
// Null filter_policy -> no filter
filter_builder.reset();
} else {
FilterBuildingContext filter_context(table_options);
filter_context.level_at_creation = tbo.level;
filter_context.column_family_name = column_family_name;
filter_context.compaction_style = ioptions.compaction_style;
filter_context.info_log = ioptions.logger;
filter_context.column_family_name = tbo.column_family_name;
filter_context.reason = tbo.reason;
// Only populate other fields if known to be in LSM rather than
// generating external SST file
if (tbo.reason != TableFileCreationReason::kMisc) {
filter_context.compaction_style = ioptions.compaction_style;
filter_context.num_levels = ioptions.num_levels;
filter_context.level_at_creation = tbo.level_at_creation;
filter_context.is_bottommost = tbo.is_bottommost;
assert(filter_context.level_at_creation < filter_context.num_levels);
}
filter_builder.reset(CreateFilterBlockBuilder(
ioptions, moptions, filter_context,

@ -67,8 +67,8 @@ class BlockBasedTableReaderTest
std::unique_ptr<TableBuilder> table_builder(table_factory_->NewTableBuilder(
TableBuilderOptions(ioptions, moptions, comparator, &factories,
compression_type, CompressionOptions(),
false /* skip_filters */, 0 /* column_family_id */,
kDefaultColumnFamilyName, -1 /* level */),
0 /* column_family_id */, kDefaultColumnFamilyName,
-1 /* level */),
writer.get()));
// Build table.

@ -558,7 +558,7 @@ void TestBoundary(InternalKey& ik1, std::string& v1, InternalKey& ik2,
TableBuilderOptions(
ioptions, moptions, internal_comparator,
&int_tbl_prop_collector_factories, options.compression,
CompressionOptions(), false /* skip_filters */,
CompressionOptions(),
TablePropertiesCollectorFactory::Context::kUnknownColumnFamily,
column_family_name, level_),
file_writer.get()));

@ -101,8 +101,8 @@ class BlockFetcherTest : public testing::Test {
std::unique_ptr<TableBuilder> table_builder(table_factory_.NewTableBuilder(
TableBuilderOptions(ioptions, moptions, comparator, &factories,
compression_type, CompressionOptions(),
false /* skip_filters */, 0 /* column_family_id */,
kDefaultColumnFamilyName, -1 /* level */),
0 /* column_family_id */, kDefaultColumnFamilyName,
-1 /* level */),
writer.get()));
// Build table.

@ -275,7 +275,7 @@ Status SstFileDumper::ShowCompressionSize(
int unknown_level = -1;
TableBuilderOptions tb_opts(
imoptions, moptions, ikc, &block_based_table_factories, compress_type,
compress_opt, false /* skip_filters */,
compress_opt,
TablePropertiesCollectorFactory::Context::kUnknownColumnFamily,
column_family_name, unknown_level);
uint64_t num_data_blocks = 0;

@ -253,10 +253,13 @@ Status SstFileWriter::Open(const std::string& file_path) {
TableBuilderOptions table_builder_options(
r->ioptions, r->mutable_cf_options, r->internal_comparator,
&int_tbl_prop_collector_factories, compression_type, compression_opts,
r->skip_filters, cf_id, r->column_family_name, unknown_level,
0 /* creation_time */, 0 /* oldest_key_time */,
0 /* file_creation_time */, "SST Writer" /* db_id */, db_session_id,
0 /* target_file_size */);
cf_id, r->column_family_name, unknown_level, false /* is_bottommost */,
TableFileCreationReason::kMisc, 0 /* creation_time */,
0 /* oldest_key_time */, 0 /* file_creation_time */,
"SST Writer" /* db_id */, db_session_id, 0 /* target_file_size */);
// XXX: when we can remove skip_filters from the SstFileWriter public API
// we can remove it from TableBuilderOptions.
table_builder_options.skip_filters = r->skip_filters;
FileTypeSet tmp_set = r->ioptions.checksum_handoff_file_types;
r->file_writer.reset(new WritableFileWriter(
std::move(sst_file), file_path, r->env_options, r->ioptions.clock,

@ -10,9 +10,11 @@
#pragma once
#include <stdint.h>
#include <string>
#include <utility>
#include <vector>
#include "db/dbformat.h"
#include "db/table_properties_collector.h"
#include "file/writable_file_writer.h"
@ -76,7 +78,8 @@ struct TableReaderOptions {
// fetch into RocksDB's buffer, rather than relying
// RandomAccessFile::Prefetch().
bool force_direct_prefetch;
// what level this table/file is on, -1 for "not set, don't know"
// What level this table/file is on, -1 for "not set, don't know." Used
// for level-specific statistics.
int level;
// largest seqno in the table
SequenceNumber largest_seqno;
@ -93,10 +96,11 @@ struct TableBuilderOptions {
const std::vector<std::unique_ptr<IntTblPropCollectorFactory>>*
_int_tbl_prop_collector_factories,
CompressionType _compression_type,
const CompressionOptions& _compression_opts, bool _skip_filters,
uint32_t _column_family_id, const std::string& _column_family_name,
int _level, const uint64_t _creation_time = 0,
const int64_t _oldest_key_time = 0,
const CompressionOptions& _compression_opts, uint32_t _column_family_id,
const std::string& _column_family_name, int _level,
bool _is_bottommost = false,
TableFileCreationReason _reason = TableFileCreationReason::kMisc,
const uint64_t _creation_time = 0, const int64_t _oldest_key_time = 0,
const uint64_t _file_creation_time = 0, const std::string& _db_id = "",
const std::string& _db_session_id = "",
const uint64_t _target_file_size = 0)
@ -106,16 +110,17 @@ struct TableBuilderOptions {
int_tbl_prop_collector_factories(_int_tbl_prop_collector_factories),
compression_type(_compression_type),
compression_opts(_compression_opts),
skip_filters(_skip_filters),
column_family_id(_column_family_id),
column_family_name(_column_family_name),
level(_level),
creation_time(_creation_time),
oldest_key_time(_oldest_key_time),
target_file_size(_target_file_size),
file_creation_time(_file_creation_time),
db_id(_db_id),
db_session_id(_db_session_id) {}
db_session_id(_db_session_id),
level_at_creation(_level),
is_bottommost(_is_bottommost),
reason(_reason) {}
const ImmutableCFOptions& ioptions;
const MutableCFOptions& moptions;
@ -124,17 +129,24 @@ struct TableBuilderOptions {
int_tbl_prop_collector_factories;
const CompressionType compression_type;
const CompressionOptions& compression_opts;
const bool skip_filters; // only used by BlockBasedTableBuilder
const uint32_t column_family_id;
const std::string& column_family_name;
// what level this table/file is on, -1 for "not set, don't know"
const int level;
const uint64_t creation_time;
const int64_t oldest_key_time;
const uint64_t target_file_size;
const uint64_t file_creation_time;
const std::string db_id;
const std::string db_session_id;
// BEGIN for FilterBuildingContext
const int level_at_creation;
const bool is_bottommost;
const TableFileCreationReason reason;
// END for FilterBuildingContext
// XXX: only used by BlockBasedTableBuilder for SstFileWriter. If you
// want to skip filters, that should be (for example) null filter_policy
// in the table options of the ioptions.table_factory
bool skip_filters = false;
};
// TableBuilder provides the interface used to build a Table

@ -103,8 +103,7 @@ void TableReaderBenchmark(Options& opts, EnvOptions& env_options,
TableBuilderOptions(
ioptions, moptions, ikc, &int_tbl_prop_collector_factories,
CompressionType::kNoCompression, CompressionOptions(),
false /* skip_filters */, 0 /* column_family_id */,
kDefaultColumnFamilyName, unknown_level),
0 /* column_family_id */, kDefaultColumnFamilyName, unknown_level),
file_writer.get());
} else {
s = DB::Open(opts, dbname, &db);

@ -367,8 +367,7 @@ class TableConstructor : public Constructor {
TableBuilderOptions(ioptions, moptions, internal_comparator,
&int_tbl_prop_collector_factories,
options.compression, options.compression_opts,
false /* skip_filters */, kUnknownColumnFamily,
column_family_name, level_),
kUnknownColumnFamily, column_family_name, level_),
file_writer_.get()));
for (const auto& kv : kv_map) {
@ -3326,8 +3325,7 @@ TEST_P(BlockBasedTableTest, NoFileChecksum) {
TableBuilderOptions(ioptions, moptions, *comparator,
&int_tbl_prop_collector_factories,
options.compression, options.compression_opts,
false /* skip_filters */, kUnknownColumnFamily,
column_family_name, level),
kUnknownColumnFamily, column_family_name, level),
f.GetFileWriter()));
ASSERT_OK(f.ResetTableBuilder(std::move(builder)));
f.AddKVtoKVMap(1000);
@ -3364,8 +3362,7 @@ TEST_P(BlockBasedTableTest, Crc32cFileChecksum) {
TableBuilderOptions(ioptions, moptions, *comparator,
&int_tbl_prop_collector_factories,
options.compression, options.compression_opts,
false /* skip_filters */, kUnknownColumnFamily,
column_family_name, level),
kUnknownColumnFamily, column_family_name, level),
f.GetFileWriter()));
ASSERT_OK(f.ResetTableBuilder(std::move(builder)));
f.AddKVtoKVMap(1000);
@ -3411,10 +3408,10 @@ TEST_F(PlainTableTest, BasicPlainTableProperties) {
std::string column_family_name;
int unknown_level = -1;
std::unique_ptr<TableBuilder> builder(factory.NewTableBuilder(
TableBuilderOptions(
ioptions, moptions, ikc, &int_tbl_prop_collector_factories,
kNoCompression, CompressionOptions(), false /* skip_filters */,
kUnknownColumnFamily, column_family_name, unknown_level),
TableBuilderOptions(ioptions, moptions, ikc,
&int_tbl_prop_collector_factories, kNoCompression,
CompressionOptions(), kUnknownColumnFamily,
column_family_name, unknown_level),
file_writer.get()));
for (char c = 'a'; c <= 'z'; ++c) {
@ -3467,10 +3464,10 @@ TEST_F(PlainTableTest, NoFileChecksum) {
f.CreateWriteableFile();
std::unique_ptr<TableBuilder> builder(factory.NewTableBuilder(
TableBuilderOptions(
ioptions, moptions, ikc, &int_tbl_prop_collector_factories,
kNoCompression, CompressionOptions(), false /* skip_filters */,
kUnknownColumnFamily, column_family_name, unknown_level),
TableBuilderOptions(ioptions, moptions, ikc,
&int_tbl_prop_collector_factories, kNoCompression,
CompressionOptions(), kUnknownColumnFamily,
column_family_name, unknown_level),
f.GetFileWriter()));
ASSERT_OK(f.ResetTableBuilder(std::move(builder)));
f.AddKVtoKVMap(1000);
@ -3508,10 +3505,10 @@ TEST_F(PlainTableTest, Crc32cFileChecksum) {
f.SetFileChecksumGenerator(checksum_crc32c_gen1.release());
std::unique_ptr<TableBuilder> builder(factory.NewTableBuilder(
TableBuilderOptions(
ioptions, moptions, ikc, &int_tbl_prop_collector_factories,
kNoCompression, CompressionOptions(), false /* skip_filters */,
kUnknownColumnFamily, column_family_name, unknown_level),
TableBuilderOptions(ioptions, moptions, ikc,
&int_tbl_prop_collector_factories, kNoCompression,
CompressionOptions(), kUnknownColumnFamily,
column_family_name, unknown_level),
f.GetFileWriter()));
ASSERT_OK(f.ResetTableBuilder(std::move(builder)));
f.AddKVtoKVMap(1000);
@ -4075,8 +4072,8 @@ TEST_P(BlockBasedTableTest, DISABLED_TableWithGlobalSeqno) {
std::unique_ptr<TableBuilder> builder(options.table_factory->NewTableBuilder(
TableBuilderOptions(ioptions, moptions, ikc,
&int_tbl_prop_collector_factories, kNoCompression,
CompressionOptions(), false /* skip_filters */,
kUnknownColumnFamily, column_family_name, -1),
CompressionOptions(), kUnknownColumnFamily,
column_family_name, -1),
file_writer.get()));
for (char c = 'a'; c <= 'z'; ++c) {
@ -4260,8 +4257,8 @@ TEST_P(BlockBasedTableTest, BlockAlignTest) {
std::unique_ptr<TableBuilder> builder(options.table_factory->NewTableBuilder(
TableBuilderOptions(ioptions, moptions, ikc,
&int_tbl_prop_collector_factories, kNoCompression,
CompressionOptions(), false /* skip_filters */,
kUnknownColumnFamily, column_family_name, -1),
CompressionOptions(), kUnknownColumnFamily,
column_family_name, -1),
file_writer.get()));
for (int i = 1; i <= 10000; ++i) {
@ -4354,8 +4351,8 @@ TEST_P(BlockBasedTableTest, PropertiesBlockRestartPointTest) {
std::unique_ptr<TableBuilder> builder(options.table_factory->NewTableBuilder(
TableBuilderOptions(ioptions, moptions, ikc,
&int_tbl_prop_collector_factories, kNoCompression,
CompressionOptions(), false /* skip_filters */,
kUnknownColumnFamily, column_family_name, -1),
CompressionOptions(), kUnknownColumnFamily,
column_family_name, -1),
file_writer.get()));
for (int i = 1; i <= 10000; ++i) {

@ -114,7 +114,6 @@ class SSTDumpToolTest : public testing::Test {
TableBuilderOptions(
imoptions, moptions, ikc, &int_tbl_prop_collector_factories,
CompressionType::kNoCompression, CompressionOptions(),
false /* skip_filters */,
TablePropertiesCollectorFactory::Context::kUnknownColumnFamily,
column_family_name, unknown_level),
file_writer.get()));

Loading…
Cancel
Save