Rename variables in ImmutableCFOptions to avoid conflicts with ImmutableDBOptions (#8227)

Summary:
Renaming ImmutableCFOptions::info_log and statistics to logger and stats.  This is stage 2 in creating an ImmutableOptions class.  It is necessary because the names match those in ImmutableOptions and have different types.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/8227

Reviewed By: jay-zhuang

Differential Revision: D28000967

Pulled By: mrambacher

fbshipit-source-id: 3bf2aa04e8f1e8724d825b7deacf41080c14420b
main
mrambacher 3 years ago committed by Facebook GitHub Bot
parent c2c7d5e916
commit 0ca6d6297f
  1. 4
      db/blob/blob_file_builder.cc
  2. 2
      db/blob/blob_file_cache.cc
  3. 2
      db/blob/blob_file_reader.cc
  4. 8
      db/builder.cc
  5. 40
      db/column_family.cc
  6. 2
      db/compaction/compaction_picker.cc
  7. 2
      db/compaction/compaction_picker_fifo.cc
  8. 2
      db/compaction/compaction_picker_universal.cc
  9. 4
      db/db_iter.cc
  10. 10
      db/memtable.cc
  11. 14
      db/table_cache.cc
  12. 11
      db/version_set.cc
  13. 8
      options/cf_options.cc
  14. 7
      options/cf_options.h
  15. 2
      options/db_options.cc
  16. 2
      options/db_options.h
  17. 31
      table/block_based/block_based_table_builder.cc
  18. 78
      table/block_based/block_based_table_reader.cc
  19. 2
      table/block_based/mock_block_based_table.h
  20. 2
      table/block_based/partitioned_filter_block.cc
  21. 2
      table/block_based/uncompression_dict_reader.cc
  22. 8
      table/block_fetcher.cc
  23. 12
      table/format.cc
  24. 2
      table/meta_blocks.cc
  25. 11
      table/plain/plain_table_builder.cc
  26. 8
      table/plain/plain_table_index.cc
  27. 2
      table/plain/plain_table_reader.cc
  28. 2
      table/table_reader_bench.cc
  29. 2
      table/table_test.cc
  30. 2
      util/filter_bench.cc

@ -185,7 +185,7 @@ Status BlobFileBuilder::OpenBlobFileIfNeeded() {
file->SetIOPriority(io_priority_);
file->SetWriteLifeTimeHint(write_hint_);
FileTypeSet tmp_set = immutable_cf_options_->checksum_handoff_file_types;
Statistics* const statistics = immutable_cf_options_->statistics;
Statistics* const statistics = immutable_cf_options_->stats;
std::unique_ptr<WritableFileWriter> file_writer(new WritableFileWriter(
std::move(file), blob_file_paths_->back(), *file_options_,
immutable_cf_options_->clock, io_tracer_, statistics,
@ -301,7 +301,7 @@ Status BlobFileBuilder::CloseBlobFile() {
std::move(checksum_value));
assert(immutable_cf_options_);
ROCKS_LOG_INFO(immutable_cf_options_->info_log,
ROCKS_LOG_INFO(immutable_cf_options_->logger,
"[%s] [JOB %d] Generated blob file #%" PRIu64 ": %" PRIu64
" total blobs, %" PRIu64 " total bytes",
column_family_name_.c_str(), job_id_, blob_file_number,

@ -64,7 +64,7 @@ Status BlobFileCache::GetBlobFileReader(
}
assert(immutable_cf_options_);
Statistics* const statistics = immutable_cf_options_->statistics;
Statistics* const statistics = immutable_cf_options_->stats;
RecordTick(statistics, NO_FILE_OPENS);

@ -119,7 +119,7 @@ Status BlobFileReader::OpenFile(
file_reader->reset(new RandomAccessFileReader(
std::move(file), blob_file_path, immutable_cf_options.clock, io_tracer,
immutable_cf_options.statistics, BLOB_DB_BLOB_FILE_READ_MICROS,
immutable_cf_options.stats, BLOB_DB_BLOB_FILE_READ_MICROS,
blob_file_read_hist, immutable_cf_options.rate_limiter.get(),
immutable_cf_options.listeners));

@ -155,7 +155,7 @@ Status BuildTable(
file->SetWriteLifeTimeHint(write_hint);
file_writer.reset(new WritableFileWriter(
std::move(file), fname, file_options, ioptions.clock, io_tracer,
ioptions.statistics, ioptions.listeners,
ioptions.stats, ioptions.listeners,
ioptions.file_checksum_gen_factory.get(),
tmp_set.Contains(FileType::kTableFile)));
@ -168,7 +168,7 @@ Status BuildTable(
}
MergeHelper merge(env, internal_comparator.user_comparator(),
ioptions.merge_operator.get(), nullptr, ioptions.info_log,
ioptions.merge_operator.get(), nullptr, ioptions.logger,
true /* internal key corruption is not ok */,
snapshots.empty() ? 0 : snapshots.back(),
snapshot_checker);
@ -185,7 +185,7 @@ Status BuildTable(
CompactionIterator c_iter(
iter, internal_comparator.user_comparator(), &merge, kMaxSequenceNumber,
&snapshots, earliest_write_conflict_snapshot, snapshot_checker, env,
ShouldReportDetailedTime(env, ioptions.statistics),
ShouldReportDetailedTime(env, ioptions.stats),
true /* internal key corruption is not ok */, range_del_agg.get(),
blob_file_builder.get(), ioptions.allow_data_in_errors,
/*compaction=*/nullptr,
@ -257,7 +257,7 @@ Status BuildTable(
// Finish and check for file errors
TEST_SYNC_POINT("BuildTable:BeforeSyncTable");
if (s.ok() && !empty) {
StopWatch sw(ioptions.clock, ioptions.statistics, TABLE_SYNC_MICROS);
StopWatch sw(ioptions.clock, ioptions.stats, TABLE_SYNC_MICROS);
*io_status = file_writer->Sync(ioptions.use_fsync);
}
TEST_SYNC_POINT("BuildTable:BeforeCloseTableFile");

@ -283,7 +283,7 @@ ColumnFamilyOptions SanitizeOptions(const ImmutableDBOptions& db_options,
}
if (result.level0_file_num_compaction_trigger == 0) {
ROCKS_LOG_WARN(db_options.info_log.get(),
ROCKS_LOG_WARN(db_options.logger,
"level0_file_num_compaction_trigger cannot be 0");
result.level0_file_num_compaction_trigger = 1;
}
@ -292,7 +292,7 @@ ColumnFamilyOptions SanitizeOptions(const ImmutableDBOptions& db_options,
result.level0_slowdown_writes_trigger ||
result.level0_slowdown_writes_trigger <
result.level0_file_num_compaction_trigger) {
ROCKS_LOG_WARN(db_options.info_log.get(),
ROCKS_LOG_WARN(db_options.logger,
"This condition must be satisfied: "
"level0_stop_writes_trigger(%d) >= "
"level0_slowdown_writes_trigger(%d) >= "
@ -309,7 +309,7 @@ ColumnFamilyOptions SanitizeOptions(const ImmutableDBOptions& db_options,
result.level0_slowdown_writes_trigger) {
result.level0_stop_writes_trigger = result.level0_slowdown_writes_trigger;
}
ROCKS_LOG_WARN(db_options.info_log.get(),
ROCKS_LOG_WARN(db_options.logger,
"Adjust the value to "
"level0_stop_writes_trigger(%d)"
"level0_slowdown_writes_trigger(%d)"
@ -545,7 +545,7 @@ ColumnFamilyData::ColumnFamilyData(
db_paths_registered_ = true;
} else {
ROCKS_LOG_ERROR(
ioptions_.info_log,
ioptions_.logger,
"Failed to register data paths of column family (id: %d, name: %s)",
id_, name_.c_str());
}
@ -578,13 +578,13 @@ ColumnFamilyData::ColumnFamilyData(
} else if (ioptions_.compaction_style == kCompactionStyleNone) {
compaction_picker_.reset(new NullCompactionPicker(
ioptions_, &internal_comparator_));
ROCKS_LOG_WARN(ioptions_.info_log,
ROCKS_LOG_WARN(ioptions_.logger,
"Column family %s does not use any background compaction. "
"Compactions can only be done via CompactFiles\n",
GetName().c_str());
#endif // !ROCKSDB_LITE
} else {
ROCKS_LOG_ERROR(ioptions_.info_log,
ROCKS_LOG_ERROR(ioptions_.logger,
"Unable to recognize the specified compaction style %d. "
"Column family %s will use kCompactionStyleLevel.\n",
ioptions_.compaction_style, GetName().c_str());
@ -593,12 +593,12 @@ ColumnFamilyData::ColumnFamilyData(
}
if (column_family_set_->NumberOfColumnFamilies() < 10) {
ROCKS_LOG_INFO(ioptions_.info_log,
ROCKS_LOG_INFO(ioptions_.logger,
"--------------- Options for column family [%s]:\n",
name.c_str());
initial_cf_options_.Dump(ioptions_.info_log);
initial_cf_options_.Dump(ioptions_.logger);
} else {
ROCKS_LOG_INFO(ioptions_.info_log, "\t(skipping printing options)\n");
ROCKS_LOG_INFO(ioptions_.logger, "\t(skipping printing options)\n");
}
}
@ -654,7 +654,7 @@ ColumnFamilyData::~ColumnFamilyData() {
Status s = ioptions_.env->UnregisterDbPaths(GetDbPaths());
if (!s.ok()) {
ROCKS_LOG_ERROR(
ioptions_.info_log,
ioptions_.logger,
"Failed to unregister data paths of column family (id: %d, name: %s)",
id_, name_.c_str());
}
@ -893,7 +893,7 @@ WriteStallCondition ColumnFamilyData::RecalculateWriteStallConditions(
write_controller_token_ = write_controller->GetStopToken();
internal_stats_->AddCFStats(InternalStats::MEMTABLE_LIMIT_STOPS, 1);
ROCKS_LOG_WARN(
ioptions_.info_log,
ioptions_.logger,
"[%s] Stopping writes because we have %d immutable memtables "
"(waiting for flush), max_write_buffer_number is set to %d",
name_.c_str(), imm()->NumNotFlushed(),
@ -906,7 +906,7 @@ WriteStallCondition ColumnFamilyData::RecalculateWriteStallConditions(
internal_stats_->AddCFStats(
InternalStats::LOCKED_L0_FILE_COUNT_LIMIT_STOPS, 1);
}
ROCKS_LOG_WARN(ioptions_.info_log,
ROCKS_LOG_WARN(ioptions_.logger,
"[%s] Stopping writes because we have %d level-0 files",
name_.c_str(), vstorage->l0_delay_trigger_count());
} else if (write_stall_condition == WriteStallCondition::kStopped &&
@ -915,7 +915,7 @@ WriteStallCondition ColumnFamilyData::RecalculateWriteStallConditions(
internal_stats_->AddCFStats(
InternalStats::PENDING_COMPACTION_BYTES_LIMIT_STOPS, 1);
ROCKS_LOG_WARN(
ioptions_.info_log,
ioptions_.logger,
"[%s] Stopping writes because of estimated pending compaction "
"bytes %" PRIu64,
name_.c_str(), compaction_needed_bytes);
@ -927,7 +927,7 @@ WriteStallCondition ColumnFamilyData::RecalculateWriteStallConditions(
mutable_cf_options.disable_auto_compactions);
internal_stats_->AddCFStats(InternalStats::MEMTABLE_LIMIT_SLOWDOWNS, 1);
ROCKS_LOG_WARN(
ioptions_.info_log,
ioptions_.logger,
"[%s] Stalling writes because we have %d immutable memtables "
"(waiting for flush), max_write_buffer_number is set to %d "
"rate %" PRIu64,
@ -949,7 +949,7 @@ WriteStallCondition ColumnFamilyData::RecalculateWriteStallConditions(
internal_stats_->AddCFStats(
InternalStats::LOCKED_L0_FILE_COUNT_LIMIT_SLOWDOWNS, 1);
}
ROCKS_LOG_WARN(ioptions_.info_log,
ROCKS_LOG_WARN(ioptions_.logger,
"[%s] Stalling writes because we have %d level-0 files "
"rate %" PRIu64,
name_.c_str(), vstorage->l0_delay_trigger_count(),
@ -974,7 +974,7 @@ WriteStallCondition ColumnFamilyData::RecalculateWriteStallConditions(
internal_stats_->AddCFStats(
InternalStats::PENDING_COMPACTION_BYTES_LIMIT_SLOWDOWNS, 1);
ROCKS_LOG_WARN(
ioptions_.info_log,
ioptions_.logger,
"[%s] Stalling writes because of estimated pending compaction "
"bytes %" PRIu64 " rate %" PRIu64,
name_.c_str(), vstorage->estimated_compaction_needed_bytes(),
@ -988,7 +988,7 @@ WriteStallCondition ColumnFamilyData::RecalculateWriteStallConditions(
write_controller_token_ =
write_controller->GetCompactionPressureToken();
ROCKS_LOG_INFO(
ioptions_.info_log,
ioptions_.logger,
"[%s] Increasing compaction threads because we have %d level-0 "
"files ",
name_.c_str(), vstorage->l0_delay_trigger_count());
@ -1002,7 +1002,7 @@ WriteStallCondition ColumnFamilyData::RecalculateWriteStallConditions(
write_controller->GetCompactionPressureToken();
if (mutable_cf_options.soft_pending_compaction_bytes_limit > 0) {
ROCKS_LOG_INFO(
ioptions_.info_log,
ioptions_.logger,
"[%s] Increasing compaction threads because of estimated pending "
"compaction "
"bytes %" PRIu64,
@ -1203,11 +1203,11 @@ SuperVersion* ColumnFamilyData::GetThreadLocalSuperVersion(DBImpl* db) {
SuperVersion* sv = static_cast<SuperVersion*>(ptr);
if (sv == SuperVersion::kSVObsolete ||
sv->version_number != super_version_number_.load()) {
RecordTick(ioptions_.statistics, NUMBER_SUPERVERSION_ACQUIRES);
RecordTick(ioptions_.stats, NUMBER_SUPERVERSION_ACQUIRES);
SuperVersion* sv_to_delete = nullptr;
if (sv && sv->Unref()) {
RecordTick(ioptions_.statistics, NUMBER_SUPERVERSION_CLEANUPS);
RecordTick(ioptions_.stats, NUMBER_SUPERVERSION_CLEANUPS);
db->mutex()->Lock();
// NOTE: underlying resources held by superversion (sst files) might
// not be released until the next background job.

@ -530,7 +530,7 @@ bool CompactionPicker::SetupOtherInputs(
}
}
if (expand_inputs) {
ROCKS_LOG_INFO(ioptions_.info_log,
ROCKS_LOG_INFO(ioptions_.logger,
"[%s] Expanding@%d %" ROCKSDB_PRIszt "+%" ROCKSDB_PRIszt
"(%" PRIu64 "+%" PRIu64 " bytes) to %" ROCKSDB_PRIszt
"+%" ROCKSDB_PRIszt " (%" PRIu64 "+%" PRIu64 " bytes)\n",

@ -244,7 +244,7 @@ Compaction* FIFOCompactionPicker::CompactRange(
assert(input_level == 0);
assert(output_level == 0);
*compaction_end = nullptr;
LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, ioptions_.info_log);
LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, ioptions_.logger);
Compaction* c = PickCompaction(cf_name, mutable_cf_options,
mutable_db_options, vstorage, &log_buffer);
log_buffer.FlushBufferToLog();

@ -486,7 +486,7 @@ Compaction* UniversalCompactionBuilder::PickCompaction() {
}
#endif
// update statistics
RecordInHistogram(ioptions_.statistics, NUM_FILES_IN_SINGLE_COMPACTION,
RecordInHistogram(ioptions_.stats, NUM_FILES_IN_SINGLE_COMPACTION,
c->inputs(0)->size());
picker_->RegisterCompaction(c);

@ -46,14 +46,14 @@ DBIter::DBIter(Env* _env, const ReadOptions& read_options,
: prefix_extractor_(mutable_cf_options.prefix_extractor.get()),
env_(_env),
clock_(cf_options.clock),
logger_(cf_options.info_log),
logger_(cf_options.logger),
user_comparator_(cmp),
merge_operator_(cf_options.merge_operator.get()),
iter_(iter),
version_(version),
read_callback_(read_callback),
sequence_(s),
statistics_(cf_options.statistics),
statistics_(cf_options.stats),
max_skip_(max_sequential_skip_in_iterations),
max_skippable_internal_keys_(read_options.max_skippable_internal_keys),
num_internal_keys_skipped_(0),

@ -58,9 +58,9 @@ ImmutableMemTableOptions::ImmutableMemTableOptions(
inplace_update_num_locks(mutable_cf_options.inplace_update_num_locks),
inplace_callback(ioptions.inplace_callback),
max_successive_merges(mutable_cf_options.max_successive_merges),
statistics(ioptions.statistics),
statistics(ioptions.stats),
merge_operator(ioptions.merge_operator.get()),
info_log(ioptions.info_log),
info_log(ioptions.logger),
allow_data_in_errors(ioptions.allow_data_in_errors) {}
MemTable::MemTable(const InternalKeyComparator& cmp,
@ -82,9 +82,9 @@ MemTable::MemTable(const InternalKeyComparator& cmp,
mutable_cf_options.memtable_huge_page_size),
table_(ioptions.memtable_factory->CreateMemTableRep(
comparator_, &arena_, mutable_cf_options.prefix_extractor.get(),
ioptions.info_log, column_family_id)),
ioptions.logger, column_family_id)),
range_del_table_(SkipListFactory().CreateMemTableRep(
comparator_, &arena_, nullptr /* transform */, ioptions.info_log,
comparator_, &arena_, nullptr /* transform */, ioptions.logger,
column_family_id)),
is_range_del_table_empty_(true),
data_size_(0),
@ -120,7 +120,7 @@ MemTable::MemTable(const InternalKeyComparator& cmp,
bloom_filter_.reset(
new DynamicBloom(&arena_, moptions_.memtable_prefix_bloom_bits,
6 /* hard coded 6 probes */,
moptions_.memtable_huge_page_size, ioptions.info_log));
moptions_.memtable_huge_page_size, ioptions.logger));
}
}

@ -110,7 +110,7 @@ Status TableCache::GetTableReader(
if (s.ok()) {
s = ioptions_.fs->NewRandomAccessFile(fname, fopts, &file, nullptr);
}
RecordTick(ioptions_.statistics, NO_FILE_OPENS);
RecordTick(ioptions_.stats, NO_FILE_OPENS);
if (s.IsPathNotFound()) {
fname = Rocks2LevelTableFileName(fname);
s = PrepareIOFromReadOptions(ro, ioptions_.clock, fopts.io_options);
@ -118,18 +118,18 @@ Status TableCache::GetTableReader(
s = ioptions_.fs->NewRandomAccessFile(fname, file_options, &file,
nullptr);
}
RecordTick(ioptions_.statistics, NO_FILE_OPENS);
RecordTick(ioptions_.stats, NO_FILE_OPENS);
}
if (s.ok()) {
if (!sequential_mode && ioptions_.advise_random_on_open) {
file->Hint(FSRandomAccessFile::kRandom);
}
StopWatch sw(ioptions_.clock, ioptions_.statistics, TABLE_OPEN_IO_MICROS);
StopWatch sw(ioptions_.clock, ioptions_.stats, TABLE_OPEN_IO_MICROS);
std::unique_ptr<RandomAccessFileReader> file_reader(
new RandomAccessFileReader(
std::move(file), fname, ioptions_.clock, io_tracer_,
record_read_stats ? ioptions_.statistics : nullptr, SST_READ_MICROS,
record_read_stats ? ioptions_.stats : nullptr, SST_READ_MICROS,
file_read_hist, ioptions_.rate_limiter.get(), ioptions_.listeners));
s = ioptions_.table_factory->NewTableReader(
ro,
@ -187,7 +187,7 @@ Status TableCache::FindTable(const ReadOptions& ro,
max_file_size_for_l0_meta_pin);
if (!s.ok()) {
assert(table_reader == nullptr);
RecordTick(ioptions_.statistics, NO_FILE_ERRORS);
RecordTick(ioptions_.stats, 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 {
@ -375,10 +375,10 @@ bool TableCache::GetFromRowCache(const Slice& user_key, IterKey& row_cache_key,
ioptions_.row_cache.get(), row_handle);
replayGetContextLog(*found_row_cache_entry, user_key, get_context,
&value_pinner);
RecordTick(ioptions_.statistics, ROW_CACHE_HIT);
RecordTick(ioptions_.stats, ROW_CACHE_HIT);
found = true;
} else {
RecordTick(ioptions_.statistics, ROW_CACHE_MISS);
RecordTick(ioptions_.stats, ROW_CACHE_MISS);
}
return found;
}

@ -1302,7 +1302,7 @@ Status Version::GetTableProperties(std::shared_ptr<const TableProperties>* tp,
if (!s.ok()) {
return s;
}
RecordTick(ioptions->statistics, NUMBER_DIRECT_LOAD_TABLE_PROPERTIES);
RecordTick(ioptions->stats, NUMBER_DIRECT_LOAD_TABLE_PROPERTIES);
*tp = std::shared_ptr<const TableProperties>(raw_table_properties);
return s;
@ -1763,9 +1763,8 @@ Version::Version(ColumnFamilyData* column_family_data, VersionSet* vset,
: env_(vset->env_),
clock_(vset->clock_),
cfd_(column_family_data),
info_log_((cfd_ == nullptr) ? nullptr : cfd_->ioptions()->info_log),
db_statistics_((cfd_ == nullptr) ? nullptr
: cfd_->ioptions()->statistics),
info_log_((cfd_ == nullptr) ? nullptr : cfd_->ioptions()->logger),
db_statistics_((cfd_ == nullptr) ? nullptr : cfd_->ioptions()->stats),
table_cache_((cfd_ == nullptr) ? nullptr : cfd_->table_cache()),
blob_file_cache_(cfd_ ? cfd_->blob_file_cache() : nullptr),
merge_operator_(
@ -2763,7 +2762,7 @@ void VersionStorageInfo::ComputeFilesMarkedForPeriodicCompaction(
status = ioptions.env->GetFileModificationTime(
file_path, &file_modification_time);
if (!status.ok()) {
ROCKS_LOG_WARN(ioptions.info_log,
ROCKS_LOG_WARN(ioptions.logger,
"Can't get file modification time: %s: %s",
file_path.c_str(), status.ToString().c_str());
continue;
@ -3486,7 +3485,7 @@ void VersionStorageInfo::CalculateBaseBytes(const ImmutableCFOptions& ioptions,
// base_bytes_min. We set it be base_bytes_min.
base_level_size = base_bytes_min + 1U;
base_level_ = first_non_empty_level;
ROCKS_LOG_INFO(ioptions.info_log,
ROCKS_LOG_INFO(ioptions.logger,
"More existing levels in DB than needed. "
"max_bytes_for_level_multiplier may not be guaranteed.");
} else {

@ -793,9 +793,7 @@ ImmutableCFOptions::ImmutableCFOptions(const Options& options)
ImmutableCFOptions::ImmutableCFOptions(const ImmutableDBOptions& db_options,
const ColumnFamilyOptions& cf_options)
: logger(db_options.info_log),
stats(db_options.statistics),
compaction_style(cf_options.compaction_style),
: compaction_style(cf_options.compaction_style),
compaction_pri(cf_options.compaction_pri),
user_comparator(cf_options.comparator),
internal_comparator(InternalKeyComparator(cf_options.comparator)),
@ -810,8 +808,8 @@ ImmutableCFOptions::ImmutableCFOptions(const ImmutableDBOptions& db_options,
cf_options.max_write_buffer_size_to_maintain),
inplace_update_support(cf_options.inplace_update_support),
inplace_callback(cf_options.inplace_callback),
info_log(logger.get()),
statistics(stats.get()),
logger(db_options.logger),
stats(db_options.stats),
rate_limiter(db_options.rate_limiter),
info_log_level(db_options.info_log_level),
env(db_options.env),

@ -20,9 +20,6 @@ namespace ROCKSDB_NAMESPACE {
// of DB. Raw pointers defined in this struct do not have ownership to the data
// they point to. Options contains std::shared_ptr to these data.
struct ImmutableCFOptions {
private:
std::shared_ptr<Logger> logger; // ImmutableDBOptions
std::shared_ptr<Statistics> stats; // ImmutableDBOptions
public:
static const char* kName() { return "ImmutableCFOptions"; }
explicit ImmutableCFOptions();
@ -57,9 +54,9 @@ struct ImmutableCFOptions {
Slice delta_value,
std::string* merged_value);
Logger* info_log; // ImmutableDBOptions
Logger* logger; // ImmutableDBOptions
Statistics* statistics; // ImmutableDBOptions
Statistics* stats; // ImmutableDBOptions
std::shared_ptr<RateLimiter> rate_limiter; // ImmutableDBOptions

@ -586,6 +586,8 @@ ImmutableDBOptions::ImmutableDBOptions(const DBOptions& options)
} else {
clock = SystemClock::Default().get();
}
logger = info_log.get();
stats = statistics.get();
}
void ImmutableDBOptions::Dump(Logger* log) const {

@ -97,6 +97,8 @@ struct ImmutableDBOptions {
// Convenience/Helper objects that are not part of the base DBOptions
std::shared_ptr<FileSystem> fs;
SystemClock* clock;
Statistics* stats;
Logger* logger;
};
struct MutableDBOptions {

@ -491,7 +491,7 @@ struct BlockBasedTableBuilder::Rep {
context.column_family_name = column_family_name;
context.compaction_style = ioptions.compaction_style;
context.level_at_creation = level_at_creation;
context.info_log = ioptions.info_log;
context.info_log = ioptions.logger;
filter_builder.reset(CreateFilterBlockBuilder(
ioptions, moptions, context, use_delta_encoding_for_index_values,
p_index_builder_));
@ -512,7 +512,7 @@ struct BlockBasedTableBuilder::Rep {
}
if (!ReifyDbHostIdProperty(ioptions.env, &db_host_id).ok()) {
ROCKS_LOG_INFO(ioptions.info_log, "db_host_id property will not be set");
ROCKS_LOG_INFO(ioptions.logger, "db_host_id property will not be set");
}
}
@ -855,7 +855,7 @@ BlockBasedTableBuilder::BlockBasedTableBuilder(
if (sanitized_table_options.format_version == 0 &&
sanitized_table_options.checksum != kCRC32c) {
ROCKS_LOG_WARN(
ioptions.info_log,
ioptions.logger,
"Silently converting format_version to 1 because checksum is "
"non-default");
// silently convert format_version to 1 to keep consistent with current
@ -959,14 +959,14 @@ void BlockBasedTableBuilder::Add(const Slice& key, const Slice& value) {
// TODO offset passed in is not accurate for parallel compression case
NotifyCollectTableCollectorsOnAdd(key, value, r->get_offset(),
r->table_properties_collectors,
r->ioptions.info_log);
r->ioptions.logger);
} else if (value_type == kTypeRangeDeletion) {
r->range_del_block.Add(key, value);
// TODO offset passed in is not accurate for parallel compression case
NotifyCollectTableCollectorsOnAdd(key, value, r->get_offset(),
r->table_properties_collectors,
r->ioptions.info_log);
r->ioptions.logger);
} else {
assert(false);
}
@ -1081,7 +1081,7 @@ void BlockBasedTableBuilder::CompressAndVerifyBlock(
StopWatchNano timer(
r->ioptions.clock,
ShouldReportDetailedTime(r->ioptions.env, r->ioptions.statistics));
ShouldReportDetailedTime(r->ioptions.env, r->ioptions.stats));
if (is_status_ok && raw_block_contents.size() < kCompressionSizeLimit) {
if (is_data_block) {
@ -1145,7 +1145,7 @@ void BlockBasedTableBuilder::CompressAndVerifyBlock(
if (!compressed_ok) {
// The result of the compression was invalid. abort.
abort_compression = true;
ROCKS_LOG_ERROR(r->ioptions.info_log,
ROCKS_LOG_ERROR(r->ioptions.logger,
"Decompressed block did not match raw block");
*out_status =
Status::Corruption("Decompressed block did not match raw block");
@ -1173,19 +1173,19 @@ void BlockBasedTableBuilder::CompressAndVerifyBlock(
// Abort compression if the block is too big, or did not pass
// verification.
if (abort_compression) {
RecordTick(r->ioptions.statistics, NUMBER_BLOCK_NOT_COMPRESSED);
RecordTick(r->ioptions.stats, NUMBER_BLOCK_NOT_COMPRESSED);
*type = kNoCompression;
*block_contents = raw_block_contents;
} else if (*type != kNoCompression) {
if (ShouldReportDetailedTime(r->ioptions.env, r->ioptions.statistics)) {
RecordTimeToHistogram(r->ioptions.statistics, COMPRESSION_TIMES_NANOS,
if (ShouldReportDetailedTime(r->ioptions.env, r->ioptions.stats)) {
RecordTimeToHistogram(r->ioptions.stats, COMPRESSION_TIMES_NANOS,
timer.ElapsedNanos());
}
RecordInHistogram(r->ioptions.statistics, BYTES_COMPRESSED,
RecordInHistogram(r->ioptions.stats, BYTES_COMPRESSED,
raw_block_contents.size());
RecordTick(r->ioptions.statistics, NUMBER_BLOCK_COMPRESSED);
RecordTick(r->ioptions.stats, NUMBER_BLOCK_COMPRESSED);
} else if (*type != r->compression_type) {
RecordTick(r->ioptions.statistics, NUMBER_BLOCK_NOT_COMPRESSED);
RecordTick(r->ioptions.stats, NUMBER_BLOCK_NOT_COMPRESSED);
}
}
@ -1196,8 +1196,7 @@ void BlockBasedTableBuilder::WriteRawBlock(const Slice& block_contents,
Rep* r = rep_;
Status s = Status::OK();
IOStatus io_s = IOStatus::OK();
StopWatch sw(r->ioptions.clock, r->ioptions.statistics,
WRITE_RAW_BLOCK_MICROS);
StopWatch sw(r->ioptions.clock, r->ioptions.stats, WRITE_RAW_BLOCK_MICROS);
handle->set_offset(r->get_offset());
handle->set_size(block_contents.size());
assert(status().ok());
@ -1590,7 +1589,7 @@ void BlockBasedTableBuilder::WritePropertiesBlock(
// Add use collected properties
NotifyCollectTableCollectorsOnFinish(rep_->table_properties_collectors,
rep_->ioptions.info_log,
rep_->ioptions.logger,
&property_block_builder);
WriteRawBlock(property_block_builder.Finish(), kNoCompression,

@ -166,8 +166,8 @@ Status ReadBlockFromFile(
Status s = block_fetcher.ReadBlockContents();
if (s.ok()) {
result->reset(BlocklikeTraits<TBlocklike>::Create(
std::move(contents), read_amp_bytes_per_bit, ioptions.statistics,
using_zstd, filter_policy));
std::move(contents), read_amp_bytes_per_bit, ioptions.stats, using_zstd,
filter_policy));
}
return s;
@ -221,7 +221,7 @@ CacheAllocationPtr CopyBufferToHeap(MemoryAllocator* allocator, Slice& buf) {
void BlockBasedTable::UpdateCacheHitMetrics(BlockType block_type,
GetContext* get_context,
size_t usage) const {
Statistics* const statistics = rep_->ioptions.statistics;
Statistics* const statistics = rep_->ioptions.stats;
PERF_COUNTER_ADD(block_cache_hit_count, 1);
PERF_COUNTER_BY_LEVEL_ADD(block_cache_hit_count, 1,
@ -279,7 +279,7 @@ void BlockBasedTable::UpdateCacheHitMetrics(BlockType block_type,
void BlockBasedTable::UpdateCacheMissMetrics(BlockType block_type,
GetContext* get_context) const {
Statistics* const statistics = rep_->ioptions.statistics;
Statistics* const statistics = rep_->ioptions.stats;
// TODO: introduce aggregate (not per-level) block cache miss count
PERF_COUNTER_BY_LEVEL_ADD(block_cache_miss_count, 1,
@ -333,7 +333,7 @@ void BlockBasedTable::UpdateCacheInsertionMetrics(BlockType block_type,
GetContext* get_context,
size_t usage,
bool redundant) const {
Statistics* const statistics = rep_->ioptions.statistics;
Statistics* const statistics = rep_->ioptions.stats;
// TODO: introduce perf counters for block cache insertions
if (get_context) {
@ -425,7 +425,7 @@ void BlockBasedTable::UpdateCacheInsertionMetrics(BlockType block_type,
Cache::Handle* BlockBasedTable::GetEntryFromCache(
Cache* block_cache, const Slice& key, BlockType block_type,
GetContext* get_context) const {
auto cache_handle = block_cache->Lookup(key, rep_->ioptions.statistics);
auto cache_handle = block_cache->Lookup(key, rep_->ioptions.stats);
if (cache_handle != nullptr) {
UpdateCacheHitMetrics(block_type, get_context,
@ -662,7 +662,7 @@ Status BlockBasedTable::Open(
PersistentCacheOptions(rep->table_options.persistent_cache,
std::string(rep->persistent_cache_key_prefix,
rep->persistent_cache_key_prefix_size),
rep->ioptions.statistics);
rep->ioptions.stats);
// Meta-blocks are not dictionary compressed. Explicitly set the dictionary
// handle to null, otherwise it may be seen as uninitialized during the below
@ -805,7 +805,7 @@ Status BlockBasedTable::ReadPropertiesBlock(
s = SeekToPropertiesBlock(meta_iter, &found_properties_block);
if (!s.ok()) {
ROCKS_LOG_WARN(rep_->ioptions.info_log,
ROCKS_LOG_WARN(rep_->ioptions.logger,
"Error when seeking to properties block from file: %s",
s.ToString().c_str());
} else if (found_properties_block) {
@ -832,7 +832,7 @@ Status BlockBasedTable::ReadPropertiesBlock(
}
if (!s.ok()) {
ROCKS_LOG_WARN(rep_->ioptions.info_log,
ROCKS_LOG_WARN(rep_->ioptions.logger,
"Encountered error while reading data from properties "
"block %s",
s.ToString().c_str());
@ -849,7 +849,7 @@ Status BlockBasedTable::ReadPropertiesBlock(
CompressionTypeToString(kZSTDNotFinalCompression));
}
} else {
ROCKS_LOG_ERROR(rep_->ioptions.info_log,
ROCKS_LOG_ERROR(rep_->ioptions.logger,
"Cannot find Properties block from file.");
}
#ifndef ROCKSDB_LITE
@ -864,11 +864,10 @@ Status BlockBasedTable::ReadPropertiesBlock(
rep_->whole_key_filtering &=
IsFeatureSupported(*(rep_->table_properties),
BlockBasedTablePropertyNames::kWholeKeyFiltering,
rep_->ioptions.info_log);
rep_->prefix_filtering &=
IsFeatureSupported(*(rep_->table_properties),
BlockBasedTablePropertyNames::kPrefixFiltering,
rep_->ioptions.info_log);
rep_->ioptions.logger);
rep_->prefix_filtering &= IsFeatureSupported(
*(rep_->table_properties),
BlockBasedTablePropertyNames::kPrefixFiltering, rep_->ioptions.logger);
rep_->index_key_includes_seq =
rep_->table_properties->index_key_is_user_key == 0;
@ -891,7 +890,7 @@ Status BlockBasedTable::ReadPropertiesBlock(
s = GetGlobalSequenceNumber(*(rep_->table_properties), largest_seqno,
&(rep_->global_seqno));
if (!s.ok()) {
ROCKS_LOG_ERROR(rep_->ioptions.info_log, "%s", s.ToString().c_str());
ROCKS_LOG_ERROR(rep_->ioptions.logger, "%s", s.ToString().c_str());
}
}
return s;
@ -908,7 +907,7 @@ Status BlockBasedTable::ReadRangeDelBlock(
s = SeekToRangeDelBlock(meta_iter, &found_range_del_block, &range_del_handle);
if (!s.ok()) {
ROCKS_LOG_WARN(
rep_->ioptions.info_log,
rep_->ioptions.logger,
"Error when seeking to range delete tombstones block from file: %s",
s.ToString().c_str());
} else if (found_range_del_block && !range_del_handle.IsNull()) {
@ -920,7 +919,7 @@ Status BlockBasedTable::ReadRangeDelBlock(
s = iter->status();
if (!s.ok()) {
ROCKS_LOG_WARN(
rep_->ioptions.info_log,
rep_->ioptions.logger,
"Encountered error while reading data from range del block %s",
s.ToString().c_str());
IGNORE_STATUS_IF_ERROR(s);
@ -1151,7 +1150,7 @@ Status BlockBasedTable::ReadMetaIndexBlock(
nullptr /* filter_policy */);
if (!s.ok()) {
ROCKS_LOG_ERROR(rep_->ioptions.info_log,
ROCKS_LOG_ERROR(rep_->ioptions.logger,
"Encountered error while reading data from properties"
" block %s",
s.ToString().c_str());
@ -1206,7 +1205,7 @@ Status BlockBasedTable::GetDataBlockFromCache(
block_cache_compressed_handle =
block_cache_compressed->Lookup(compressed_block_cache_key);
Statistics* statistics = rep_->ioptions.statistics;
Statistics* statistics = rep_->ioptions.stats;
// if we found in the compressed cache, then uncompress and insert into
// uncompressed cache
@ -1291,7 +1290,7 @@ Status BlockBasedTable::PutDataBlockToCache(
assert(cached_block->IsEmpty());
Status s;
Statistics* statistics = ioptions.statistics;
Statistics* statistics = ioptions.stats;
std::unique_ptr<TBlocklike> block_holder;
if (raw_block_comp_type != kNoCompression) {
@ -1422,8 +1421,7 @@ DataBlockIter* BlockBasedTable::InitBlockIterator<DataBlockIter>(
DataBlockIter* input_iter, bool block_contents_pinned) {
return block->NewDataIterator(rep->internal_comparator.user_comparator(),
rep->get_global_seqno(block_type), input_iter,
rep->ioptions.statistics,
block_contents_pinned);
rep->ioptions.stats, block_contents_pinned);
}
template <>
@ -1432,7 +1430,7 @@ IndexBlockIter* BlockBasedTable::InitBlockIterator<IndexBlockIter>(
IndexBlockIter* input_iter, bool block_contents_pinned) {
return block->NewIndexIterator(
rep->internal_comparator.user_comparator(),
rep->get_global_seqno(block_type), input_iter, rep->ioptions.statistics,
rep->get_global_seqno(block_type), input_iter, rep->ioptions.stats,
/* total_order_seek */ true, rep->index_has_first_key,
rep->index_key_includes_seq, rep->index_value_is_full,
block_contents_pinned);
@ -1492,7 +1490,7 @@ Status BlockBasedTable::MaybeReadBlockAndLoadToCache(
// Can't find the block from the cache. If I/O is allowed, read from the
// file.
if (block_entry->GetValue() == nullptr && !no_io && ro.fill_cache) {
Statistics* statistics = rep_->ioptions.statistics;
Statistics* statistics = rep_->ioptions.stats;
const bool maybe_compressed =
block_type != BlockType::kFilter &&
block_type != BlockType::kCompressionDictionary &&
@ -1886,7 +1884,7 @@ void BlockBasedTable::RetrieveMultipleBlocks(
}
if (s.ok()) {
(*results)[idx_in_batch].SetOwnedValue(new Block(
std::move(contents), read_amp_bytes_per_bit, ioptions.statistics));
std::move(contents), read_amp_bytes_per_bit, ioptions.stats));
}
}
(*statuses)[idx_in_batch] = s;
@ -1935,7 +1933,7 @@ Status BlockBasedTable::RetrieveBlock(
std::unique_ptr<TBlocklike> block;
{
StopWatch sw(rep_->ioptions.clock, rep_->ioptions.statistics,
StopWatch sw(rep_->ioptions.clock, rep_->ioptions.stats,
READ_BLOCK_GET_MICROS);
s = ReadBlockFromFile(
rep_->file.get(), prefetch_buffer, rep_->footer, ro, handle, &block,
@ -2149,7 +2147,7 @@ bool BlockBasedTable::PrefixMayMatch(
}
if (filter_checked) {
Statistics* statistics = rep_->ioptions.statistics;
Statistics* statistics = rep_->ioptions.stats;
RecordTick(statistics, BLOOM_FILTER_PREFIX_CHECKED);
if (!may_match) {
RecordTick(statistics, BLOOM_FILTER_PREFIX_USEFUL);
@ -2232,7 +2230,7 @@ bool BlockBasedTable::FullFilterKeyMayMatch(
may_match = false;
}
if (may_match) {
RecordTick(rep_->ioptions.statistics, BLOOM_FILTER_FULL_POSITIVE);
RecordTick(rep_->ioptions.stats, BLOOM_FILTER_FULL_POSITIVE);
PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_full_positive, 1, rep_->level);
}
return may_match;
@ -2253,14 +2251,13 @@ void BlockBasedTable::FullFilterKeysMayMatch(
lookup_context);
uint64_t after_keys = range->KeysLeft();
if (after_keys) {
RecordTick(rep_->ioptions.statistics, BLOOM_FILTER_FULL_POSITIVE,
after_keys);
RecordTick(rep_->ioptions.stats, BLOOM_FILTER_FULL_POSITIVE, after_keys);
PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_full_positive, after_keys,
rep_->level);
}
uint64_t filtered_keys = before_keys - after_keys;
if (filtered_keys) {
RecordTick(rep_->ioptions.statistics, BLOOM_FILTER_USEFUL, filtered_keys);
RecordTick(rep_->ioptions.stats, BLOOM_FILTER_USEFUL, filtered_keys);
PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_useful, filtered_keys,
rep_->level);
}
@ -2269,12 +2266,11 @@ void BlockBasedTable::FullFilterKeysMayMatch(
prefix_extractor->Name()) == 0) {
filter->PrefixesMayMatch(range, prefix_extractor, kNotValid, false,
lookup_context);
RecordTick(rep_->ioptions.statistics, BLOOM_FILTER_PREFIX_CHECKED,
before_keys);
RecordTick(rep_->ioptions.stats, BLOOM_FILTER_PREFIX_CHECKED, before_keys);
uint64_t after_keys = range->KeysLeft();
uint64_t filtered_keys = before_keys - after_keys;
if (filtered_keys) {
RecordTick(rep_->ioptions.statistics, BLOOM_FILTER_PREFIX_USEFUL,
RecordTick(rep_->ioptions.stats, BLOOM_FILTER_PREFIX_USEFUL,
filtered_keys);
}
}
@ -2310,7 +2306,7 @@ Status BlockBasedTable::Get(const ReadOptions& read_options, const Slice& key,
get_context, &lookup_context);
TEST_SYNC_POINT("BlockBasedTable::Get:AfterFilterMatch");
if (!may_match) {
RecordTick(rep_->ioptions.statistics, BLOOM_FILTER_USEFUL);
RecordTick(rep_->ioptions.stats, BLOOM_FILTER_USEFUL);
PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_useful, 1, rep_->level);
} else {
IndexBlockIter iiter_on_stack;
@ -2347,7 +2343,7 @@ Status BlockBasedTable::Get(const ReadOptions& read_options, const Slice& key,
// Not found
// TODO: think about interaction with Merge. If a user key cannot
// cross one data block, we should be fine.
RecordTick(rep_->ioptions.statistics, BLOOM_FILTER_USEFUL);
RecordTick(rep_->ioptions.stats, BLOOM_FILTER_USEFUL);
PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_useful, 1, rep_->level);
break;
}
@ -2455,7 +2451,7 @@ Status BlockBasedTable::Get(const ReadOptions& read_options, const Slice& key,
}
}
if (matched && filter != nullptr && !filter->IsBlockBased()) {
RecordTick(rep_->ioptions.statistics, BLOOM_FILTER_FULL_TRUE_POSITIVE);
RecordTick(rep_->ioptions.stats, BLOOM_FILTER_FULL_TRUE_POSITIVE);
PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_full_true_positive, 1,
rep_->level);
}
@ -2799,7 +2795,7 @@ void BlockBasedTable::MultiGet(const ReadOptions& read_options,
} while (iiter->Valid());
if (matched && filter != nullptr && !filter->IsBlockBased()) {
RecordTick(rep_->ioptions.statistics, BLOOM_FILTER_FULL_TRUE_POSITIVE);
RecordTick(rep_->ioptions.stats, BLOOM_FILTER_FULL_TRUE_POSITIVE);
PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_full_true_positive, 1,
rep_->level);
}
@ -3091,7 +3087,7 @@ Status BlockBasedTable::CreateIndexReader(
auto meta_index_iter = preloaded_meta_index_iter;
bool should_fallback = false;
if (rep_->internal_prefix_transform.get() == nullptr) {
ROCKS_LOG_WARN(rep_->ioptions.info_log,
ROCKS_LOG_WARN(rep_->ioptions.logger,
"No prefix extractor passed in. Fall back to binary"
" search index.");
should_fallback = true;
@ -3101,7 +3097,7 @@ Status BlockBasedTable::CreateIndexReader(
if (!s.ok()) {
// we simply fall back to binary search in case there is any
// problem with prefix hash index loading.
ROCKS_LOG_WARN(rep_->ioptions.info_log,
ROCKS_LOG_WARN(rep_->ioptions.logger,
"Unable to read the metaindex block."
" Fall back to binary search index.");
should_fallback = true;

@ -47,7 +47,7 @@ class MockBlockBasedTableTester {
context.column_family_name = "mock_cf";
context.compaction_style = ioptions_.compaction_style;
context.level_at_creation = kMockLevel;
context.info_log = ioptions_.info_log;
context.info_log = ioptions_.logger;
return BloomFilterPolicy::GetBuilderFromContext(context);
}
};

@ -430,7 +430,7 @@ Status PartitionedFilterBlockReader::CacheDependencies(const ReadOptions& ro,
Status s = GetOrReadFilterBlock(false /* no_io */, nullptr /* get_context */,
&lookup_context, &filter_block);
if (!s.ok()) {
ROCKS_LOG_ERROR(rep->ioptions.info_log,
ROCKS_LOG_ERROR(rep->ioptions.logger,
"Error retrieving top-level filter block while trying to "
"cache filter partitions: %s",
s.ToString().c_str());

@ -64,7 +64,7 @@ Status UncompressionDictReader::ReadUncompressionDictionary(
if (!s.ok()) {
ROCKS_LOG_WARN(
rep->ioptions.info_log,
rep->ioptions.logger,
"Encountered error while reading data from compression dictionary "
"block %s",
s.ToString().c_str());

@ -45,9 +45,9 @@ inline bool BlockFetcher::TryGetUncompressBlockFromPersistentCache() {
return true;
} else {
// uncompressed page is not found
if (ioptions_.info_log && !status.IsNotFound()) {
if (ioptions_.logger && !status.IsNotFound()) {
assert(!status.ok());
ROCKS_LOG_INFO(ioptions_.info_log,
ROCKS_LOG_INFO(ioptions_.logger,
"Error reading from persistent cache. %s",
status.ToString().c_str());
}
@ -89,9 +89,9 @@ inline bool BlockFetcher::TryGetCompressedBlockFromPersistentCache() {
used_buf_ = heap_buf_.get();
slice_ = Slice(heap_buf_.get(), block_size_);
return true;
} else if (!io_status_.IsNotFound() && ioptions_.info_log) {
} else if (!io_status_.IsNotFound() && ioptions_.logger) {
assert(!io_status_.ok());
ROCKS_LOG_INFO(ioptions_.info_log,
ROCKS_LOG_INFO(ioptions_.logger,
"Error reading from persistent cache. %s",
io_status_.ToString().c_str());
}

@ -353,8 +353,8 @@ Status UncompressBlockContentsForCompressionType(
assert(uncompression_info.type() != kNoCompression &&
"Invalid compression type");
StopWatchNano timer(ioptions.clock, ShouldReportDetailedTime(
ioptions.env, ioptions.statistics));
StopWatchNano timer(ioptions.clock,
ShouldReportDetailedTime(ioptions.env, ioptions.stats));
size_t uncompressed_size = 0;
CacheAllocationPtr ubuf =
UncompressData(uncompression_info, data, n, &uncompressed_size,
@ -367,13 +367,13 @@ Status UncompressBlockContentsForCompressionType(
*contents = BlockContents(std::move(ubuf), uncompressed_size);
if (ShouldReportDetailedTime(ioptions.env, ioptions.statistics)) {
RecordTimeToHistogram(ioptions.statistics, DECOMPRESSION_TIMES_NANOS,
if (ShouldReportDetailedTime(ioptions.env, ioptions.stats)) {
RecordTimeToHistogram(ioptions.stats, DECOMPRESSION_TIMES_NANOS,
timer.ElapsedNanos());
}
RecordTimeToHistogram(ioptions.statistics, BYTES_DECOMPRESSED,
RecordTimeToHistogram(ioptions.stats, BYTES_DECOMPRESSED,
contents->data.size());
RecordTick(ioptions.statistics, NUMBER_BLOCK_DECOMPRESSED);
RecordTick(ioptions.stats, NUMBER_BLOCK_DECOMPRESSED);
TEST_SYNC_POINT_CALLBACK(
"UncompressBlockContentsForCompressionType:TamperWithReturnValue",

@ -329,7 +329,7 @@ Status ReadProperties(const ReadOptions& read_options,
auto error_msg =
"Detect malformed value in properties meta-block:"
"\tkey: " + key + "\tval: " + raw_val.ToString();
ROCKS_LOG_ERROR(ioptions.info_log, "%s", error_msg.c_str());
ROCKS_LOG_ERROR(ioptions.logger, "%s", error_msg.c_str());
continue;
}
*(pos->second) = val;

@ -102,7 +102,7 @@ PlainTableBuilder::PlainTableBuilder(
properties_.db_session_id = db_session_id;
properties_.db_host_id = ioptions.db_host_id;
if (!ReifyDbHostIdProperty(ioptions_.env, &properties_.db_host_id).ok()) {
ROCKS_LOG_INFO(ioptions_.info_log, "db_host_id property will not be set");
ROCKS_LOG_INFO(ioptions_.logger, "db_host_id property will not be set");
}
properties_.prefix_extractor_name = moptions_.prefix_extractor != nullptr
? moptions_.prefix_extractor->Name()
@ -193,7 +193,7 @@ void PlainTableBuilder::Add(const Slice& key, const Slice& value) {
// notify property collectors
NotifyCollectTableCollectorsOnAdd(
key, value, offset_, table_properties_collectors_, ioptions_.info_log);
key, value, offset_, table_properties_collectors_, ioptions_.logger);
status_ = io_status_;
}
@ -219,7 +219,7 @@ Status PlainTableBuilder::Finish() {
bloom_block_.SetTotalBits(
&arena_,
static_cast<uint32_t>(properties_.num_entries) * bloom_bits_per_key_,
ioptions_.bloom_locality, huge_page_tlb_size_, ioptions_.info_log);
ioptions_.bloom_locality, huge_page_tlb_size_, ioptions_.logger);
PutVarint32(&properties_.user_collected_properties
[PlainTablePropertyNames::kNumBloomBlocks],
@ -263,9 +263,8 @@ Status PlainTableBuilder::Finish() {
property_block_builder.Add(properties_.user_collected_properties);
// -- Add user collected properties
NotifyCollectTableCollectorsOnFinish(table_properties_collectors_,
ioptions_.info_log,
&property_block_builder);
NotifyCollectTableCollectorsOnFinish(
table_properties_collectors_, ioptions_.logger, &property_block_builder);
// -- Write property block
BlockHandle property_block_handle;

@ -98,7 +98,7 @@ Slice PlainTableIndexBuilder::Finish() {
BucketizeIndexes(&hash_to_offsets, &entries_per_bucket);
keys_per_prefix_hist_.Add(num_keys_per_prefix_);
ROCKS_LOG_INFO(ioptions_.info_log, "Number of Keys per prefix Histogram: %s",
ROCKS_LOG_INFO(ioptions_.logger, "Number of Keys per prefix Histogram: %s",
keys_per_prefix_hist_.ToString().c_str());
// From the temp data structure, populate indexes.
@ -153,12 +153,12 @@ void PlainTableIndexBuilder::BucketizeIndexes(
Slice PlainTableIndexBuilder::FillIndexes(
const std::vector<IndexRecord*>& hash_to_offsets,
const std::vector<uint32_t>& entries_per_bucket) {
ROCKS_LOG_DEBUG(ioptions_.info_log,
ROCKS_LOG_DEBUG(ioptions_.logger,
"Reserving %" PRIu32 " 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_, ioptions_.info_log);
total_allocate_size, huge_page_tlb_size_, ioptions_.logger);
auto temp_ptr = EncodeVarint32(allocated, index_size_);
uint32_t* index =
@ -198,7 +198,7 @@ Slice PlainTableIndexBuilder::FillIndexes(
}
assert(sub_index_offset == sub_index_size_);
ROCKS_LOG_DEBUG(ioptions_.info_log,
ROCKS_LOG_DEBUG(ioptions_.logger,
"hash table size: %" PRIu32 ", suffix_map length %" PRIu32,
index_size_, sub_index_size_);
return Slice(allocated, GetTotalSize());

@ -277,7 +277,7 @@ void PlainTableReader::AllocateBloom(int bloom_bits_per_key, int num_keys,
if (bloom_total_bits > 0) {
enable_bloom_ = true;
bloom_.SetTotalBits(&arena_, bloom_total_bits, ioptions_.bloom_locality,
huge_page_tlb_size, ioptions_.info_log);
huge_page_tlb_size, ioptions_.logger);
}
}

@ -177,7 +177,7 @@ void TableReaderBenchmark(Options& opts, EnvOptions& env_options,
SequenceNumber max_covering_tombstone_seq = 0;
GetContext get_context(
ioptions.user_comparator, ioptions.merge_operator.get(),
ioptions.info_log, ioptions.statistics, GetContext::kNotFound,
ioptions.logger, ioptions.stats, GetContext::kNotFound,
Slice(key), &value, nullptr, &merge_context, true,
&max_covering_tombstone_seq, clock);
s = table_reader->Get(read_options, key, &get_context, nullptr);

@ -1412,7 +1412,6 @@ TEST_P(BlockBasedTableTest, BasicBlockBasedTableProperties) {
ImmutableCFOptions ioptions(options);
MutableCFOptions moptions(options);
ioptions.statistics = options.statistics.get();
c.Finish(options, ioptions, moptions, table_options,
GetPlainInternalComparator(options.comparator), &keys, &kvmap);
ASSERT_EQ(options.statistics->getTickerCount(NUMBER_BLOCK_NOT_COMPRESSED), 0);
@ -1461,7 +1460,6 @@ uint64_t BlockBasedTableTest::IndexUncompressedHelper(bool compressed) {
ImmutableCFOptions ioptions(options);
MutableCFOptions moptions(options);
ioptions.statistics = options.statistics.get();
c.Finish(options, ioptions, moptions, table_options,
GetPlainInternalComparator(options.comparator), &keys, &kvmap);
c.ResetTableReader();

@ -282,7 +282,7 @@ struct FilterBench : public MockBlockBasedTableTester {
for (uint32_t i = 0; i < FLAGS_batch_size; ++i) {
kms_.emplace_back(FLAGS_key_size < 8 ? 8 : FLAGS_key_size);
}
ioptions_.info_log = &stderr_logger_;
ioptions_.logger = &stderr_logger_;
table_options_.optimize_filters_for_memory =
FLAGS_optimize_filters_for_memory;
}

Loading…
Cancel
Save