From d2ca04e3ed237e202306865db6201be1161cbdc2 Mon Sep 17 00:00:00 2001 From: Peter Dillinger Date: Fri, 30 Apr 2021 13:49:24 -0700 Subject: [PATCH] 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 --- HISTORY.md | 2 + db/builder.cc | 16 +++--- db/builder.h | 4 +- db/compaction/compaction_job.cc | 11 ++--- db/db_bloom_filter_test.cc | 49 +++++++++++++++---- db/db_impl/db_impl_open.cc | 12 ++--- db/flush_job.cc | 13 +++-- db/repair.cc | 12 ++--- db/table_properties_collector_test.cc | 4 +- db/version_set_test.cc | 1 - include/rocksdb/filter_policy.h | 28 ++++++++--- include/rocksdb/listener.h | 7 --- include/rocksdb/sst_file_writer.h | 3 ++ include/rocksdb/types.h | 7 +++ .../block_based/block_based_table_builder.cc | 29 ++++++++--- .../block_based_table_reader_test.cc | 4 +- .../block_based/data_block_hash_index_test.cc | 2 +- table/block_fetcher_test.cc | 4 +- table/sst_file_dumper.cc | 2 +- table/sst_file_writer.cc | 11 +++-- table/table_builder.h | 34 ++++++++----- table/table_reader_bench.cc | 3 +- table/table_test.cc | 45 ++++++++--------- tools/sst_dump_test.cc | 1 - 24 files changed, 186 insertions(+), 118 deletions(-) diff --git a/HISTORY.md b/HISTORY.md index 9a8325116..c427800ea 100644 --- a/HISTORY.md +++ b/HISTORY.md @@ -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 diff --git a/db/builder.cc b/db/builder.cc index 6edf1b47a..ef671570b 100644 --- a/db/builder.cc +++ b/db/builder.cc @@ -64,9 +64,9 @@ Status BuildTable( std::vector 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& io_tracer, - EventLogger* event_logger, int job_id, const Env::IOPriority io_priority, + InternalStats* internal_stats, IOStatus* io_status, + const std::shared_ptr& 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; } diff --git a/db/builder.h b/db/builder.h index c6d01fa31..529d618d8 100644 --- a/db/builder.h +++ b/db/builder.h @@ -58,8 +58,8 @@ extern Status BuildTable( std::vector 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& io_tracer, + InternalStats* internal_stats, IOStatus* io_status, + const std::shared_ptr& io_tracer, EventLogger* event_logger = nullptr, int job_id = 0, const Env::IOPriority io_priority = Env::IO_HIGH, TableProperties* table_properties = nullptr, diff --git a/db/compaction/compaction_job.cc b/db/compaction/compaction_job.cc index 44e613fe9..e0f034aeb 100644 --- a/db/compaction/compaction_job.cc +++ b/db/compaction/compaction_job.cc @@ -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( diff --git a/db/db_bloom_filter_test.cc b/db/db_bloom_filter_test.cc index 18b34f0c5..64e75b620 100644 --- a/db/db_bloom_filter_test.cc +++ b/db/db_bloom_filter_test.cc @@ -774,6 +774,14 @@ class LevelAndStyleCustomFilterPolicy : public FilterPolicy { const std::unique_ptr policy_otherwise_; }; +static std::map + 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(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(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 diff --git a/db/db_impl/db_impl_open.cc b/db/db_impl/db_impl_open.cc index 71148a9de..a57675c4d 100644 --- a/db/db_impl/db_impl_open.cc +++ b/db/db_impl/db_impl_open.cc @@ -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]" diff --git a/db/flush_job.cc b/db/flush_job.cc index f96240719..92787e728 100644 --- a/db/flush_job.cc +++ b/db/flush_job.cc @@ -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; } diff --git a/db/repair.cc b/db/repair.cc index 5d3a0c097..9d103a710 100644 --- a/db/repair.cc +++ b/db/repair.cc @@ -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(), diff --git a/db/table_properties_collector_test.cc b/db/table_properties_collector_test.cc index 6c29d365d..aff95c40d 100644 --- a/db/table_properties_collector_test.cc +++ b/db/table_properties_collector_test.cc @@ -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 diff --git a/db/version_set_test.cc b/db/version_set_test.cc index afabeab8e..b03fe6a5e 100644 --- a/db/version_set_test.cc +++ b/db/version_set_test.cc @@ -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())); diff --git a/include/rocksdb/filter_policy.h b/include/rocksdb/filter_policy.h index 3e61d5033..5b7678a7c 100644 --- a/include/rocksdb/filter_policy.h +++ b/include/rocksdb/filter_policy.h @@ -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 diff --git a/include/rocksdb/listener.h b/include/rocksdb/listener.h index af4423de6..cfa327d86 100644 --- a/include/rocksdb/listener.h +++ b/include/rocksdb/listener.h @@ -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; diff --git a/include/rocksdb/sst_file_writer.h b/include/rocksdb/sst_file_writer.h index c7a8203e1..3bcc8a693 100644 --- a/include/rocksdb/sst_file_writer.h +++ b/include/rocksdb/sst_file_writer.h @@ -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, diff --git a/include/rocksdb/types.h b/include/rocksdb/types.h index d56a7cc94..f495fcd22 100644 --- a/include/rocksdb/types.h +++ b/include/rocksdb/types.h @@ -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 { diff --git a/table/block_based/block_based_table_builder.cc b/table/block_based/block_based_table_builder.cc index b1dc037e9..8f971446a 100644 --- a/table/block_based/block_based_table_builder.cc +++ b/table/block_based/block_based_table_builder.cc @@ -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, diff --git a/table/block_based/block_based_table_reader_test.cc b/table/block_based/block_based_table_reader_test.cc index d76be0902..bb39df5f5 100644 --- a/table/block_based/block_based_table_reader_test.cc +++ b/table/block_based/block_based_table_reader_test.cc @@ -67,8 +67,8 @@ class BlockBasedTableReaderTest std::unique_ptr 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. diff --git a/table/block_based/data_block_hash_index_test.cc b/table/block_based/data_block_hash_index_test.cc index 4e1a419d2..200be925a 100644 --- a/table/block_based/data_block_hash_index_test.cc +++ b/table/block_based/data_block_hash_index_test.cc @@ -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())); diff --git a/table/block_fetcher_test.cc b/table/block_fetcher_test.cc index ce0b7f8ad..963c1b359 100644 --- a/table/block_fetcher_test.cc +++ b/table/block_fetcher_test.cc @@ -101,8 +101,8 @@ class BlockFetcherTest : public testing::Test { std::unique_ptr 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. diff --git a/table/sst_file_dumper.cc b/table/sst_file_dumper.cc index a5f528ce3..816d76c51 100644 --- a/table/sst_file_dumper.cc +++ b/table/sst_file_dumper.cc @@ -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; diff --git a/table/sst_file_writer.cc b/table/sst_file_writer.cc index e3d67e002..af3ee9f8d 100644 --- a/table/sst_file_writer.cc +++ b/table/sst_file_writer.cc @@ -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, diff --git a/table/table_builder.h b/table/table_builder.h index 6d2800da0..a0854ba08 100644 --- a/table/table_builder.h +++ b/table/table_builder.h @@ -10,9 +10,11 @@ #pragma once #include + #include #include #include + #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>* _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 diff --git a/table/table_reader_bench.cc b/table/table_reader_bench.cc index 84919fe5c..774f2edfc 100644 --- a/table/table_reader_bench.cc +++ b/table/table_reader_bench.cc @@ -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); diff --git a/table/table_test.cc b/table/table_test.cc index 5c4c97028..e0abce8e5 100644 --- a/table/table_test.cc +++ b/table/table_test.cc @@ -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 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 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 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 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 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 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) { diff --git a/tools/sst_dump_test.cc b/tools/sst_dump_test.cc index 24f772942..dd397a701 100644 --- a/tools/sst_dump_test.cc +++ b/tools/sst_dump_test.cc @@ -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()));