From 5665e5e285c25c1674567f747df92c131037d2dc Mon Sep 17 00:00:00 2001 From: Lei Jin Date: Thu, 4 Sep 2014 16:18:36 -0700 Subject: [PATCH] introduce ImmutableOptions Summary: As a preparation to support updating some options dynamically, I'd like to first introduce ImmutableOptions, which is a subset of Options that cannot be changed during the course of a DB lifetime without restart. ColumnFamily will keep both Options and ImmutableOptions. Any component below ColumnFamily should only take ImmutableOptions in their constructor. Other options should be taken from APIs, which will be allowed to adjust dynamically. I am yet to make changes to memtable and other related classes to take ImmutableOptions in their ctor. That can be done in a seprate diff as this one is already pretty big. Test Plan: make all check Reviewers: yhchiang, igor, sdong Reviewed By: sdong Subscribers: leveldb, dhruba Differential Revision: https://reviews.facebook.net/D22545 --- db/builder.cc | 40 +++++++------ db/builder.h | 11 +++- db/column_family.cc | 15 ++--- db/column_family.h | 13 ++-- db/db_impl.cc | 37 ++++++------ db/db_impl.h | 2 +- db/plain_table_db_test.cc | 22 +++---- db/repair.cc | 23 ++++--- db/simple_table_db_test.cc | 69 +++++++++++---------- db/table_cache.cc | 43 +++++++------- db/table_cache.h | 11 ++-- db/table_properties_collector_test.cc | 3 +- include/rocksdb/immutable_options.h | 62 +++++++++++++++++++ include/rocksdb/table.h | 23 ++++--- table/adaptive_table_factory.cc | 18 +++--- table/adaptive_table_factory.h | 24 ++++---- table/block_based_table_builder.cc | 49 ++++++++------- table/block_based_table_builder.h | 6 +- table/block_based_table_factory.cc | 13 ++-- table/block_based_table_factory.h | 17 +++--- table/block_based_table_reader.cc | 67 +++++++++++---------- table/block_based_table_reader.h | 5 +- table/cuckoo_table_factory.cc | 18 +++--- table/cuckoo_table_factory.h | 7 ++- table/cuckoo_table_reader.cc | 6 +- table/cuckoo_table_reader.h | 5 +- table/cuckoo_table_reader_test.cc | 15 +++-- table/filter_block.cc | 9 +-- table/filter_block.h | 5 +- table/filter_block_test.cc | 18 +++--- table/plain_table_builder.cc | 29 ++++----- table/plain_table_builder.h | 4 +- table/plain_table_factory.cc | 14 +++-- table/plain_table_factory.h | 21 +++---- table/plain_table_index.cc | 8 +-- table/plain_table_index.h | 8 +-- table/plain_table_reader.cc | 51 ++++++++-------- table/plain_table_reader.h | 10 ++-- table/table_reader_bench.cc | 8 ++- table/table_test.cc | 86 ++++++++++++++++++--------- tools/sst_dump.cc | 6 +- util/options.cc | 21 +++++++ 42 files changed, 554 insertions(+), 368 deletions(-) create mode 100644 include/rocksdb/immutable_options.h diff --git a/db/builder.cc b/db/builder.cc index 1084f0413..2c5094370 100644 --- a/db/builder.cc +++ b/db/builder.cc @@ -26,21 +26,24 @@ namespace rocksdb { class TableFactory; -TableBuilder* NewTableBuilder(const Options& options, +TableBuilder* NewTableBuilder(const ImmutableCFOptions& ioptions, const InternalKeyComparator& internal_comparator, WritableFile* file, - CompressionType compression_type) { - return options.table_factory->NewTableBuilder(options, internal_comparator, - file, compression_type); + const CompressionType compression_type, + const CompressionOptions& compression_opts) { + return ioptions.table_factory->NewTableBuilder( + ioptions, internal_comparator, file, compression_type, compression_opts); } -Status BuildTable(const std::string& dbname, Env* env, const Options& options, - const EnvOptions& soptions, TableCache* table_cache, +Status BuildTable(const std::string& dbname, Env* env, + const ImmutableCFOptions& ioptions, + const EnvOptions& env_options, TableCache* table_cache, Iterator* iter, FileMetaData* meta, const InternalKeyComparator& internal_comparator, const SequenceNumber newest_snapshot, const SequenceNumber earliest_seqno_in_memtable, const CompressionType compression, + const CompressionOptions& compression_opts, const Env::IOPriority io_priority) { Status s; meta->fd.file_size = 0; @@ -50,23 +53,24 @@ Status BuildTable(const std::string& dbname, Env* env, const Options& options, // If the sequence number of the smallest entry in the memtable is // smaller than the most recent snapshot, then we do not trigger // removal of duplicate/deleted keys as part of this builder. - bool purge = options.purge_redundant_kvs_while_flush; + bool purge = ioptions.purge_redundant_kvs_while_flush; if (earliest_seqno_in_memtable <= newest_snapshot) { purge = false; } - std::string fname = TableFileName(options.db_paths, meta->fd.GetNumber(), + std::string fname = TableFileName(ioptions.db_paths, meta->fd.GetNumber(), meta->fd.GetPathId()); if (iter->Valid()) { unique_ptr file; - s = env->NewWritableFile(fname, &file, soptions); + s = env->NewWritableFile(fname, &file, env_options); if (!s.ok()) { return s; } file->SetIOPriority(io_priority); - TableBuilder* builder = - NewTableBuilder(options, internal_comparator, file.get(), compression); + TableBuilder* builder = NewTableBuilder( + ioptions, internal_comparator, file.get(), + compression, compression_opts); // the first key is the smallest key Slice key = iter->key(); @@ -75,8 +79,8 @@ Status BuildTable(const std::string& dbname, Env* env, const Options& options, meta->largest_seqno = meta->smallest_seqno; MergeHelper merge(internal_comparator.user_comparator(), - options.merge_operator.get(), options.info_log.get(), - options.min_partial_merge_operands, + ioptions.merge_operator, ioptions.info_log, + ioptions.min_partial_merge_operands, true /* internal key corruption is not ok */); if (purge) { @@ -196,12 +200,12 @@ Status BuildTable(const std::string& dbname, Env* env, const Options& options, delete builder; // Finish and check for file errors - if (s.ok() && !options.disableDataSync) { - if (options.use_fsync) { - StopWatch sw(env, options.statistics.get(), TABLE_SYNC_MICROS); + if (s.ok() && !ioptions.disable_data_sync) { + if (ioptions.use_fsync) { + StopWatch sw(env, ioptions.statistics, TABLE_SYNC_MICROS); s = file->Fsync(); } else { - StopWatch sw(env, options.statistics.get(), TABLE_SYNC_MICROS); + StopWatch sw(env, ioptions.statistics, TABLE_SYNC_MICROS); s = file->Sync(); } } @@ -211,7 +215,7 @@ Status BuildTable(const std::string& dbname, Env* env, const Options& options, if (s.ok()) { // Verify that the table is usable - Iterator* it = table_cache->NewIterator(ReadOptions(), soptions, + Iterator* it = table_cache->NewIterator(ReadOptions(), env_options, internal_comparator, meta->fd); s = it->status(); delete it; diff --git a/db/builder.h b/db/builder.h index f57501abd..cf3ebd1ae 100644 --- a/db/builder.h +++ b/db/builder.h @@ -11,6 +11,7 @@ #include "rocksdb/status.h" #include "rocksdb/types.h" #include "rocksdb/options.h" +#include "rocksdb/immutable_options.h" namespace rocksdb { @@ -26,8 +27,10 @@ class TableBuilder; class WritableFile; extern TableBuilder* NewTableBuilder( - const Options& options, const InternalKeyComparator& internal_comparator, - WritableFile* file, CompressionType compression_type); + const ImmutableCFOptions& options, + const InternalKeyComparator& internal_comparator, + WritableFile* file, const CompressionType compression_type, + const CompressionOptions& compression_opts); // Build a Table file from the contents of *iter. The generated file // will be named according to number specified in meta. On success, the rest of @@ -35,13 +38,15 @@ extern TableBuilder* NewTableBuilder( // If no data is present in *iter, meta->file_size will be set to // zero, and no Table file will be produced. extern Status BuildTable(const std::string& dbname, Env* env, - const Options& options, const EnvOptions& soptions, + const ImmutableCFOptions& options, + const EnvOptions& env_options, TableCache* table_cache, Iterator* iter, FileMetaData* meta, const InternalKeyComparator& internal_comparator, const SequenceNumber newest_snapshot, const SequenceNumber earliest_seqno_in_memtable, const CompressionType compression, + const CompressionOptions& compression_opts, const Env::IOPriority io_priority = Env::IO_HIGH); } // namespace rocksdb diff --git a/db/column_family.cc b/db/column_family.cc index b1c9ba7e8..7e06c9bd7 100644 --- a/db/column_family.cc +++ b/db/column_family.cc @@ -178,7 +178,7 @@ ColumnFamilyData::ColumnFamilyData(uint32_t id, const std::string& name, Version* dummy_versions, Cache* table_cache, const ColumnFamilyOptions& options, const DBOptions* db_options, - const EnvOptions& storage_options, + const EnvOptions& env_options, ColumnFamilySet* column_family_set) : id_(id), name_(name), @@ -188,6 +188,7 @@ ColumnFamilyData::ColumnFamilyData(uint32_t id, const std::string& name, dropped_(false), internal_comparator_(options.comparator), options_(*db_options, SanitizeOptions(&internal_comparator_, options)), + ioptions_(options_), mem_(nullptr), imm_(options_.min_write_buffer_number_to_merge), super_version_(nullptr), @@ -204,7 +205,7 @@ ColumnFamilyData::ColumnFamilyData(uint32_t id, const std::string& name, if (dummy_versions != nullptr) { internal_stats_.reset( new InternalStats(options_.num_levels, db_options->env, this)); - table_cache_.reset(new TableCache(&options_, storage_options, table_cache)); + table_cache_.reset(new TableCache(ioptions_, env_options, table_cache)); if (options_.compaction_style == kCompactionStyleUniversal) { compaction_picker_.reset( new UniversalCompactionPicker(&options_, &internal_comparator_)); @@ -306,7 +307,7 @@ void ColumnFamilyData::RecalculateWriteStallRateLimitsConditions() { } const EnvOptions* ColumnFamilyData::soptions() const { - return &(column_family_set_->storage_options_); + return &(column_family_set_->env_options_); } void ColumnFamilyData::SetCurrent(Version* current) { @@ -462,16 +463,16 @@ void ColumnFamilyData::ResetThreadLocalSuperVersions() { ColumnFamilySet::ColumnFamilySet(const std::string& dbname, const DBOptions* db_options, - const EnvOptions& storage_options, + const EnvOptions& env_options, Cache* table_cache) : max_column_family_(0), dummy_cfd_(new ColumnFamilyData(0, "", nullptr, nullptr, ColumnFamilyOptions(), db_options, - storage_options_, nullptr)), + env_options_, nullptr)), default_cfd_cache_(nullptr), db_name_(dbname), db_options_(db_options), - storage_options_(storage_options), + env_options_(env_options), table_cache_(table_cache), spin_lock_(ATOMIC_FLAG_INIT) { // initialize linked list @@ -537,7 +538,7 @@ ColumnFamilyData* ColumnFamilySet::CreateColumnFamily( assert(column_families_.find(name) == column_families_.end()); ColumnFamilyData* new_cfd = new ColumnFamilyData(id, name, dummy_versions, table_cache_, options, - db_options_, storage_options_, this); + db_options_, env_options_, this); Lock(); column_families_.insert({name, id}); column_family_data_.insert({id, new_cfd}); diff --git a/db/column_family.h b/db/column_family.h index 33bceadc6..a68189d51 100644 --- a/db/column_family.h +++ b/db/column_family.h @@ -165,9 +165,11 @@ class ColumnFamilyData { void SetLogNumber(uint64_t log_number) { log_number_ = log_number; } uint64_t GetLogNumber() const { return log_number_; } - // thread-safe + // TODO(ljin): make this API thread-safe once we allow updating options_ const Options* options() const { return &options_; } + // thread-safe const EnvOptions* soptions() const; + const ImmutableCFOptions* ioptions() const { return &ioptions_; } InternalStats* internal_stats() { return internal_stats_.get(); } @@ -251,7 +253,7 @@ class ColumnFamilyData { Version* dummy_versions, Cache* table_cache, const ColumnFamilyOptions& options, const DBOptions* db_options, - const EnvOptions& storage_options, + const EnvOptions& env_options, ColumnFamilySet* column_family_set); // Recalculate some small conditions, which are changed only during @@ -272,7 +274,8 @@ class ColumnFamilyData { const InternalKeyComparator internal_comparator_; - Options const options_; + const Options options_; + const ImmutableCFOptions ioptions_; std::unique_ptr table_cache_; @@ -367,7 +370,7 @@ class ColumnFamilySet { }; ColumnFamilySet(const std::string& dbname, const DBOptions* db_options, - const EnvOptions& storage_options, Cache* table_cache); + const EnvOptions& env_options, Cache* table_cache); ~ColumnFamilySet(); ColumnFamilyData* GetDefault() const; @@ -420,7 +423,7 @@ class ColumnFamilySet { const std::string db_name_; const DBOptions* const db_options_; - const EnvOptions storage_options_; + const EnvOptions env_options_; Cache* table_cache_; std::atomic_flag spin_lock_; }; diff --git a/db/db_impl.cc b/db/db_impl.cc index f18bb2141..049d40c7b 100644 --- a/db/db_impl.cc +++ b/db/db_impl.cc @@ -356,7 +356,7 @@ DBImpl::DBImpl(const DBOptions& options, const std::string& dbname) default_interval_to_delete_obsolete_WAL_(600), flush_on_destroy_(false), delayed_writes_(0), - storage_options_(options), + env_options_(options), bg_work_gate_closed_(false), refitting_level_(false), opened_successfully_(false) { @@ -372,7 +372,7 @@ DBImpl::DBImpl(const DBOptions& options, const std::string& dbname) options_.table_cache_remove_scan_count_limit); versions_.reset( - new VersionSet(dbname_, &options_, storage_options_, table_cache_.get())); + new VersionSet(dbname_, &options_, env_options_, table_cache_.get())); column_family_memtables_.reset( new ColumnFamilyMemTablesImpl(versions_->GetColumnFamilySet())); @@ -453,7 +453,7 @@ Status DBImpl::NewDB() { const std::string manifest = DescriptorFileName(dbname_, 1); unique_ptr file; Status s = env_->NewWritableFile( - manifest, &file, env_->OptimizeForManifestWrite(storage_options_)); + manifest, &file, env_->OptimizeForManifestWrite(env_options_)); if (!s.ok()) { return s; } @@ -1075,7 +1075,7 @@ Status DBImpl::ReadFirstLine(const std::string& fname, }; unique_ptr file; - Status status = env_->NewSequentialFile(fname, &file, storage_options_); + Status status = env_->NewSequentialFile(fname, &file, env_options_); if (!status.ok()) { return status; @@ -1275,7 +1275,7 @@ Status DBImpl::RecoverLogFile(uint64_t log_number, SequenceNumber* max_sequence, // Open the log file std::string fname = LogFileName(options_.wal_dir, log_number); unique_ptr file; - Status status = env_->NewSequentialFile(fname, &file, storage_options_); + Status status = env_->NewSequentialFile(fname, &file, env_options_); if (!status.ok()) { MaybeIgnoreError(&status); return status; @@ -1425,10 +1425,11 @@ Status DBImpl::WriteLevel0TableForRecovery(ColumnFamilyData* cfd, MemTable* mem, Status s; { mutex_.Unlock(); - s = BuildTable(dbname_, env_, *cfd->options(), storage_options_, + s = BuildTable(dbname_, env_, *cfd->ioptions(), env_options_, cfd->table_cache(), iter, &meta, cfd->internal_comparator(), newest_snapshot, earliest_seqno_in_memtable, - GetCompressionFlush(*cfd->options()), Env::IO_HIGH); + GetCompressionFlush(*cfd->options()), + cfd->options()->compression_opts, Env::IO_HIGH); LogFlush(options_.info_log); mutex_.Lock(); } @@ -1495,10 +1496,11 @@ Status DBImpl::WriteLevel0Table(ColumnFamilyData* cfd, Log(options_.info_log, "[%s] Level-0 flush table #%" PRIu64 ": started", cfd->GetName().c_str(), meta.fd.GetNumber()); - s = BuildTable(dbname_, env_, *cfd->options(), storage_options_, + s = BuildTable(dbname_, env_, *cfd->ioptions(), env_options_, cfd->table_cache(), iter, &meta, cfd->internal_comparator(), newest_snapshot, earliest_seqno_in_memtable, - GetCompressionFlush(*cfd->options()), Env::IO_HIGH); + GetCompressionFlush(*cfd->options()), + cfd->options()->compression_opts, Env::IO_HIGH); LogFlush(options_.info_log); delete iter; Log(options_.info_log, @@ -2447,7 +2449,7 @@ Status DBImpl::OpenCompactionOutputFile(CompactionState* compact) { // Make the output file std::string fname = TableFileName(options_.db_paths, file_number, compact->compaction->GetOutputPathId()); - Status s = env_->NewWritableFile(fname, &compact->outfile, storage_options_); + Status s = env_->NewWritableFile(fname, &compact->outfile, env_options_); if (s.ok()) { compact->outfile->SetIOPriority(Env::IO_LOW); @@ -2456,8 +2458,9 @@ Status DBImpl::OpenCompactionOutputFile(CompactionState* compact) { ColumnFamilyData* cfd = compact->compaction->column_family_data(); compact->builder.reset(NewTableBuilder( - *cfd->options(), cfd->internal_comparator(), compact->outfile.get(), - compact->compaction->OutputCompressionType())); + *cfd->ioptions(), cfd->internal_comparator(), compact->outfile.get(), + compact->compaction->OutputCompressionType(), + cfd->options()->compression_opts)); } LogFlush(options_.info_log); return s; @@ -2506,7 +2509,7 @@ Status DBImpl::FinishCompactionOutputFile(CompactionState* compact, ColumnFamilyData* cfd = compact->compaction->column_family_data(); FileDescriptor fd(output_number, output_path_id, current_bytes); Iterator* iter = cfd->table_cache()->NewIterator( - ReadOptions(), storage_options_, cfd->internal_comparator(), fd); + ReadOptions(), env_options_, cfd->internal_comparator(), fd); s = iter->status(); delete iter; if (s.ok()) { @@ -3355,7 +3358,7 @@ Iterator* DBImpl::NewInternalIterator(const ReadOptions& options, // Collect all needed child iterators for immutable memtables super_version->imm->AddIterators(options, &merge_iter_builder); // Collect iterators for files in L0 - Ln - super_version->current->AddIterators(options, storage_options_, + super_version->current->AddIterators(options, env_options_, &merge_iter_builder); internal_iter = merge_iter_builder.Finish(); } else { @@ -3366,7 +3369,7 @@ Iterator* DBImpl::NewInternalIterator(const ReadOptions& options, // Collect all needed child iterators for immutable memtables super_version->imm->AddIterators(options, &iterator_list); // Collect iterators for files in L0 - Ln - super_version->current->AddIterators(options, storage_options_, + super_version->current->AddIterators(options, env_options_, &iterator_list); internal_iter = NewMergingIterator(&cfd->internal_comparator(), &iterator_list[0], iterator_list.size()); @@ -4377,7 +4380,7 @@ Status DBImpl::SetNewMemtableAndNewLogFile(ColumnFamilyData* cfd, if (creating_new_log) { s = env_->NewWritableFile(LogFileName(options_.wal_dir, new_log_number), &lfile, - env_->OptimizeForLogWrite(storage_options_)); + env_->OptimizeForLogWrite(env_options_)); if (s.ok()) { // Our final size should be less than write_buffer_size // (compression, etc) but err on the side of caution. @@ -4615,7 +4618,7 @@ Status DBImpl::GetUpdatesSince( return s; } iter->reset(new TransactionLogIteratorImpl(options_.wal_dir, &options_, - read_options, storage_options_, + read_options, env_options_, seq, std::move(wal_files), this)); return (*iter)->status(); } diff --git a/db/db_impl.h b/db/db_impl.h index 086ac9fd4..caacd012a 100644 --- a/db/db_impl.h +++ b/db/db_impl.h @@ -607,7 +607,7 @@ class DBImpl : public DB { int delayed_writes_; // The options to access storage files - const EnvOptions storage_options_; + const EnvOptions env_options_; // A value of true temporarily disables scheduling of background work bool bg_work_gate_closed_; diff --git a/db/plain_table_db_test.cc b/db/plain_table_db_test.cc index bb0f96f15..1750d265c 100644 --- a/db/plain_table_db_test.cc +++ b/db/plain_table_db_test.cc @@ -192,16 +192,17 @@ extern const uint64_t kPlainTableMagicNumber; class TestPlainTableReader : public PlainTableReader { public: - TestPlainTableReader(const EnvOptions& storage_options, + TestPlainTableReader(const EnvOptions& env_options, const InternalKeyComparator& icomparator, EncodingType encoding_type, uint64_t file_size, int bloom_bits_per_key, double hash_table_ratio, size_t index_sparseness, const TableProperties* table_properties, unique_ptr&& file, - const Options& options, bool* expect_bloom_not_match, + const ImmutableCFOptions& ioptions, + bool* expect_bloom_not_match, bool store_index_in_file) - : PlainTableReader(options, std::move(file), storage_options, icomparator, + : PlainTableReader(ioptions, std::move(file), env_options, icomparator, encoding_type, file_size, table_properties), expect_bloom_not_match_(expect_bloom_not_match) { Status s = MmapDataFile(); @@ -218,7 +219,7 @@ class TestPlainTableReader : public PlainTableReader { PlainTablePropertyNames::kBloomVersion); ASSERT_TRUE(bloom_version_ptr != props->user_collected_properties.end()); ASSERT_EQ(bloom_version_ptr->second, std::string("1")); - if (options.bloom_locality > 0) { + if (ioptions.bloom_locality > 0) { auto num_blocks_ptr = props->user_collected_properties.find( PlainTablePropertyNames::kNumBloomBlocks); ASSERT_TRUE(num_blocks_ptr != props->user_collected_properties.end()); @@ -253,25 +254,26 @@ class TestPlainTableFactory : public PlainTableFactory { store_index_in_file_(options.store_index_in_file), expect_bloom_not_match_(expect_bloom_not_match) {} - Status NewTableReader(const Options& options, const EnvOptions& soptions, + Status NewTableReader(const ImmutableCFOptions& ioptions, + const EnvOptions& env_options, const InternalKeyComparator& internal_comparator, unique_ptr&& file, uint64_t file_size, unique_ptr* table) const override { TableProperties* props = nullptr; auto s = ReadTableProperties(file.get(), file_size, kPlainTableMagicNumber, - options.env, options.info_log.get(), &props); + ioptions.env, ioptions.info_log, &props); ASSERT_TRUE(s.ok()); if (store_index_in_file_) { BlockHandle bloom_block_handle; s = FindMetaBlock(file.get(), file_size, kPlainTableMagicNumber, - options.env, BloomBlockBuilder::kBloomBlock, + ioptions.env, BloomBlockBuilder::kBloomBlock, &bloom_block_handle); ASSERT_TRUE(s.ok()); BlockHandle index_block_handle; s = FindMetaBlock( - file.get(), file_size, kPlainTableMagicNumber, options.env, + file.get(), file_size, kPlainTableMagicNumber, ioptions.env, PlainTableIndexBuilder::kPlainTableIndexBlock, &index_block_handle); ASSERT_TRUE(s.ok()); } @@ -284,9 +286,9 @@ class TestPlainTableFactory : public PlainTableFactory { DecodeFixed32(encoding_type_prop->second.c_str())); std::unique_ptr new_reader(new TestPlainTableReader( - soptions, internal_comparator, encoding_type, file_size, + env_options, internal_comparator, encoding_type, file_size, bloom_bits_per_key_, hash_table_ratio_, index_sparseness_, props, - std::move(file), options, expect_bloom_not_match_, + std::move(file), ioptions, expect_bloom_not_match_, store_index_in_file_)); *table = std::move(new_reader); diff --git a/db/repair.cc b/db/repair.cc index 820cc1924..3c64449d1 100644 --- a/db/repair.cc +++ b/db/repair.cc @@ -46,6 +46,8 @@ #include "rocksdb/comparator.h" #include "rocksdb/db.h" #include "rocksdb/env.h" +#include "rocksdb/options.h" +#include "rocksdb/immutable_options.h" namespace rocksdb { @@ -58,6 +60,7 @@ class Repairer { env_(options.env), icmp_(options.comparator), options_(SanitizeOptions(dbname, &icmp_, options)), + ioptions_(options_), raw_table_cache_( // TableCache can be small since we expect each table to be opened // once. @@ -65,7 +68,7 @@ class Repairer { options_.table_cache_remove_scan_count_limit)), next_file_number_(1) { table_cache_ = - new TableCache(&options_, storage_options_, raw_table_cache_.get()); + new TableCache(ioptions_, env_options_, raw_table_cache_.get()); edit_ = new VersionEdit(); } @@ -107,8 +110,9 @@ class Repairer { std::string const dbname_; Env* const env_; - InternalKeyComparator const icmp_; - Options const options_; + const InternalKeyComparator icmp_; + const Options options_; + const ImmutableCFOptions ioptions_; std::shared_ptr raw_table_cache_; TableCache* table_cache_; VersionEdit* edit_; @@ -118,7 +122,7 @@ class Repairer { std::vector logs_; std::vector tables_; uint64_t next_file_number_; - const EnvOptions storage_options_; + const EnvOptions env_options_; Status FindFiles() { std::vector filenames; @@ -190,7 +194,7 @@ class Repairer { // Open the log file std::string logname = LogFileName(dbname_, log); unique_ptr lfile; - Status status = env_->NewSequentialFile(logname, &lfile, storage_options_); + Status status = env_->NewSequentialFile(logname, &lfile, env_options_); if (!status.ok()) { return status; } @@ -239,8 +243,9 @@ class Repairer { ReadOptions ro; ro.total_order_seek = true; Iterator* iter = mem->NewIterator(ro); - status = BuildTable(dbname_, env_, options_, storage_options_, table_cache_, - iter, &meta, icmp_, 0, 0, kNoCompression); + status = BuildTable(dbname_, env_, ioptions_, env_options_, table_cache_, + iter, &meta, icmp_, 0, 0, kNoCompression, + CompressionOptions()); delete iter; delete mem->Unref(); delete cf_mems_default; @@ -286,7 +291,7 @@ class Repairer { file_size); if (status.ok()) { Iterator* iter = table_cache_->NewIterator( - ReadOptions(), storage_options_, icmp_, t->meta.fd); + ReadOptions(), env_options_, icmp_, t->meta.fd); bool empty = true; ParsedInternalKey parsed; t->min_sequence = 0; @@ -326,7 +331,7 @@ class Repairer { std::string tmp = TempFileName(dbname_, 1); unique_ptr file; Status status = env_->NewWritableFile( - tmp, &file, env_->OptimizeForManifestWrite(storage_options_)); + tmp, &file, env_->OptimizeForManifestWrite(env_options_)); if (!status.ok()) { return status; } diff --git a/db/simple_table_db_test.cc b/db/simple_table_db_test.cc index e88485070..0a0ecf064 100644 --- a/db/simple_table_db_test.cc +++ b/db/simple_table_db_test.cc @@ -79,7 +79,8 @@ public: // for the duration of the returned table's lifetime. // // *file must remain live while this Table is in use. - static Status Open(const Options& options, const EnvOptions& soptions, + static Status Open(const ImmutableCFOptions& options, + const EnvOptions& env_options, unique_ptr && file, uint64_t file_size, unique_ptr* table_reader); @@ -160,14 +161,14 @@ private: struct SimpleTableReader::Rep { ~Rep() { } - Rep(const EnvOptions& storage_options, uint64_t index_start_offset, - int num_entries) : - soptions(storage_options), index_start_offset(index_start_offset), - num_entries(num_entries) { + Rep(const ImmutableCFOptions& ioptions, const EnvOptions& env_options, + uint64_t index_start_offset, int num_entries) : + ioptions(ioptions), env_options(env_options), + index_start_offset(index_start_offset), num_entries(num_entries) { } - Options options; - const EnvOptions& soptions; + const ImmutableCFOptions& ioptions; + const EnvOptions& env_options; Status status; unique_ptr file; uint64_t index_start_offset; @@ -187,8 +188,8 @@ SimpleTableReader::~SimpleTableReader() { delete rep_; } -Status SimpleTableReader::Open(const Options& options, - const EnvOptions& soptions, +Status SimpleTableReader::Open(const ImmutableCFOptions& ioptions, + const EnvOptions& env_options, unique_ptr && file, uint64_t size, unique_ptr* table_reader) { @@ -201,12 +202,10 @@ Status SimpleTableReader::Open(const Options& options, int num_entries = (size - Rep::offset_length - index_start_offset) / (Rep::GetInternalKeyLength() + Rep::offset_length); - SimpleTableReader::Rep* rep = new SimpleTableReader::Rep(soptions, - index_start_offset, - num_entries); + SimpleTableReader::Rep* rep = new SimpleTableReader::Rep( + ioptions, env_options, index_start_offset, num_entries); rep->file = std::move(file); - rep->options = options; table_reader->reset(new SimpleTableReader(rep)); } return s; @@ -248,7 +247,7 @@ Status SimpleTableReader::GetOffset(const Slice& target, uint64_t* offset) { return s; } - InternalKeyComparator ikc(rep_->options.comparator); + InternalKeyComparator ikc(rep_->ioptions.comparator); int compare_result = ikc.Compare(tmp_slice, target); if (compare_result < 0) { @@ -382,7 +381,7 @@ void SimpleTableIterator::Prev() { } Slice SimpleTableIterator::key() const { - Log(table_->rep_->options.info_log, "key!!!!"); + Log(table_->rep_->ioptions.info_log, "key!!!!"); return key_; } @@ -401,7 +400,7 @@ public: // caller to close the file after calling Finish(). The output file // will be part of level specified by 'level'. A value of -1 means // that the caller does not know which level the output file will reside. - SimpleTableBuilder(const Options& options, WritableFile* file, + SimpleTableBuilder(const ImmutableCFOptions& ioptions, WritableFile* file, CompressionType compression_type); // REQUIRES: Either Finish() or Abandon() has been called. @@ -444,7 +443,7 @@ private: }; struct SimpleTableBuilder::Rep { - Options options; + const ImmutableCFOptions& ioptions; WritableFile* file; uint64_t offset = 0; Status status; @@ -463,17 +462,17 @@ struct SimpleTableBuilder::Rep { std::string index; - Rep(const Options& opt, WritableFile* f) : - options(opt), file(f) { + Rep(const ImmutableCFOptions& iopt, WritableFile* f) : + ioptions(iopt), file(f) { } ~Rep() { } }; -SimpleTableBuilder::SimpleTableBuilder(const Options& options, +SimpleTableBuilder::SimpleTableBuilder(const ImmutableCFOptions& ioptions, WritableFile* file, CompressionType compression_type) : - rep_(new SimpleTableBuilder::Rep(options, file)) { + rep_(new SimpleTableBuilder::Rep(ioptions, file)) { } SimpleTableBuilder::~SimpleTableBuilder() { @@ -546,15 +545,18 @@ public: const char* Name() const override { return "SimpleTable"; } - Status NewTableReader(const Options& options, const EnvOptions& soptions, + Status NewTableReader(const ImmutableCFOptions& ioptions, + const EnvOptions& env_options, const InternalKeyComparator& internal_key, unique_ptr&& file, uint64_t file_size, unique_ptr* table_reader) const; - TableBuilder* NewTableBuilder(const Options& options, - const InternalKeyComparator& internal_key, - WritableFile* file, - CompressionType compression_type) const; + TableBuilder* NewTableBuilder( + const ImmutableCFOptions& ioptions, + const InternalKeyComparator& internal_key, + WritableFile* file, + const CompressionType compression_type, + const CompressionOptions& compression_opts) const; virtual Status SanitizeDBOptions(const DBOptions* db_opts) const override { return Status::OK(); @@ -566,19 +568,22 @@ public: }; Status SimpleTableFactory::NewTableReader( - const Options& options, const EnvOptions& soptions, + const ImmutableCFOptions& ioptions, + const EnvOptions& env_options, const InternalKeyComparator& internal_key, unique_ptr&& file, uint64_t file_size, unique_ptr* table_reader) const { - return SimpleTableReader::Open(options, soptions, std::move(file), file_size, - table_reader); + return SimpleTableReader::Open(ioptions, env_options, std::move(file), + file_size, table_reader); } TableBuilder* SimpleTableFactory::NewTableBuilder( - const Options& options, const InternalKeyComparator& internal_key, - WritableFile* file, CompressionType compression_type) const { - return new SimpleTableBuilder(options, file, compression_type); + const ImmutableCFOptions& ioptions, + const InternalKeyComparator& internal_key, + WritableFile* file, const CompressionType compression_type, + const CompressionOptions& compression_opts) const { + return new SimpleTableBuilder(ioptions, file, compression_type); } class SimpleTableDBTest { diff --git a/db/table_cache.cc b/db/table_cache.cc index c362499a6..5cb96f8bf 100644 --- a/db/table_cache.cc +++ b/db/table_cache.cc @@ -36,12 +36,10 @@ static Slice GetSliceForFileNumber(const uint64_t* file_number) { sizeof(*file_number)); } -TableCache::TableCache(const Options* options, - const EnvOptions& storage_options, Cache* const cache) - : env_(options->env), - db_paths_(options->db_paths), - options_(options), - storage_options_(storage_options), +TableCache::TableCache(const ImmutableCFOptions& ioptions, + const EnvOptions& env_options, Cache* const cache) + : ioptions_(ioptions), + env_options_(env_options), cache_(cache) {} TableCache::~TableCache() { @@ -55,7 +53,7 @@ void TableCache::ReleaseHandle(Cache::Handle* handle) { cache_->Release(handle); } -Status TableCache::FindTable(const EnvOptions& toptions, +Status TableCache::FindTable(const EnvOptions& env_options, const InternalKeyComparator& internal_comparator, const FileDescriptor& fd, Cache::Handle** handle, const bool no_io) { @@ -68,24 +66,24 @@ Status TableCache::FindTable(const EnvOptions& toptions, return Status::Incomplete("Table not found in table_cache, no_io is set"); } std::string fname = - TableFileName(db_paths_, fd.GetNumber(), fd.GetPathId()); + TableFileName(ioptions_.db_paths, fd.GetNumber(), fd.GetPathId()); unique_ptr file; unique_ptr table_reader; - s = env_->NewRandomAccessFile(fname, &file, toptions); - RecordTick(options_->statistics.get(), NO_FILE_OPENS); + s = ioptions_.env->NewRandomAccessFile(fname, &file, env_options); + RecordTick(ioptions_.statistics, NO_FILE_OPENS); if (s.ok()) { - if (options_->advise_random_on_open) { + if (ioptions_.advise_random_on_open) { file->Hint(RandomAccessFile::RANDOM); } - StopWatch sw(env_, options_->statistics.get(), TABLE_OPEN_IO_MICROS); - s = options_->table_factory->NewTableReader( - *options_, toptions, internal_comparator, std::move(file), + StopWatch sw(ioptions_.env, ioptions_.statistics, TABLE_OPEN_IO_MICROS); + s = ioptions_.table_factory->NewTableReader( + ioptions_, env_options, internal_comparator, std::move(file), fd.GetFileSize(), &table_reader); } if (!s.ok()) { assert(table_reader == nullptr); - RecordTick(options_->statistics.get(), NO_FILE_ERRORS); + RecordTick(ioptions_.statistics, NO_FILE_ERRORS); // We do not cache error results so that if the error is transient, // or somebody repairs the file, we recover automatically. } else { @@ -97,7 +95,7 @@ Status TableCache::FindTable(const EnvOptions& toptions, } Iterator* TableCache::NewIterator(const ReadOptions& options, - const EnvOptions& toptions, + const EnvOptions& env_options, const InternalKeyComparator& icomparator, const FileDescriptor& fd, TableReader** table_reader_ptr, @@ -109,7 +107,7 @@ Iterator* TableCache::NewIterator(const ReadOptions& options, Cache::Handle* handle = nullptr; Status s; if (table_reader == nullptr) { - s = FindTable(toptions, icomparator, fd, &handle, + s = FindTable(env_options, icomparator, fd, &handle, options.read_tier == kBlockCacheTier); if (!s.ok()) { return NewErrorIterator(s, arena); @@ -142,7 +140,7 @@ Status TableCache::Get(const ReadOptions& options, Status s; Cache::Handle* handle = nullptr; if (!t) { - s = FindTable(storage_options_, internal_comparator, fd, &handle, + s = FindTable(env_options_, internal_comparator, fd, &handle, options.read_tier == kBlockCacheTier); if (s.ok()) { t = GetTableReaderFromHandle(handle); @@ -160,8 +158,9 @@ Status TableCache::Get(const ReadOptions& options, } return s; } + Status TableCache::GetTableProperties( - const EnvOptions& toptions, + const EnvOptions& env_options, const InternalKeyComparator& internal_comparator, const FileDescriptor& fd, std::shared_ptr* properties, bool no_io) { Status s; @@ -174,7 +173,7 @@ Status TableCache::GetTableProperties( } Cache::Handle* table_handle = nullptr; - s = FindTable(toptions, internal_comparator, fd, &table_handle, no_io); + s = FindTable(env_options, internal_comparator, fd, &table_handle, no_io); if (!s.ok()) { return s; } @@ -186,7 +185,7 @@ Status TableCache::GetTableProperties( } size_t TableCache::GetMemoryUsageByTableReader( - const EnvOptions& toptions, + const EnvOptions& env_options, const InternalKeyComparator& internal_comparator, const FileDescriptor& fd) { Status s; @@ -197,7 +196,7 @@ size_t TableCache::GetMemoryUsageByTableReader( } Cache::Handle* table_handle = nullptr; - s = FindTable(toptions, internal_comparator, fd, &table_handle, true); + s = FindTable(env_options, internal_comparator, fd, &table_handle, true); if (!s.ok()) { return 0; } diff --git a/db/table_cache.h b/db/table_cache.h index 79090e064..2f6740d9f 100644 --- a/db/table_cache.h +++ b/db/table_cache.h @@ -19,6 +19,7 @@ #include "rocksdb/cache.h" #include "rocksdb/env.h" #include "rocksdb/table.h" +#include "rocksdb/options.h" #include "table/table_reader.h" namespace rocksdb { @@ -29,8 +30,8 @@ struct FileDescriptor; class TableCache { public: - TableCache(const Options* options, const EnvOptions& storage_options, - Cache* cache); + TableCache(const ImmutableCFOptions& ioptions, + const EnvOptions& storage_options, Cache* cache); ~TableCache(); // Return an iterator for the specified file number (the corresponding @@ -91,10 +92,8 @@ class TableCache { void ReleaseHandle(Cache::Handle* handle); private: - Env* const env_; - const std::vector db_paths_; - const Options* options_; - const EnvOptions& storage_options_; + const ImmutableCFOptions& ioptions_; + const EnvOptions& env_options_; Cache* const cache_; }; diff --git a/db/table_properties_collector_test.cc b/db/table_properties_collector_test.cc index 638b259f2..8168ca5d6 100644 --- a/db/table_properties_collector_test.cc +++ b/db/table_properties_collector_test.cc @@ -90,7 +90,8 @@ void MakeBuilder(const Options& options, std::unique_ptr* builder) { writable->reset(new FakeWritableFile); builder->reset(options.table_factory->NewTableBuilder( - options, internal_comparator, writable->get(), options.compression)); + ImmutableCFOptions(options), internal_comparator, writable->get(), + options.compression, options.compression_opts)); } } // namespace diff --git a/include/rocksdb/immutable_options.h b/include/rocksdb/immutable_options.h new file mode 100644 index 000000000..22084f6f0 --- /dev/null +++ b/include/rocksdb/immutable_options.h @@ -0,0 +1,62 @@ +// 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 +#include "rocksdb/options.h" + +namespace rocksdb { + +// ImmutableCFOptions is a data struct used by RocksDB internal. It contains a +// subset of Options that should not be changed during the entire lifetime +// of DB. You shouldn't need to access this data structure unless you are +// implementing a new TableFactory. +struct ImmutableCFOptions { + explicit ImmutableCFOptions(const Options& options); + + const SliceTransform* prefix_extractor; + + const Comparator* comparator; + + MergeOperator* merge_operator; + + Logger* info_log; + + Statistics* statistics; + + InfoLogLevel info_log_level; + + Env* env; + + // Allow the OS to mmap file for reading sst tables. Default: false + bool allow_mmap_reads; + + // Allow the OS to mmap file for writing. Default: false + bool allow_mmap_writes; + + std::vector db_paths; + + TableFactory* table_factory; + + Options::TablePropertiesCollectorFactories + table_properties_collector_factories; + + bool advise_random_on_open; + + // This options is required by PlainTableReader. May need to move it + // to PlainTalbeOptions just like bloom_bits_per_key + uint32_t bloom_locality; + + bool purge_redundant_kvs_while_flush; + + uint32_t min_partial_merge_operands; + + bool disable_data_sync; + + bool use_fsync; +}; + +} // namespace rocksdb diff --git a/include/rocksdb/table.h b/include/rocksdb/table.h index 0f8b41074..2fb4f50dd 100644 --- a/include/rocksdb/table.h +++ b/include/rocksdb/table.h @@ -23,6 +23,7 @@ #include "rocksdb/env.h" #include "rocksdb/iterator.h" #include "rocksdb/options.h" +#include "rocksdb/immutable_options.h" #include "rocksdb/status.h" namespace rocksdb { @@ -293,14 +294,15 @@ class TableFactory { // and cache the table object returned. // (1) SstFileReader (for SST Dump) opens the table and dump the table // contents using the interator of the table. - // options and soptions are options. options is the general options. + // ImmutableCFOptions is a subset of Options that can not be altered. + // EnvOptions is a subset of Options that will be used by Env. // Multiple configured can be accessed from there, including and not // limited to block cache and key comparators. // file is a file handler to handle the file for the table // file_size is the physical file size of the file // table_reader is the output table reader virtual Status NewTableReader( - const Options& options, const EnvOptions& soptions, + const ImmutableCFOptions& ioptions, const EnvOptions& env_options, const InternalKeyComparator& internal_comparator, unique_ptr&& file, uint64_t file_size, unique_ptr* table_reader) const = 0; @@ -318,14 +320,17 @@ class TableFactory { // (4) When running Repairer, it creates a table builder to convert logs to // SST files (In Repairer::ConvertLogToTable() by calling BuildTable()) // - // options is the general options. Multiple configured can be acceseed from - // there, including and not limited to compression options. - // file is a handle of a writable file. It is the caller's responsibility to - // keep the file open and close the file after closing the table builder. - // compression_type is the compression type to use in this table. + // ImmutableCFOptions is a subset of Options that can not be altered. + // Multiple configured can be acceseed from there, including and not limited + // to compression options. file is a handle of a writable file. + // It is the caller's responsibility to keep the file open and close the file + // after closing the table builder. compression_type is the compression type + // to use in this table. virtual TableBuilder* NewTableBuilder( - const Options& options, const InternalKeyComparator& internal_comparator, - WritableFile* file, CompressionType compression_type) const = 0; + const ImmutableCFOptions& ioptions, + const InternalKeyComparator& internal_comparator, + WritableFile* file, const CompressionType compression_type, + const CompressionOptions& compression_opts) const = 0; // Sanitizes the specified DB Options. // diff --git a/table/adaptive_table_factory.cc b/table/adaptive_table_factory.cc index a259e79d8..c693064af 100644 --- a/table/adaptive_table_factory.cc +++ b/table/adaptive_table_factory.cc @@ -39,7 +39,7 @@ extern const uint64_t kLegacyBlockBasedTableMagicNumber; extern const uint64_t kCuckooTableMagicNumber; Status AdaptiveTableFactory::NewTableReader( - const Options& options, const EnvOptions& soptions, + const ImmutableCFOptions& ioptions, const EnvOptions& env_options, const InternalKeyComparator& icomp, unique_ptr&& file, uint64_t file_size, unique_ptr* table) const { Footer footer; @@ -50,24 +50,26 @@ Status AdaptiveTableFactory::NewTableReader( if (footer.table_magic_number() == kPlainTableMagicNumber || footer.table_magic_number() == kLegacyPlainTableMagicNumber) { return plain_table_factory_->NewTableReader( - options, soptions, icomp, std::move(file), file_size, table); + ioptions, env_options, icomp, std::move(file), file_size, table); } else if (footer.table_magic_number() == kBlockBasedTableMagicNumber || footer.table_magic_number() == kLegacyBlockBasedTableMagicNumber) { return block_based_table_factory_->NewTableReader( - options, soptions, icomp, std::move(file), file_size, table); + ioptions, env_options, icomp, std::move(file), file_size, table); } else if (footer.table_magic_number() == kCuckooTableMagicNumber) { return cuckoo_table_factory_->NewTableReader( - options, soptions, icomp, std::move(file), file_size, table); + ioptions, env_options, icomp, std::move(file), file_size, table); } else { return Status::NotSupported("Unidentified table format"); } } TableBuilder* AdaptiveTableFactory::NewTableBuilder( - const Options& options, const InternalKeyComparator& internal_comparator, - WritableFile* file, CompressionType compression_type) const { - return table_factory_to_write_->NewTableBuilder(options, internal_comparator, - file, compression_type); + const ImmutableCFOptions& ioptions, + const InternalKeyComparator& internal_comparator, + WritableFile* file, const CompressionType compression_type, + const CompressionOptions& compression_opts) const { + return table_factory_to_write_->NewTableBuilder( + ioptions, internal_comparator, file, compression_type, compression_opts); } std::string AdaptiveTableFactory::GetPrintableTableOptions() const { diff --git a/table/adaptive_table_factory.h b/table/adaptive_table_factory.h index f119d97b1..f0920db97 100644 --- a/table/adaptive_table_factory.h +++ b/table/adaptive_table_factory.h @@ -12,7 +12,6 @@ namespace rocksdb { -struct Options; struct EnvOptions; using std::unique_ptr; @@ -31,16 +30,21 @@ class AdaptiveTableFactory : public TableFactory { std::shared_ptr block_based_table_factory, std::shared_ptr plain_table_factory, std::shared_ptr cuckoo_table_factory); + const char* Name() const override { return "AdaptiveTableFactory"; } - Status NewTableReader(const Options& options, const EnvOptions& soptions, - const InternalKeyComparator& internal_comparator, - unique_ptr&& file, uint64_t file_size, - unique_ptr* table) const override; - TableBuilder* NewTableBuilder(const Options& options, - const InternalKeyComparator& icomparator, - WritableFile* file, - CompressionType compression_type) const - override; + + Status NewTableReader( + const ImmutableCFOptions& ioptions, const EnvOptions& env_options, + const InternalKeyComparator& internal_comparator, + unique_ptr&& file, uint64_t file_size, + unique_ptr* table) const override; + + TableBuilder* NewTableBuilder( + const ImmutableCFOptions& ioptions, + const InternalKeyComparator& icomparator, + WritableFile* file, + const CompressionType compression_type, + const CompressionOptions& compression_opts) const override; // Sanitizes the specified DB Options. Status SanitizeDBOptions(const DBOptions* db_opts) const override { diff --git a/table/block_based_table_builder.cc b/table/block_based_table_builder.cc index ddfbe74a6..fde363760 100644 --- a/table/block_based_table_builder.cc +++ b/table/block_based_table_builder.cc @@ -25,7 +25,6 @@ #include "rocksdb/env.h" #include "rocksdb/filter_policy.h" #include "rocksdb/flush_block_policy.h" -#include "rocksdb/options.h" #include "rocksdb/table.h" #include "table/block.h" @@ -385,7 +384,7 @@ class BlockBasedTableBuilder::BlockBasedTablePropertiesCollector }; struct BlockBasedTableBuilder::Rep { - const Options options; + const ImmutableCFOptions ioptions; const BlockBasedTableOptions table_options; const InternalKeyComparator& internal_comparator; WritableFile* file; @@ -397,7 +396,8 @@ struct BlockBasedTableBuilder::Rep { std::unique_ptr index_builder; std::string last_key; - CompressionType compression_type; + const CompressionType compression_type; + const CompressionOptions compression_opts; TableProperties props; bool closed = false; // Either Finish() or Abandon() has been called. @@ -413,27 +413,31 @@ struct BlockBasedTableBuilder::Rep { std::vector> table_properties_collectors; - Rep(const Options& opt, const BlockBasedTableOptions& table_opt, + Rep(const ImmutableCFOptions& ioptions, + const BlockBasedTableOptions& table_opt, const InternalKeyComparator& icomparator, - WritableFile* f, CompressionType compression_type) - : options(opt), + WritableFile* f, const CompressionType compression_type, + const CompressionOptions& compression_opts) + : ioptions(ioptions), table_options(table_opt), internal_comparator(icomparator), file(f), data_block(table_options.block_restart_interval), - internal_prefix_transform(options.prefix_extractor.get()), + internal_prefix_transform(ioptions.prefix_extractor), index_builder(CreateIndexBuilder( table_options.index_type, &internal_comparator, &this->internal_prefix_transform)), compression_type(compression_type), + compression_opts(compression_opts), filter_block(table_options.filter_policy == nullptr ? nullptr : - new FilterBlockBuilder(opt, table_options, &internal_comparator)), + new FilterBlockBuilder(ioptions.prefix_extractor, + table_options, &internal_comparator)), flush_block_policy( table_options.flush_block_policy_factory->NewFlushBlockPolicy( table_options, data_block)) { for (auto& collector_factories : - options.table_properties_collector_factories) { + ioptions.table_properties_collector_factories) { table_properties_collectors.emplace_back( collector_factories->CreateTablePropertiesCollector()); } @@ -443,11 +447,13 @@ struct BlockBasedTableBuilder::Rep { }; BlockBasedTableBuilder::BlockBasedTableBuilder( - const Options& options, const BlockBasedTableOptions& table_options, + const ImmutableCFOptions& ioptions, + const BlockBasedTableOptions& table_options, const InternalKeyComparator& internal_comparator, WritableFile* file, - CompressionType compression_type) - : rep_(new Rep(options, table_options, internal_comparator, - file, compression_type)) { + const CompressionType compression_type, + const CompressionOptions& compression_opts) + : rep_(new Rep(ioptions, table_options, internal_comparator, + file, compression_type, compression_opts)) { if (rep_->filter_block != nullptr) { rep_->filter_block->StartBlock(0); } @@ -502,7 +508,7 @@ void BlockBasedTableBuilder::Add(const Slice& key, const Slice& value) { r->index_builder->OnKeyAdded(key); NotifyCollectTableCollectorsOnAdd(key, value, r->table_properties_collectors, - r->options.info_log.get()); + r->ioptions.info_log); } void BlockBasedTableBuilder::Flush() { @@ -540,10 +546,10 @@ void BlockBasedTableBuilder::WriteBlock(const Slice& raw_block_contents, Slice block_contents; if (raw_block_contents.size() < kCompressionSizeLimit) { block_contents = - CompressBlock(raw_block_contents, r->options.compression_opts, &type, + CompressBlock(raw_block_contents, r->compression_opts, &type, &r->compressed_output); } else { - RecordTick(r->options.statistics.get(), NUMBER_BLOCK_NOT_COMPRESSED); + RecordTick(r->ioptions.statistics, NUMBER_BLOCK_NOT_COMPRESSED); type = kNoCompression; block_contents = raw_block_contents; } @@ -555,8 +561,7 @@ void BlockBasedTableBuilder::WriteRawBlock(const Slice& block_contents, CompressionType type, BlockHandle* handle) { Rep* r = rep_; - StopWatch sw(r->options.env, r->options.statistics.get(), - WRITE_RAW_BLOCK_MICROS); + StopWatch sw(r->ioptions.env, r->ioptions.statistics, WRITE_RAW_BLOCK_MICROS); handle->set_offset(r->offset); handle->set_size(block_contents.size()); r->status = r->file->Append(block_contents); @@ -717,7 +722,7 @@ Status BlockBasedTableBuilder::Finish() { // Add use collected properties NotifyCollectTableCollectorsOnFinish(r->table_properties_collectors, - r->options.info_log.get(), + r->ioptions.info_log, &property_block_builder); BlockHandle properties_block_handle; @@ -776,14 +781,12 @@ Status BlockBasedTableBuilder::Finish() { } } - Log( - r->options.info_log, + Log(r->ioptions.info_log, "Table was constructed:\n" " [basic properties]: %s\n" " [user collected properties]: %s", r->props.ToString().c_str(), - user_collected.c_str() - ); + user_collected.c_str()); } return r->status; diff --git a/table/block_based_table_builder.h b/table/block_based_table_builder.h index 72a2f207a..6fde32919 100644 --- a/table/block_based_table_builder.h +++ b/table/block_based_table_builder.h @@ -28,10 +28,12 @@ class BlockBasedTableBuilder : public TableBuilder { // Create a builder that will store the contents of the table it is // building in *file. Does not close the file. It is up to the // caller to close the file after calling Finish(). - BlockBasedTableBuilder(const Options& options, + BlockBasedTableBuilder(const ImmutableCFOptions& ioptions, const BlockBasedTableOptions& table_options, const InternalKeyComparator& internal_comparator, - WritableFile* file, CompressionType compression_type); + WritableFile* file, + const CompressionType compression_type, + const CompressionOptions& compression_opts); // REQUIRES: Either Finish() or Abandon() has been called. ~BlockBasedTableBuilder(); diff --git a/table/block_based_table_factory.cc b/table/block_based_table_factory.cc index de30fb383..b4e2e7d1f 100644 --- a/table/block_based_table_factory.cc +++ b/table/block_based_table_factory.cc @@ -41,21 +41,24 @@ BlockBasedTableFactory::BlockBasedTableFactory( } Status BlockBasedTableFactory::NewTableReader( - const Options& options, const EnvOptions& soptions, + const ImmutableCFOptions& ioptions, const EnvOptions& soptions, const InternalKeyComparator& internal_comparator, unique_ptr&& file, uint64_t file_size, unique_ptr* table_reader) const { - return BlockBasedTable::Open(options, soptions, table_options_, + return BlockBasedTable::Open(ioptions, soptions, table_options_, internal_comparator, std::move(file), file_size, table_reader); } TableBuilder* BlockBasedTableFactory::NewTableBuilder( - const Options& options, const InternalKeyComparator& internal_comparator, - WritableFile* file, CompressionType compression_type) const { + const ImmutableCFOptions& ioptions, + const InternalKeyComparator& internal_comparator, + WritableFile* file, const CompressionType compression_type, + const CompressionOptions& compression_opts) const { auto table_builder = new BlockBasedTableBuilder( - options, table_options_, internal_comparator, file, compression_type); + ioptions, table_options_, internal_comparator, file, + compression_type, compression_opts); return table_builder; } diff --git a/table/block_based_table_factory.h b/table/block_based_table_factory.h index d7045346a..2dcfda6d4 100644 --- a/table/block_based_table_factory.h +++ b/table/block_based_table_factory.h @@ -14,13 +14,11 @@ #include #include "rocksdb/flush_block_policy.h" -#include "rocksdb/options.h" #include "rocksdb/table.h" #include "db/dbformat.h" namespace rocksdb { -struct Options; struct EnvOptions; using std::unique_ptr; @@ -35,14 +33,17 @@ class BlockBasedTableFactory : public TableFactory { const char* Name() const override { return "BlockBasedTable"; } - Status NewTableReader(const Options& options, const EnvOptions& soptions, - const InternalKeyComparator& internal_comparator, - unique_ptr&& file, uint64_t file_size, - unique_ptr* table_reader) const override; + Status NewTableReader( + const ImmutableCFOptions& ioptions, const EnvOptions& soptions, + const InternalKeyComparator& internal_comparator, + unique_ptr&& file, uint64_t file_size, + unique_ptr* table_reader) const override; TableBuilder* NewTableBuilder( - const Options& options, const InternalKeyComparator& internal_comparator, - WritableFile* file, CompressionType compression_type) const override; + const ImmutableCFOptions& ioptions, + const InternalKeyComparator& internal_comparator, + WritableFile* file, const CompressionType compression_type, + const CompressionOptions& compression_opts) const override; // Sanitizes the specified DB Options. Status SanitizeDBOptions(const DBOptions* db_opts) const override { diff --git a/table/block_based_table_reader.cc b/table/block_based_table_reader.cc index 0be38a1dc..cf915e105 100644 --- a/table/block_based_table_reader.cc +++ b/table/block_based_table_reader.cc @@ -336,15 +336,16 @@ class HashIndexReader : public IndexReader { struct BlockBasedTable::Rep { - Rep(const EnvOptions& storage_options, + Rep(const ImmutableCFOptions& ioptions, + const EnvOptions& env_options, const BlockBasedTableOptions& table_opt, const InternalKeyComparator& internal_comparator) - : soptions(storage_options), table_options(table_opt), + : ioptions(ioptions), env_options(env_options), table_options(table_opt), filter_policy(table_opt.filter_policy.get()), internal_comparator(internal_comparator) {} - Options options; - const EnvOptions& soptions; + const ImmutableCFOptions& ioptions; + const EnvOptions& env_options; const BlockBasedTableOptions& table_options; const FilterPolicy* const filter_policy; const InternalKeyComparator& internal_comparator; @@ -446,7 +447,8 @@ void BlockBasedTable::GenerateCachePrefix(Cache* cc, } } -Status BlockBasedTable::Open(const Options& options, const EnvOptions& soptions, +Status BlockBasedTable::Open(const ImmutableCFOptions& ioptions, + const EnvOptions& env_options, const BlockBasedTableOptions& table_options, const InternalKeyComparator& internal_comparator, unique_ptr&& file, @@ -461,8 +463,7 @@ Status BlockBasedTable::Open(const Options& options, const EnvOptions& soptions, // We've successfully read the footer and the index block: we're // ready to serve requests. Rep* rep = new BlockBasedTable::Rep( - soptions, table_options, internal_comparator); - rep->options = options; + ioptions, env_options, table_options, internal_comparator); rep->file = std::move(file); rep->footer = footer; rep->index_type = table_options.index_type; @@ -484,7 +485,7 @@ Status BlockBasedTable::Open(const Options& options, const EnvOptions& soptions, TableProperties* table_properties = nullptr; if (s.ok()) { s = ReadProperties(meta_iter->value(), rep->file.get(), rep->footer, - rep->options.env, rep->options.info_log.get(), + rep->ioptions.env, rep->ioptions.info_log, &table_properties); } @@ -492,12 +493,12 @@ Status BlockBasedTable::Open(const Options& options, const EnvOptions& soptions, auto err_msg = "[Warning] Encountered error while reading data from properties " "block " + s.ToString(); - Log(rep->options.info_log, "%s", err_msg.c_str()); + Log(rep->ioptions.info_log, "%s", err_msg.c_str()); } else { rep->table_properties.reset(table_properties); } } else { - Log(WARN_LEVEL, rep->options.info_log, + Log(WARN_LEVEL, rep->ioptions.info_log, "Cannot find Properties block from file."); } @@ -546,7 +547,8 @@ Status BlockBasedTable::Open(const Options& options, const EnvOptions& soptions, } void BlockBasedTable::SetupForCompaction() { - switch (rep_->options.access_hint_on_compaction_start) { + /* + switch (.access_hint_on_compaction_start) { case Options::NONE: break; case Options::NORMAL: @@ -562,6 +564,7 @@ void BlockBasedTable::SetupForCompaction() { assert(false); } compaction_optimized_ = true; + */ } std::shared_ptr BlockBasedTable::GetTableProperties() @@ -596,13 +599,13 @@ Status BlockBasedTable::ReadMetaBlock( ReadOptions(), rep->footer.metaindex_handle(), &meta, - rep->options.env); + rep->ioptions.env); if (!s.ok()) { auto err_msg = "[Warning] Encountered error while reading data from properties" "block " + s.ToString(); - Log(rep->options.info_log, "%s", err_msg.c_str()); + Log(rep->ioptions.info_log, "%s", err_msg.c_str()); } if (!s.ok()) { delete meta; @@ -746,7 +749,7 @@ FilterBlockReader* BlockBasedTable::ReadFilter(const BlockHandle& filter_handle, ReadOptions opt; BlockContents block; if (!ReadBlockContents(rep->file.get(), rep->footer, opt, filter_handle, - &block, rep->options.env, false).ok()) { + &block, rep->ioptions.env, false).ok()) { return nullptr; } @@ -755,7 +758,8 @@ FilterBlockReader* BlockBasedTable::ReadFilter(const BlockHandle& filter_handle, } return new FilterBlockReader( - rep->options, rep->table_options, block.data, block.heap_allocated); + rep->ioptions.prefix_extractor, rep->table_options, + block.data, block.heap_allocated); } BlockBasedTable::CachableEntry BlockBasedTable::GetFilter( @@ -780,7 +784,7 @@ BlockBasedTable::CachableEntry BlockBasedTable::GetFilter( cache_key ); - Statistics* statistics = rep_->options.statistics.get(); + Statistics* statistics = rep_->ioptions.statistics; auto cache_handle = GetEntryFromCache(block_cache, key, BLOCK_CACHE_FILTER_MISS, BLOCK_CACHE_FILTER_HIT, statistics); @@ -830,7 +834,7 @@ Iterator* BlockBasedTable::NewIndexIterator(const ReadOptions& read_options, char cache_key[kMaxCacheKeyPrefixSize + kMaxVarint64Length]; auto key = GetCacheKey(rep_->cache_key_prefix, rep_->cache_key_prefix_size, rep_->footer.index_handle(), cache_key); - Statistics* statistics = rep_->options.statistics.get(); + Statistics* statistics = rep_->ioptions.statistics; auto cache_handle = GetEntryFromCache(block_cache, key, BLOCK_CACHE_INDEX_MISS, BLOCK_CACHE_INDEX_HIT, statistics); @@ -906,7 +910,7 @@ Iterator* BlockBasedTable::NewDataBlockIterator(Rep* rep, // If either block cache is enabled, we'll try to read from it. if (block_cache != nullptr || block_cache_compressed != nullptr) { - Statistics* statistics = rep->options.statistics.get(); + Statistics* statistics = rep->ioptions.statistics; char cache_key[kMaxCacheKeyPrefixSize + kMaxVarint64Length]; char compressed_cache_key[kMaxCacheKeyPrefixSize + kMaxVarint64Length]; Slice key, /* key to the block cache */ @@ -930,9 +934,9 @@ Iterator* BlockBasedTable::NewDataBlockIterator(Rep* rep, if (block.value == nullptr && !no_io && ro.fill_cache) { Block* raw_block = nullptr; { - StopWatch sw(rep->options.env, statistics, READ_BLOCK_GET_MICROS); + StopWatch sw(rep->ioptions.env, statistics, READ_BLOCK_GET_MICROS); s = ReadBlockFromFile(rep->file.get(), rep->footer, ro, handle, - &raw_block, rep->options.env, + &raw_block, rep->ioptions.env, block_cache_compressed == nullptr); } @@ -955,7 +959,7 @@ Iterator* BlockBasedTable::NewDataBlockIterator(Rep* rep, } } s = ReadBlockFromFile(rep->file.get(), rep->footer, ro, handle, - &block.value, rep->options.env); + &block.value, rep->ioptions.env); } Iterator* iter; @@ -982,7 +986,8 @@ class BlockBasedTable::BlockEntryIteratorState : public TwoLevelIteratorState { public: BlockEntryIteratorState(BlockBasedTable* table, const ReadOptions& read_options) - : TwoLevelIteratorState(table->rep_->options.prefix_extractor != nullptr), + : TwoLevelIteratorState( + table->rep_->ioptions.prefix_extractor != nullptr), table_(table), read_options_(read_options) {} @@ -1020,8 +1025,8 @@ bool BlockBasedTable::PrefixMayMatch(const Slice& internal_key) { return true; } - assert(rep_->options.prefix_extractor != nullptr); - auto prefix = rep_->options.prefix_extractor->Transform( + assert(rep_->ioptions.prefix_extractor != nullptr); + auto prefix = rep_->ioptions.prefix_extractor->Transform( ExtractUserKey(internal_key)); InternalKey internal_key_prefix(prefix, 0, kTypeValue); auto internal_prefix = internal_key_prefix.Encode(); @@ -1072,7 +1077,7 @@ bool BlockBasedTable::PrefixMayMatch(const Slice& internal_key) { filter_entry.Release(rep_->table_options.block_cache.get()); } - Statistics* statistics = rep_->options.statistics.get(); + Statistics* statistics = rep_->ioptions.statistics; RecordTick(statistics, BLOOM_FILTER_PREFIX_CHECKED); if (!may_match) { RecordTick(statistics, BLOOM_FILTER_PREFIX_USEFUL); @@ -1111,7 +1116,7 @@ Status BlockBasedTable::Get( // Not found // TODO: think about interaction with Merge. If a user key cannot // cross one data block, we should be fine. - RecordTick(rep_->options.statistics.get(), BLOOM_FILTER_USEFUL); + RecordTick(rep_->ioptions.statistics, BLOOM_FILTER_USEFUL); break; } else { BlockIter biter; @@ -1205,13 +1210,13 @@ Status BlockBasedTable::CreateIndexReader(IndexReader** index_reader, } auto file = rep_->file.get(); - auto env = rep_->options.env; + auto env = rep_->ioptions.env; auto comparator = &rep_->internal_comparator; const Footer& footer = rep_->footer; if (index_type_on_file == BlockBasedTableOptions::kHashSearch && - rep_->options.prefix_extractor == nullptr) { - Log(rep_->options.info_log, + rep_->ioptions.prefix_extractor == nullptr) { + Log(rep_->ioptions.info_log, "BlockBasedTableOptions::kHashSearch requires " "options.prefix_extractor to be set." " Fall back to binary seach index."); @@ -1232,7 +1237,7 @@ Status BlockBasedTable::CreateIndexReader(IndexReader** index_reader, if (!s.ok()) { // we simply fall back to binary search in case there is any // problem with prefix hash index loading. - Log(rep_->options.info_log, + Log(rep_->ioptions.info_log, "Unable to read the metaindex block." " Fall back to binary seach index."); return BinarySearchIndexReader::Create( @@ -1244,7 +1249,7 @@ Status BlockBasedTable::CreateIndexReader(IndexReader** index_reader, // We need to wrap data with internal_prefix_transform to make sure it can // handle prefix correctly. rep_->internal_prefix_transform.reset( - new InternalKeySliceTransform(rep_->options.prefix_extractor.get())); + new InternalKeySliceTransform(rep_->ioptions.prefix_extractor)); return HashIndexReader::Create( rep_->internal_prefix_transform.get(), footer, file, env, comparator, footer.index_handle(), meta_index_iter, index_reader, diff --git a/table/block_based_table_reader.h b/table/block_based_table_reader.h index 3ff97dda6..b5686d265 100644 --- a/table/block_based_table_reader.h +++ b/table/block_based_table_reader.h @@ -14,6 +14,7 @@ #include #include +#include "rocksdb/options.h" #include "rocksdb/statistics.h" #include "rocksdb/status.h" #include "rocksdb/table.h" @@ -36,7 +37,6 @@ class TableReader; class WritableFile; struct BlockBasedTableOptions; struct EnvOptions; -struct Options; struct ReadOptions; using std::unique_ptr; @@ -58,7 +58,8 @@ class BlockBasedTable : public TableReader { // to nullptr and returns a non-ok status. // // *file must remain live while this Table is in use. - static Status Open(const Options& db_options, const EnvOptions& env_options, + static Status Open(const ImmutableCFOptions& ioptions, + const EnvOptions& env_options, const BlockBasedTableOptions& table_options, const InternalKeyComparator& internal_key_comparator, unique_ptr&& file, uint64_t file_size, diff --git a/table/cuckoo_table_factory.cc b/table/cuckoo_table_factory.cc index e2cc6fd89..5727a91c0 100644 --- a/table/cuckoo_table_factory.cc +++ b/table/cuckoo_table_factory.cc @@ -11,11 +11,12 @@ #include "table/cuckoo_table_reader.h" namespace rocksdb { -Status CuckooTableFactory::NewTableReader(const Options& options, - const EnvOptions& soptions, const InternalKeyComparator& icomp, + +Status CuckooTableFactory::NewTableReader(const ImmutableCFOptions& ioptions, + const EnvOptions& env_options, const InternalKeyComparator& icomp, std::unique_ptr&& file, uint64_t file_size, std::unique_ptr* table) const { - std::unique_ptr new_reader(new CuckooTableReader(options, + std::unique_ptr new_reader(new CuckooTableReader(ioptions, std::move(file), file_size, icomp.user_comparator(), nullptr)); Status s = new_reader->status(); if (s.ok()) { @@ -25,10 +26,13 @@ Status CuckooTableFactory::NewTableReader(const Options& options, } TableBuilder* CuckooTableFactory::NewTableBuilder( - const Options& options, const InternalKeyComparator& internal_comparator, - WritableFile* file, CompressionType compression_type) const { - return new CuckooTableBuilder(file, hash_table_ratio_, 64, max_search_depth_, - internal_comparator.user_comparator(), cuckoo_block_size_, nullptr); + const ImmutableCFOptions& ioptions, + const InternalKeyComparator& internal_comparator, + WritableFile* file, const CompressionType, + const CompressionOptions&) const { + return new CuckooTableBuilder(file, hash_table_ratio_, 64, + max_search_depth_, internal_comparator.user_comparator(), + cuckoo_block_size_, nullptr); } std::string CuckooTableFactory::GetPrintableTableOptions() const { diff --git a/table/cuckoo_table_factory.h b/table/cuckoo_table_factory.h index 5799a7f23..2b575dc45 100644 --- a/table/cuckoo_table_factory.h +++ b/table/cuckoo_table_factory.h @@ -9,6 +9,7 @@ #include #include "rocksdb/table.h" #include "util/murmurhash.h" +#include "rocksdb/options.h" namespace rocksdb { @@ -45,14 +46,14 @@ class CuckooTableFactory : public TableFactory { const char* Name() const override { return "CuckooTable"; } Status NewTableReader( - const Options& options, const EnvOptions& soptions, + const ImmutableCFOptions& ioptions, const EnvOptions& env_options, const InternalKeyComparator& internal_comparator, unique_ptr&& file, uint64_t file_size, unique_ptr* table) const override; - TableBuilder* NewTableBuilder(const Options& options, + TableBuilder* NewTableBuilder(const ImmutableCFOptions& options, const InternalKeyComparator& icomparator, WritableFile* file, - CompressionType compression_type) const override; + const CompressionType, const CompressionOptions&) const override; // Sanitizes the specified DB Options. Status SanitizeDBOptions(const DBOptions* db_opts) const override { diff --git a/table/cuckoo_table_reader.cc b/table/cuckoo_table_reader.cc index f1dcbc3bb..1fdbc4475 100644 --- a/table/cuckoo_table_reader.cc +++ b/table/cuckoo_table_reader.cc @@ -29,7 +29,7 @@ namespace { extern const uint64_t kCuckooTableMagicNumber; CuckooTableReader::CuckooTableReader( - const Options& options, + const ImmutableCFOptions& ioptions, std::unique_ptr&& file, uint64_t file_size, const Comparator* comparator, @@ -37,12 +37,12 @@ CuckooTableReader::CuckooTableReader( : file_(std::move(file)), ucomp_(comparator), get_slice_hash_(get_slice_hash) { - if (!options.allow_mmap_reads) { + if (!ioptions.allow_mmap_reads) { status_ = Status::InvalidArgument("File is not mmaped"); } TableProperties* props = nullptr; status_ = ReadTableProperties(file_.get(), file_size, kCuckooTableMagicNumber, - options.env, options.info_log.get(), &props); + ioptions.env, ioptions.info_log, &props); if (!status_.ok()) { return; } diff --git a/table/cuckoo_table_reader.h b/table/cuckoo_table_reader.h index 05d5c3397..61e048eb6 100644 --- a/table/cuckoo_table_reader.h +++ b/table/cuckoo_table_reader.h @@ -16,6 +16,7 @@ #include "db/dbformat.h" #include "rocksdb/env.h" +#include "rocksdb/options.h" #include "table/table_reader.h" namespace rocksdb { @@ -26,7 +27,7 @@ class TableReader; class CuckooTableReader: public TableReader { public: CuckooTableReader( - const Options& options, + const ImmutableCFOptions& ioptions, std::unique_ptr&& file, uint64_t file_size, const Comparator* user_comparator, @@ -40,7 +41,7 @@ class CuckooTableReader: public TableReader { Status status() const { return status_; } Status Get( - const ReadOptions& readOptions, const Slice& key, void* handle_context, + const ReadOptions& read_options, const Slice& key, void* handle_context, bool (*result_handler)(void* arg, const ParsedInternalKey& k, const Slice& v), void (*mark_key_may_exist_handler)(void* handle_context) = nullptr) diff --git a/table/cuckoo_table_reader_test.cc b/table/cuckoo_table_reader_test.cc index 63fe0ae5b..53946e71b 100644 --- a/table/cuckoo_table_reader_test.cc +++ b/table/cuckoo_table_reader_test.cc @@ -121,8 +121,9 @@ class CuckooReaderTest { // Check reader now. std::unique_ptr read_file; ASSERT_OK(env->NewRandomAccessFile(fname, &read_file, env_options)); + const ImmutableCFOptions ioptions(options); CuckooTableReader reader( - options, + ioptions, std::move(read_file), file_size, ucomp, @@ -147,8 +148,9 @@ class CuckooReaderTest { void CheckIterator(const Comparator* ucomp = BytewiseComparator()) { std::unique_ptr read_file; ASSERT_OK(env->NewRandomAccessFile(fname, &read_file, env_options)); + const ImmutableCFOptions ioptions(options); CuckooTableReader reader( - options, + ioptions, std::move(read_file), file_size, ucomp, @@ -325,8 +327,9 @@ TEST(CuckooReaderTest, WhenKeyNotFound) { CreateCuckooFileAndCheckReader(); std::unique_ptr read_file; ASSERT_OK(env->NewRandomAccessFile(fname, &read_file, env_options)); + const ImmutableCFOptions ioptions(options); CuckooTableReader reader( - options, + ioptions, std::move(read_file), file_size, BytewiseComparator(), @@ -433,8 +436,9 @@ void WriteFile(const std::vector& keys, std::unique_ptr read_file; ASSERT_OK(env->NewRandomAccessFile(fname, &read_file, env_options)); + const ImmutableCFOptions ioptions(options); CuckooTableReader reader( - options, std::move(read_file), file_size, + ioptions, std::move(read_file), file_size, test::Uint64Comparator(), nullptr); ASSERT_OK(reader.status()); ReadOptions r_options; @@ -460,8 +464,9 @@ void ReadKeys(uint64_t num, uint32_t batch_size) { std::unique_ptr read_file; ASSERT_OK(env->NewRandomAccessFile(fname, &read_file, env_options)); + const ImmutableCFOptions ioptions(options); CuckooTableReader reader( - options, std::move(read_file), file_size, test::Uint64Comparator(), + ioptions, std::move(read_file), file_size, test::Uint64Comparator(), nullptr); ASSERT_OK(reader.status()); const UserCollectedProperties user_props = diff --git a/table/filter_block.cc b/table/filter_block.cc index 6b4ff1c10..30284017b 100644 --- a/table/filter_block.cc +++ b/table/filter_block.cc @@ -21,11 +21,11 @@ namespace rocksdb { static const size_t kFilterBaseLg = 11; static const size_t kFilterBase = 1 << kFilterBaseLg; -FilterBlockBuilder::FilterBlockBuilder(const Options& opt, +FilterBlockBuilder::FilterBlockBuilder(const SliceTransform* prefix_extractor, const BlockBasedTableOptions& table_opt, const Comparator* internal_comparator) : policy_(table_opt.filter_policy.get()), - prefix_extractor_(opt.prefix_extractor.get()), + prefix_extractor_(prefix_extractor), whole_key_filtering_(table_opt.whole_key_filtering), comparator_(internal_comparator) {} @@ -126,10 +126,11 @@ void FilterBlockBuilder::GenerateFilter() { } FilterBlockReader::FilterBlockReader( - const Options& opt, const BlockBasedTableOptions& table_opt, + const SliceTransform* prefix_extractor, + const BlockBasedTableOptions& table_opt, const Slice& contents, bool delete_contents_after_use) : policy_(table_opt.filter_policy.get()), - prefix_extractor_(opt.prefix_extractor.get()), + prefix_extractor_(prefix_extractor), whole_key_filtering_(table_opt.whole_key_filtering), data_(nullptr), offset_(nullptr), diff --git a/table/filter_block.h b/table/filter_block.h index 5041393f6..efee5ac71 100644 --- a/table/filter_block.h +++ b/table/filter_block.h @@ -18,7 +18,6 @@ #include #include #include -#include "rocksdb/options.h" #include "rocksdb/slice.h" #include "rocksdb/slice_transform.h" #include "rocksdb/table.h" @@ -36,7 +35,7 @@ class FilterPolicy; // (StartBlock AddKey*)* Finish class FilterBlockBuilder { public: - explicit FilterBlockBuilder(const Options& opt, + explicit FilterBlockBuilder(const SliceTransform* prefix_extractor, const BlockBasedTableOptions& table_opt, const Comparator* internal_comparator); @@ -71,7 +70,7 @@ class FilterBlockReader { public: // REQUIRES: "contents" and *policy must stay live while *this is live. FilterBlockReader( - const Options& opt, + const SliceTransform* prefix_extractor, const BlockBasedTableOptions& table_opt, const Slice& contents, bool delete_contents_after_use = false); diff --git a/table/filter_block_test.cc b/table/filter_block_test.cc index 95496a82c..903247e80 100644 --- a/table/filter_block_test.cc +++ b/table/filter_block_test.cc @@ -45,26 +45,26 @@ class TestHashFilter : public FilterPolicy { class FilterBlockTest { public: - Options options_; + const Comparator* comparator_; BlockBasedTableOptions table_options_; - FilterBlockTest() { - options_ = Options(); + FilterBlockTest() + : comparator_(BytewiseComparator()) { table_options_.filter_policy.reset(new TestHashFilter()); } }; TEST(FilterBlockTest, EmptyBuilder) { - FilterBlockBuilder builder(options_, table_options_, options_.comparator); + FilterBlockBuilder builder(nullptr, table_options_, comparator_); Slice block = builder.Finish(); ASSERT_EQ("\\x00\\x00\\x00\\x00\\x0b", EscapeString(block)); - FilterBlockReader reader(options_, table_options_, block); + FilterBlockReader reader(nullptr, table_options_, block); ASSERT_TRUE(reader.KeyMayMatch(0, "foo")); ASSERT_TRUE(reader.KeyMayMatch(100000, "foo")); } TEST(FilterBlockTest, SingleChunk) { - FilterBlockBuilder builder(options_, table_options_, options_.comparator); + FilterBlockBuilder builder(nullptr, table_options_, comparator_); builder.StartBlock(100); builder.AddKey("foo"); builder.AddKey("bar"); @@ -74,7 +74,7 @@ TEST(FilterBlockTest, SingleChunk) { builder.StartBlock(300); builder.AddKey("hello"); Slice block = builder.Finish(); - FilterBlockReader reader(options_, table_options_, block); + FilterBlockReader reader(nullptr, table_options_, block); ASSERT_TRUE(reader.KeyMayMatch(100, "foo")); ASSERT_TRUE(reader.KeyMayMatch(100, "bar")); ASSERT_TRUE(reader.KeyMayMatch(100, "box")); @@ -85,7 +85,7 @@ TEST(FilterBlockTest, SingleChunk) { } TEST(FilterBlockTest, MultiChunk) { - FilterBlockBuilder builder(options_, table_options_, options_.comparator); + FilterBlockBuilder builder(nullptr, table_options_, comparator_); // First filter builder.StartBlock(0); @@ -105,7 +105,7 @@ TEST(FilterBlockTest, MultiChunk) { builder.AddKey("hello"); Slice block = builder.Finish(); - FilterBlockReader reader(options_, table_options_, block); + FilterBlockReader reader(nullptr, table_options_, block); // Check first filter ASSERT_TRUE(reader.KeyMayMatch(0, "foo")); diff --git a/table/plain_table_builder.cc b/table/plain_table_builder.cc index 4f3b62ad4..49489ed64 100644 --- a/table/plain_table_builder.cc +++ b/table/plain_table_builder.cc @@ -58,24 +58,24 @@ extern const uint64_t kPlainTableMagicNumber = 0x8242229663bf9564ull; extern const uint64_t kLegacyPlainTableMagicNumber = 0x4f3418eb7a8f13b8ull; PlainTableBuilder::PlainTableBuilder( - const Options& options, WritableFile* file, uint32_t user_key_len, - EncodingType encoding_type, size_t index_sparseness, + const ImmutableCFOptions& ioptions, WritableFile* file, + uint32_t user_key_len, EncodingType encoding_type, size_t index_sparseness, uint32_t bloom_bits_per_key, uint32_t num_probes, size_t huge_page_tlb_size, double hash_table_ratio, bool store_index_in_file) - : options_(options), + : ioptions_(ioptions), bloom_block_(num_probes), file_(file), bloom_bits_per_key_(bloom_bits_per_key), huge_page_tlb_size_(huge_page_tlb_size), - encoder_(encoding_type, user_key_len, options.prefix_extractor.get(), + encoder_(encoding_type, user_key_len, ioptions.prefix_extractor, index_sparseness), store_index_in_file_(store_index_in_file), - prefix_extractor_(options.prefix_extractor.get()) { + prefix_extractor_(ioptions.prefix_extractor) { // Build index block and save it in the file if hash_table_ratio > 0 if (store_index_in_file_) { assert(hash_table_ratio > 0 || IsTotalOrderMode()); index_builder_.reset( - new PlainTableIndexBuilder(&arena_, options, index_sparseness, + new PlainTableIndexBuilder(&arena_, ioptions, index_sparseness, hash_table_ratio, huge_page_tlb_size_)); assert(bloom_bits_per_key_ > 0); properties_.user_collected_properties @@ -93,10 +93,10 @@ PlainTableBuilder::PlainTableBuilder( // plain encoding. properties_.format_version = (encoding_type == kPlain) ? 0 : 1; - if (options_.prefix_extractor) { + if (ioptions_.prefix_extractor) { properties_.user_collected_properties [PlainTablePropertyNames::kPrefixExtractorName] = - options_.prefix_extractor->Name(); + ioptions_.prefix_extractor->Name(); } std::string val; @@ -105,7 +105,7 @@ PlainTableBuilder::PlainTableBuilder( [PlainTablePropertyNames::kEncodingType] = val; for (auto& collector_factories : - options.table_properties_collector_factories) { + ioptions.table_properties_collector_factories) { table_properties_collectors_.emplace_back( collector_factories->CreateTablePropertiesCollector()); } @@ -124,11 +124,11 @@ void PlainTableBuilder::Add(const Slice& key, const Slice& value) { // Store key hash if (store_index_in_file_) { - if (options_.prefix_extractor.get() == nullptr) { + if (ioptions_.prefix_extractor == nullptr) { keys_or_prefixes_hashes_.push_back(GetSliceHash(internal_key.user_key)); } else { Slice prefix = - options_.prefix_extractor->Transform(internal_key.user_key); + ioptions_.prefix_extractor->Transform(internal_key.user_key); keys_or_prefixes_hashes_.push_back(GetSliceHash(prefix)); } } @@ -160,7 +160,7 @@ void PlainTableBuilder::Add(const Slice& key, const Slice& value) { // notify property collectors NotifyCollectTableCollectorsOnAdd(key, value, table_properties_collectors_, - options_.info_log.get()); + ioptions_.info_log); } Status PlainTableBuilder::status() const { return status_; } @@ -183,7 +183,8 @@ Status PlainTableBuilder::Finish() { if (store_index_in_file_ && (properties_.num_entries > 0)) { bloom_block_.SetTotalBits( &arena_, properties_.num_entries * bloom_bits_per_key_, - options_.bloom_locality, huge_page_tlb_size_, options_.info_log.get()); + ioptions_.bloom_locality, huge_page_tlb_size_, + ioptions_.info_log); PutVarint32(&properties_.user_collected_properties [PlainTablePropertyNames::kNumBloomBlocks], @@ -224,7 +225,7 @@ Status PlainTableBuilder::Finish() { // -- Add user collected properties NotifyCollectTableCollectorsOnFinish(table_properties_collectors_, - options_.info_log.get(), + ioptions_.info_log, &property_block_builder); // -- Write property block diff --git a/table/plain_table_builder.h b/table/plain_table_builder.h index 2871d887e..c3af08072 100644 --- a/table/plain_table_builder.h +++ b/table/plain_table_builder.h @@ -30,7 +30,7 @@ class PlainTableBuilder: public TableBuilder { // caller to close the file after calling Finish(). The output file // will be part of level specified by 'level'. A value of -1 means // that the caller does not know which level the output file will reside. - PlainTableBuilder(const Options& options, WritableFile* file, + PlainTableBuilder(const ImmutableCFOptions& ioptions, WritableFile* file, uint32_t user_key_size, EncodingType encoding_type, size_t index_sparseness, uint32_t bloom_bits_per_key, uint32_t num_probes = 6, size_t huge_page_tlb_size = 0, @@ -71,7 +71,7 @@ class PlainTableBuilder: public TableBuilder { private: Arena arena_; - Options options_; + const ImmutableCFOptions& ioptions_; std::vector> table_properties_collectors_; diff --git a/table/plain_table_factory.cc b/table/plain_table_factory.cc index 145179bae..de23cc902 100644 --- a/table/plain_table_factory.cc +++ b/table/plain_table_factory.cc @@ -14,22 +14,24 @@ namespace rocksdb { -Status PlainTableFactory::NewTableReader(const Options& options, - const EnvOptions& soptions, +Status PlainTableFactory::NewTableReader(const ImmutableCFOptions& ioptions, + const EnvOptions& env_options, const InternalKeyComparator& icomp, unique_ptr&& file, uint64_t file_size, unique_ptr* table) const { - return PlainTableReader::Open(options, soptions, icomp, std::move(file), + return PlainTableReader::Open(ioptions, env_options, icomp, std::move(file), file_size, table, bloom_bits_per_key_, hash_table_ratio_, index_sparseness_, huge_page_tlb_size_, full_scan_mode_); } TableBuilder* PlainTableFactory::NewTableBuilder( - const Options& options, const InternalKeyComparator& internal_comparator, - WritableFile* file, CompressionType compression_type) const { - return new PlainTableBuilder(options, file, user_key_len_, encoding_type_, + const ImmutableCFOptions& ioptions, + const InternalKeyComparator& internal_comparator, + WritableFile* file, const CompressionType, + const CompressionOptions&) const { + return new PlainTableBuilder(ioptions, file, user_key_len_, encoding_type_, index_sparseness_, bloom_bits_per_key_, 6, huge_page_tlb_size_, hash_table_ratio_, store_index_in_file_); diff --git a/table/plain_table_factory.h b/table/plain_table_factory.h index d1cf0cae6..54c628c15 100644 --- a/table/plain_table_factory.h +++ b/table/plain_table_factory.h @@ -14,7 +14,6 @@ namespace rocksdb { -struct Options; struct EnvOptions; using std::unique_ptr; @@ -154,15 +153,17 @@ class PlainTableFactory : public TableFactory { full_scan_mode_(options.full_scan_mode), store_index_in_file_(options.store_index_in_file) {} const char* Name() const override { return "PlainTable"; } - Status NewTableReader(const Options& options, const EnvOptions& soptions, - const InternalKeyComparator& internal_comparator, - unique_ptr&& file, uint64_t file_size, - unique_ptr* table) const override; - TableBuilder* NewTableBuilder(const Options& options, - const InternalKeyComparator& icomparator, - WritableFile* file, - CompressionType compression_type) const - override; + Status NewTableReader( + const ImmutableCFOptions& options, const EnvOptions& soptions, + const InternalKeyComparator& internal_comparator, + unique_ptr&& file, uint64_t file_size, + unique_ptr* table) const override; + TableBuilder* NewTableBuilder( + const ImmutableCFOptions& options, + const InternalKeyComparator& icomparator, + WritableFile* file, + const CompressionType, + const CompressionOptions&) const override; std::string GetPrintableTableOptions() const override; diff --git a/table/plain_table_index.cc b/table/plain_table_index.cc index efba9b71d..61f9e335b 100644 --- a/table/plain_table_index.cc +++ b/table/plain_table_index.cc @@ -93,7 +93,7 @@ Slice PlainTableIndexBuilder::Finish() { BucketizeIndexes(&hash_to_offsets, &entries_per_bucket); keys_per_prefix_hist_.Add(num_keys_per_prefix_); - Log(options_.info_log, "Number of Keys per prefix Histogram: %s", + Log(ioptions_.info_log, "Number of Keys per prefix Histogram: %s", keys_per_prefix_hist_.ToString().c_str()); // From the temp data structure, populate indexes. @@ -147,11 +147,11 @@ void PlainTableIndexBuilder::BucketizeIndexes( Slice PlainTableIndexBuilder::FillIndexes( const std::vector& hash_to_offsets, const std::vector& entries_per_bucket) { - Log(options_.info_log, "Reserving %zu bytes for plain table's sub_index", + Log(ioptions_.info_log, "Reserving %zu bytes for plain table's sub_index", sub_index_size_); auto total_allocate_size = GetTotalSize(); char* allocated = arena_->AllocateAligned( - total_allocate_size, huge_page_tlb_size_, options_.info_log.get()); + total_allocate_size, huge_page_tlb_size_, ioptions_.info_log); auto temp_ptr = EncodeVarint32(allocated, index_size_); uint32_t* index = @@ -191,7 +191,7 @@ Slice PlainTableIndexBuilder::FillIndexes( } assert(sub_index_offset == sub_index_size_); - Log(options_.info_log, "hash table size: %d, suffix_map length %zu", + Log(ioptions_.info_log, "hash table size: %d, suffix_map length %zu", index_size_, sub_index_size_); return Slice(allocated, GetTotalSize()); } diff --git a/table/plain_table_index.h b/table/plain_table_index.h index f63bbd0d5..0b26ecd0d 100644 --- a/table/plain_table_index.h +++ b/table/plain_table_index.h @@ -108,11 +108,11 @@ class PlainTableIndex { // #wiki-in-memory-index-format class PlainTableIndexBuilder { public: - PlainTableIndexBuilder(Arena* arena, const Options& options, + PlainTableIndexBuilder(Arena* arena, const ImmutableCFOptions& ioptions, uint32_t index_sparseness, double hash_table_ratio, double huge_page_tlb_size) : arena_(arena), - options_(options), + ioptions_(ioptions), record_list_(kRecordsPerGroup), is_first_record_(true), due_index_(false), @@ -120,7 +120,7 @@ class PlainTableIndexBuilder { num_keys_per_prefix_(0), prev_key_prefix_hash_(0), index_sparseness_(index_sparseness), - prefix_extractor_(options.prefix_extractor.get()), + prefix_extractor_(ioptions.prefix_extractor), hash_table_ratio_(hash_table_ratio), huge_page_tlb_size_(huge_page_tlb_size) {} @@ -196,7 +196,7 @@ class PlainTableIndexBuilder { const std::vector& entries_per_bucket); Arena* arena_; - Options options_; + const ImmutableCFOptions ioptions_; HistogramImpl keys_per_prefix_hist_; IndexRecordList record_list_; bool is_first_record_; diff --git a/table/plain_table_reader.cc b/table/plain_table_reader.cc index b5eccd310..3a6d48be8 100644 --- a/table/plain_table_reader.cc +++ b/table/plain_table_reader.cc @@ -87,7 +87,7 @@ class PlainTableIterator : public Iterator { }; extern const uint64_t kPlainTableMagicNumber; -PlainTableReader::PlainTableReader(const Options& options, +PlainTableReader::PlainTableReader(const ImmutableCFOptions& ioptions, unique_ptr&& file, const EnvOptions& storage_options, const InternalKeyComparator& icomparator, @@ -99,10 +99,10 @@ PlainTableReader::PlainTableReader(const Options& options, full_scan_mode_(false), data_end_offset_(table_properties->data_size), user_key_len_(table_properties->fixed_key_len), - prefix_extractor_(options.prefix_extractor.get()), + prefix_extractor_(ioptions.prefix_extractor), enable_bloom_(false), bloom_(6, nullptr), - options_(options), + ioptions_(ioptions), file_(std::move(file)), file_size_(file_size), table_properties_(nullptr) {} @@ -110,8 +110,8 @@ PlainTableReader::PlainTableReader(const Options& options, PlainTableReader::~PlainTableReader() { } -Status PlainTableReader::Open(const Options& options, - const EnvOptions& soptions, +Status PlainTableReader::Open(const ImmutableCFOptions& ioptions, + const EnvOptions& env_options, const InternalKeyComparator& internal_comparator, unique_ptr&& file, uint64_t file_size, @@ -119,14 +119,14 @@ Status PlainTableReader::Open(const Options& options, const int bloom_bits_per_key, double hash_table_ratio, size_t index_sparseness, size_t huge_page_tlb_size, bool full_scan_mode) { - assert(options.allow_mmap_reads); + assert(ioptions.allow_mmap_reads); if (file_size > PlainTableIndex::kMaxFileSize) { return Status::NotSupported("File is too large for PlainTableReader!"); } TableProperties* props = nullptr; auto s = ReadTableProperties(file.get(), file_size, kPlainTableMagicNumber, - options.env, options.info_log.get(), &props); + ioptions.env, ioptions.info_log, &props); if (!s.ok()) { return s; } @@ -137,12 +137,12 @@ Status PlainTableReader::Open(const Options& options, user_props.find(PlainTablePropertyNames::kPrefixExtractorName); if (!full_scan_mode && prefix_extractor_in_file != user_props.end()) { - if (!options.prefix_extractor) { + if (!ioptions.prefix_extractor) { return Status::InvalidArgument( "Prefix extractor is missing when opening a PlainTable built " "using a prefix extractor"); } else if (prefix_extractor_in_file->second.compare( - options.prefix_extractor->Name()) != 0) { + ioptions.prefix_extractor->Name()) != 0) { return Status::InvalidArgument( "Prefix extractor given doesn't match the one used to build " "PlainTable"); @@ -158,8 +158,8 @@ Status PlainTableReader::Open(const Options& options, } std::unique_ptr new_reader(new PlainTableReader( - options, std::move(file), soptions, internal_comparator, encoding_type, - file_size, props)); + ioptions, std::move(file), env_options, internal_comparator, + encoding_type, file_size, props)); s = new_reader->MmapDataFile(); if (!s.ok()) { @@ -207,7 +207,7 @@ Status PlainTableReader::PopulateIndexRecordList( bool is_first_record = true; Slice key_prefix_slice; PlainTableKeyDecoder decoder(encoding_type_, user_key_len_, - options_.prefix_extractor.get()); + ioptions_.prefix_extractor); while (pos < data_end_offset_) { uint32_t key_offset = pos; ParsedInternalKey key; @@ -252,8 +252,8 @@ void PlainTableReader::AllocateAndFillBloom(int bloom_bits_per_key, uint32_t bloom_total_bits = num_prefixes * bloom_bits_per_key; if (bloom_total_bits > 0) { enable_bloom_ = true; - bloom_.SetTotalBits(&arena_, bloom_total_bits, options_.bloom_locality, - huge_page_tlb_size, options_.info_log.get()); + bloom_.SetTotalBits(&arena_, bloom_total_bits, ioptions_.bloom_locality, + huge_page_tlb_size, ioptions_.info_log); FillBloom(prefix_hashes); } } @@ -281,14 +281,14 @@ Status PlainTableReader::PopulateIndex(TableProperties* props, BlockContents bloom_block_contents; auto s = ReadMetaBlock(file_.get(), file_size_, kPlainTableMagicNumber, - options_.env, BloomBlockBuilder::kBloomBlock, + ioptions_.env, BloomBlockBuilder::kBloomBlock, &bloom_block_contents); bool index_in_file = s.ok(); BlockContents index_block_contents; s = ReadMetaBlock(file_.get(), file_size_, kPlainTableMagicNumber, - options_.env, PlainTableIndexBuilder::kPlainTableIndexBlock, - &index_block_contents); + ioptions_.env, PlainTableIndexBuilder::kPlainTableIndexBlock, + &index_block_contents); index_in_file &= s.ok(); @@ -310,8 +310,9 @@ Status PlainTableReader::PopulateIndex(TableProperties* props, index_block = nullptr; } - if ((options_.prefix_extractor.get() == nullptr) && (hash_table_ratio != 0)) { - // options.prefix_extractor is requried for a hash-based look-up. + if ((ioptions_.prefix_extractor == nullptr) && + (hash_table_ratio != 0)) { + // ioptions.prefix_extractor is requried for a hash-based look-up. return Status::NotSupported( "PlainTable requires a prefix extractor enable prefix hash mode."); } @@ -328,8 +329,8 @@ Status PlainTableReader::PopulateIndex(TableProperties* props, table_properties_->num_entries * bloom_bits_per_key; if (num_bloom_bits > 0) { enable_bloom_ = true; - bloom_.SetTotalBits(&arena_, num_bloom_bits, options_.bloom_locality, - huge_page_tlb_size, options_.info_log.get()); + bloom_.SetTotalBits(&arena_, num_bloom_bits, ioptions_.bloom_locality, + huge_page_tlb_size, ioptions_.info_log); } } } else { @@ -351,7 +352,7 @@ Status PlainTableReader::PopulateIndex(TableProperties* props, bloom_block->size() * 8, num_blocks); } - PlainTableIndexBuilder index_builder(&arena_, options_, index_sparseness, + PlainTableIndexBuilder index_builder(&arena_, ioptions_, index_sparseness, hash_table_ratio, huge_page_tlb_size); std::vector prefix_hashes; @@ -422,7 +423,7 @@ Status PlainTableReader::GetOffset(const Slice& target, const Slice& prefix, uint32_t file_offset = GetFixed32Element(base_ptr, mid); size_t tmp; Status s = PlainTableKeyDecoder(encoding_type_, user_key_len_, - options_.prefix_extractor.get()) + ioptions_.prefix_extractor) .NextKey(file_data_.data() + file_offset, file_data_.data() + data_end_offset_, &mid_key, nullptr, &tmp); @@ -451,7 +452,7 @@ Status PlainTableReader::GetOffset(const Slice& target, const Slice& prefix, size_t tmp; uint32_t low_key_offset = GetFixed32Element(base_ptr, low); Status s = PlainTableKeyDecoder(encoding_type_, user_key_len_, - options_.prefix_extractor.get()) + ioptions_.prefix_extractor) .NextKey(file_data_.data() + low_key_offset, file_data_.data() + data_end_offset_, &low_key, nullptr, &tmp); @@ -565,7 +566,7 @@ Status PlainTableReader::Get(const ReadOptions& ro, const Slice& target, } Slice found_value; PlainTableKeyDecoder decoder(encoding_type_, user_key_len_, - options_.prefix_extractor.get()); + ioptions_.prefix_extractor); while (offset < data_end_offset_) { Status s = Next(&decoder, &offset, &found_key, nullptr, &found_value); if (!s.ok()) { diff --git a/table/plain_table_reader.h b/table/plain_table_reader.h index 4a626979a..fcc94a53e 100644 --- a/table/plain_table_reader.h +++ b/table/plain_table_reader.h @@ -52,7 +52,8 @@ extern const uint32_t kPlainTableVariableLength; // The implementation of IndexedTableReader requires output file is mmaped class PlainTableReader: public TableReader { public: - static Status Open(const Options& options, const EnvOptions& soptions, + static Status Open(const ImmutableCFOptions& ioptions, + const EnvOptions& env_options, const InternalKeyComparator& internal_comparator, unique_ptr&& file, uint64_t file_size, unique_ptr* table, @@ -82,8 +83,9 @@ class PlainTableReader: public TableReader { return arena_.MemoryAllocatedBytes(); } - PlainTableReader(const Options& options, unique_ptr&& file, - const EnvOptions& storage_options, + PlainTableReader(const ImmutableCFOptions& ioptions, + unique_ptr&& file, + const EnvOptions& env_options, const InternalKeyComparator& internal_comparator, EncodingType encoding_type, uint64_t file_size, const TableProperties* table_properties); @@ -132,7 +134,7 @@ class PlainTableReader: public TableReader { DynamicBloom bloom_; Arena arena_; - const Options& options_; + const ImmutableCFOptions& ioptions_; unique_ptr file_; uint32_t file_size_; std::shared_ptr table_properties_; diff --git a/table/table_reader_bench.cc b/table/table_reader_bench.cc index ed2c7c52d..584937587 100644 --- a/table/table_reader_bench.cc +++ b/table/table_reader_bench.cc @@ -88,10 +88,12 @@ void TableReaderBenchmark(Options& opts, EnvOptions& env_options, TableBuilder* tb = nullptr; DB* db = nullptr; Status s; + const ImmutableCFOptions ioptions(opts); if (!through_db) { env->NewWritableFile(file_name, &file, env_options); - tb = opts.table_factory->NewTableBuilder(opts, ikc, file.get(), - CompressionType::kNoCompression); + tb = opts.table_factory->NewTableBuilder(ioptions, ikc, file.get(), + CompressionType::kNoCompression, + CompressionOptions()); } else { s = DB::Open(opts, dbname, &db); ASSERT_OK(s); @@ -122,7 +124,7 @@ void TableReaderBenchmark(Options& opts, EnvOptions& env_options, uint64_t file_size; env->GetFileSize(file_name, &file_size); s = opts.table_factory->NewTableReader( - opts, env_options, ikc, std::move(raf), file_size, &table_reader); + ioptions, env_options, ikc, std::move(raf), file_size, &table_reader); } Random rnd(301); diff --git a/table/table_test.cc b/table/table_test.cc index 500abf48f..df4997588 100644 --- a/table/table_test.cc +++ b/table/table_test.cc @@ -194,6 +194,7 @@ class Constructor { // been added so far. Returns the keys in sorted order in "*keys" // and stores the key/value pairs in "*kvmap" void Finish(const Options& options, + const ImmutableCFOptions& ioptions, const BlockBasedTableOptions& table_options, const InternalKeyComparator& internal_comparator, std::vector* keys, KVMap* kvmap) { @@ -206,12 +207,14 @@ class Constructor { keys->push_back(it->first); } data_.clear(); - Status s = FinishImpl(options, table_options, internal_comparator, *kvmap); + Status s = FinishImpl(options, ioptions, table_options, + internal_comparator, *kvmap); ASSERT_TRUE(s.ok()) << s.ToString(); } // Construct the data structure from the data in "data" virtual Status FinishImpl(const Options& options, + const ImmutableCFOptions& ioptions, const BlockBasedTableOptions& table_options, const InternalKeyComparator& internal_comparator, const KVMap& data) = 0; @@ -239,6 +242,7 @@ class BlockConstructor: public Constructor { delete block_; } virtual Status FinishImpl(const Options& options, + const ImmutableCFOptions& ioptions, const BlockBasedTableOptions& table_options, const InternalKeyComparator& internal_comparator, const KVMap& data) { @@ -322,14 +326,16 @@ class TableConstructor: public Constructor { ~TableConstructor() { Reset(); } virtual Status FinishImpl(const Options& options, + const ImmutableCFOptions& ioptions, const BlockBasedTableOptions& table_options, const InternalKeyComparator& internal_comparator, const KVMap& data) { Reset(); sink_.reset(new StringSink()); unique_ptr builder; - builder.reset(options.table_factory->NewTableBuilder( - options, internal_comparator, sink_.get(), options.compression)); + builder.reset(ioptions.table_factory->NewTableBuilder( + ioptions, internal_comparator, sink_.get(), options.compression, + CompressionOptions())); for (KVMap::const_iterator it = data.begin(); it != data.end(); @@ -352,9 +358,9 @@ class TableConstructor: public Constructor { // Open the table uniq_id_ = cur_uniq_id_++; source_.reset(new StringSource(sink_->contents(), uniq_id_, - options.allow_mmap_reads)); - return options.table_factory->NewTableReader( - options, soptions, internal_comparator, std::move(source_), + ioptions.allow_mmap_reads)); + return ioptions.table_factory->NewTableReader( + ioptions, soptions, internal_comparator, std::move(source_), sink_->contents().size(), &table_reader_); } @@ -372,12 +378,12 @@ class TableConstructor: public Constructor { return table_reader_->ApproximateOffsetOf(key); } - virtual Status Reopen(const Options& options) { + virtual Status Reopen(const ImmutableCFOptions& ioptions) { source_.reset( new StringSource(sink_->contents(), uniq_id_, - options.allow_mmap_reads)); - return options.table_factory->NewTableReader( - options, soptions, *last_internal_key_, std::move(source_), + ioptions.allow_mmap_reads)); + return ioptions.table_factory->NewTableReader( + ioptions, soptions, *last_internal_key_, std::move(source_), sink_->contents().size(), &table_reader_); } @@ -421,6 +427,7 @@ class MemTableConstructor: public Constructor { delete memtable_->Unref(); } virtual Status FinishImpl(const Options& options, + const ImmutableCFOptions& ioptions, const BlockBasedTableOptions& table_options, const InternalKeyComparator& internal_comparator, const KVMap& data) { @@ -460,6 +467,7 @@ class DBConstructor: public Constructor { delete db_; } virtual Status FinishImpl(const Options& options, + const ImmutableCFOptions& ioptions, const BlockBasedTableOptions& table_options, const InternalKeyComparator& internal_comparator, const KVMap& data) { @@ -670,7 +678,7 @@ class FixedOrLessPrefixTransform : public SliceTransform { class Harness { public: - Harness() : constructor_(nullptr) { } + Harness() : ioptions_(options_), constructor_(nullptr) {} void Init(const TestArgs& args) { delete constructor_; @@ -756,6 +764,7 @@ class Harness { constructor_ = new DBConstructor(options_.comparator); break; } + ioptions_ = ImmutableCFOptions(options_); } ~Harness() { @@ -769,8 +778,8 @@ class Harness { void Test(Random* rnd) { std::vector keys; KVMap data; - constructor_->Finish(options_, table_options_, *internal_comparator_, - &keys, &data); + constructor_->Finish(options_, ioptions_, table_options_, + *internal_comparator_, &keys, &data); TestForwardScan(keys, data); if (support_prev_) { @@ -939,6 +948,7 @@ class Harness { private: Options options_ = Options(); + ImmutableCFOptions ioptions_; BlockBasedTableOptions table_options_ = BlockBasedTableOptions(); Constructor* constructor_; bool support_prev_; @@ -1038,7 +1048,8 @@ TEST(BlockBasedTableTest, BasicBlockBasedTableProperties) { table_options.block_restart_interval = 1; options.table_factory.reset(NewBlockBasedTableFactory(table_options)); - c.Finish(options, table_options, + const ImmutableCFOptions ioptions(options); + c.Finish(options, ioptions, table_options, GetPlainInternalComparator(options.comparator), &keys, &kvmap); auto& props = *c.GetTableReader()->GetTableProperties(); @@ -1071,7 +1082,8 @@ TEST(BlockBasedTableTest, FilterPolicyNameProperties) { Options options; options.table_factory.reset(NewBlockBasedTableFactory(table_options)); - c.Finish(options, table_options, + const ImmutableCFOptions ioptions(options); + c.Finish(options, ioptions, table_options, GetPlainInternalComparator(options.comparator), &keys, &kvmap); auto& props = *c.GetTableReader()->GetTableProperties(); ASSERT_EQ("rocksdb.BuiltinBloomFilter", props.filter_policy_name); @@ -1122,7 +1134,8 @@ TEST(BlockBasedTableTest, TotalOrderSeekOnHashIndex) { c.Add("cccc2", std::string('a', 56)); std::vector keys; KVMap kvmap; - c.Finish(options, table_options, + const ImmutableCFOptions ioptions(options); + c.Finish(options, ioptions, table_options, GetPlainInternalComparator(options.comparator), &keys, &kvmap); auto props = c.GetTableReader()->GetTableProperties(); ASSERT_EQ(7u, props->num_data_blocks); @@ -1206,7 +1219,8 @@ TEST(TableTest, HashIndexTest) { std::unique_ptr comparator( new InternalKeyComparator(BytewiseComparator())); - c.Finish(options, table_options, *comparator, &keys, &kvmap); + const ImmutableCFOptions ioptions(options); + c.Finish(options, ioptions, table_options, *comparator, &keys, &kvmap); auto reader = c.GetTableReader(); auto props = reader->GetTableProperties(); @@ -1314,7 +1328,8 @@ TEST(BlockBasedTableTest, IndexSizeStat) { table_options.block_restart_interval = 1; options.table_factory.reset(NewBlockBasedTableFactory(table_options)); - c.Finish(options, table_options, + const ImmutableCFOptions ioptions(options); + c.Finish(options, ioptions, table_options, GetPlainInternalComparator(options.comparator), &ks, &kvmap); auto index_size = c.GetTableReader()->GetTableProperties()->index_size; ASSERT_GT(index_size, last_index_size); @@ -1340,7 +1355,8 @@ TEST(BlockBasedTableTest, NumBlockStat) { std::vector ks; KVMap kvmap; - c.Finish(options, table_options, + const ImmutableCFOptions ioptions(options); + c.Finish(options, ioptions, table_options, GetPlainInternalComparator(options.comparator), &ks, &kvmap); ASSERT_EQ(kvmap.size(), c.GetTableReader()->GetTableProperties()->num_data_blocks); @@ -1416,7 +1432,8 @@ TEST(BlockBasedTableTest, BlockCacheDisabledTest) { TableConstructor c(BytewiseComparator(), true); c.Add("key", "value"); - c.Finish(options, table_options, + const ImmutableCFOptions ioptions(options); + c.Finish(options, ioptions, table_options, GetPlainInternalComparator(options.comparator), &keys, &kvmap); // preloading filter/index blocks is enabled. @@ -1458,7 +1475,8 @@ TEST(BlockBasedTableTest, FilterBlockInBlockCache) { TableConstructor c(BytewiseComparator()); c.Add("key", "value"); - c.Finish(options, table_options, + const ImmutableCFOptions ioptions(options); + c.Finish(options, ioptions, table_options, GetPlainInternalComparator(options.comparator), &keys, &kvmap); // preloading filter/index blocks is prohibited. auto reader = dynamic_cast(c.GetTableReader()); @@ -1512,7 +1530,8 @@ TEST(BlockBasedTableTest, FilterBlockInBlockCache) { table_options.block_cache.reset(); options.table_factory.reset(new BlockBasedTableFactory(table_options)); options.statistics = CreateDBStatistics(); // reset the stats - c.Reopen(options); + const ImmutableCFOptions ioptions1(options); + c.Reopen(ioptions1); table_options.no_block_cache = false; { @@ -1529,7 +1548,8 @@ TEST(BlockBasedTableTest, FilterBlockInBlockCache) { // too small to fit even one entry. table_options.block_cache = NewLRUCache(1); options.table_factory.reset(new BlockBasedTableFactory(table_options)); - c.Reopen(options); + const ImmutableCFOptions ioptions2(options); + c.Reopen(ioptions2); { BlockCachePropertiesSnapshot props(options.statistics.get()); props.AssertEqual(1, // index block miss @@ -1583,7 +1603,8 @@ TEST(BlockBasedTableTest, BlockCacheLeak) { c.Add("k07", std::string(100000, 'x')); std::vector keys; KVMap kvmap; - c.Finish(opt, table_options, *ikc, &keys, &kvmap); + const ImmutableCFOptions ioptions(opt); + c.Finish(opt, ioptions, table_options, *ikc, &keys, &kvmap); unique_ptr iter(c.NewIterator()); iter->SeekToFirst(); @@ -1594,7 +1615,8 @@ TEST(BlockBasedTableTest, BlockCacheLeak) { } ASSERT_OK(iter->status()); - ASSERT_OK(c.Reopen(opt)); + const ImmutableCFOptions ioptions1(opt); + ASSERT_OK(c.Reopen(ioptions1)); auto table_reader = dynamic_cast(c.GetTableReader()); for (const std::string& key : keys) { ASSERT_TRUE(table_reader->TEST_KeyInCache(ReadOptions(), key)); @@ -1603,7 +1625,8 @@ TEST(BlockBasedTableTest, BlockCacheLeak) { // rerun with different block cache table_options.block_cache = NewLRUCache(16 * 1024 * 1024); opt.table_factory.reset(NewBlockBasedTableFactory(table_options)); - ASSERT_OK(c.Reopen(opt)); + const ImmutableCFOptions ioptions2(opt); + ASSERT_OK(c.Reopen(ioptions2)); table_reader = dynamic_cast(c.GetTableReader()); for (const std::string& key : keys) { ASSERT_TRUE(!table_reader->TEST_KeyInCache(ReadOptions(), key)); @@ -1619,9 +1642,11 @@ TEST(PlainTableTest, BasicPlainTableProperties) { PlainTableFactory factory(plain_table_options); StringSink sink; Options options; + const ImmutableCFOptions ioptions(options); InternalKeyComparator ikc(options.comparator); std::unique_ptr builder( - factory.NewTableBuilder(options, ikc, &sink, kNoCompression)); + factory.NewTableBuilder(ioptions, ikc, &sink, kNoCompression, + CompressionOptions())); for (char c = 'a'; c <= 'z'; ++c) { std::string key(8, c); @@ -1664,7 +1689,9 @@ TEST(GeneralTableTest, ApproximateOffsetOfPlain) { options.compression = kNoCompression; BlockBasedTableOptions table_options; table_options.block_size = 1024; - c.Finish(options, table_options, internal_comparator, &keys, &kvmap); + const ImmutableCFOptions ioptions(options); + c.Finish(options, ioptions, table_options, internal_comparator, + &keys, &kvmap); ASSERT_TRUE(Between(c.ApproximateOffsetOf("abc"), 0, 0)); ASSERT_TRUE(Between(c.ApproximateOffsetOf("k01"), 0, 0)); @@ -1694,7 +1721,8 @@ static void DoCompressionTest(CompressionType comp) { options.compression = comp; BlockBasedTableOptions table_options; table_options.block_size = 1024; - c.Finish(options, table_options, ikc, &keys, &kvmap); + const ImmutableCFOptions ioptions(options); + c.Finish(options, ioptions, table_options, ikc, &keys, &kvmap); ASSERT_TRUE(Between(c.ApproximateOffsetOf("abc"), 0, 0)); ASSERT_TRUE(Between(c.ApproximateOffsetOf("k01"), 0, 0)); diff --git a/tools/sst_dump.cc b/tools/sst_dump.cc index 9b130c7c6..6c496e8dd 100644 --- a/tools/sst_dump.cc +++ b/tools/sst_dump.cc @@ -68,6 +68,7 @@ class SstFileReader { // options_ and internal_comparator_ will also be used in // ReadSequential internally (specifically, seek-related operations) Options options_; + const ImmutableCFOptions ioptions_; InternalKeyComparator internal_comparator_; unique_ptr table_properties_; }; @@ -76,7 +77,8 @@ SstFileReader::SstFileReader(const std::string& file_path, bool verify_checksum, bool output_hex) :file_name_(file_path), read_num_(0), verify_checksum_(verify_checksum), - output_hex_(output_hex), internal_comparator_(BytewiseComparator()) { + output_hex_(output_hex), ioptions_(options_), + internal_comparator_(BytewiseComparator()) { fprintf(stdout, "Process %s\n", file_path.c_str()); init_result_ = NewTableReader(file_name_); @@ -123,7 +125,7 @@ Status SstFileReader::NewTableReader(const std::string& file_path) { if (s.ok()) { s = options_.table_factory->NewTableReader( - options_, soptions_, internal_comparator_, std::move(file_), file_size, + ioptions_, soptions_, internal_comparator_, std::move(file_), file_size, &table_reader_); } return s; diff --git a/util/options.cc b/util/options.cc index b16c6f2f5..fc9285442 100644 --- a/util/options.cc +++ b/util/options.cc @@ -8,6 +8,7 @@ // found in the LICENSE file. See the AUTHORS file for names of contributors. #include "rocksdb/options.h" +#include "rocksdb/immutable_options.h" #define __STDC_FORMAT_MACROS #include @@ -28,6 +29,26 @@ namespace rocksdb { +ImmutableCFOptions::ImmutableCFOptions(const Options& options) + : prefix_extractor(options.prefix_extractor.get()), + comparator(options.comparator), + merge_operator(options.merge_operator.get()), + info_log(options.info_log.get()), + statistics(options.statistics.get()), + env(options.env), + allow_mmap_reads(options.allow_mmap_reads), + allow_mmap_writes(options.allow_mmap_writes), + db_paths(options.db_paths), + table_factory(options.table_factory.get()), + table_properties_collector_factories( + options.table_properties_collector_factories), + advise_random_on_open(options.advise_random_on_open), + bloom_locality(options.bloom_locality), + purge_redundant_kvs_while_flush(options.purge_redundant_kvs_while_flush), + min_partial_merge_operands(options.min_partial_merge_operands), + disable_data_sync(options.disableDataSync), + use_fsync(options.use_fsync) {} + ColumnFamilyOptions::ColumnFamilyOptions() : comparator(BytewiseComparator()), merge_operator(nullptr),