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),