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

@ -11,6 +11,7 @@
#include "rocksdb/status.h"
#include "rocksdb/types.h"
#include "rocksdb/options.h"
#include "rocksdb/immutable_options.h"
namespace rocksdb {
@ -26,8 +27,10 @@ class TableBuilder;
class WritableFile;
extern TableBuilder* NewTableBuilder(
const Options& options, const InternalKeyComparator& internal_comparator,
WritableFile* file, CompressionType compression_type);
const ImmutableCFOptions& options,
const InternalKeyComparator& internal_comparator,
WritableFile* file, const CompressionType compression_type,
const CompressionOptions& compression_opts);
// Build a Table file from the contents of *iter. The generated file
// will be named according to number specified in meta. On success, the rest of
@ -35,13 +38,15 @@ extern TableBuilder* NewTableBuilder(
// If no data is present in *iter, meta->file_size will be set to
// zero, and no Table file will be produced.
extern Status BuildTable(const std::string& dbname, Env* env,
const Options& options, const EnvOptions& soptions,
const ImmutableCFOptions& options,
const EnvOptions& env_options,
TableCache* table_cache, Iterator* iter,
FileMetaData* meta,
const InternalKeyComparator& internal_comparator,
const SequenceNumber newest_snapshot,
const SequenceNumber earliest_seqno_in_memtable,
const CompressionType compression,
const CompressionOptions& compression_opts,
const Env::IOPriority io_priority = Env::IO_HIGH);
} // namespace rocksdb

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

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

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

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

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

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

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

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

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

@ -90,7 +90,8 @@ void MakeBuilder(const Options& options,
std::unique_ptr<TableBuilder>* builder) {
writable->reset(new FakeWritableFile);
builder->reset(options.table_factory->NewTableBuilder(
options, internal_comparator, writable->get(), options.compression));
ImmutableCFOptions(options), internal_comparator, writable->get(),
options.compression, options.compression_opts));
}
} // namespace

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

@ -93,7 +93,7 @@ Slice PlainTableIndexBuilder::Finish() {
BucketizeIndexes(&hash_to_offsets, &entries_per_bucket);
keys_per_prefix_hist_.Add(num_keys_per_prefix_);
Log(options_.info_log, "Number of Keys per prefix Histogram: %s",
Log(ioptions_.info_log, "Number of Keys per prefix Histogram: %s",
keys_per_prefix_hist_.ToString().c_str());
// From the temp data structure, populate indexes.
@ -147,11 +147,11 @@ void PlainTableIndexBuilder::BucketizeIndexes(
Slice PlainTableIndexBuilder::FillIndexes(
const std::vector<IndexRecord*>& hash_to_offsets,
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_);
auto total_allocate_size = GetTotalSize();
char* allocated = arena_->AllocateAligned(
total_allocate_size, huge_page_tlb_size_, options_.info_log.get());
total_allocate_size, huge_page_tlb_size_, ioptions_.info_log);
auto temp_ptr = EncodeVarint32(allocated, index_size_);
uint32_t* index =
@ -191,7 +191,7 @@ Slice PlainTableIndexBuilder::FillIndexes(
}
assert(sub_index_offset == sub_index_size_);
Log(options_.info_log, "hash table size: %d, suffix_map length %zu",
Log(ioptions_.info_log, "hash table size: %d, suffix_map length %zu",
index_size_, sub_index_size_);
return Slice(allocated, GetTotalSize());
}

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

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

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

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

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

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

@ -8,6 +8,7 @@
// found in the LICENSE file. See the AUTHORS file for names of contributors.
#include "rocksdb/options.h"
#include "rocksdb/immutable_options.h"
#define __STDC_FORMAT_MACROS
#include <inttypes.h>
@ -28,6 +29,26 @@
namespace rocksdb {
ImmutableCFOptions::ImmutableCFOptions(const Options& options)
: prefix_extractor(options.prefix_extractor.get()),
comparator(options.comparator),
merge_operator(options.merge_operator.get()),
info_log(options.info_log.get()),
statistics(options.statistics.get()),
env(options.env),
allow_mmap_reads(options.allow_mmap_reads),
allow_mmap_writes(options.allow_mmap_writes),
db_paths(options.db_paths),
table_factory(options.table_factory.get()),
table_properties_collector_factories(
options.table_properties_collector_factories),
advise_random_on_open(options.advise_random_on_open),
bloom_locality(options.bloom_locality),
purge_redundant_kvs_while_flush(options.purge_redundant_kvs_while_flush),
min_partial_merge_operands(options.min_partial_merge_operands),
disable_data_sync(options.disableDataSync),
use_fsync(options.use_fsync) {}
ColumnFamilyOptions::ColumnFamilyOptions()
: comparator(BytewiseComparator()),
merge_operator(nullptr),

Loading…
Cancel
Save