introduce ImmutableOptions

Summary:
As a preparation to support updating some options dynamically, I'd like
to first introduce ImmutableOptions, which is a subset of Options that
cannot be changed during the course of a DB lifetime without restart.

ColumnFamily will keep both Options and ImmutableOptions. Any component
below ColumnFamily should only take ImmutableOptions in their
constructor. Other options should be taken from APIs, which will be
allowed to adjust dynamically.

I am yet to make changes to memtable and other related classes to take
ImmutableOptions in their ctor. That can be done in a seprate diff as
this one is already pretty big.

Test Plan: make all check

Reviewers: yhchiang, igor, sdong

Reviewed By: sdong

Subscribers: leveldb, dhruba

Differential Revision: https://reviews.facebook.net/D22545
main
Lei Jin 10 years ago
parent e0b99d4f5d
commit 5665e5e285
  1. 40
      db/builder.cc
  2. 11
      db/builder.h
  3. 15
      db/column_family.cc
  4. 13
      db/column_family.h
  5. 37
      db/db_impl.cc
  6. 2
      db/db_impl.h
  7. 22
      db/plain_table_db_test.cc
  8. 23
      db/repair.cc
  9. 65
      db/simple_table_db_test.cc
  10. 43
      db/table_cache.cc
  11. 11
      db/table_cache.h
  12. 3
      db/table_properties_collector_test.cc
  13. 62
      include/rocksdb/immutable_options.h
  14. 23
      include/rocksdb/table.h
  15. 18
      table/adaptive_table_factory.cc
  16. 14
      table/adaptive_table_factory.h
  17. 49
      table/block_based_table_builder.cc
  18. 6
      table/block_based_table_builder.h
  19. 13
      table/block_based_table_factory.cc
  20. 11
      table/block_based_table_factory.h
  21. 67
      table/block_based_table_reader.cc
  22. 5
      table/block_based_table_reader.h
  23. 18
      table/cuckoo_table_factory.cc
  24. 7
      table/cuckoo_table_factory.h
  25. 6
      table/cuckoo_table_reader.cc
  26. 5
      table/cuckoo_table_reader.h
  27. 15
      table/cuckoo_table_reader_test.cc
  28. 9
      table/filter_block.cc
  29. 5
      table/filter_block.h
  30. 18
      table/filter_block_test.cc
  31. 29
      table/plain_table_builder.cc
  32. 4
      table/plain_table_builder.h
  33. 14
      table/plain_table_factory.cc
  34. 11
      table/plain_table_factory.h
  35. 8
      table/plain_table_index.cc
  36. 8
      table/plain_table_index.h
  37. 49
      table/plain_table_reader.cc
  38. 10
      table/plain_table_reader.h
  39. 8
      table/table_reader_bench.cc
  40. 86
      table/table_test.cc
  41. 6
      tools/sst_dump.cc
  42. 21
      util/options.cc

@ -26,21 +26,24 @@ namespace rocksdb {
class TableFactory; class TableFactory;
TableBuilder* NewTableBuilder(const Options& options, TableBuilder* NewTableBuilder(const ImmutableCFOptions& ioptions,
const InternalKeyComparator& internal_comparator, const InternalKeyComparator& internal_comparator,
WritableFile* file, WritableFile* file,
CompressionType compression_type) { const CompressionType compression_type,
return options.table_factory->NewTableBuilder(options, internal_comparator, const CompressionOptions& compression_opts) {
file, compression_type); return ioptions.table_factory->NewTableBuilder(
ioptions, internal_comparator, file, compression_type, compression_opts);
} }
Status BuildTable(const std::string& dbname, Env* env, const Options& options, Status BuildTable(const std::string& dbname, Env* env,
const EnvOptions& soptions, TableCache* table_cache, const ImmutableCFOptions& ioptions,
const EnvOptions& env_options, TableCache* table_cache,
Iterator* iter, FileMetaData* meta, Iterator* iter, FileMetaData* meta,
const InternalKeyComparator& internal_comparator, const InternalKeyComparator& internal_comparator,
const SequenceNumber newest_snapshot, const SequenceNumber newest_snapshot,
const SequenceNumber earliest_seqno_in_memtable, const SequenceNumber earliest_seqno_in_memtable,
const CompressionType compression, const CompressionType compression,
const CompressionOptions& compression_opts,
const Env::IOPriority io_priority) { const Env::IOPriority io_priority) {
Status s; Status s;
meta->fd.file_size = 0; 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 // If the sequence number of the smallest entry in the memtable is
// smaller than the most recent snapshot, then we do not trigger // smaller than the most recent snapshot, then we do not trigger
// removal of duplicate/deleted keys as part of this builder. // 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) { if (earliest_seqno_in_memtable <= newest_snapshot) {
purge = false; 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()); meta->fd.GetPathId());
if (iter->Valid()) { if (iter->Valid()) {
unique_ptr<WritableFile> file; unique_ptr<WritableFile> file;
s = env->NewWritableFile(fname, &file, soptions); s = env->NewWritableFile(fname, &file, env_options);
if (!s.ok()) { if (!s.ok()) {
return s; return s;
} }
file->SetIOPriority(io_priority); file->SetIOPriority(io_priority);
TableBuilder* builder = TableBuilder* builder = NewTableBuilder(
NewTableBuilder(options, internal_comparator, file.get(), compression); ioptions, internal_comparator, file.get(),
compression, compression_opts);
// the first key is the smallest key // the first key is the smallest key
Slice key = iter->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; meta->largest_seqno = meta->smallest_seqno;
MergeHelper merge(internal_comparator.user_comparator(), MergeHelper merge(internal_comparator.user_comparator(),
options.merge_operator.get(), options.info_log.get(), ioptions.merge_operator, ioptions.info_log,
options.min_partial_merge_operands, ioptions.min_partial_merge_operands,
true /* internal key corruption is not ok */); true /* internal key corruption is not ok */);
if (purge) { if (purge) {
@ -196,12 +200,12 @@ Status BuildTable(const std::string& dbname, Env* env, const Options& options,
delete builder; delete builder;
// Finish and check for file errors // Finish and check for file errors
if (s.ok() && !options.disableDataSync) { if (s.ok() && !ioptions.disable_data_sync) {
if (options.use_fsync) { if (ioptions.use_fsync) {
StopWatch sw(env, options.statistics.get(), TABLE_SYNC_MICROS); StopWatch sw(env, ioptions.statistics, TABLE_SYNC_MICROS);
s = file->Fsync(); s = file->Fsync();
} else { } else {
StopWatch sw(env, options.statistics.get(), TABLE_SYNC_MICROS); StopWatch sw(env, ioptions.statistics, TABLE_SYNC_MICROS);
s = file->Sync(); s = file->Sync();
} }
} }
@ -211,7 +215,7 @@ Status BuildTable(const std::string& dbname, Env* env, const Options& options,
if (s.ok()) { if (s.ok()) {
// Verify that the table is usable // 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); internal_comparator, meta->fd);
s = it->status(); s = it->status();
delete it; delete it;

@ -11,6 +11,7 @@
#include "rocksdb/status.h" #include "rocksdb/status.h"
#include "rocksdb/types.h" #include "rocksdb/types.h"
#include "rocksdb/options.h" #include "rocksdb/options.h"
#include "rocksdb/immutable_options.h"
namespace rocksdb { namespace rocksdb {
@ -26,8 +27,10 @@ class TableBuilder;
class WritableFile; class WritableFile;
extern TableBuilder* NewTableBuilder( extern TableBuilder* NewTableBuilder(
const Options& options, const InternalKeyComparator& internal_comparator, const ImmutableCFOptions& options,
WritableFile* file, CompressionType compression_type); 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 // 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 // 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 // If no data is present in *iter, meta->file_size will be set to
// zero, and no Table file will be produced. // zero, and no Table file will be produced.
extern Status BuildTable(const std::string& dbname, Env* env, 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, TableCache* table_cache, Iterator* iter,
FileMetaData* meta, FileMetaData* meta,
const InternalKeyComparator& internal_comparator, const InternalKeyComparator& internal_comparator,
const SequenceNumber newest_snapshot, const SequenceNumber newest_snapshot,
const SequenceNumber earliest_seqno_in_memtable, const SequenceNumber earliest_seqno_in_memtable,
const CompressionType compression, const CompressionType compression,
const CompressionOptions& compression_opts,
const Env::IOPriority io_priority = Env::IO_HIGH); const Env::IOPriority io_priority = Env::IO_HIGH);
} // namespace rocksdb } // namespace rocksdb

@ -178,7 +178,7 @@ ColumnFamilyData::ColumnFamilyData(uint32_t id, const std::string& name,
Version* dummy_versions, Cache* table_cache, Version* dummy_versions, Cache* table_cache,
const ColumnFamilyOptions& options, const ColumnFamilyOptions& options,
const DBOptions* db_options, const DBOptions* db_options,
const EnvOptions& storage_options, const EnvOptions& env_options,
ColumnFamilySet* column_family_set) ColumnFamilySet* column_family_set)
: id_(id), : id_(id),
name_(name), name_(name),
@ -188,6 +188,7 @@ ColumnFamilyData::ColumnFamilyData(uint32_t id, const std::string& name,
dropped_(false), dropped_(false),
internal_comparator_(options.comparator), internal_comparator_(options.comparator),
options_(*db_options, SanitizeOptions(&internal_comparator_, options)), options_(*db_options, SanitizeOptions(&internal_comparator_, options)),
ioptions_(options_),
mem_(nullptr), mem_(nullptr),
imm_(options_.min_write_buffer_number_to_merge), imm_(options_.min_write_buffer_number_to_merge),
super_version_(nullptr), super_version_(nullptr),
@ -204,7 +205,7 @@ ColumnFamilyData::ColumnFamilyData(uint32_t id, const std::string& name,
if (dummy_versions != nullptr) { if (dummy_versions != nullptr) {
internal_stats_.reset( internal_stats_.reset(
new InternalStats(options_.num_levels, db_options->env, this)); 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) { if (options_.compaction_style == kCompactionStyleUniversal) {
compaction_picker_.reset( compaction_picker_.reset(
new UniversalCompactionPicker(&options_, &internal_comparator_)); new UniversalCompactionPicker(&options_, &internal_comparator_));
@ -306,7 +307,7 @@ void ColumnFamilyData::RecalculateWriteStallRateLimitsConditions() {
} }
const EnvOptions* ColumnFamilyData::soptions() const { const EnvOptions* ColumnFamilyData::soptions() const {
return &(column_family_set_->storage_options_); return &(column_family_set_->env_options_);
} }
void ColumnFamilyData::SetCurrent(Version* current) { void ColumnFamilyData::SetCurrent(Version* current) {
@ -462,16 +463,16 @@ void ColumnFamilyData::ResetThreadLocalSuperVersions() {
ColumnFamilySet::ColumnFamilySet(const std::string& dbname, ColumnFamilySet::ColumnFamilySet(const std::string& dbname,
const DBOptions* db_options, const DBOptions* db_options,
const EnvOptions& storage_options, const EnvOptions& env_options,
Cache* table_cache) Cache* table_cache)
: max_column_family_(0), : max_column_family_(0),
dummy_cfd_(new ColumnFamilyData(0, "", nullptr, nullptr, dummy_cfd_(new ColumnFamilyData(0, "", nullptr, nullptr,
ColumnFamilyOptions(), db_options, ColumnFamilyOptions(), db_options,
storage_options_, nullptr)), env_options_, nullptr)),
default_cfd_cache_(nullptr), default_cfd_cache_(nullptr),
db_name_(dbname), db_name_(dbname),
db_options_(db_options), db_options_(db_options),
storage_options_(storage_options), env_options_(env_options),
table_cache_(table_cache), table_cache_(table_cache),
spin_lock_(ATOMIC_FLAG_INIT) { spin_lock_(ATOMIC_FLAG_INIT) {
// initialize linked list // initialize linked list
@ -537,7 +538,7 @@ ColumnFamilyData* ColumnFamilySet::CreateColumnFamily(
assert(column_families_.find(name) == column_families_.end()); assert(column_families_.find(name) == column_families_.end());
ColumnFamilyData* new_cfd = ColumnFamilyData* new_cfd =
new ColumnFamilyData(id, name, dummy_versions, table_cache_, options, new ColumnFamilyData(id, name, dummy_versions, table_cache_, options,
db_options_, storage_options_, this); db_options_, env_options_, this);
Lock(); Lock();
column_families_.insert({name, id}); column_families_.insert({name, id});
column_family_data_.insert({id, new_cfd}); column_family_data_.insert({id, new_cfd});

@ -165,9 +165,11 @@ class ColumnFamilyData {
void SetLogNumber(uint64_t log_number) { log_number_ = log_number; } void SetLogNumber(uint64_t log_number) { log_number_ = log_number; }
uint64_t GetLogNumber() const { return 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_; } const Options* options() const { return &options_; }
// thread-safe
const EnvOptions* soptions() const; const EnvOptions* soptions() const;
const ImmutableCFOptions* ioptions() const { return &ioptions_; }
InternalStats* internal_stats() { return internal_stats_.get(); } InternalStats* internal_stats() { return internal_stats_.get(); }
@ -251,7 +253,7 @@ class ColumnFamilyData {
Version* dummy_versions, Cache* table_cache, Version* dummy_versions, Cache* table_cache,
const ColumnFamilyOptions& options, const ColumnFamilyOptions& options,
const DBOptions* db_options, const DBOptions* db_options,
const EnvOptions& storage_options, const EnvOptions& env_options,
ColumnFamilySet* column_family_set); ColumnFamilySet* column_family_set);
// Recalculate some small conditions, which are changed only during // Recalculate some small conditions, which are changed only during
@ -272,7 +274,8 @@ class ColumnFamilyData {
const InternalKeyComparator internal_comparator_; const InternalKeyComparator internal_comparator_;
Options const options_; const Options options_;
const ImmutableCFOptions ioptions_;
std::unique_ptr<TableCache> table_cache_; std::unique_ptr<TableCache> table_cache_;
@ -367,7 +370,7 @@ class ColumnFamilySet {
}; };
ColumnFamilySet(const std::string& dbname, const DBOptions* db_options, ColumnFamilySet(const std::string& dbname, const DBOptions* db_options,
const EnvOptions& storage_options, Cache* table_cache); const EnvOptions& env_options, Cache* table_cache);
~ColumnFamilySet(); ~ColumnFamilySet();
ColumnFamilyData* GetDefault() const; ColumnFamilyData* GetDefault() const;
@ -420,7 +423,7 @@ class ColumnFamilySet {
const std::string db_name_; const std::string db_name_;
const DBOptions* const db_options_; const DBOptions* const db_options_;
const EnvOptions storage_options_; const EnvOptions env_options_;
Cache* table_cache_; Cache* table_cache_;
std::atomic_flag spin_lock_; std::atomic_flag spin_lock_;
}; };

@ -356,7 +356,7 @@ DBImpl::DBImpl(const DBOptions& options, const std::string& dbname)
default_interval_to_delete_obsolete_WAL_(600), default_interval_to_delete_obsolete_WAL_(600),
flush_on_destroy_(false), flush_on_destroy_(false),
delayed_writes_(0), delayed_writes_(0),
storage_options_(options), env_options_(options),
bg_work_gate_closed_(false), bg_work_gate_closed_(false),
refitting_level_(false), refitting_level_(false),
opened_successfully_(false) { opened_successfully_(false) {
@ -372,7 +372,7 @@ DBImpl::DBImpl(const DBOptions& options, const std::string& dbname)
options_.table_cache_remove_scan_count_limit); options_.table_cache_remove_scan_count_limit);
versions_.reset( versions_.reset(
new VersionSet(dbname_, &options_, storage_options_, table_cache_.get())); new VersionSet(dbname_, &options_, env_options_, table_cache_.get()));
column_family_memtables_.reset( column_family_memtables_.reset(
new ColumnFamilyMemTablesImpl(versions_->GetColumnFamilySet())); new ColumnFamilyMemTablesImpl(versions_->GetColumnFamilySet()));
@ -453,7 +453,7 @@ Status DBImpl::NewDB() {
const std::string manifest = DescriptorFileName(dbname_, 1); const std::string manifest = DescriptorFileName(dbname_, 1);
unique_ptr<WritableFile> file; unique_ptr<WritableFile> file;
Status s = env_->NewWritableFile( Status s = env_->NewWritableFile(
manifest, &file, env_->OptimizeForManifestWrite(storage_options_)); manifest, &file, env_->OptimizeForManifestWrite(env_options_));
if (!s.ok()) { if (!s.ok()) {
return s; return s;
} }
@ -1075,7 +1075,7 @@ Status DBImpl::ReadFirstLine(const std::string& fname,
}; };
unique_ptr<SequentialFile> file; unique_ptr<SequentialFile> file;
Status status = env_->NewSequentialFile(fname, &file, storage_options_); Status status = env_->NewSequentialFile(fname, &file, env_options_);
if (!status.ok()) { if (!status.ok()) {
return status; return status;
@ -1275,7 +1275,7 @@ Status DBImpl::RecoverLogFile(uint64_t log_number, SequenceNumber* max_sequence,
// Open the log file // Open the log file
std::string fname = LogFileName(options_.wal_dir, log_number); std::string fname = LogFileName(options_.wal_dir, log_number);
unique_ptr<SequentialFile> file; unique_ptr<SequentialFile> file;
Status status = env_->NewSequentialFile(fname, &file, storage_options_); Status status = env_->NewSequentialFile(fname, &file, env_options_);
if (!status.ok()) { if (!status.ok()) {
MaybeIgnoreError(&status); MaybeIgnoreError(&status);
return status; return status;
@ -1425,10 +1425,11 @@ Status DBImpl::WriteLevel0TableForRecovery(ColumnFamilyData* cfd, MemTable* mem,
Status s; Status s;
{ {
mutex_.Unlock(); 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(), cfd->table_cache(), iter, &meta, cfd->internal_comparator(),
newest_snapshot, earliest_seqno_in_memtable, 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); LogFlush(options_.info_log);
mutex_.Lock(); mutex_.Lock();
} }
@ -1495,10 +1496,11 @@ Status DBImpl::WriteLevel0Table(ColumnFamilyData* cfd,
Log(options_.info_log, "[%s] Level-0 flush table #%" PRIu64 ": started", Log(options_.info_log, "[%s] Level-0 flush table #%" PRIu64 ": started",
cfd->GetName().c_str(), meta.fd.GetNumber()); 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(), cfd->table_cache(), iter, &meta, cfd->internal_comparator(),
newest_snapshot, earliest_seqno_in_memtable, 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); LogFlush(options_.info_log);
delete iter; delete iter;
Log(options_.info_log, Log(options_.info_log,
@ -2447,7 +2449,7 @@ Status DBImpl::OpenCompactionOutputFile(CompactionState* compact) {
// Make the output file // Make the output file
std::string fname = TableFileName(options_.db_paths, file_number, std::string fname = TableFileName(options_.db_paths, file_number,
compact->compaction->GetOutputPathId()); compact->compaction->GetOutputPathId());
Status s = env_->NewWritableFile(fname, &compact->outfile, storage_options_); Status s = env_->NewWritableFile(fname, &compact->outfile, env_options_);
if (s.ok()) { if (s.ok()) {
compact->outfile->SetIOPriority(Env::IO_LOW); compact->outfile->SetIOPriority(Env::IO_LOW);
@ -2456,8 +2458,9 @@ Status DBImpl::OpenCompactionOutputFile(CompactionState* compact) {
ColumnFamilyData* cfd = compact->compaction->column_family_data(); ColumnFamilyData* cfd = compact->compaction->column_family_data();
compact->builder.reset(NewTableBuilder( compact->builder.reset(NewTableBuilder(
*cfd->options(), cfd->internal_comparator(), compact->outfile.get(), *cfd->ioptions(), cfd->internal_comparator(), compact->outfile.get(),
compact->compaction->OutputCompressionType())); compact->compaction->OutputCompressionType(),
cfd->options()->compression_opts));
} }
LogFlush(options_.info_log); LogFlush(options_.info_log);
return s; return s;
@ -2506,7 +2509,7 @@ Status DBImpl::FinishCompactionOutputFile(CompactionState* compact,
ColumnFamilyData* cfd = compact->compaction->column_family_data(); ColumnFamilyData* cfd = compact->compaction->column_family_data();
FileDescriptor fd(output_number, output_path_id, current_bytes); FileDescriptor fd(output_number, output_path_id, current_bytes);
Iterator* iter = cfd->table_cache()->NewIterator( Iterator* iter = cfd->table_cache()->NewIterator(
ReadOptions(), storage_options_, cfd->internal_comparator(), fd); ReadOptions(), env_options_, cfd->internal_comparator(), fd);
s = iter->status(); s = iter->status();
delete iter; delete iter;
if (s.ok()) { if (s.ok()) {
@ -3355,7 +3358,7 @@ Iterator* DBImpl::NewInternalIterator(const ReadOptions& options,
// Collect all needed child iterators for immutable memtables // Collect all needed child iterators for immutable memtables
super_version->imm->AddIterators(options, &merge_iter_builder); super_version->imm->AddIterators(options, &merge_iter_builder);
// Collect iterators for files in L0 - Ln // Collect iterators for files in L0 - Ln
super_version->current->AddIterators(options, storage_options_, super_version->current->AddIterators(options, env_options_,
&merge_iter_builder); &merge_iter_builder);
internal_iter = merge_iter_builder.Finish(); internal_iter = merge_iter_builder.Finish();
} else { } else {
@ -3366,7 +3369,7 @@ Iterator* DBImpl::NewInternalIterator(const ReadOptions& options,
// Collect all needed child iterators for immutable memtables // Collect all needed child iterators for immutable memtables
super_version->imm->AddIterators(options, &iterator_list); super_version->imm->AddIterators(options, &iterator_list);
// Collect iterators for files in L0 - Ln // Collect iterators for files in L0 - Ln
super_version->current->AddIterators(options, storage_options_, super_version->current->AddIterators(options, env_options_,
&iterator_list); &iterator_list);
internal_iter = NewMergingIterator(&cfd->internal_comparator(), internal_iter = NewMergingIterator(&cfd->internal_comparator(),
&iterator_list[0], iterator_list.size()); &iterator_list[0], iterator_list.size());
@ -4377,7 +4380,7 @@ Status DBImpl::SetNewMemtableAndNewLogFile(ColumnFamilyData* cfd,
if (creating_new_log) { if (creating_new_log) {
s = env_->NewWritableFile(LogFileName(options_.wal_dir, new_log_number), s = env_->NewWritableFile(LogFileName(options_.wal_dir, new_log_number),
&lfile, &lfile,
env_->OptimizeForLogWrite(storage_options_)); env_->OptimizeForLogWrite(env_options_));
if (s.ok()) { if (s.ok()) {
// Our final size should be less than write_buffer_size // Our final size should be less than write_buffer_size
// (compression, etc) but err on the side of caution. // (compression, etc) but err on the side of caution.
@ -4615,7 +4618,7 @@ Status DBImpl::GetUpdatesSince(
return s; return s;
} }
iter->reset(new TransactionLogIteratorImpl(options_.wal_dir, &options_, iter->reset(new TransactionLogIteratorImpl(options_.wal_dir, &options_,
read_options, storage_options_, read_options, env_options_,
seq, std::move(wal_files), this)); seq, std::move(wal_files), this));
return (*iter)->status(); return (*iter)->status();
} }

@ -607,7 +607,7 @@ class DBImpl : public DB {
int delayed_writes_; int delayed_writes_;
// The options to access storage files // The options to access storage files
const EnvOptions storage_options_; const EnvOptions env_options_;
// A value of true temporarily disables scheduling of background work // A value of true temporarily disables scheduling of background work
bool bg_work_gate_closed_; bool bg_work_gate_closed_;

@ -192,16 +192,17 @@ extern const uint64_t kPlainTableMagicNumber;
class TestPlainTableReader : public PlainTableReader { class TestPlainTableReader : public PlainTableReader {
public: public:
TestPlainTableReader(const EnvOptions& storage_options, TestPlainTableReader(const EnvOptions& env_options,
const InternalKeyComparator& icomparator, const InternalKeyComparator& icomparator,
EncodingType encoding_type, uint64_t file_size, EncodingType encoding_type, uint64_t file_size,
int bloom_bits_per_key, double hash_table_ratio, int bloom_bits_per_key, double hash_table_ratio,
size_t index_sparseness, size_t index_sparseness,
const TableProperties* table_properties, const TableProperties* table_properties,
unique_ptr<RandomAccessFile>&& file, unique_ptr<RandomAccessFile>&& file,
const Options& options, bool* expect_bloom_not_match, const ImmutableCFOptions& ioptions,
bool* expect_bloom_not_match,
bool store_index_in_file) 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), encoding_type, file_size, table_properties),
expect_bloom_not_match_(expect_bloom_not_match) { expect_bloom_not_match_(expect_bloom_not_match) {
Status s = MmapDataFile(); Status s = MmapDataFile();
@ -218,7 +219,7 @@ class TestPlainTableReader : public PlainTableReader {
PlainTablePropertyNames::kBloomVersion); PlainTablePropertyNames::kBloomVersion);
ASSERT_TRUE(bloom_version_ptr != props->user_collected_properties.end()); ASSERT_TRUE(bloom_version_ptr != props->user_collected_properties.end());
ASSERT_EQ(bloom_version_ptr->second, std::string("1")); 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( auto num_blocks_ptr = props->user_collected_properties.find(
PlainTablePropertyNames::kNumBloomBlocks); PlainTablePropertyNames::kNumBloomBlocks);
ASSERT_TRUE(num_blocks_ptr != props->user_collected_properties.end()); 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), store_index_in_file_(options.store_index_in_file),
expect_bloom_not_match_(expect_bloom_not_match) {} 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, const InternalKeyComparator& internal_comparator,
unique_ptr<RandomAccessFile>&& file, uint64_t file_size, unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
unique_ptr<TableReader>* table) const override { unique_ptr<TableReader>* table) const override {
TableProperties* props = nullptr; TableProperties* props = nullptr;
auto s = ReadTableProperties(file.get(), file_size, kPlainTableMagicNumber, 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()); ASSERT_TRUE(s.ok());
if (store_index_in_file_) { if (store_index_in_file_) {
BlockHandle bloom_block_handle; BlockHandle bloom_block_handle;
s = FindMetaBlock(file.get(), file_size, kPlainTableMagicNumber, s = FindMetaBlock(file.get(), file_size, kPlainTableMagicNumber,
options.env, BloomBlockBuilder::kBloomBlock, ioptions.env, BloomBlockBuilder::kBloomBlock,
&bloom_block_handle); &bloom_block_handle);
ASSERT_TRUE(s.ok()); ASSERT_TRUE(s.ok());
BlockHandle index_block_handle; BlockHandle index_block_handle;
s = FindMetaBlock( s = FindMetaBlock(
file.get(), file_size, kPlainTableMagicNumber, options.env, file.get(), file_size, kPlainTableMagicNumber, ioptions.env,
PlainTableIndexBuilder::kPlainTableIndexBlock, &index_block_handle); PlainTableIndexBuilder::kPlainTableIndexBlock, &index_block_handle);
ASSERT_TRUE(s.ok()); ASSERT_TRUE(s.ok());
} }
@ -284,9 +286,9 @@ class TestPlainTableFactory : public PlainTableFactory {
DecodeFixed32(encoding_type_prop->second.c_str())); DecodeFixed32(encoding_type_prop->second.c_str()));
std::unique_ptr<PlainTableReader> new_reader(new TestPlainTableReader( std::unique_ptr<PlainTableReader> 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, 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_)); store_index_in_file_));
*table = std::move(new_reader); *table = std::move(new_reader);

@ -46,6 +46,8 @@
#include "rocksdb/comparator.h" #include "rocksdb/comparator.h"
#include "rocksdb/db.h" #include "rocksdb/db.h"
#include "rocksdb/env.h" #include "rocksdb/env.h"
#include "rocksdb/options.h"
#include "rocksdb/immutable_options.h"
namespace rocksdb { namespace rocksdb {
@ -58,6 +60,7 @@ class Repairer {
env_(options.env), env_(options.env),
icmp_(options.comparator), icmp_(options.comparator),
options_(SanitizeOptions(dbname, &icmp_, options)), options_(SanitizeOptions(dbname, &icmp_, options)),
ioptions_(options_),
raw_table_cache_( raw_table_cache_(
// TableCache can be small since we expect each table to be opened // TableCache can be small since we expect each table to be opened
// once. // once.
@ -65,7 +68,7 @@ class Repairer {
options_.table_cache_remove_scan_count_limit)), options_.table_cache_remove_scan_count_limit)),
next_file_number_(1) { next_file_number_(1) {
table_cache_ = table_cache_ =
new TableCache(&options_, storage_options_, raw_table_cache_.get()); new TableCache(ioptions_, env_options_, raw_table_cache_.get());
edit_ = new VersionEdit(); edit_ = new VersionEdit();
} }
@ -107,8 +110,9 @@ class Repairer {
std::string const dbname_; std::string const dbname_;
Env* const env_; Env* const env_;
InternalKeyComparator const icmp_; const InternalKeyComparator icmp_;
Options const options_; const Options options_;
const ImmutableCFOptions ioptions_;
std::shared_ptr<Cache> raw_table_cache_; std::shared_ptr<Cache> raw_table_cache_;
TableCache* table_cache_; TableCache* table_cache_;
VersionEdit* edit_; VersionEdit* edit_;
@ -118,7 +122,7 @@ class Repairer {
std::vector<uint64_t> logs_; std::vector<uint64_t> logs_;
std::vector<TableInfo> tables_; std::vector<TableInfo> tables_;
uint64_t next_file_number_; uint64_t next_file_number_;
const EnvOptions storage_options_; const EnvOptions env_options_;
Status FindFiles() { Status FindFiles() {
std::vector<std::string> filenames; std::vector<std::string> filenames;
@ -190,7 +194,7 @@ class Repairer {
// Open the log file // Open the log file
std::string logname = LogFileName(dbname_, log); std::string logname = LogFileName(dbname_, log);
unique_ptr<SequentialFile> lfile; unique_ptr<SequentialFile> lfile;
Status status = env_->NewSequentialFile(logname, &lfile, storage_options_); Status status = env_->NewSequentialFile(logname, &lfile, env_options_);
if (!status.ok()) { if (!status.ok()) {
return status; return status;
} }
@ -239,8 +243,9 @@ class Repairer {
ReadOptions ro; ReadOptions ro;
ro.total_order_seek = true; ro.total_order_seek = true;
Iterator* iter = mem->NewIterator(ro); Iterator* iter = mem->NewIterator(ro);
status = BuildTable(dbname_, env_, options_, storage_options_, table_cache_, status = BuildTable(dbname_, env_, ioptions_, env_options_, table_cache_,
iter, &meta, icmp_, 0, 0, kNoCompression); iter, &meta, icmp_, 0, 0, kNoCompression,
CompressionOptions());
delete iter; delete iter;
delete mem->Unref(); delete mem->Unref();
delete cf_mems_default; delete cf_mems_default;
@ -286,7 +291,7 @@ class Repairer {
file_size); file_size);
if (status.ok()) { if (status.ok()) {
Iterator* iter = table_cache_->NewIterator( Iterator* iter = table_cache_->NewIterator(
ReadOptions(), storage_options_, icmp_, t->meta.fd); ReadOptions(), env_options_, icmp_, t->meta.fd);
bool empty = true; bool empty = true;
ParsedInternalKey parsed; ParsedInternalKey parsed;
t->min_sequence = 0; t->min_sequence = 0;
@ -326,7 +331,7 @@ class Repairer {
std::string tmp = TempFileName(dbname_, 1); std::string tmp = TempFileName(dbname_, 1);
unique_ptr<WritableFile> file; unique_ptr<WritableFile> file;
Status status = env_->NewWritableFile( Status status = env_->NewWritableFile(
tmp, &file, env_->OptimizeForManifestWrite(storage_options_)); tmp, &file, env_->OptimizeForManifestWrite(env_options_));
if (!status.ok()) { if (!status.ok()) {
return status; return status;
} }

@ -79,7 +79,8 @@ public:
// for the duration of the returned table's lifetime. // for the duration of the returned table's lifetime.
// //
// *file must remain live while this Table is in use. // *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<RandomAccessFile> && file, uint64_t file_size, unique_ptr<RandomAccessFile> && file, uint64_t file_size,
unique_ptr<TableReader>* table_reader); unique_ptr<TableReader>* table_reader);
@ -160,14 +161,14 @@ private:
struct SimpleTableReader::Rep { struct SimpleTableReader::Rep {
~Rep() { ~Rep() {
} }
Rep(const EnvOptions& storage_options, uint64_t index_start_offset, Rep(const ImmutableCFOptions& ioptions, const EnvOptions& env_options,
int num_entries) : uint64_t index_start_offset, int num_entries) :
soptions(storage_options), index_start_offset(index_start_offset), ioptions(ioptions), env_options(env_options),
num_entries(num_entries) { index_start_offset(index_start_offset), num_entries(num_entries) {
} }
Options options; const ImmutableCFOptions& ioptions;
const EnvOptions& soptions; const EnvOptions& env_options;
Status status; Status status;
unique_ptr<RandomAccessFile> file; unique_ptr<RandomAccessFile> file;
uint64_t index_start_offset; uint64_t index_start_offset;
@ -187,8 +188,8 @@ SimpleTableReader::~SimpleTableReader() {
delete rep_; delete rep_;
} }
Status SimpleTableReader::Open(const Options& options, Status SimpleTableReader::Open(const ImmutableCFOptions& ioptions,
const EnvOptions& soptions, const EnvOptions& env_options,
unique_ptr<RandomAccessFile> && file, unique_ptr<RandomAccessFile> && file,
uint64_t size, uint64_t size,
unique_ptr<TableReader>* table_reader) { unique_ptr<TableReader>* table_reader) {
@ -201,12 +202,10 @@ Status SimpleTableReader::Open(const Options& options,
int num_entries = (size - Rep::offset_length - index_start_offset) int num_entries = (size - Rep::offset_length - index_start_offset)
/ (Rep::GetInternalKeyLength() + Rep::offset_length); / (Rep::GetInternalKeyLength() + Rep::offset_length);
SimpleTableReader::Rep* rep = new SimpleTableReader::Rep(soptions, SimpleTableReader::Rep* rep = new SimpleTableReader::Rep(
index_start_offset, ioptions, env_options, index_start_offset, num_entries);
num_entries);
rep->file = std::move(file); rep->file = std::move(file);
rep->options = options;
table_reader->reset(new SimpleTableReader(rep)); table_reader->reset(new SimpleTableReader(rep));
} }
return s; return s;
@ -248,7 +247,7 @@ Status SimpleTableReader::GetOffset(const Slice& target, uint64_t* offset) {
return s; return s;
} }
InternalKeyComparator ikc(rep_->options.comparator); InternalKeyComparator ikc(rep_->ioptions.comparator);
int compare_result = ikc.Compare(tmp_slice, target); int compare_result = ikc.Compare(tmp_slice, target);
if (compare_result < 0) { if (compare_result < 0) {
@ -382,7 +381,7 @@ void SimpleTableIterator::Prev() {
} }
Slice SimpleTableIterator::key() const { Slice SimpleTableIterator::key() const {
Log(table_->rep_->options.info_log, "key!!!!"); Log(table_->rep_->ioptions.info_log, "key!!!!");
return key_; return key_;
} }
@ -401,7 +400,7 @@ public:
// caller to close the file after calling Finish(). The output file // caller to close the file after calling Finish(). The output file
// will be part of level specified by 'level'. A value of -1 means // 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. // 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); CompressionType compression_type);
// REQUIRES: Either Finish() or Abandon() has been called. // REQUIRES: Either Finish() or Abandon() has been called.
@ -444,7 +443,7 @@ private:
}; };
struct SimpleTableBuilder::Rep { struct SimpleTableBuilder::Rep {
Options options; const ImmutableCFOptions& ioptions;
WritableFile* file; WritableFile* file;
uint64_t offset = 0; uint64_t offset = 0;
Status status; Status status;
@ -463,17 +462,17 @@ struct SimpleTableBuilder::Rep {
std::string index; std::string index;
Rep(const Options& opt, WritableFile* f) : Rep(const ImmutableCFOptions& iopt, WritableFile* f) :
options(opt), file(f) { ioptions(iopt), file(f) {
} }
~Rep() { ~Rep() {
} }
}; };
SimpleTableBuilder::SimpleTableBuilder(const Options& options, SimpleTableBuilder::SimpleTableBuilder(const ImmutableCFOptions& ioptions,
WritableFile* file, WritableFile* file,
CompressionType compression_type) : CompressionType compression_type) :
rep_(new SimpleTableBuilder::Rep(options, file)) { rep_(new SimpleTableBuilder::Rep(ioptions, file)) {
} }
SimpleTableBuilder::~SimpleTableBuilder() { SimpleTableBuilder::~SimpleTableBuilder() {
@ -546,15 +545,18 @@ public:
const char* Name() const override { const char* Name() const override {
return "SimpleTable"; return "SimpleTable";
} }
Status NewTableReader(const Options& options, const EnvOptions& soptions, Status NewTableReader(const ImmutableCFOptions& ioptions,
const EnvOptions& env_options,
const InternalKeyComparator& internal_key, const InternalKeyComparator& internal_key,
unique_ptr<RandomAccessFile>&& file, uint64_t file_size, unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
unique_ptr<TableReader>* table_reader) const; unique_ptr<TableReader>* table_reader) const;
TableBuilder* NewTableBuilder(const Options& options, TableBuilder* NewTableBuilder(
const ImmutableCFOptions& ioptions,
const InternalKeyComparator& internal_key, const InternalKeyComparator& internal_key,
WritableFile* file, WritableFile* file,
CompressionType compression_type) const; const CompressionType compression_type,
const CompressionOptions& compression_opts) const;
virtual Status SanitizeDBOptions(const DBOptions* db_opts) const override { virtual Status SanitizeDBOptions(const DBOptions* db_opts) const override {
return Status::OK(); return Status::OK();
@ -566,19 +568,22 @@ public:
}; };
Status SimpleTableFactory::NewTableReader( Status SimpleTableFactory::NewTableReader(
const Options& options, const EnvOptions& soptions, const ImmutableCFOptions& ioptions,
const EnvOptions& env_options,
const InternalKeyComparator& internal_key, const InternalKeyComparator& internal_key,
unique_ptr<RandomAccessFile>&& file, uint64_t file_size, unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
unique_ptr<TableReader>* table_reader) const { unique_ptr<TableReader>* table_reader) const {
return SimpleTableReader::Open(options, soptions, std::move(file), file_size, return SimpleTableReader::Open(ioptions, env_options, std::move(file),
table_reader); file_size, table_reader);
} }
TableBuilder* SimpleTableFactory::NewTableBuilder( TableBuilder* SimpleTableFactory::NewTableBuilder(
const Options& options, const InternalKeyComparator& internal_key, const ImmutableCFOptions& ioptions,
WritableFile* file, CompressionType compression_type) const { const InternalKeyComparator& internal_key,
return new SimpleTableBuilder(options, file, compression_type); WritableFile* file, const CompressionType compression_type,
const CompressionOptions& compression_opts) const {
return new SimpleTableBuilder(ioptions, file, compression_type);
} }
class SimpleTableDBTest { class SimpleTableDBTest {

@ -36,12 +36,10 @@ static Slice GetSliceForFileNumber(const uint64_t* file_number) {
sizeof(*file_number)); sizeof(*file_number));
} }
TableCache::TableCache(const Options* options, TableCache::TableCache(const ImmutableCFOptions& ioptions,
const EnvOptions& storage_options, Cache* const cache) const EnvOptions& env_options, Cache* const cache)
: env_(options->env), : ioptions_(ioptions),
db_paths_(options->db_paths), env_options_(env_options),
options_(options),
storage_options_(storage_options),
cache_(cache) {} cache_(cache) {}
TableCache::~TableCache() { TableCache::~TableCache() {
@ -55,7 +53,7 @@ void TableCache::ReleaseHandle(Cache::Handle* handle) {
cache_->Release(handle); cache_->Release(handle);
} }
Status TableCache::FindTable(const EnvOptions& toptions, Status TableCache::FindTable(const EnvOptions& env_options,
const InternalKeyComparator& internal_comparator, const InternalKeyComparator& internal_comparator,
const FileDescriptor& fd, Cache::Handle** handle, const FileDescriptor& fd, Cache::Handle** handle,
const bool no_io) { 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"); return Status::Incomplete("Table not found in table_cache, no_io is set");
} }
std::string fname = std::string fname =
TableFileName(db_paths_, fd.GetNumber(), fd.GetPathId()); TableFileName(ioptions_.db_paths, fd.GetNumber(), fd.GetPathId());
unique_ptr<RandomAccessFile> file; unique_ptr<RandomAccessFile> file;
unique_ptr<TableReader> table_reader; unique_ptr<TableReader> table_reader;
s = env_->NewRandomAccessFile(fname, &file, toptions); s = ioptions_.env->NewRandomAccessFile(fname, &file, env_options);
RecordTick(options_->statistics.get(), NO_FILE_OPENS); RecordTick(ioptions_.statistics, NO_FILE_OPENS);
if (s.ok()) { if (s.ok()) {
if (options_->advise_random_on_open) { if (ioptions_.advise_random_on_open) {
file->Hint(RandomAccessFile::RANDOM); file->Hint(RandomAccessFile::RANDOM);
} }
StopWatch sw(env_, options_->statistics.get(), TABLE_OPEN_IO_MICROS); StopWatch sw(ioptions_.env, ioptions_.statistics, TABLE_OPEN_IO_MICROS);
s = options_->table_factory->NewTableReader( s = ioptions_.table_factory->NewTableReader(
*options_, toptions, internal_comparator, std::move(file), ioptions_, env_options, internal_comparator, std::move(file),
fd.GetFileSize(), &table_reader); fd.GetFileSize(), &table_reader);
} }
if (!s.ok()) { if (!s.ok()) {
assert(table_reader == nullptr); 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, // We do not cache error results so that if the error is transient,
// or somebody repairs the file, we recover automatically. // or somebody repairs the file, we recover automatically.
} else { } else {
@ -97,7 +95,7 @@ Status TableCache::FindTable(const EnvOptions& toptions,
} }
Iterator* TableCache::NewIterator(const ReadOptions& options, Iterator* TableCache::NewIterator(const ReadOptions& options,
const EnvOptions& toptions, const EnvOptions& env_options,
const InternalKeyComparator& icomparator, const InternalKeyComparator& icomparator,
const FileDescriptor& fd, const FileDescriptor& fd,
TableReader** table_reader_ptr, TableReader** table_reader_ptr,
@ -109,7 +107,7 @@ Iterator* TableCache::NewIterator(const ReadOptions& options,
Cache::Handle* handle = nullptr; Cache::Handle* handle = nullptr;
Status s; Status s;
if (table_reader == nullptr) { if (table_reader == nullptr) {
s = FindTable(toptions, icomparator, fd, &handle, s = FindTable(env_options, icomparator, fd, &handle,
options.read_tier == kBlockCacheTier); options.read_tier == kBlockCacheTier);
if (!s.ok()) { if (!s.ok()) {
return NewErrorIterator(s, arena); return NewErrorIterator(s, arena);
@ -142,7 +140,7 @@ Status TableCache::Get(const ReadOptions& options,
Status s; Status s;
Cache::Handle* handle = nullptr; Cache::Handle* handle = nullptr;
if (!t) { if (!t) {
s = FindTable(storage_options_, internal_comparator, fd, &handle, s = FindTable(env_options_, internal_comparator, fd, &handle,
options.read_tier == kBlockCacheTier); options.read_tier == kBlockCacheTier);
if (s.ok()) { if (s.ok()) {
t = GetTableReaderFromHandle(handle); t = GetTableReaderFromHandle(handle);
@ -160,8 +158,9 @@ Status TableCache::Get(const ReadOptions& options,
} }
return s; return s;
} }
Status TableCache::GetTableProperties( Status TableCache::GetTableProperties(
const EnvOptions& toptions, const EnvOptions& env_options,
const InternalKeyComparator& internal_comparator, const FileDescriptor& fd, const InternalKeyComparator& internal_comparator, const FileDescriptor& fd,
std::shared_ptr<const TableProperties>* properties, bool no_io) { std::shared_ptr<const TableProperties>* properties, bool no_io) {
Status s; Status s;
@ -174,7 +173,7 @@ Status TableCache::GetTableProperties(
} }
Cache::Handle* table_handle = nullptr; 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()) { if (!s.ok()) {
return s; return s;
} }
@ -186,7 +185,7 @@ Status TableCache::GetTableProperties(
} }
size_t TableCache::GetMemoryUsageByTableReader( size_t TableCache::GetMemoryUsageByTableReader(
const EnvOptions& toptions, const EnvOptions& env_options,
const InternalKeyComparator& internal_comparator, const InternalKeyComparator& internal_comparator,
const FileDescriptor& fd) { const FileDescriptor& fd) {
Status s; Status s;
@ -197,7 +196,7 @@ size_t TableCache::GetMemoryUsageByTableReader(
} }
Cache::Handle* table_handle = nullptr; 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()) { if (!s.ok()) {
return 0; return 0;
} }

@ -19,6 +19,7 @@
#include "rocksdb/cache.h" #include "rocksdb/cache.h"
#include "rocksdb/env.h" #include "rocksdb/env.h"
#include "rocksdb/table.h" #include "rocksdb/table.h"
#include "rocksdb/options.h"
#include "table/table_reader.h" #include "table/table_reader.h"
namespace rocksdb { namespace rocksdb {
@ -29,8 +30,8 @@ struct FileDescriptor;
class TableCache { class TableCache {
public: public:
TableCache(const Options* options, const EnvOptions& storage_options, TableCache(const ImmutableCFOptions& ioptions,
Cache* cache); const EnvOptions& storage_options, Cache* cache);
~TableCache(); ~TableCache();
// Return an iterator for the specified file number (the corresponding // Return an iterator for the specified file number (the corresponding
@ -91,10 +92,8 @@ class TableCache {
void ReleaseHandle(Cache::Handle* handle); void ReleaseHandle(Cache::Handle* handle);
private: private:
Env* const env_; const ImmutableCFOptions& ioptions_;
const std::vector<DbPath> db_paths_; const EnvOptions& env_options_;
const Options* options_;
const EnvOptions& storage_options_;
Cache* const cache_; Cache* const cache_;
}; };

@ -90,7 +90,8 @@ void MakeBuilder(const Options& options,
std::unique_ptr<TableBuilder>* builder) { std::unique_ptr<TableBuilder>* builder) {
writable->reset(new FakeWritableFile); writable->reset(new FakeWritableFile);
builder->reset(options.table_factory->NewTableBuilder( 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 } // namespace

@ -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 <vector>
#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<DbPath> 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

@ -23,6 +23,7 @@
#include "rocksdb/env.h" #include "rocksdb/env.h"
#include "rocksdb/iterator.h" #include "rocksdb/iterator.h"
#include "rocksdb/options.h" #include "rocksdb/options.h"
#include "rocksdb/immutable_options.h"
#include "rocksdb/status.h" #include "rocksdb/status.h"
namespace rocksdb { namespace rocksdb {
@ -293,14 +294,15 @@ class TableFactory {
// and cache the table object returned. // and cache the table object returned.
// (1) SstFileReader (for SST Dump) opens the table and dump the table // (1) SstFileReader (for SST Dump) opens the table and dump the table
// contents using the interator of 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 // Multiple configured can be accessed from there, including and not
// limited to block cache and key comparators. // limited to block cache and key comparators.
// file is a file handler to handle the file for the table // file is a file handler to handle the file for the table
// file_size is the physical file size of the file // file_size is the physical file size of the file
// table_reader is the output table reader // table_reader is the output table reader
virtual Status NewTableReader( virtual Status NewTableReader(
const Options& options, const EnvOptions& soptions, const ImmutableCFOptions& ioptions, const EnvOptions& env_options,
const InternalKeyComparator& internal_comparator, const InternalKeyComparator& internal_comparator,
unique_ptr<RandomAccessFile>&& file, uint64_t file_size, unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
unique_ptr<TableReader>* table_reader) const = 0; unique_ptr<TableReader>* table_reader) const = 0;
@ -318,14 +320,17 @@ class TableFactory {
// (4) When running Repairer, it creates a table builder to convert logs to // (4) When running Repairer, it creates a table builder to convert logs to
// SST files (In Repairer::ConvertLogToTable() by calling BuildTable()) // SST files (In Repairer::ConvertLogToTable() by calling BuildTable())
// //
// options is the general options. Multiple configured can be acceseed from // ImmutableCFOptions is a subset of Options that can not be altered.
// there, including and not limited to compression options. // Multiple configured can be acceseed from there, including and not limited
// file is a handle of a writable file. It is the caller's responsibility to // to compression options. file is a handle of a writable file.
// keep the file open and close the file after closing the table builder. // It is the caller's responsibility to keep the file open and close the file
// compression_type is the compression type to use in this table. // after closing the table builder. compression_type is the compression type
// to use in this table.
virtual TableBuilder* NewTableBuilder( virtual TableBuilder* NewTableBuilder(
const Options& options, const InternalKeyComparator& internal_comparator, const ImmutableCFOptions& ioptions,
WritableFile* file, CompressionType compression_type) const = 0; const InternalKeyComparator& internal_comparator,
WritableFile* file, const CompressionType compression_type,
const CompressionOptions& compression_opts) const = 0;
// Sanitizes the specified DB Options. // Sanitizes the specified DB Options.
// //

@ -39,7 +39,7 @@ extern const uint64_t kLegacyBlockBasedTableMagicNumber;
extern const uint64_t kCuckooTableMagicNumber; extern const uint64_t kCuckooTableMagicNumber;
Status AdaptiveTableFactory::NewTableReader( Status AdaptiveTableFactory::NewTableReader(
const Options& options, const EnvOptions& soptions, const ImmutableCFOptions& ioptions, const EnvOptions& env_options,
const InternalKeyComparator& icomp, unique_ptr<RandomAccessFile>&& file, const InternalKeyComparator& icomp, unique_ptr<RandomAccessFile>&& file,
uint64_t file_size, unique_ptr<TableReader>* table) const { uint64_t file_size, unique_ptr<TableReader>* table) const {
Footer footer; Footer footer;
@ -50,24 +50,26 @@ Status AdaptiveTableFactory::NewTableReader(
if (footer.table_magic_number() == kPlainTableMagicNumber || if (footer.table_magic_number() == kPlainTableMagicNumber ||
footer.table_magic_number() == kLegacyPlainTableMagicNumber) { footer.table_magic_number() == kLegacyPlainTableMagicNumber) {
return plain_table_factory_->NewTableReader( 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 || } else if (footer.table_magic_number() == kBlockBasedTableMagicNumber ||
footer.table_magic_number() == kLegacyBlockBasedTableMagicNumber) { footer.table_magic_number() == kLegacyBlockBasedTableMagicNumber) {
return block_based_table_factory_->NewTableReader( 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) { } else if (footer.table_magic_number() == kCuckooTableMagicNumber) {
return cuckoo_table_factory_->NewTableReader( 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 { } else {
return Status::NotSupported("Unidentified table format"); return Status::NotSupported("Unidentified table format");
} }
} }
TableBuilder* AdaptiveTableFactory::NewTableBuilder( TableBuilder* AdaptiveTableFactory::NewTableBuilder(
const Options& options, const InternalKeyComparator& internal_comparator, const ImmutableCFOptions& ioptions,
WritableFile* file, CompressionType compression_type) const { const InternalKeyComparator& internal_comparator,
return table_factory_to_write_->NewTableBuilder(options, internal_comparator, WritableFile* file, const CompressionType compression_type,
file, 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 { std::string AdaptiveTableFactory::GetPrintableTableOptions() const {

@ -12,7 +12,6 @@
namespace rocksdb { namespace rocksdb {
struct Options;
struct EnvOptions; struct EnvOptions;
using std::unique_ptr; using std::unique_ptr;
@ -31,16 +30,21 @@ class AdaptiveTableFactory : public TableFactory {
std::shared_ptr<TableFactory> block_based_table_factory, std::shared_ptr<TableFactory> block_based_table_factory,
std::shared_ptr<TableFactory> plain_table_factory, std::shared_ptr<TableFactory> plain_table_factory,
std::shared_ptr<TableFactory> cuckoo_table_factory); std::shared_ptr<TableFactory> cuckoo_table_factory);
const char* Name() const override { return "AdaptiveTableFactory"; } const char* Name() const override { return "AdaptiveTableFactory"; }
Status NewTableReader(const Options& options, const EnvOptions& soptions,
Status NewTableReader(
const ImmutableCFOptions& ioptions, const EnvOptions& env_options,
const InternalKeyComparator& internal_comparator, const InternalKeyComparator& internal_comparator,
unique_ptr<RandomAccessFile>&& file, uint64_t file_size, unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
unique_ptr<TableReader>* table) const override; unique_ptr<TableReader>* table) const override;
TableBuilder* NewTableBuilder(const Options& options,
TableBuilder* NewTableBuilder(
const ImmutableCFOptions& ioptions,
const InternalKeyComparator& icomparator, const InternalKeyComparator& icomparator,
WritableFile* file, WritableFile* file,
CompressionType compression_type) const const CompressionType compression_type,
override; const CompressionOptions& compression_opts) const override;
// Sanitizes the specified DB Options. // Sanitizes the specified DB Options.
Status SanitizeDBOptions(const DBOptions* db_opts) const override { Status SanitizeDBOptions(const DBOptions* db_opts) const override {

@ -25,7 +25,6 @@
#include "rocksdb/env.h" #include "rocksdb/env.h"
#include "rocksdb/filter_policy.h" #include "rocksdb/filter_policy.h"
#include "rocksdb/flush_block_policy.h" #include "rocksdb/flush_block_policy.h"
#include "rocksdb/options.h"
#include "rocksdb/table.h" #include "rocksdb/table.h"
#include "table/block.h" #include "table/block.h"
@ -385,7 +384,7 @@ class BlockBasedTableBuilder::BlockBasedTablePropertiesCollector
}; };
struct BlockBasedTableBuilder::Rep { struct BlockBasedTableBuilder::Rep {
const Options options; const ImmutableCFOptions ioptions;
const BlockBasedTableOptions table_options; const BlockBasedTableOptions table_options;
const InternalKeyComparator& internal_comparator; const InternalKeyComparator& internal_comparator;
WritableFile* file; WritableFile* file;
@ -397,7 +396,8 @@ struct BlockBasedTableBuilder::Rep {
std::unique_ptr<IndexBuilder> index_builder; std::unique_ptr<IndexBuilder> index_builder;
std::string last_key; std::string last_key;
CompressionType compression_type; const CompressionType compression_type;
const CompressionOptions compression_opts;
TableProperties props; TableProperties props;
bool closed = false; // Either Finish() or Abandon() has been called. bool closed = false; // Either Finish() or Abandon() has been called.
@ -413,27 +413,31 @@ struct BlockBasedTableBuilder::Rep {
std::vector<std::unique_ptr<TablePropertiesCollector>> std::vector<std::unique_ptr<TablePropertiesCollector>>
table_properties_collectors; table_properties_collectors;
Rep(const Options& opt, const BlockBasedTableOptions& table_opt, Rep(const ImmutableCFOptions& ioptions,
const BlockBasedTableOptions& table_opt,
const InternalKeyComparator& icomparator, const InternalKeyComparator& icomparator,
WritableFile* f, CompressionType compression_type) WritableFile* f, const CompressionType compression_type,
: options(opt), const CompressionOptions& compression_opts)
: ioptions(ioptions),
table_options(table_opt), table_options(table_opt),
internal_comparator(icomparator), internal_comparator(icomparator),
file(f), file(f),
data_block(table_options.block_restart_interval), data_block(table_options.block_restart_interval),
internal_prefix_transform(options.prefix_extractor.get()), internal_prefix_transform(ioptions.prefix_extractor),
index_builder(CreateIndexBuilder( index_builder(CreateIndexBuilder(
table_options.index_type, &internal_comparator, table_options.index_type, &internal_comparator,
&this->internal_prefix_transform)), &this->internal_prefix_transform)),
compression_type(compression_type), compression_type(compression_type),
compression_opts(compression_opts),
filter_block(table_options.filter_policy == nullptr ? filter_block(table_options.filter_policy == nullptr ?
nullptr : nullptr :
new FilterBlockBuilder(opt, table_options, &internal_comparator)), new FilterBlockBuilder(ioptions.prefix_extractor,
table_options, &internal_comparator)),
flush_block_policy( flush_block_policy(
table_options.flush_block_policy_factory->NewFlushBlockPolicy( table_options.flush_block_policy_factory->NewFlushBlockPolicy(
table_options, data_block)) { table_options, data_block)) {
for (auto& collector_factories : for (auto& collector_factories :
options.table_properties_collector_factories) { ioptions.table_properties_collector_factories) {
table_properties_collectors.emplace_back( table_properties_collectors.emplace_back(
collector_factories->CreateTablePropertiesCollector()); collector_factories->CreateTablePropertiesCollector());
} }
@ -443,11 +447,13 @@ struct BlockBasedTableBuilder::Rep {
}; };
BlockBasedTableBuilder::BlockBasedTableBuilder( BlockBasedTableBuilder::BlockBasedTableBuilder(
const Options& options, const BlockBasedTableOptions& table_options, const ImmutableCFOptions& ioptions,
const BlockBasedTableOptions& table_options,
const InternalKeyComparator& internal_comparator, WritableFile* file, const InternalKeyComparator& internal_comparator, WritableFile* file,
CompressionType compression_type) const CompressionType compression_type,
: rep_(new Rep(options, table_options, internal_comparator, const CompressionOptions& compression_opts)
file, compression_type)) { : rep_(new Rep(ioptions, table_options, internal_comparator,
file, compression_type, compression_opts)) {
if (rep_->filter_block != nullptr) { if (rep_->filter_block != nullptr) {
rep_->filter_block->StartBlock(0); rep_->filter_block->StartBlock(0);
} }
@ -502,7 +508,7 @@ void BlockBasedTableBuilder::Add(const Slice& key, const Slice& value) {
r->index_builder->OnKeyAdded(key); r->index_builder->OnKeyAdded(key);
NotifyCollectTableCollectorsOnAdd(key, value, r->table_properties_collectors, NotifyCollectTableCollectorsOnAdd(key, value, r->table_properties_collectors,
r->options.info_log.get()); r->ioptions.info_log);
} }
void BlockBasedTableBuilder::Flush() { void BlockBasedTableBuilder::Flush() {
@ -540,10 +546,10 @@ void BlockBasedTableBuilder::WriteBlock(const Slice& raw_block_contents,
Slice block_contents; Slice block_contents;
if (raw_block_contents.size() < kCompressionSizeLimit) { if (raw_block_contents.size() < kCompressionSizeLimit) {
block_contents = block_contents =
CompressBlock(raw_block_contents, r->options.compression_opts, &type, CompressBlock(raw_block_contents, r->compression_opts, &type,
&r->compressed_output); &r->compressed_output);
} else { } else {
RecordTick(r->options.statistics.get(), NUMBER_BLOCK_NOT_COMPRESSED); RecordTick(r->ioptions.statistics, NUMBER_BLOCK_NOT_COMPRESSED);
type = kNoCompression; type = kNoCompression;
block_contents = raw_block_contents; block_contents = raw_block_contents;
} }
@ -555,8 +561,7 @@ void BlockBasedTableBuilder::WriteRawBlock(const Slice& block_contents,
CompressionType type, CompressionType type,
BlockHandle* handle) { BlockHandle* handle) {
Rep* r = rep_; Rep* r = rep_;
StopWatch sw(r->options.env, r->options.statistics.get(), StopWatch sw(r->ioptions.env, r->ioptions.statistics, WRITE_RAW_BLOCK_MICROS);
WRITE_RAW_BLOCK_MICROS);
handle->set_offset(r->offset); handle->set_offset(r->offset);
handle->set_size(block_contents.size()); handle->set_size(block_contents.size());
r->status = r->file->Append(block_contents); r->status = r->file->Append(block_contents);
@ -717,7 +722,7 @@ Status BlockBasedTableBuilder::Finish() {
// Add use collected properties // Add use collected properties
NotifyCollectTableCollectorsOnFinish(r->table_properties_collectors, NotifyCollectTableCollectorsOnFinish(r->table_properties_collectors,
r->options.info_log.get(), r->ioptions.info_log,
&property_block_builder); &property_block_builder);
BlockHandle properties_block_handle; BlockHandle properties_block_handle;
@ -776,14 +781,12 @@ Status BlockBasedTableBuilder::Finish() {
} }
} }
Log( Log(r->ioptions.info_log,
r->options.info_log,
"Table was constructed:\n" "Table was constructed:\n"
" [basic properties]: %s\n" " [basic properties]: %s\n"
" [user collected properties]: %s", " [user collected properties]: %s",
r->props.ToString().c_str(), r->props.ToString().c_str(),
user_collected.c_str() user_collected.c_str());
);
} }
return r->status; return r->status;

@ -28,10 +28,12 @@ class BlockBasedTableBuilder : public TableBuilder {
// Create a builder that will store the contents of the table it is // 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 // building in *file. Does not close the file. It is up to the
// caller to close the file after calling Finish(). // caller to close the file after calling Finish().
BlockBasedTableBuilder(const Options& options, BlockBasedTableBuilder(const ImmutableCFOptions& ioptions,
const BlockBasedTableOptions& table_options, const BlockBasedTableOptions& table_options,
const InternalKeyComparator& internal_comparator, 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. // REQUIRES: Either Finish() or Abandon() has been called.
~BlockBasedTableBuilder(); ~BlockBasedTableBuilder();

@ -41,21 +41,24 @@ BlockBasedTableFactory::BlockBasedTableFactory(
} }
Status BlockBasedTableFactory::NewTableReader( Status BlockBasedTableFactory::NewTableReader(
const Options& options, const EnvOptions& soptions, const ImmutableCFOptions& ioptions, const EnvOptions& soptions,
const InternalKeyComparator& internal_comparator, const InternalKeyComparator& internal_comparator,
unique_ptr<RandomAccessFile>&& file, uint64_t file_size, unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
unique_ptr<TableReader>* table_reader) const { unique_ptr<TableReader>* table_reader) const {
return BlockBasedTable::Open(options, soptions, table_options_, return BlockBasedTable::Open(ioptions, soptions, table_options_,
internal_comparator, std::move(file), file_size, internal_comparator, std::move(file), file_size,
table_reader); table_reader);
} }
TableBuilder* BlockBasedTableFactory::NewTableBuilder( TableBuilder* BlockBasedTableFactory::NewTableBuilder(
const Options& options, const InternalKeyComparator& internal_comparator, const ImmutableCFOptions& ioptions,
WritableFile* file, CompressionType compression_type) const { const InternalKeyComparator& internal_comparator,
WritableFile* file, const CompressionType compression_type,
const CompressionOptions& compression_opts) const {
auto table_builder = new BlockBasedTableBuilder( 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; return table_builder;
} }

@ -14,13 +14,11 @@
#include <string> #include <string>
#include "rocksdb/flush_block_policy.h" #include "rocksdb/flush_block_policy.h"
#include "rocksdb/options.h"
#include "rocksdb/table.h" #include "rocksdb/table.h"
#include "db/dbformat.h" #include "db/dbformat.h"
namespace rocksdb { namespace rocksdb {
struct Options;
struct EnvOptions; struct EnvOptions;
using std::unique_ptr; using std::unique_ptr;
@ -35,14 +33,17 @@ class BlockBasedTableFactory : public TableFactory {
const char* Name() const override { return "BlockBasedTable"; } const char* Name() const override { return "BlockBasedTable"; }
Status NewTableReader(const Options& options, const EnvOptions& soptions, Status NewTableReader(
const ImmutableCFOptions& ioptions, const EnvOptions& soptions,
const InternalKeyComparator& internal_comparator, const InternalKeyComparator& internal_comparator,
unique_ptr<RandomAccessFile>&& file, uint64_t file_size, unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
unique_ptr<TableReader>* table_reader) const override; unique_ptr<TableReader>* table_reader) const override;
TableBuilder* NewTableBuilder( TableBuilder* NewTableBuilder(
const Options& options, const InternalKeyComparator& internal_comparator, const ImmutableCFOptions& ioptions,
WritableFile* file, CompressionType compression_type) const override; const InternalKeyComparator& internal_comparator,
WritableFile* file, const CompressionType compression_type,
const CompressionOptions& compression_opts) const override;
// Sanitizes the specified DB Options. // Sanitizes the specified DB Options.
Status SanitizeDBOptions(const DBOptions* db_opts) const override { Status SanitizeDBOptions(const DBOptions* db_opts) const override {

@ -336,15 +336,16 @@ class HashIndexReader : public IndexReader {
struct BlockBasedTable::Rep { struct BlockBasedTable::Rep {
Rep(const EnvOptions& storage_options, Rep(const ImmutableCFOptions& ioptions,
const EnvOptions& env_options,
const BlockBasedTableOptions& table_opt, const BlockBasedTableOptions& table_opt,
const InternalKeyComparator& internal_comparator) 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()), filter_policy(table_opt.filter_policy.get()),
internal_comparator(internal_comparator) {} internal_comparator(internal_comparator) {}
Options options; const ImmutableCFOptions& ioptions;
const EnvOptions& soptions; const EnvOptions& env_options;
const BlockBasedTableOptions& table_options; const BlockBasedTableOptions& table_options;
const FilterPolicy* const filter_policy; const FilterPolicy* const filter_policy;
const InternalKeyComparator& internal_comparator; 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 BlockBasedTableOptions& table_options,
const InternalKeyComparator& internal_comparator, const InternalKeyComparator& internal_comparator,
unique_ptr<RandomAccessFile>&& file, unique_ptr<RandomAccessFile>&& 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 // We've successfully read the footer and the index block: we're
// ready to serve requests. // ready to serve requests.
Rep* rep = new BlockBasedTable::Rep( Rep* rep = new BlockBasedTable::Rep(
soptions, table_options, internal_comparator); ioptions, env_options, table_options, internal_comparator);
rep->options = options;
rep->file = std::move(file); rep->file = std::move(file);
rep->footer = footer; rep->footer = footer;
rep->index_type = table_options.index_type; rep->index_type = table_options.index_type;
@ -484,7 +485,7 @@ Status BlockBasedTable::Open(const Options& options, const EnvOptions& soptions,
TableProperties* table_properties = nullptr; TableProperties* table_properties = nullptr;
if (s.ok()) { if (s.ok()) {
s = ReadProperties(meta_iter->value(), rep->file.get(), rep->footer, 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); &table_properties);
} }
@ -492,12 +493,12 @@ Status BlockBasedTable::Open(const Options& options, const EnvOptions& soptions,
auto err_msg = auto err_msg =
"[Warning] Encountered error while reading data from properties " "[Warning] Encountered error while reading data from properties "
"block " + s.ToString(); "block " + s.ToString();
Log(rep->options.info_log, "%s", err_msg.c_str()); Log(rep->ioptions.info_log, "%s", err_msg.c_str());
} else { } else {
rep->table_properties.reset(table_properties); rep->table_properties.reset(table_properties);
} }
} else { } else {
Log(WARN_LEVEL, rep->options.info_log, Log(WARN_LEVEL, rep->ioptions.info_log,
"Cannot find Properties block from file."); "Cannot find Properties block from file.");
} }
@ -546,7 +547,8 @@ Status BlockBasedTable::Open(const Options& options, const EnvOptions& soptions,
} }
void BlockBasedTable::SetupForCompaction() { void BlockBasedTable::SetupForCompaction() {
switch (rep_->options.access_hint_on_compaction_start) { /*
switch (.access_hint_on_compaction_start) {
case Options::NONE: case Options::NONE:
break; break;
case Options::NORMAL: case Options::NORMAL:
@ -562,6 +564,7 @@ void BlockBasedTable::SetupForCompaction() {
assert(false); assert(false);
} }
compaction_optimized_ = true; compaction_optimized_ = true;
*/
} }
std::shared_ptr<const TableProperties> BlockBasedTable::GetTableProperties() std::shared_ptr<const TableProperties> BlockBasedTable::GetTableProperties()
@ -596,13 +599,13 @@ Status BlockBasedTable::ReadMetaBlock(
ReadOptions(), ReadOptions(),
rep->footer.metaindex_handle(), rep->footer.metaindex_handle(),
&meta, &meta,
rep->options.env); rep->ioptions.env);
if (!s.ok()) { if (!s.ok()) {
auto err_msg = auto err_msg =
"[Warning] Encountered error while reading data from properties" "[Warning] Encountered error while reading data from properties"
"block " + s.ToString(); "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()) { if (!s.ok()) {
delete meta; delete meta;
@ -746,7 +749,7 @@ FilterBlockReader* BlockBasedTable::ReadFilter(const BlockHandle& filter_handle,
ReadOptions opt; ReadOptions opt;
BlockContents block; BlockContents block;
if (!ReadBlockContents(rep->file.get(), rep->footer, opt, filter_handle, if (!ReadBlockContents(rep->file.get(), rep->footer, opt, filter_handle,
&block, rep->options.env, false).ok()) { &block, rep->ioptions.env, false).ok()) {
return nullptr; return nullptr;
} }
@ -755,7 +758,8 @@ FilterBlockReader* BlockBasedTable::ReadFilter(const BlockHandle& filter_handle,
} }
return new FilterBlockReader( 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<FilterBlockReader> BlockBasedTable::GetFilter( BlockBasedTable::CachableEntry<FilterBlockReader> BlockBasedTable::GetFilter(
@ -780,7 +784,7 @@ BlockBasedTable::CachableEntry<FilterBlockReader> BlockBasedTable::GetFilter(
cache_key cache_key
); );
Statistics* statistics = rep_->options.statistics.get(); Statistics* statistics = rep_->ioptions.statistics;
auto cache_handle = auto cache_handle =
GetEntryFromCache(block_cache, key, BLOCK_CACHE_FILTER_MISS, GetEntryFromCache(block_cache, key, BLOCK_CACHE_FILTER_MISS,
BLOCK_CACHE_FILTER_HIT, statistics); BLOCK_CACHE_FILTER_HIT, statistics);
@ -830,7 +834,7 @@ Iterator* BlockBasedTable::NewIndexIterator(const ReadOptions& read_options,
char cache_key[kMaxCacheKeyPrefixSize + kMaxVarint64Length]; char cache_key[kMaxCacheKeyPrefixSize + kMaxVarint64Length];
auto key = GetCacheKey(rep_->cache_key_prefix, rep_->cache_key_prefix_size, auto key = GetCacheKey(rep_->cache_key_prefix, rep_->cache_key_prefix_size,
rep_->footer.index_handle(), cache_key); rep_->footer.index_handle(), cache_key);
Statistics* statistics = rep_->options.statistics.get(); Statistics* statistics = rep_->ioptions.statistics;
auto cache_handle = auto cache_handle =
GetEntryFromCache(block_cache, key, BLOCK_CACHE_INDEX_MISS, GetEntryFromCache(block_cache, key, BLOCK_CACHE_INDEX_MISS,
BLOCK_CACHE_INDEX_HIT, statistics); 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 either block cache is enabled, we'll try to read from it.
if (block_cache != nullptr || block_cache_compressed != nullptr) { 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 cache_key[kMaxCacheKeyPrefixSize + kMaxVarint64Length];
char compressed_cache_key[kMaxCacheKeyPrefixSize + kMaxVarint64Length]; char compressed_cache_key[kMaxCacheKeyPrefixSize + kMaxVarint64Length];
Slice key, /* key to the block cache */ 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) { if (block.value == nullptr && !no_io && ro.fill_cache) {
Block* raw_block = nullptr; 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, s = ReadBlockFromFile(rep->file.get(), rep->footer, ro, handle,
&raw_block, rep->options.env, &raw_block, rep->ioptions.env,
block_cache_compressed == nullptr); block_cache_compressed == nullptr);
} }
@ -955,7 +959,7 @@ Iterator* BlockBasedTable::NewDataBlockIterator(Rep* rep,
} }
} }
s = ReadBlockFromFile(rep->file.get(), rep->footer, ro, handle, s = ReadBlockFromFile(rep->file.get(), rep->footer, ro, handle,
&block.value, rep->options.env); &block.value, rep->ioptions.env);
} }
Iterator* iter; Iterator* iter;
@ -982,7 +986,8 @@ class BlockBasedTable::BlockEntryIteratorState : public TwoLevelIteratorState {
public: public:
BlockEntryIteratorState(BlockBasedTable* table, BlockEntryIteratorState(BlockBasedTable* table,
const ReadOptions& read_options) const ReadOptions& read_options)
: TwoLevelIteratorState(table->rep_->options.prefix_extractor != nullptr), : TwoLevelIteratorState(
table->rep_->ioptions.prefix_extractor != nullptr),
table_(table), table_(table),
read_options_(read_options) {} read_options_(read_options) {}
@ -1020,8 +1025,8 @@ bool BlockBasedTable::PrefixMayMatch(const Slice& internal_key) {
return true; return true;
} }
assert(rep_->options.prefix_extractor != nullptr); assert(rep_->ioptions.prefix_extractor != nullptr);
auto prefix = rep_->options.prefix_extractor->Transform( auto prefix = rep_->ioptions.prefix_extractor->Transform(
ExtractUserKey(internal_key)); ExtractUserKey(internal_key));
InternalKey internal_key_prefix(prefix, 0, kTypeValue); InternalKey internal_key_prefix(prefix, 0, kTypeValue);
auto internal_prefix = internal_key_prefix.Encode(); 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()); 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); RecordTick(statistics, BLOOM_FILTER_PREFIX_CHECKED);
if (!may_match) { if (!may_match) {
RecordTick(statistics, BLOOM_FILTER_PREFIX_USEFUL); RecordTick(statistics, BLOOM_FILTER_PREFIX_USEFUL);
@ -1111,7 +1116,7 @@ Status BlockBasedTable::Get(
// Not found // Not found
// TODO: think about interaction with Merge. If a user key cannot // TODO: think about interaction with Merge. If a user key cannot
// cross one data block, we should be fine. // cross one data block, we should be fine.
RecordTick(rep_->options.statistics.get(), BLOOM_FILTER_USEFUL); RecordTick(rep_->ioptions.statistics, BLOOM_FILTER_USEFUL);
break; break;
} else { } else {
BlockIter biter; BlockIter biter;
@ -1205,13 +1210,13 @@ Status BlockBasedTable::CreateIndexReader(IndexReader** index_reader,
} }
auto file = rep_->file.get(); auto file = rep_->file.get();
auto env = rep_->options.env; auto env = rep_->ioptions.env;
auto comparator = &rep_->internal_comparator; auto comparator = &rep_->internal_comparator;
const Footer& footer = rep_->footer; const Footer& footer = rep_->footer;
if (index_type_on_file == BlockBasedTableOptions::kHashSearch && if (index_type_on_file == BlockBasedTableOptions::kHashSearch &&
rep_->options.prefix_extractor == nullptr) { rep_->ioptions.prefix_extractor == nullptr) {
Log(rep_->options.info_log, Log(rep_->ioptions.info_log,
"BlockBasedTableOptions::kHashSearch requires " "BlockBasedTableOptions::kHashSearch requires "
"options.prefix_extractor to be set." "options.prefix_extractor to be set."
" Fall back to binary seach index."); " Fall back to binary seach index.");
@ -1232,7 +1237,7 @@ Status BlockBasedTable::CreateIndexReader(IndexReader** index_reader,
if (!s.ok()) { if (!s.ok()) {
// we simply fall back to binary search in case there is any // we simply fall back to binary search in case there is any
// problem with prefix hash index loading. // problem with prefix hash index loading.
Log(rep_->options.info_log, Log(rep_->ioptions.info_log,
"Unable to read the metaindex block." "Unable to read the metaindex block."
" Fall back to binary seach index."); " Fall back to binary seach index.");
return BinarySearchIndexReader::Create( 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 // We need to wrap data with internal_prefix_transform to make sure it can
// handle prefix correctly. // handle prefix correctly.
rep_->internal_prefix_transform.reset( rep_->internal_prefix_transform.reset(
new InternalKeySliceTransform(rep_->options.prefix_extractor.get())); new InternalKeySliceTransform(rep_->ioptions.prefix_extractor));
return HashIndexReader::Create( return HashIndexReader::Create(
rep_->internal_prefix_transform.get(), footer, file, env, comparator, rep_->internal_prefix_transform.get(), footer, file, env, comparator,
footer.index_handle(), meta_index_iter, index_reader, footer.index_handle(), meta_index_iter, index_reader,

@ -14,6 +14,7 @@
#include <utility> #include <utility>
#include <string> #include <string>
#include "rocksdb/options.h"
#include "rocksdb/statistics.h" #include "rocksdb/statistics.h"
#include "rocksdb/status.h" #include "rocksdb/status.h"
#include "rocksdb/table.h" #include "rocksdb/table.h"
@ -36,7 +37,6 @@ class TableReader;
class WritableFile; class WritableFile;
struct BlockBasedTableOptions; struct BlockBasedTableOptions;
struct EnvOptions; struct EnvOptions;
struct Options;
struct ReadOptions; struct ReadOptions;
using std::unique_ptr; using std::unique_ptr;
@ -58,7 +58,8 @@ class BlockBasedTable : public TableReader {
// to nullptr and returns a non-ok status. // to nullptr and returns a non-ok status.
// //
// *file must remain live while this Table is in use. // *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 BlockBasedTableOptions& table_options,
const InternalKeyComparator& internal_key_comparator, const InternalKeyComparator& internal_key_comparator,
unique_ptr<RandomAccessFile>&& file, uint64_t file_size, unique_ptr<RandomAccessFile>&& file, uint64_t file_size,

@ -11,11 +11,12 @@
#include "table/cuckoo_table_reader.h" #include "table/cuckoo_table_reader.h"
namespace rocksdb { 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<RandomAccessFile>&& file, uint64_t file_size, std::unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
std::unique_ptr<TableReader>* table) const { std::unique_ptr<TableReader>* table) const {
std::unique_ptr<CuckooTableReader> new_reader(new CuckooTableReader(options, std::unique_ptr<CuckooTableReader> new_reader(new CuckooTableReader(ioptions,
std::move(file), file_size, icomp.user_comparator(), nullptr)); std::move(file), file_size, icomp.user_comparator(), nullptr));
Status s = new_reader->status(); Status s = new_reader->status();
if (s.ok()) { if (s.ok()) {
@ -25,10 +26,13 @@ Status CuckooTableFactory::NewTableReader(const Options& options,
} }
TableBuilder* CuckooTableFactory::NewTableBuilder( TableBuilder* CuckooTableFactory::NewTableBuilder(
const Options& options, const InternalKeyComparator& internal_comparator, const ImmutableCFOptions& ioptions,
WritableFile* file, CompressionType compression_type) const { const InternalKeyComparator& internal_comparator,
return new CuckooTableBuilder(file, hash_table_ratio_, 64, max_search_depth_, WritableFile* file, const CompressionType,
internal_comparator.user_comparator(), cuckoo_block_size_, nullptr); 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 { std::string CuckooTableFactory::GetPrintableTableOptions() const {

@ -9,6 +9,7 @@
#include <string> #include <string>
#include "rocksdb/table.h" #include "rocksdb/table.h"
#include "util/murmurhash.h" #include "util/murmurhash.h"
#include "rocksdb/options.h"
namespace rocksdb { namespace rocksdb {
@ -45,14 +46,14 @@ class CuckooTableFactory : public TableFactory {
const char* Name() const override { return "CuckooTable"; } const char* Name() const override { return "CuckooTable"; }
Status NewTableReader( Status NewTableReader(
const Options& options, const EnvOptions& soptions, const ImmutableCFOptions& ioptions, const EnvOptions& env_options,
const InternalKeyComparator& internal_comparator, const InternalKeyComparator& internal_comparator,
unique_ptr<RandomAccessFile>&& file, uint64_t file_size, unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
unique_ptr<TableReader>* table) const override; unique_ptr<TableReader>* table) const override;
TableBuilder* NewTableBuilder(const Options& options, TableBuilder* NewTableBuilder(const ImmutableCFOptions& options,
const InternalKeyComparator& icomparator, WritableFile* file, const InternalKeyComparator& icomparator, WritableFile* file,
CompressionType compression_type) const override; const CompressionType, const CompressionOptions&) const override;
// Sanitizes the specified DB Options. // Sanitizes the specified DB Options.
Status SanitizeDBOptions(const DBOptions* db_opts) const override { Status SanitizeDBOptions(const DBOptions* db_opts) const override {

@ -29,7 +29,7 @@ namespace {
extern const uint64_t kCuckooTableMagicNumber; extern const uint64_t kCuckooTableMagicNumber;
CuckooTableReader::CuckooTableReader( CuckooTableReader::CuckooTableReader(
const Options& options, const ImmutableCFOptions& ioptions,
std::unique_ptr<RandomAccessFile>&& file, std::unique_ptr<RandomAccessFile>&& file,
uint64_t file_size, uint64_t file_size,
const Comparator* comparator, const Comparator* comparator,
@ -37,12 +37,12 @@ CuckooTableReader::CuckooTableReader(
: file_(std::move(file)), : file_(std::move(file)),
ucomp_(comparator), ucomp_(comparator),
get_slice_hash_(get_slice_hash) { get_slice_hash_(get_slice_hash) {
if (!options.allow_mmap_reads) { if (!ioptions.allow_mmap_reads) {
status_ = Status::InvalidArgument("File is not mmaped"); status_ = Status::InvalidArgument("File is not mmaped");
} }
TableProperties* props = nullptr; TableProperties* props = nullptr;
status_ = ReadTableProperties(file_.get(), file_size, kCuckooTableMagicNumber, status_ = ReadTableProperties(file_.get(), file_size, kCuckooTableMagicNumber,
options.env, options.info_log.get(), &props); ioptions.env, ioptions.info_log, &props);
if (!status_.ok()) { if (!status_.ok()) {
return; return;
} }

@ -16,6 +16,7 @@
#include "db/dbformat.h" #include "db/dbformat.h"
#include "rocksdb/env.h" #include "rocksdb/env.h"
#include "rocksdb/options.h"
#include "table/table_reader.h" #include "table/table_reader.h"
namespace rocksdb { namespace rocksdb {
@ -26,7 +27,7 @@ class TableReader;
class CuckooTableReader: public TableReader { class CuckooTableReader: public TableReader {
public: public:
CuckooTableReader( CuckooTableReader(
const Options& options, const ImmutableCFOptions& ioptions,
std::unique_ptr<RandomAccessFile>&& file, std::unique_ptr<RandomAccessFile>&& file,
uint64_t file_size, uint64_t file_size,
const Comparator* user_comparator, const Comparator* user_comparator,
@ -40,7 +41,7 @@ class CuckooTableReader: public TableReader {
Status status() const { return status_; } Status status() const { return status_; }
Status Get( 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, bool (*result_handler)(void* arg, const ParsedInternalKey& k,
const Slice& v), const Slice& v),
void (*mark_key_may_exist_handler)(void* handle_context) = nullptr) void (*mark_key_may_exist_handler)(void* handle_context) = nullptr)

@ -121,8 +121,9 @@ class CuckooReaderTest {
// Check reader now. // Check reader now.
std::unique_ptr<RandomAccessFile> read_file; std::unique_ptr<RandomAccessFile> read_file;
ASSERT_OK(env->NewRandomAccessFile(fname, &read_file, env_options)); ASSERT_OK(env->NewRandomAccessFile(fname, &read_file, env_options));
const ImmutableCFOptions ioptions(options);
CuckooTableReader reader( CuckooTableReader reader(
options, ioptions,
std::move(read_file), std::move(read_file),
file_size, file_size,
ucomp, ucomp,
@ -147,8 +148,9 @@ class CuckooReaderTest {
void CheckIterator(const Comparator* ucomp = BytewiseComparator()) { void CheckIterator(const Comparator* ucomp = BytewiseComparator()) {
std::unique_ptr<RandomAccessFile> read_file; std::unique_ptr<RandomAccessFile> read_file;
ASSERT_OK(env->NewRandomAccessFile(fname, &read_file, env_options)); ASSERT_OK(env->NewRandomAccessFile(fname, &read_file, env_options));
const ImmutableCFOptions ioptions(options);
CuckooTableReader reader( CuckooTableReader reader(
options, ioptions,
std::move(read_file), std::move(read_file),
file_size, file_size,
ucomp, ucomp,
@ -325,8 +327,9 @@ TEST(CuckooReaderTest, WhenKeyNotFound) {
CreateCuckooFileAndCheckReader(); CreateCuckooFileAndCheckReader();
std::unique_ptr<RandomAccessFile> read_file; std::unique_ptr<RandomAccessFile> read_file;
ASSERT_OK(env->NewRandomAccessFile(fname, &read_file, env_options)); ASSERT_OK(env->NewRandomAccessFile(fname, &read_file, env_options));
const ImmutableCFOptions ioptions(options);
CuckooTableReader reader( CuckooTableReader reader(
options, ioptions,
std::move(read_file), std::move(read_file),
file_size, file_size,
BytewiseComparator(), BytewiseComparator(),
@ -433,8 +436,9 @@ void WriteFile(const std::vector<std::string>& keys,
std::unique_ptr<RandomAccessFile> read_file; std::unique_ptr<RandomAccessFile> read_file;
ASSERT_OK(env->NewRandomAccessFile(fname, &read_file, env_options)); ASSERT_OK(env->NewRandomAccessFile(fname, &read_file, env_options));
const ImmutableCFOptions ioptions(options);
CuckooTableReader reader( CuckooTableReader reader(
options, std::move(read_file), file_size, ioptions, std::move(read_file), file_size,
test::Uint64Comparator(), nullptr); test::Uint64Comparator(), nullptr);
ASSERT_OK(reader.status()); ASSERT_OK(reader.status());
ReadOptions r_options; ReadOptions r_options;
@ -460,8 +464,9 @@ void ReadKeys(uint64_t num, uint32_t batch_size) {
std::unique_ptr<RandomAccessFile> read_file; std::unique_ptr<RandomAccessFile> read_file;
ASSERT_OK(env->NewRandomAccessFile(fname, &read_file, env_options)); ASSERT_OK(env->NewRandomAccessFile(fname, &read_file, env_options));
const ImmutableCFOptions ioptions(options);
CuckooTableReader reader( CuckooTableReader reader(
options, std::move(read_file), file_size, test::Uint64Comparator(), ioptions, std::move(read_file), file_size, test::Uint64Comparator(),
nullptr); nullptr);
ASSERT_OK(reader.status()); ASSERT_OK(reader.status());
const UserCollectedProperties user_props = const UserCollectedProperties user_props =

@ -21,11 +21,11 @@ namespace rocksdb {
static const size_t kFilterBaseLg = 11; static const size_t kFilterBaseLg = 11;
static const size_t kFilterBase = 1 << kFilterBaseLg; static const size_t kFilterBase = 1 << kFilterBaseLg;
FilterBlockBuilder::FilterBlockBuilder(const Options& opt, FilterBlockBuilder::FilterBlockBuilder(const SliceTransform* prefix_extractor,
const BlockBasedTableOptions& table_opt, const BlockBasedTableOptions& table_opt,
const Comparator* internal_comparator) const Comparator* internal_comparator)
: policy_(table_opt.filter_policy.get()), : policy_(table_opt.filter_policy.get()),
prefix_extractor_(opt.prefix_extractor.get()), prefix_extractor_(prefix_extractor),
whole_key_filtering_(table_opt.whole_key_filtering), whole_key_filtering_(table_opt.whole_key_filtering),
comparator_(internal_comparator) {} comparator_(internal_comparator) {}
@ -126,10 +126,11 @@ void FilterBlockBuilder::GenerateFilter() {
} }
FilterBlockReader::FilterBlockReader( 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) const Slice& contents, bool delete_contents_after_use)
: policy_(table_opt.filter_policy.get()), : policy_(table_opt.filter_policy.get()),
prefix_extractor_(opt.prefix_extractor.get()), prefix_extractor_(prefix_extractor),
whole_key_filtering_(table_opt.whole_key_filtering), whole_key_filtering_(table_opt.whole_key_filtering),
data_(nullptr), data_(nullptr),
offset_(nullptr), offset_(nullptr),

@ -18,7 +18,6 @@
#include <stdint.h> #include <stdint.h>
#include <string> #include <string>
#include <vector> #include <vector>
#include "rocksdb/options.h"
#include "rocksdb/slice.h" #include "rocksdb/slice.h"
#include "rocksdb/slice_transform.h" #include "rocksdb/slice_transform.h"
#include "rocksdb/table.h" #include "rocksdb/table.h"
@ -36,7 +35,7 @@ class FilterPolicy;
// (StartBlock AddKey*)* Finish // (StartBlock AddKey*)* Finish
class FilterBlockBuilder { class FilterBlockBuilder {
public: public:
explicit FilterBlockBuilder(const Options& opt, explicit FilterBlockBuilder(const SliceTransform* prefix_extractor,
const BlockBasedTableOptions& table_opt, const BlockBasedTableOptions& table_opt,
const Comparator* internal_comparator); const Comparator* internal_comparator);
@ -71,7 +70,7 @@ class FilterBlockReader {
public: public:
// REQUIRES: "contents" and *policy must stay live while *this is live. // REQUIRES: "contents" and *policy must stay live while *this is live.
FilterBlockReader( FilterBlockReader(
const Options& opt, const SliceTransform* prefix_extractor,
const BlockBasedTableOptions& table_opt, const BlockBasedTableOptions& table_opt,
const Slice& contents, const Slice& contents,
bool delete_contents_after_use = false); bool delete_contents_after_use = false);

@ -45,26 +45,26 @@ class TestHashFilter : public FilterPolicy {
class FilterBlockTest { class FilterBlockTest {
public: public:
Options options_; const Comparator* comparator_;
BlockBasedTableOptions table_options_; BlockBasedTableOptions table_options_;
FilterBlockTest() { FilterBlockTest()
options_ = Options(); : comparator_(BytewiseComparator()) {
table_options_.filter_policy.reset(new TestHashFilter()); table_options_.filter_policy.reset(new TestHashFilter());
} }
}; };
TEST(FilterBlockTest, EmptyBuilder) { TEST(FilterBlockTest, EmptyBuilder) {
FilterBlockBuilder builder(options_, table_options_, options_.comparator); FilterBlockBuilder builder(nullptr, table_options_, comparator_);
Slice block = builder.Finish(); Slice block = builder.Finish();
ASSERT_EQ("\\x00\\x00\\x00\\x00\\x0b", EscapeString(block)); 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(0, "foo"));
ASSERT_TRUE(reader.KeyMayMatch(100000, "foo")); ASSERT_TRUE(reader.KeyMayMatch(100000, "foo"));
} }
TEST(FilterBlockTest, SingleChunk) { TEST(FilterBlockTest, SingleChunk) {
FilterBlockBuilder builder(options_, table_options_, options_.comparator); FilterBlockBuilder builder(nullptr, table_options_, comparator_);
builder.StartBlock(100); builder.StartBlock(100);
builder.AddKey("foo"); builder.AddKey("foo");
builder.AddKey("bar"); builder.AddKey("bar");
@ -74,7 +74,7 @@ TEST(FilterBlockTest, SingleChunk) {
builder.StartBlock(300); builder.StartBlock(300);
builder.AddKey("hello"); builder.AddKey("hello");
Slice block = builder.Finish(); 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, "foo"));
ASSERT_TRUE(reader.KeyMayMatch(100, "bar")); ASSERT_TRUE(reader.KeyMayMatch(100, "bar"));
ASSERT_TRUE(reader.KeyMayMatch(100, "box")); ASSERT_TRUE(reader.KeyMayMatch(100, "box"));
@ -85,7 +85,7 @@ TEST(FilterBlockTest, SingleChunk) {
} }
TEST(FilterBlockTest, MultiChunk) { TEST(FilterBlockTest, MultiChunk) {
FilterBlockBuilder builder(options_, table_options_, options_.comparator); FilterBlockBuilder builder(nullptr, table_options_, comparator_);
// First filter // First filter
builder.StartBlock(0); builder.StartBlock(0);
@ -105,7 +105,7 @@ TEST(FilterBlockTest, MultiChunk) {
builder.AddKey("hello"); builder.AddKey("hello");
Slice block = builder.Finish(); Slice block = builder.Finish();
FilterBlockReader reader(options_, table_options_, block); FilterBlockReader reader(nullptr, table_options_, block);
// Check first filter // Check first filter
ASSERT_TRUE(reader.KeyMayMatch(0, "foo")); ASSERT_TRUE(reader.KeyMayMatch(0, "foo"));

@ -58,24 +58,24 @@ extern const uint64_t kPlainTableMagicNumber = 0x8242229663bf9564ull;
extern const uint64_t kLegacyPlainTableMagicNumber = 0x4f3418eb7a8f13b8ull; extern const uint64_t kLegacyPlainTableMagicNumber = 0x4f3418eb7a8f13b8ull;
PlainTableBuilder::PlainTableBuilder( PlainTableBuilder::PlainTableBuilder(
const Options& options, WritableFile* file, uint32_t user_key_len, const ImmutableCFOptions& ioptions, WritableFile* file,
EncodingType encoding_type, size_t index_sparseness, 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, 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) double hash_table_ratio, bool store_index_in_file)
: options_(options), : ioptions_(ioptions),
bloom_block_(num_probes), bloom_block_(num_probes),
file_(file), file_(file),
bloom_bits_per_key_(bloom_bits_per_key), bloom_bits_per_key_(bloom_bits_per_key),
huge_page_tlb_size_(huge_page_tlb_size), 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), index_sparseness),
store_index_in_file_(store_index_in_file), 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 // Build index block and save it in the file if hash_table_ratio > 0
if (store_index_in_file_) { if (store_index_in_file_) {
assert(hash_table_ratio > 0 || IsTotalOrderMode()); assert(hash_table_ratio > 0 || IsTotalOrderMode());
index_builder_.reset( index_builder_.reset(
new PlainTableIndexBuilder(&arena_, options, index_sparseness, new PlainTableIndexBuilder(&arena_, ioptions, index_sparseness,
hash_table_ratio, huge_page_tlb_size_)); hash_table_ratio, huge_page_tlb_size_));
assert(bloom_bits_per_key_ > 0); assert(bloom_bits_per_key_ > 0);
properties_.user_collected_properties properties_.user_collected_properties
@ -93,10 +93,10 @@ PlainTableBuilder::PlainTableBuilder(
// plain encoding. // plain encoding.
properties_.format_version = (encoding_type == kPlain) ? 0 : 1; properties_.format_version = (encoding_type == kPlain) ? 0 : 1;
if (options_.prefix_extractor) { if (ioptions_.prefix_extractor) {
properties_.user_collected_properties properties_.user_collected_properties
[PlainTablePropertyNames::kPrefixExtractorName] = [PlainTablePropertyNames::kPrefixExtractorName] =
options_.prefix_extractor->Name(); ioptions_.prefix_extractor->Name();
} }
std::string val; std::string val;
@ -105,7 +105,7 @@ PlainTableBuilder::PlainTableBuilder(
[PlainTablePropertyNames::kEncodingType] = val; [PlainTablePropertyNames::kEncodingType] = val;
for (auto& collector_factories : for (auto& collector_factories :
options.table_properties_collector_factories) { ioptions.table_properties_collector_factories) {
table_properties_collectors_.emplace_back( table_properties_collectors_.emplace_back(
collector_factories->CreateTablePropertiesCollector()); collector_factories->CreateTablePropertiesCollector());
} }
@ -124,11 +124,11 @@ void PlainTableBuilder::Add(const Slice& key, const Slice& value) {
// Store key hash // Store key hash
if (store_index_in_file_) { 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)); keys_or_prefixes_hashes_.push_back(GetSliceHash(internal_key.user_key));
} else { } else {
Slice prefix = 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)); keys_or_prefixes_hashes_.push_back(GetSliceHash(prefix));
} }
} }
@ -160,7 +160,7 @@ void PlainTableBuilder::Add(const Slice& key, const Slice& value) {
// notify property collectors // notify property collectors
NotifyCollectTableCollectorsOnAdd(key, value, table_properties_collectors_, NotifyCollectTableCollectorsOnAdd(key, value, table_properties_collectors_,
options_.info_log.get()); ioptions_.info_log);
} }
Status PlainTableBuilder::status() const { return status_; } Status PlainTableBuilder::status() const { return status_; }
@ -183,7 +183,8 @@ Status PlainTableBuilder::Finish() {
if (store_index_in_file_ && (properties_.num_entries > 0)) { if (store_index_in_file_ && (properties_.num_entries > 0)) {
bloom_block_.SetTotalBits( bloom_block_.SetTotalBits(
&arena_, properties_.num_entries * bloom_bits_per_key_, &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 PutVarint32(&properties_.user_collected_properties
[PlainTablePropertyNames::kNumBloomBlocks], [PlainTablePropertyNames::kNumBloomBlocks],
@ -224,7 +225,7 @@ Status PlainTableBuilder::Finish() {
// -- Add user collected properties // -- Add user collected properties
NotifyCollectTableCollectorsOnFinish(table_properties_collectors_, NotifyCollectTableCollectorsOnFinish(table_properties_collectors_,
options_.info_log.get(), ioptions_.info_log,
&property_block_builder); &property_block_builder);
// -- Write property block // -- Write property block

@ -30,7 +30,7 @@ class PlainTableBuilder: public TableBuilder {
// caller to close the file after calling Finish(). The output file // caller to close the file after calling Finish(). The output file
// will be part of level specified by 'level'. A value of -1 means // 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. // 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, uint32_t user_key_size, EncodingType encoding_type,
size_t index_sparseness, uint32_t bloom_bits_per_key, size_t index_sparseness, uint32_t bloom_bits_per_key,
uint32_t num_probes = 6, size_t huge_page_tlb_size = 0, uint32_t num_probes = 6, size_t huge_page_tlb_size = 0,
@ -71,7 +71,7 @@ class PlainTableBuilder: public TableBuilder {
private: private:
Arena arena_; Arena arena_;
Options options_; const ImmutableCFOptions& ioptions_;
std::vector<std::unique_ptr<TablePropertiesCollector>> std::vector<std::unique_ptr<TablePropertiesCollector>>
table_properties_collectors_; table_properties_collectors_;

@ -14,22 +14,24 @@
namespace rocksdb { namespace rocksdb {
Status PlainTableFactory::NewTableReader(const Options& options, Status PlainTableFactory::NewTableReader(const ImmutableCFOptions& ioptions,
const EnvOptions& soptions, const EnvOptions& env_options,
const InternalKeyComparator& icomp, const InternalKeyComparator& icomp,
unique_ptr<RandomAccessFile>&& file, unique_ptr<RandomAccessFile>&& file,
uint64_t file_size, uint64_t file_size,
unique_ptr<TableReader>* table) const { unique_ptr<TableReader>* 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_, file_size, table, bloom_bits_per_key_,
hash_table_ratio_, index_sparseness_, hash_table_ratio_, index_sparseness_,
huge_page_tlb_size_, full_scan_mode_); huge_page_tlb_size_, full_scan_mode_);
} }
TableBuilder* PlainTableFactory::NewTableBuilder( TableBuilder* PlainTableFactory::NewTableBuilder(
const Options& options, const InternalKeyComparator& internal_comparator, const ImmutableCFOptions& ioptions,
WritableFile* file, CompressionType compression_type) const { const InternalKeyComparator& internal_comparator,
return new PlainTableBuilder(options, file, user_key_len_, encoding_type_, WritableFile* file, const CompressionType,
const CompressionOptions&) const {
return new PlainTableBuilder(ioptions, file, user_key_len_, encoding_type_,
index_sparseness_, bloom_bits_per_key_, 6, index_sparseness_, bloom_bits_per_key_, 6,
huge_page_tlb_size_, hash_table_ratio_, huge_page_tlb_size_, hash_table_ratio_,
store_index_in_file_); store_index_in_file_);

@ -14,7 +14,6 @@
namespace rocksdb { namespace rocksdb {
struct Options;
struct EnvOptions; struct EnvOptions;
using std::unique_ptr; using std::unique_ptr;
@ -154,15 +153,17 @@ class PlainTableFactory : public TableFactory {
full_scan_mode_(options.full_scan_mode), full_scan_mode_(options.full_scan_mode),
store_index_in_file_(options.store_index_in_file) {} store_index_in_file_(options.store_index_in_file) {}
const char* Name() const override { return "PlainTable"; } const char* Name() const override { return "PlainTable"; }
Status NewTableReader(const Options& options, const EnvOptions& soptions, Status NewTableReader(
const ImmutableCFOptions& options, const EnvOptions& soptions,
const InternalKeyComparator& internal_comparator, const InternalKeyComparator& internal_comparator,
unique_ptr<RandomAccessFile>&& file, uint64_t file_size, unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
unique_ptr<TableReader>* table) const override; unique_ptr<TableReader>* table) const override;
TableBuilder* NewTableBuilder(const Options& options, TableBuilder* NewTableBuilder(
const ImmutableCFOptions& options,
const InternalKeyComparator& icomparator, const InternalKeyComparator& icomparator,
WritableFile* file, WritableFile* file,
CompressionType compression_type) const const CompressionType,
override; const CompressionOptions&) const override;
std::string GetPrintableTableOptions() const override; std::string GetPrintableTableOptions() const override;

@ -93,7 +93,7 @@ Slice PlainTableIndexBuilder::Finish() {
BucketizeIndexes(&hash_to_offsets, &entries_per_bucket); BucketizeIndexes(&hash_to_offsets, &entries_per_bucket);
keys_per_prefix_hist_.Add(num_keys_per_prefix_); 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()); keys_per_prefix_hist_.ToString().c_str());
// From the temp data structure, populate indexes. // From the temp data structure, populate indexes.
@ -147,11 +147,11 @@ void PlainTableIndexBuilder::BucketizeIndexes(
Slice PlainTableIndexBuilder::FillIndexes( Slice PlainTableIndexBuilder::FillIndexes(
const std::vector<IndexRecord*>& hash_to_offsets, const std::vector<IndexRecord*>& hash_to_offsets,
const std::vector<uint32_t>& entries_per_bucket) { const std::vector<uint32_t>& 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_); sub_index_size_);
auto total_allocate_size = GetTotalSize(); auto total_allocate_size = GetTotalSize();
char* allocated = arena_->AllocateAligned( 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_); auto temp_ptr = EncodeVarint32(allocated, index_size_);
uint32_t* index = uint32_t* index =
@ -191,7 +191,7 @@ Slice PlainTableIndexBuilder::FillIndexes(
} }
assert(sub_index_offset == sub_index_size_); 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_); index_size_, sub_index_size_);
return Slice(allocated, GetTotalSize()); return Slice(allocated, GetTotalSize());
} }

@ -108,11 +108,11 @@ class PlainTableIndex {
// #wiki-in-memory-index-format // #wiki-in-memory-index-format
class PlainTableIndexBuilder { class PlainTableIndexBuilder {
public: public:
PlainTableIndexBuilder(Arena* arena, const Options& options, PlainTableIndexBuilder(Arena* arena, const ImmutableCFOptions& ioptions,
uint32_t index_sparseness, double hash_table_ratio, uint32_t index_sparseness, double hash_table_ratio,
double huge_page_tlb_size) double huge_page_tlb_size)
: arena_(arena), : arena_(arena),
options_(options), ioptions_(ioptions),
record_list_(kRecordsPerGroup), record_list_(kRecordsPerGroup),
is_first_record_(true), is_first_record_(true),
due_index_(false), due_index_(false),
@ -120,7 +120,7 @@ class PlainTableIndexBuilder {
num_keys_per_prefix_(0), num_keys_per_prefix_(0),
prev_key_prefix_hash_(0), prev_key_prefix_hash_(0),
index_sparseness_(index_sparseness), index_sparseness_(index_sparseness),
prefix_extractor_(options.prefix_extractor.get()), prefix_extractor_(ioptions.prefix_extractor),
hash_table_ratio_(hash_table_ratio), hash_table_ratio_(hash_table_ratio),
huge_page_tlb_size_(huge_page_tlb_size) {} huge_page_tlb_size_(huge_page_tlb_size) {}
@ -196,7 +196,7 @@ class PlainTableIndexBuilder {
const std::vector<uint32_t>& entries_per_bucket); const std::vector<uint32_t>& entries_per_bucket);
Arena* arena_; Arena* arena_;
Options options_; const ImmutableCFOptions ioptions_;
HistogramImpl keys_per_prefix_hist_; HistogramImpl keys_per_prefix_hist_;
IndexRecordList record_list_; IndexRecordList record_list_;
bool is_first_record_; bool is_first_record_;

@ -87,7 +87,7 @@ class PlainTableIterator : public Iterator {
}; };
extern const uint64_t kPlainTableMagicNumber; extern const uint64_t kPlainTableMagicNumber;
PlainTableReader::PlainTableReader(const Options& options, PlainTableReader::PlainTableReader(const ImmutableCFOptions& ioptions,
unique_ptr<RandomAccessFile>&& file, unique_ptr<RandomAccessFile>&& file,
const EnvOptions& storage_options, const EnvOptions& storage_options,
const InternalKeyComparator& icomparator, const InternalKeyComparator& icomparator,
@ -99,10 +99,10 @@ PlainTableReader::PlainTableReader(const Options& options,
full_scan_mode_(false), full_scan_mode_(false),
data_end_offset_(table_properties->data_size), data_end_offset_(table_properties->data_size),
user_key_len_(table_properties->fixed_key_len), user_key_len_(table_properties->fixed_key_len),
prefix_extractor_(options.prefix_extractor.get()), prefix_extractor_(ioptions.prefix_extractor),
enable_bloom_(false), enable_bloom_(false),
bloom_(6, nullptr), bloom_(6, nullptr),
options_(options), ioptions_(ioptions),
file_(std::move(file)), file_(std::move(file)),
file_size_(file_size), file_size_(file_size),
table_properties_(nullptr) {} table_properties_(nullptr) {}
@ -110,8 +110,8 @@ PlainTableReader::PlainTableReader(const Options& options,
PlainTableReader::~PlainTableReader() { PlainTableReader::~PlainTableReader() {
} }
Status PlainTableReader::Open(const Options& options, Status PlainTableReader::Open(const ImmutableCFOptions& ioptions,
const EnvOptions& soptions, const EnvOptions& env_options,
const InternalKeyComparator& internal_comparator, const InternalKeyComparator& internal_comparator,
unique_ptr<RandomAccessFile>&& file, unique_ptr<RandomAccessFile>&& file,
uint64_t file_size, uint64_t file_size,
@ -119,14 +119,14 @@ Status PlainTableReader::Open(const Options& options,
const int bloom_bits_per_key, const int bloom_bits_per_key,
double hash_table_ratio, size_t index_sparseness, double hash_table_ratio, size_t index_sparseness,
size_t huge_page_tlb_size, bool full_scan_mode) { 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) { if (file_size > PlainTableIndex::kMaxFileSize) {
return Status::NotSupported("File is too large for PlainTableReader!"); return Status::NotSupported("File is too large for PlainTableReader!");
} }
TableProperties* props = nullptr; TableProperties* props = nullptr;
auto s = ReadTableProperties(file.get(), file_size, kPlainTableMagicNumber, auto s = ReadTableProperties(file.get(), file_size, kPlainTableMagicNumber,
options.env, options.info_log.get(), &props); ioptions.env, ioptions.info_log, &props);
if (!s.ok()) { if (!s.ok()) {
return s; return s;
} }
@ -137,12 +137,12 @@ Status PlainTableReader::Open(const Options& options,
user_props.find(PlainTablePropertyNames::kPrefixExtractorName); user_props.find(PlainTablePropertyNames::kPrefixExtractorName);
if (!full_scan_mode && prefix_extractor_in_file != user_props.end()) { if (!full_scan_mode && prefix_extractor_in_file != user_props.end()) {
if (!options.prefix_extractor) { if (!ioptions.prefix_extractor) {
return Status::InvalidArgument( return Status::InvalidArgument(
"Prefix extractor is missing when opening a PlainTable built " "Prefix extractor is missing when opening a PlainTable built "
"using a prefix extractor"); "using a prefix extractor");
} else if (prefix_extractor_in_file->second.compare( } else if (prefix_extractor_in_file->second.compare(
options.prefix_extractor->Name()) != 0) { ioptions.prefix_extractor->Name()) != 0) {
return Status::InvalidArgument( return Status::InvalidArgument(
"Prefix extractor given doesn't match the one used to build " "Prefix extractor given doesn't match the one used to build "
"PlainTable"); "PlainTable");
@ -158,8 +158,8 @@ Status PlainTableReader::Open(const Options& options,
} }
std::unique_ptr<PlainTableReader> new_reader(new PlainTableReader( std::unique_ptr<PlainTableReader> new_reader(new PlainTableReader(
options, std::move(file), soptions, internal_comparator, encoding_type, ioptions, std::move(file), env_options, internal_comparator,
file_size, props)); encoding_type, file_size, props));
s = new_reader->MmapDataFile(); s = new_reader->MmapDataFile();
if (!s.ok()) { if (!s.ok()) {
@ -207,7 +207,7 @@ Status PlainTableReader::PopulateIndexRecordList(
bool is_first_record = true; bool is_first_record = true;
Slice key_prefix_slice; Slice key_prefix_slice;
PlainTableKeyDecoder decoder(encoding_type_, user_key_len_, PlainTableKeyDecoder decoder(encoding_type_, user_key_len_,
options_.prefix_extractor.get()); ioptions_.prefix_extractor);
while (pos < data_end_offset_) { while (pos < data_end_offset_) {
uint32_t key_offset = pos; uint32_t key_offset = pos;
ParsedInternalKey key; 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; uint32_t bloom_total_bits = num_prefixes * bloom_bits_per_key;
if (bloom_total_bits > 0) { if (bloom_total_bits > 0) {
enable_bloom_ = true; enable_bloom_ = true;
bloom_.SetTotalBits(&arena_, bloom_total_bits, options_.bloom_locality, bloom_.SetTotalBits(&arena_, bloom_total_bits, ioptions_.bloom_locality,
huge_page_tlb_size, options_.info_log.get()); huge_page_tlb_size, ioptions_.info_log);
FillBloom(prefix_hashes); FillBloom(prefix_hashes);
} }
} }
@ -281,13 +281,13 @@ Status PlainTableReader::PopulateIndex(TableProperties* props,
BlockContents bloom_block_contents; BlockContents bloom_block_contents;
auto s = ReadMetaBlock(file_.get(), file_size_, kPlainTableMagicNumber, auto s = ReadMetaBlock(file_.get(), file_size_, kPlainTableMagicNumber,
options_.env, BloomBlockBuilder::kBloomBlock, ioptions_.env, BloomBlockBuilder::kBloomBlock,
&bloom_block_contents); &bloom_block_contents);
bool index_in_file = s.ok(); bool index_in_file = s.ok();
BlockContents index_block_contents; BlockContents index_block_contents;
s = ReadMetaBlock(file_.get(), file_size_, kPlainTableMagicNumber, s = ReadMetaBlock(file_.get(), file_size_, kPlainTableMagicNumber,
options_.env, PlainTableIndexBuilder::kPlainTableIndexBlock, ioptions_.env, PlainTableIndexBuilder::kPlainTableIndexBlock,
&index_block_contents); &index_block_contents);
index_in_file &= s.ok(); index_in_file &= s.ok();
@ -310,8 +310,9 @@ Status PlainTableReader::PopulateIndex(TableProperties* props,
index_block = nullptr; index_block = nullptr;
} }
if ((options_.prefix_extractor.get() == nullptr) && (hash_table_ratio != 0)) { if ((ioptions_.prefix_extractor == nullptr) &&
// options.prefix_extractor is requried for a hash-based look-up. (hash_table_ratio != 0)) {
// ioptions.prefix_extractor is requried for a hash-based look-up.
return Status::NotSupported( return Status::NotSupported(
"PlainTable requires a prefix extractor enable prefix hash mode."); "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; table_properties_->num_entries * bloom_bits_per_key;
if (num_bloom_bits > 0) { if (num_bloom_bits > 0) {
enable_bloom_ = true; enable_bloom_ = true;
bloom_.SetTotalBits(&arena_, num_bloom_bits, options_.bloom_locality, bloom_.SetTotalBits(&arena_, num_bloom_bits, ioptions_.bloom_locality,
huge_page_tlb_size, options_.info_log.get()); huge_page_tlb_size, ioptions_.info_log);
} }
} }
} else { } else {
@ -351,7 +352,7 @@ Status PlainTableReader::PopulateIndex(TableProperties* props,
bloom_block->size() * 8, num_blocks); 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); hash_table_ratio, huge_page_tlb_size);
std::vector<uint32_t> prefix_hashes; std::vector<uint32_t> prefix_hashes;
@ -422,7 +423,7 @@ Status PlainTableReader::GetOffset(const Slice& target, const Slice& prefix,
uint32_t file_offset = GetFixed32Element(base_ptr, mid); uint32_t file_offset = GetFixed32Element(base_ptr, mid);
size_t tmp; size_t tmp;
Status s = PlainTableKeyDecoder(encoding_type_, user_key_len_, Status s = PlainTableKeyDecoder(encoding_type_, user_key_len_,
options_.prefix_extractor.get()) ioptions_.prefix_extractor)
.NextKey(file_data_.data() + file_offset, .NextKey(file_data_.data() + file_offset,
file_data_.data() + data_end_offset_, &mid_key, file_data_.data() + data_end_offset_, &mid_key,
nullptr, &tmp); nullptr, &tmp);
@ -451,7 +452,7 @@ Status PlainTableReader::GetOffset(const Slice& target, const Slice& prefix,
size_t tmp; size_t tmp;
uint32_t low_key_offset = GetFixed32Element(base_ptr, low); uint32_t low_key_offset = GetFixed32Element(base_ptr, low);
Status s = PlainTableKeyDecoder(encoding_type_, user_key_len_, Status s = PlainTableKeyDecoder(encoding_type_, user_key_len_,
options_.prefix_extractor.get()) ioptions_.prefix_extractor)
.NextKey(file_data_.data() + low_key_offset, .NextKey(file_data_.data() + low_key_offset,
file_data_.data() + data_end_offset_, &low_key, file_data_.data() + data_end_offset_, &low_key,
nullptr, &tmp); nullptr, &tmp);
@ -565,7 +566,7 @@ Status PlainTableReader::Get(const ReadOptions& ro, const Slice& target,
} }
Slice found_value; Slice found_value;
PlainTableKeyDecoder decoder(encoding_type_, user_key_len_, PlainTableKeyDecoder decoder(encoding_type_, user_key_len_,
options_.prefix_extractor.get()); ioptions_.prefix_extractor);
while (offset < data_end_offset_) { while (offset < data_end_offset_) {
Status s = Next(&decoder, &offset, &found_key, nullptr, &found_value); Status s = Next(&decoder, &offset, &found_key, nullptr, &found_value);
if (!s.ok()) { if (!s.ok()) {

@ -52,7 +52,8 @@ extern const uint32_t kPlainTableVariableLength;
// The implementation of IndexedTableReader requires output file is mmaped // The implementation of IndexedTableReader requires output file is mmaped
class PlainTableReader: public TableReader { class PlainTableReader: public TableReader {
public: public:
static Status Open(const Options& options, const EnvOptions& soptions, static Status Open(const ImmutableCFOptions& ioptions,
const EnvOptions& env_options,
const InternalKeyComparator& internal_comparator, const InternalKeyComparator& internal_comparator,
unique_ptr<RandomAccessFile>&& file, uint64_t file_size, unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
unique_ptr<TableReader>* table, unique_ptr<TableReader>* table,
@ -82,8 +83,9 @@ class PlainTableReader: public TableReader {
return arena_.MemoryAllocatedBytes(); return arena_.MemoryAllocatedBytes();
} }
PlainTableReader(const Options& options, unique_ptr<RandomAccessFile>&& file, PlainTableReader(const ImmutableCFOptions& ioptions,
const EnvOptions& storage_options, unique_ptr<RandomAccessFile>&& file,
const EnvOptions& env_options,
const InternalKeyComparator& internal_comparator, const InternalKeyComparator& internal_comparator,
EncodingType encoding_type, uint64_t file_size, EncodingType encoding_type, uint64_t file_size,
const TableProperties* table_properties); const TableProperties* table_properties);
@ -132,7 +134,7 @@ class PlainTableReader: public TableReader {
DynamicBloom bloom_; DynamicBloom bloom_;
Arena arena_; Arena arena_;
const Options& options_; const ImmutableCFOptions& ioptions_;
unique_ptr<RandomAccessFile> file_; unique_ptr<RandomAccessFile> file_;
uint32_t file_size_; uint32_t file_size_;
std::shared_ptr<const TableProperties> table_properties_; std::shared_ptr<const TableProperties> table_properties_;

@ -88,10 +88,12 @@ void TableReaderBenchmark(Options& opts, EnvOptions& env_options,
TableBuilder* tb = nullptr; TableBuilder* tb = nullptr;
DB* db = nullptr; DB* db = nullptr;
Status s; Status s;
const ImmutableCFOptions ioptions(opts);
if (!through_db) { if (!through_db) {
env->NewWritableFile(file_name, &file, env_options); env->NewWritableFile(file_name, &file, env_options);
tb = opts.table_factory->NewTableBuilder(opts, ikc, file.get(), tb = opts.table_factory->NewTableBuilder(ioptions, ikc, file.get(),
CompressionType::kNoCompression); CompressionType::kNoCompression,
CompressionOptions());
} else { } else {
s = DB::Open(opts, dbname, &db); s = DB::Open(opts, dbname, &db);
ASSERT_OK(s); ASSERT_OK(s);
@ -122,7 +124,7 @@ void TableReaderBenchmark(Options& opts, EnvOptions& env_options,
uint64_t file_size; uint64_t file_size;
env->GetFileSize(file_name, &file_size); env->GetFileSize(file_name, &file_size);
s = opts.table_factory->NewTableReader( 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); Random rnd(301);

@ -194,6 +194,7 @@ class Constructor {
// been added so far. Returns the keys in sorted order in "*keys" // been added so far. Returns the keys in sorted order in "*keys"
// and stores the key/value pairs in "*kvmap" // and stores the key/value pairs in "*kvmap"
void Finish(const Options& options, void Finish(const Options& options,
const ImmutableCFOptions& ioptions,
const BlockBasedTableOptions& table_options, const BlockBasedTableOptions& table_options,
const InternalKeyComparator& internal_comparator, const InternalKeyComparator& internal_comparator,
std::vector<std::string>* keys, KVMap* kvmap) { std::vector<std::string>* keys, KVMap* kvmap) {
@ -206,12 +207,14 @@ class Constructor {
keys->push_back(it->first); keys->push_back(it->first);
} }
data_.clear(); 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(); ASSERT_TRUE(s.ok()) << s.ToString();
} }
// Construct the data structure from the data in "data" // Construct the data structure from the data in "data"
virtual Status FinishImpl(const Options& options, virtual Status FinishImpl(const Options& options,
const ImmutableCFOptions& ioptions,
const BlockBasedTableOptions& table_options, const BlockBasedTableOptions& table_options,
const InternalKeyComparator& internal_comparator, const InternalKeyComparator& internal_comparator,
const KVMap& data) = 0; const KVMap& data) = 0;
@ -239,6 +242,7 @@ class BlockConstructor: public Constructor {
delete block_; delete block_;
} }
virtual Status FinishImpl(const Options& options, virtual Status FinishImpl(const Options& options,
const ImmutableCFOptions& ioptions,
const BlockBasedTableOptions& table_options, const BlockBasedTableOptions& table_options,
const InternalKeyComparator& internal_comparator, const InternalKeyComparator& internal_comparator,
const KVMap& data) { const KVMap& data) {
@ -322,14 +326,16 @@ class TableConstructor: public Constructor {
~TableConstructor() { Reset(); } ~TableConstructor() { Reset(); }
virtual Status FinishImpl(const Options& options, virtual Status FinishImpl(const Options& options,
const ImmutableCFOptions& ioptions,
const BlockBasedTableOptions& table_options, const BlockBasedTableOptions& table_options,
const InternalKeyComparator& internal_comparator, const InternalKeyComparator& internal_comparator,
const KVMap& data) { const KVMap& data) {
Reset(); Reset();
sink_.reset(new StringSink()); sink_.reset(new StringSink());
unique_ptr<TableBuilder> builder; unique_ptr<TableBuilder> builder;
builder.reset(options.table_factory->NewTableBuilder( builder.reset(ioptions.table_factory->NewTableBuilder(
options, internal_comparator, sink_.get(), options.compression)); ioptions, internal_comparator, sink_.get(), options.compression,
CompressionOptions()));
for (KVMap::const_iterator it = data.begin(); for (KVMap::const_iterator it = data.begin();
it != data.end(); it != data.end();
@ -352,9 +358,9 @@ class TableConstructor: public Constructor {
// Open the table // Open the table
uniq_id_ = cur_uniq_id_++; uniq_id_ = cur_uniq_id_++;
source_.reset(new StringSource(sink_->contents(), uniq_id_, source_.reset(new StringSource(sink_->contents(), uniq_id_,
options.allow_mmap_reads)); ioptions.allow_mmap_reads));
return options.table_factory->NewTableReader( return ioptions.table_factory->NewTableReader(
options, soptions, internal_comparator, std::move(source_), ioptions, soptions, internal_comparator, std::move(source_),
sink_->contents().size(), &table_reader_); sink_->contents().size(), &table_reader_);
} }
@ -372,12 +378,12 @@ class TableConstructor: public Constructor {
return table_reader_->ApproximateOffsetOf(key); return table_reader_->ApproximateOffsetOf(key);
} }
virtual Status Reopen(const Options& options) { virtual Status Reopen(const ImmutableCFOptions& ioptions) {
source_.reset( source_.reset(
new StringSource(sink_->contents(), uniq_id_, new StringSource(sink_->contents(), uniq_id_,
options.allow_mmap_reads)); ioptions.allow_mmap_reads));
return options.table_factory->NewTableReader( return ioptions.table_factory->NewTableReader(
options, soptions, *last_internal_key_, std::move(source_), ioptions, soptions, *last_internal_key_, std::move(source_),
sink_->contents().size(), &table_reader_); sink_->contents().size(), &table_reader_);
} }
@ -421,6 +427,7 @@ class MemTableConstructor: public Constructor {
delete memtable_->Unref(); delete memtable_->Unref();
} }
virtual Status FinishImpl(const Options& options, virtual Status FinishImpl(const Options& options,
const ImmutableCFOptions& ioptions,
const BlockBasedTableOptions& table_options, const BlockBasedTableOptions& table_options,
const InternalKeyComparator& internal_comparator, const InternalKeyComparator& internal_comparator,
const KVMap& data) { const KVMap& data) {
@ -460,6 +467,7 @@ class DBConstructor: public Constructor {
delete db_; delete db_;
} }
virtual Status FinishImpl(const Options& options, virtual Status FinishImpl(const Options& options,
const ImmutableCFOptions& ioptions,
const BlockBasedTableOptions& table_options, const BlockBasedTableOptions& table_options,
const InternalKeyComparator& internal_comparator, const InternalKeyComparator& internal_comparator,
const KVMap& data) { const KVMap& data) {
@ -670,7 +678,7 @@ class FixedOrLessPrefixTransform : public SliceTransform {
class Harness { class Harness {
public: public:
Harness() : constructor_(nullptr) { } Harness() : ioptions_(options_), constructor_(nullptr) {}
void Init(const TestArgs& args) { void Init(const TestArgs& args) {
delete constructor_; delete constructor_;
@ -756,6 +764,7 @@ class Harness {
constructor_ = new DBConstructor(options_.comparator); constructor_ = new DBConstructor(options_.comparator);
break; break;
} }
ioptions_ = ImmutableCFOptions(options_);
} }
~Harness() { ~Harness() {
@ -769,8 +778,8 @@ class Harness {
void Test(Random* rnd) { void Test(Random* rnd) {
std::vector<std::string> keys; std::vector<std::string> keys;
KVMap data; KVMap data;
constructor_->Finish(options_, table_options_, *internal_comparator_, constructor_->Finish(options_, ioptions_, table_options_,
&keys, &data); *internal_comparator_, &keys, &data);
TestForwardScan(keys, data); TestForwardScan(keys, data);
if (support_prev_) { if (support_prev_) {
@ -939,6 +948,7 @@ class Harness {
private: private:
Options options_ = Options(); Options options_ = Options();
ImmutableCFOptions ioptions_;
BlockBasedTableOptions table_options_ = BlockBasedTableOptions(); BlockBasedTableOptions table_options_ = BlockBasedTableOptions();
Constructor* constructor_; Constructor* constructor_;
bool support_prev_; bool support_prev_;
@ -1038,7 +1048,8 @@ TEST(BlockBasedTableTest, BasicBlockBasedTableProperties) {
table_options.block_restart_interval = 1; table_options.block_restart_interval = 1;
options.table_factory.reset(NewBlockBasedTableFactory(table_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); GetPlainInternalComparator(options.comparator), &keys, &kvmap);
auto& props = *c.GetTableReader()->GetTableProperties(); auto& props = *c.GetTableReader()->GetTableProperties();
@ -1071,7 +1082,8 @@ TEST(BlockBasedTableTest, FilterPolicyNameProperties) {
Options options; Options options;
options.table_factory.reset(NewBlockBasedTableFactory(table_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); GetPlainInternalComparator(options.comparator), &keys, &kvmap);
auto& props = *c.GetTableReader()->GetTableProperties(); auto& props = *c.GetTableReader()->GetTableProperties();
ASSERT_EQ("rocksdb.BuiltinBloomFilter", props.filter_policy_name); ASSERT_EQ("rocksdb.BuiltinBloomFilter", props.filter_policy_name);
@ -1122,7 +1134,8 @@ TEST(BlockBasedTableTest, TotalOrderSeekOnHashIndex) {
c.Add("cccc2", std::string('a', 56)); c.Add("cccc2", std::string('a', 56));
std::vector<std::string> keys; std::vector<std::string> keys;
KVMap kvmap; KVMap kvmap;
c.Finish(options, table_options, const ImmutableCFOptions ioptions(options);
c.Finish(options, ioptions, table_options,
GetPlainInternalComparator(options.comparator), &keys, &kvmap); GetPlainInternalComparator(options.comparator), &keys, &kvmap);
auto props = c.GetTableReader()->GetTableProperties(); auto props = c.GetTableReader()->GetTableProperties();
ASSERT_EQ(7u, props->num_data_blocks); ASSERT_EQ(7u, props->num_data_blocks);
@ -1206,7 +1219,8 @@ TEST(TableTest, HashIndexTest) {
std::unique_ptr<InternalKeyComparator> comparator( std::unique_ptr<InternalKeyComparator> comparator(
new InternalKeyComparator(BytewiseComparator())); 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 reader = c.GetTableReader();
auto props = reader->GetTableProperties(); auto props = reader->GetTableProperties();
@ -1314,7 +1328,8 @@ TEST(BlockBasedTableTest, IndexSizeStat) {
table_options.block_restart_interval = 1; table_options.block_restart_interval = 1;
options.table_factory.reset(NewBlockBasedTableFactory(table_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), &ks, &kvmap); GetPlainInternalComparator(options.comparator), &ks, &kvmap);
auto index_size = c.GetTableReader()->GetTableProperties()->index_size; auto index_size = c.GetTableReader()->GetTableProperties()->index_size;
ASSERT_GT(index_size, last_index_size); ASSERT_GT(index_size, last_index_size);
@ -1340,7 +1355,8 @@ TEST(BlockBasedTableTest, NumBlockStat) {
std::vector<std::string> ks; std::vector<std::string> ks;
KVMap kvmap; KVMap kvmap;
c.Finish(options, table_options, const ImmutableCFOptions ioptions(options);
c.Finish(options, ioptions, table_options,
GetPlainInternalComparator(options.comparator), &ks, &kvmap); GetPlainInternalComparator(options.comparator), &ks, &kvmap);
ASSERT_EQ(kvmap.size(), ASSERT_EQ(kvmap.size(),
c.GetTableReader()->GetTableProperties()->num_data_blocks); c.GetTableReader()->GetTableProperties()->num_data_blocks);
@ -1416,7 +1432,8 @@ TEST(BlockBasedTableTest, BlockCacheDisabledTest) {
TableConstructor c(BytewiseComparator(), true); TableConstructor c(BytewiseComparator(), true);
c.Add("key", "value"); c.Add("key", "value");
c.Finish(options, table_options, const ImmutableCFOptions ioptions(options);
c.Finish(options, ioptions, table_options,
GetPlainInternalComparator(options.comparator), &keys, &kvmap); GetPlainInternalComparator(options.comparator), &keys, &kvmap);
// preloading filter/index blocks is enabled. // preloading filter/index blocks is enabled.
@ -1458,7 +1475,8 @@ TEST(BlockBasedTableTest, FilterBlockInBlockCache) {
TableConstructor c(BytewiseComparator()); TableConstructor c(BytewiseComparator());
c.Add("key", "value"); c.Add("key", "value");
c.Finish(options, table_options, const ImmutableCFOptions ioptions(options);
c.Finish(options, ioptions, table_options,
GetPlainInternalComparator(options.comparator), &keys, &kvmap); GetPlainInternalComparator(options.comparator), &keys, &kvmap);
// preloading filter/index blocks is prohibited. // preloading filter/index blocks is prohibited.
auto reader = dynamic_cast<BlockBasedTable*>(c.GetTableReader()); auto reader = dynamic_cast<BlockBasedTable*>(c.GetTableReader());
@ -1512,7 +1530,8 @@ TEST(BlockBasedTableTest, FilterBlockInBlockCache) {
table_options.block_cache.reset(); table_options.block_cache.reset();
options.table_factory.reset(new BlockBasedTableFactory(table_options)); options.table_factory.reset(new BlockBasedTableFactory(table_options));
options.statistics = CreateDBStatistics(); // reset the stats options.statistics = CreateDBStatistics(); // reset the stats
c.Reopen(options); const ImmutableCFOptions ioptions1(options);
c.Reopen(ioptions1);
table_options.no_block_cache = false; table_options.no_block_cache = false;
{ {
@ -1529,7 +1548,8 @@ TEST(BlockBasedTableTest, FilterBlockInBlockCache) {
// too small to fit even one entry. // too small to fit even one entry.
table_options.block_cache = NewLRUCache(1); table_options.block_cache = NewLRUCache(1);
options.table_factory.reset(new BlockBasedTableFactory(table_options)); options.table_factory.reset(new BlockBasedTableFactory(table_options));
c.Reopen(options); const ImmutableCFOptions ioptions2(options);
c.Reopen(ioptions2);
{ {
BlockCachePropertiesSnapshot props(options.statistics.get()); BlockCachePropertiesSnapshot props(options.statistics.get());
props.AssertEqual(1, // index block miss props.AssertEqual(1, // index block miss
@ -1583,7 +1603,8 @@ TEST(BlockBasedTableTest, BlockCacheLeak) {
c.Add("k07", std::string(100000, 'x')); c.Add("k07", std::string(100000, 'x'));
std::vector<std::string> keys; std::vector<std::string> keys;
KVMap kvmap; 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<Iterator> iter(c.NewIterator()); unique_ptr<Iterator> iter(c.NewIterator());
iter->SeekToFirst(); iter->SeekToFirst();
@ -1594,7 +1615,8 @@ TEST(BlockBasedTableTest, BlockCacheLeak) {
} }
ASSERT_OK(iter->status()); ASSERT_OK(iter->status());
ASSERT_OK(c.Reopen(opt)); const ImmutableCFOptions ioptions1(opt);
ASSERT_OK(c.Reopen(ioptions1));
auto table_reader = dynamic_cast<BlockBasedTable*>(c.GetTableReader()); auto table_reader = dynamic_cast<BlockBasedTable*>(c.GetTableReader());
for (const std::string& key : keys) { for (const std::string& key : keys) {
ASSERT_TRUE(table_reader->TEST_KeyInCache(ReadOptions(), key)); ASSERT_TRUE(table_reader->TEST_KeyInCache(ReadOptions(), key));
@ -1603,7 +1625,8 @@ TEST(BlockBasedTableTest, BlockCacheLeak) {
// rerun with different block cache // rerun with different block cache
table_options.block_cache = NewLRUCache(16 * 1024 * 1024); table_options.block_cache = NewLRUCache(16 * 1024 * 1024);
opt.table_factory.reset(NewBlockBasedTableFactory(table_options)); 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<BlockBasedTable*>(c.GetTableReader()); table_reader = dynamic_cast<BlockBasedTable*>(c.GetTableReader());
for (const std::string& key : keys) { for (const std::string& key : keys) {
ASSERT_TRUE(!table_reader->TEST_KeyInCache(ReadOptions(), key)); ASSERT_TRUE(!table_reader->TEST_KeyInCache(ReadOptions(), key));
@ -1619,9 +1642,11 @@ TEST(PlainTableTest, BasicPlainTableProperties) {
PlainTableFactory factory(plain_table_options); PlainTableFactory factory(plain_table_options);
StringSink sink; StringSink sink;
Options options; Options options;
const ImmutableCFOptions ioptions(options);
InternalKeyComparator ikc(options.comparator); InternalKeyComparator ikc(options.comparator);
std::unique_ptr<TableBuilder> builder( std::unique_ptr<TableBuilder> builder(
factory.NewTableBuilder(options, ikc, &sink, kNoCompression)); factory.NewTableBuilder(ioptions, ikc, &sink, kNoCompression,
CompressionOptions()));
for (char c = 'a'; c <= 'z'; ++c) { for (char c = 'a'; c <= 'z'; ++c) {
std::string key(8, c); std::string key(8, c);
@ -1664,7 +1689,9 @@ TEST(GeneralTableTest, ApproximateOffsetOfPlain) {
options.compression = kNoCompression; options.compression = kNoCompression;
BlockBasedTableOptions table_options; BlockBasedTableOptions table_options;
table_options.block_size = 1024; 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("abc"), 0, 0));
ASSERT_TRUE(Between(c.ApproximateOffsetOf("k01"), 0, 0)); ASSERT_TRUE(Between(c.ApproximateOffsetOf("k01"), 0, 0));
@ -1694,7 +1721,8 @@ static void DoCompressionTest(CompressionType comp) {
options.compression = comp; options.compression = comp;
BlockBasedTableOptions table_options; BlockBasedTableOptions table_options;
table_options.block_size = 1024; 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("abc"), 0, 0));
ASSERT_TRUE(Between(c.ApproximateOffsetOf("k01"), 0, 0)); ASSERT_TRUE(Between(c.ApproximateOffsetOf("k01"), 0, 0));

@ -68,6 +68,7 @@ class SstFileReader {
// options_ and internal_comparator_ will also be used in // options_ and internal_comparator_ will also be used in
// ReadSequential internally (specifically, seek-related operations) // ReadSequential internally (specifically, seek-related operations)
Options options_; Options options_;
const ImmutableCFOptions ioptions_;
InternalKeyComparator internal_comparator_; InternalKeyComparator internal_comparator_;
unique_ptr<TableProperties> table_properties_; unique_ptr<TableProperties> table_properties_;
}; };
@ -76,7 +77,8 @@ SstFileReader::SstFileReader(const std::string& file_path,
bool verify_checksum, bool verify_checksum,
bool output_hex) bool output_hex)
:file_name_(file_path), read_num_(0), verify_checksum_(verify_checksum), :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()); fprintf(stdout, "Process %s\n", file_path.c_str());
init_result_ = NewTableReader(file_name_); init_result_ = NewTableReader(file_name_);
@ -123,7 +125,7 @@ Status SstFileReader::NewTableReader(const std::string& file_path) {
if (s.ok()) { if (s.ok()) {
s = options_.table_factory->NewTableReader( 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_); &table_reader_);
} }
return s; return s;

@ -8,6 +8,7 @@
// found in the LICENSE file. See the AUTHORS file for names of contributors. // found in the LICENSE file. See the AUTHORS file for names of contributors.
#include "rocksdb/options.h" #include "rocksdb/options.h"
#include "rocksdb/immutable_options.h"
#define __STDC_FORMAT_MACROS #define __STDC_FORMAT_MACROS
#include <inttypes.h> #include <inttypes.h>
@ -28,6 +29,26 @@
namespace rocksdb { 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() ColumnFamilyOptions::ColumnFamilyOptions()
: comparator(BytewiseComparator()), : comparator(BytewiseComparator()),
merge_operator(nullptr), merge_operator(nullptr),

Loading…
Cancel
Save