From 3778470061c77f773fab1e433c2ecad7ff02f293 Mon Sep 17 00:00:00 2001 From: haoyuhuang Date: Mon, 22 Jul 2019 17:47:54 -0700 Subject: [PATCH] Block cache analyzer: Compute correlation of features and human readable trace file. (#5596) Summary: - Compute correlation between a few features and predictions, e.g., number of accesses since the last access vs number of accesses till the next access on a block. - Output human readable trace file so python can consume it. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5596 Test Plan: make clean && USE_CLANG=1 make check -j32 Differential Revision: D16373200 Pulled By: HaoyuHuang fbshipit-source-id: c848d26bc2e9210461f317d7dbee42d55be5a0cc --- tools/block_cache_trace_analyzer.cc | 475 ++++++++++++++++-- tools/block_cache_trace_analyzer.h | 95 +++- tools/block_cache_trace_analyzer_test.cc | 76 ++- trace_replay/block_cache_tracer.cc | 14 + trace_replay/block_cache_tracer.h | 5 +- utilities/simulator_cache/cache_simulator.cc | 66 +-- utilities/simulator_cache/cache_simulator.h | 85 ++-- .../simulator_cache/cache_simulator_test.cc | 79 +-- 8 files changed, 753 insertions(+), 142 deletions(-) diff --git a/tools/block_cache_trace_analyzer.cc b/tools/block_cache_trace_analyzer.cc index 766338462..08143ebcf 100644 --- a/tools/block_cache_trace_analyzer.cc +++ b/tools/block_cache_trace_analyzer.cc @@ -7,11 +7,16 @@ #ifdef GFLAGS #include "tools/block_cache_trace_analyzer.h" +#include #include +#include +#include #include #include #include +#include #include + #include "monitoring/histogram.h" #include "util/gflags_compat.h" #include "util/string_util.h" @@ -122,6 +127,20 @@ DEFINE_string(analyze_get_spatial_locality_labels, "", "Group data blocks using these labels."); DEFINE_string(analyze_get_spatial_locality_buckets, "", "Group data blocks by their statistics using these buckets."); +DEFINE_bool(mrc_only, false, + "Evaluate alternative cache policies only. When this flag is true, " + "the analyzer does NOT maintain states of each block in memory for " + "analysis. It only feeds the accesses into the cache simulators."); +DEFINE_string( + analyze_correlation_coefficients_labels, "", + "Analyze the correlation coefficients of features such as number of past " + "accesses with regard to the number of accesses till the next access."); +DEFINE_int32(analyze_correlation_coefficients_max_number_of_values, 1000000, + "The maximum number of values for a feature. If the number of " + "values for a feature is larger than this max, it randomly " + "selects 'max' number of values."); +DEFINE_string(human_readable_trace_file_path, "", + "The filt path that saves human readable access records."); namespace rocksdb { namespace { @@ -143,7 +162,10 @@ const std::string kSupportedCacheNames = "ghost_lru_hybrid_no_insert_on_row_miss "; // The suffix for the generated csv files. +const std::string kFileNameSuffixMissRatioTimeline = "miss_ratio_timeline"; +const std::string kFileNameSuffixMissTimeline = "miss_timeline"; const std::string kFileNameSuffixAccessTimeline = "access_timeline"; +const std::string kFileNameSuffixCorrelation = "correlation_input"; const std::string kFileNameSuffixAvgReuseIntervalNaccesses = "avg_reuse_interval_naccesses"; const std::string kFileNameSuffixAvgReuseInterval = "avg_reuse_interval"; @@ -279,6 +301,18 @@ double percent(uint64_t numerator, uint64_t denomenator) { return static_cast(numerator * 100.0 / denomenator); } +std::map adjust_time_unit( + const std::map& time_stats, uint64_t time_unit) { + if (time_unit == 1) { + return time_stats; + } + std::map adjusted_time_stats; + for (auto const& time : time_stats) { + adjusted_time_stats[static_cast(time.first / time_unit)] += + time.second; + } + return adjusted_time_stats; +} } // namespace void BlockCacheTraceAnalyzer::WriteMissRatioCurves() const { @@ -288,8 +322,12 @@ void BlockCacheTraceAnalyzer::WriteMissRatioCurves() const { if (output_dir_.empty()) { return; } + uint64_t trace_duration = + trace_end_timestamp_in_seconds_ - trace_start_timestamp_in_seconds_; + uint64_t total_accesses = access_sequence_number_; const std::string output_miss_ratio_curve_path = - output_dir_ + "/" + kMissRatioCurveFileName; + output_dir_ + "/" + std::to_string(trace_duration) + "_" + + std::to_string(total_accesses) + "_" + kMissRatioCurveFileName; std::ofstream out(output_miss_ratio_curve_path); if (!out.is_open()) { return; @@ -302,7 +340,8 @@ void BlockCacheTraceAnalyzer::WriteMissRatioCurves() const { for (auto const& config_caches : cache_simulator_->sim_caches()) { const CacheConfiguration& config = config_caches.first; for (uint32_t i = 0; i < config.cache_capacities.size(); i++) { - double miss_ratio = config_caches.second[i]->miss_ratio(); + double miss_ratio = + config_caches.second[i]->miss_ratio_stats().miss_ratio(); // Write the body. out << config.cache_name; out << ","; @@ -314,13 +353,287 @@ void BlockCacheTraceAnalyzer::WriteMissRatioCurves() const { out << ","; out << std::fixed << std::setprecision(4) << miss_ratio; out << ","; - out << config_caches.second[i]->total_accesses(); + out << config_caches.second[i]->miss_ratio_stats().total_accesses(); out << std::endl; } } out.close(); } +void BlockCacheTraceAnalyzer::UpdateFeatureVectors( + const std::vector& access_sequence_number_timeline, + const std::vector& access_timeline, const std::string& label, + std::map* label_features, + std::map* label_predictions) const { + if (access_sequence_number_timeline.empty() || access_timeline.empty()) { + return; + } + assert(access_timeline.size() == access_sequence_number_timeline.size()); + uint64_t prev_access_sequence_number = access_sequence_number_timeline[0]; + uint64_t prev_access_timestamp = access_timeline[0]; + for (uint32_t i = 0; i < access_sequence_number_timeline.size(); i++) { + uint64_t num_accesses_since_last_access = + access_sequence_number_timeline[i] - prev_access_sequence_number; + uint64_t elapsed_time_since_last_access = + access_timeline[i] - prev_access_timestamp; + prev_access_sequence_number = access_sequence_number_timeline[i]; + prev_access_timestamp = access_timeline[i]; + if (i < access_sequence_number_timeline.size() - 1) { + (*label_features)[label].num_accesses_since_last_access.push_back( + num_accesses_since_last_access); + (*label_features)[label].num_past_accesses.push_back(i); + (*label_features)[label].elapsed_time_since_last_access.push_back( + elapsed_time_since_last_access); + } + if (i >= 1) { + (*label_predictions)[label].num_accesses_till_next_access.push_back( + num_accesses_since_last_access); + (*label_predictions)[label].elapsed_time_till_next_access.push_back( + elapsed_time_since_last_access); + } + } +} + +void BlockCacheTraceAnalyzer::WriteMissRatioTimeline(uint64_t time_unit) const { + if (!cache_simulator_ || output_dir_.empty()) { + return; + } + std::map>> + cs_name_timeline; + uint64_t start_time = port::kMaxUint64; + uint64_t end_time = 0; + const std::map& trace_num_misses = + adjust_time_unit(miss_ratio_stats_.num_misses_timeline(), time_unit); + const std::map& trace_num_accesses = + adjust_time_unit(miss_ratio_stats_.num_accesses_timeline(), time_unit); + assert(trace_num_misses.size() == trace_num_accesses.size()); + for (auto const& num_miss : trace_num_misses) { + uint64_t time = num_miss.first; + start_time = std::min(start_time, time); + end_time = std::max(end_time, time); + uint64_t miss = num_miss.second; + auto it = trace_num_accesses.find(time); + assert(it != trace_num_accesses.end()); + uint64_t access = it->second; + cs_name_timeline[port::kMaxUint64]["trace"][time] = percent(miss, access); + } + for (auto const& config_caches : cache_simulator_->sim_caches()) { + const CacheConfiguration& config = config_caches.first; + std::string cache_label = config.cache_name + "-" + + std::to_string(config.num_shard_bits) + "-" + + std::to_string(config.ghost_cache_capacity); + for (uint32_t i = 0; i < config.cache_capacities.size(); i++) { + const std::map& num_misses = adjust_time_unit( + config_caches.second[i]->miss_ratio_stats().num_misses_timeline(), + time_unit); + const std::map& num_accesses = adjust_time_unit( + config_caches.second[i]->miss_ratio_stats().num_accesses_timeline(), + time_unit); + assert(num_misses.size() == num_accesses.size()); + for (auto const& num_miss : num_misses) { + uint64_t time = num_miss.first; + start_time = std::min(start_time, time); + end_time = std::max(end_time, time); + uint64_t miss = num_miss.second; + auto it = num_accesses.find(time); + assert(it != num_accesses.end()); + uint64_t access = it->second; + cs_name_timeline[config.cache_capacities[i]][cache_label][time] = + percent(miss, access); + } + } + } + for (auto const& it : cs_name_timeline) { + const std::string output_miss_ratio_timeline_path = + output_dir_ + "/" + std::to_string(it.first) + "_" + + std::to_string(time_unit) + "_" + kFileNameSuffixMissRatioTimeline; + std::ofstream out(output_miss_ratio_timeline_path); + if (!out.is_open()) { + return; + } + std::string header("time"); + for (uint64_t now = start_time; now <= end_time; now++) { + header += ","; + header += std::to_string(now); + } + out << header << std::endl; + for (auto const& label : it.second) { + std::string row(label.first); + for (uint64_t now = start_time; now <= end_time; now++) { + auto misses = label.second.find(now); + row += ","; + if (misses != label.second.end()) { + row += std::to_string(misses->second); + } else { + row += "0"; + } + } + out << row << std::endl; + } + out.close(); + } +} + +void BlockCacheTraceAnalyzer::WriteMissTimeline(uint64_t time_unit) const { + if (!cache_simulator_ || output_dir_.empty()) { + return; + } + std::map>> + cs_name_timeline; + uint64_t start_time = port::kMaxUint64; + uint64_t end_time = 0; + const std::map& trace_num_misses = + adjust_time_unit(miss_ratio_stats_.num_misses_timeline(), time_unit); + for (auto const& num_miss : trace_num_misses) { + uint64_t time = num_miss.first; + start_time = std::min(start_time, time); + end_time = std::max(end_time, time); + uint64_t miss = num_miss.second; + cs_name_timeline[port::kMaxUint64]["trace"][time] = miss; + } + for (auto const& config_caches : cache_simulator_->sim_caches()) { + const CacheConfiguration& config = config_caches.first; + std::string cache_label = config.cache_name + "-" + + std::to_string(config.num_shard_bits) + "-" + + std::to_string(config.ghost_cache_capacity); + for (uint32_t i = 0; i < config.cache_capacities.size(); i++) { + const std::map& num_misses = adjust_time_unit( + config_caches.second[i]->miss_ratio_stats().num_misses_timeline(), + time_unit); + for (auto const& num_miss : num_misses) { + uint64_t time = num_miss.first; + start_time = std::min(start_time, time); + end_time = std::max(end_time, time); + uint64_t miss = num_miss.second; + cs_name_timeline[config.cache_capacities[i]][cache_label][time] = miss; + } + } + } + for (auto const& it : cs_name_timeline) { + const std::string output_miss_ratio_timeline_path = + output_dir_ + "/" + std::to_string(it.first) + "_" + + std::to_string(time_unit) + "_" + kFileNameSuffixMissTimeline; + std::ofstream out(output_miss_ratio_timeline_path); + if (!out.is_open()) { + return; + } + std::string header("time"); + for (uint64_t now = start_time; now <= end_time; now++) { + header += ","; + header += std::to_string(now); + } + out << header << std::endl; + for (auto const& label : it.second) { + std::string row(label.first); + for (uint64_t now = start_time; now <= end_time; now++) { + auto misses = label.second.find(now); + row += ","; + if (misses != label.second.end()) { + row += std::to_string(misses->second); + } else { + row += "0"; + } + } + out << row << std::endl; + } + out.close(); + } +} + +void BlockCacheTraceAnalyzer::WriteCorrelationFeatures( + const std::string& label_str, uint32_t max_number_of_values) const { + std::set labels = ParseLabelStr(label_str); + std::map label_features; + std::map label_predictions; + auto block_callback = + [&](const std::string& cf_name, uint64_t fd, uint32_t level, + TraceType block_type, const std::string& /*block_key*/, + uint64_t /*block_key_id*/, const BlockAccessInfo& block) { + if (labels.find(kGroupbyCaller) != labels.end()) { + // Group by caller. + for (auto const& caller_map : block.caller_access_timeline) { + const std::string label = + BuildLabel(labels, cf_name, fd, level, block_type, + caller_map.first, /*block_id=*/0); + auto it = block.caller_access_sequence__number_timeline.find( + caller_map.first); + assert(it != block.caller_access_sequence__number_timeline.end()); + UpdateFeatureVectors(it->second, caller_map.second, label, + &label_features, &label_predictions); + } + return; + } + const std::string label = BuildLabel( + labels, cf_name, fd, level, block_type, + TableReaderCaller::kMaxBlockCacheLookupCaller, /*block_id=*/0); + UpdateFeatureVectors(block.access_sequence_number_timeline, + block.access_timeline, label, &label_features, + &label_predictions); + }; + TraverseBlocks(block_callback); + WriteCorrelationFeaturesToFile(label_str, label_features, label_predictions, + max_number_of_values); +} + +void BlockCacheTraceAnalyzer::WriteCorrelationFeaturesToFile( + const std::string& label, + const std::map& label_features, + const std::map& label_predictions, + uint32_t max_number_of_values) const { + std::default_random_engine rand_engine(env_->NowMicros()); + for (auto const& label_feature_vectors : label_features) { + const Features& past = label_feature_vectors.second; + auto it = label_predictions.find(label_feature_vectors.first); + assert(it != label_predictions.end()); + const Predictions& future = it->second; + const std::string output_path = output_dir_ + "/" + label + "_" + + label_feature_vectors.first + "_" + + kFileNameSuffixCorrelation; + std::ofstream out(output_path); + if (!out.is_open()) { + return; + } + std::string header( + "num_accesses_since_last_access,elapsed_time_since_last_access,num_" + "past_accesses,num_accesses_till_next_access,elapsed_time_till_next_" + "access"); + out << header << std::endl; + std::vector indexes; + for (uint32_t i = 0; i < past.num_accesses_since_last_access.size(); i++) { + indexes.push_back(i); + } + std::shuffle(indexes.begin(), indexes.end(), rand_engine); + for (uint32_t i = 0; i < max_number_of_values && i < indexes.size(); i++) { + uint32_t rand_index = indexes[i]; + out << std::to_string(past.num_accesses_since_last_access[rand_index]) + << ","; + out << std::to_string(past.elapsed_time_since_last_access[rand_index]) + << ","; + out << std::to_string(past.num_past_accesses[rand_index]) << ","; + out << std::to_string(future.num_accesses_till_next_access[rand_index]) + << ","; + out << std::to_string(future.elapsed_time_till_next_access[rand_index]) + << std::endl; + } + out.close(); + } +} + +void BlockCacheTraceAnalyzer::WriteCorrelationFeaturesForGet( + uint32_t max_number_of_values) const { + std::string label = "GetKeyInfo"; + std::map label_features; + std::map label_predictions; + for (auto const& get_info : get_key_info_map_) { + const GetKeyInfo& info = get_info.second; + UpdateFeatureVectors(info.access_sequence_number_timeline, + info.access_timeline, label, &label_features, + &label_predictions); + } + WriteCorrelationFeaturesToFile(label, label_features, label_predictions, + max_number_of_values); +} + std::set BlockCacheTraceAnalyzer::ParseLabelStr( const std::string& label_str) const { std::stringstream ss(label_str); @@ -371,7 +684,6 @@ void BlockCacheTraceAnalyzer::TraverseBlocks( uint64_t /*block_key_id*/, const BlockAccessInfo& /*block_access_info*/)> block_callback) const { - uint64_t block_id = 0; for (auto const& cf_aggregates : cf_aggregates_map_) { // Stats per column family. const std::string& cf_name = cf_aggregates.first; @@ -387,8 +699,8 @@ void BlockCacheTraceAnalyzer::TraverseBlocks( block_type_aggregates.second.block_access_info_map) { // Stats per block. block_callback(cf_name, fd, level, type, block_access_info.first, - block_id, block_access_info.second); - block_id++; + block_access_info.second.block_id, + block_access_info.second); } } } @@ -1046,12 +1358,15 @@ void BlockCacheTraceAnalyzer::WriteAccessCountSummaryStats( BlockCacheTraceAnalyzer::BlockCacheTraceAnalyzer( const std::string& trace_file_path, const std::string& output_dir, - bool compute_reuse_distance, + const std::string& human_readable_trace_file_path, + bool compute_reuse_distance, bool mrc_only, std::unique_ptr&& cache_simulator) : env_(rocksdb::Env::Default()), trace_file_path_(trace_file_path), output_dir_(output_dir), + human_readable_trace_file_path_(human_readable_trace_file_path), compute_reuse_distance_(compute_reuse_distance), + mrc_only_(mrc_only), cache_simulator_(std::move(cache_simulator)) {} void BlockCacheTraceAnalyzer::ComputeReuseDistance( @@ -1072,7 +1387,29 @@ void BlockCacheTraceAnalyzer::ComputeReuseDistance( info->unique_blocks_since_last_access.clear(); } -void BlockCacheTraceAnalyzer::RecordAccess( +Status BlockCacheTraceAnalyzer::WriteHumanReadableTraceRecord( + const BlockCacheTraceRecord& access, uint64_t block_id, + uint64_t get_key_id) { + if (!human_readable_trace_file_writer_) { + return Status::OK(); + } + int ret = snprintf( + trace_record_buffer_, sizeof(trace_record_buffer_), + "%" PRIu64 ",%" PRIu64 ",%u,%" PRIu64 ",%" PRIu64 ",%" PRIu32 ",%" PRIu64 + "" + ",%u,%u,%" PRIu64 ",%" PRIu64 ",%" PRIu64 ",%u\n", + access.access_timestamp, block_id, access.block_type, access.block_size, + access.cf_id, access.level, access.sst_fd_number, access.caller, + access.no_insert, access.get_id, get_key_id, access.referenced_data_size, + access.is_cache_hit); + if (ret < 0) { + return Status::IOError("failed to format the output"); + } + std::string printout(trace_record_buffer_); + return human_readable_trace_file_writer_->Append(printout); +} + +Status BlockCacheTraceAnalyzer::RecordAccess( const BlockCacheTraceRecord& access) { ColumnFamilyAccessInfoAggregate& cf_aggr = cf_aggregates_map_[access.cf_name]; SSTFileAccessInfoAggregate& file_aggr = @@ -1080,18 +1417,30 @@ void BlockCacheTraceAnalyzer::RecordAccess( file_aggr.level = access.level; BlockTypeAccessInfoAggregate& block_type_aggr = file_aggr.block_type_aggregates_map[access.block_type]; + if (block_type_aggr.block_access_info_map.find(access.block_key) == + block_type_aggr.block_access_info_map.end()) { + block_type_aggr.block_access_info_map[access.block_key].block_id = + unique_block_id_; + unique_block_id_++; + } BlockAccessInfo& block_access_info = block_type_aggr.block_access_info_map[access.block_key]; if (compute_reuse_distance_) { ComputeReuseDistance(&block_access_info); } - block_access_info.AddAccess(access); + block_access_info.AddAccess(access, access_sequence_number_); block_info_map_[access.block_key] = &block_access_info; - if (trace_start_timestamp_in_seconds_ == 0) { - trace_start_timestamp_in_seconds_ = - access.access_timestamp / kMicrosInSecond; + uint64_t get_key_id = 0; + if (access.caller == TableReaderCaller::kUserGet && + access.get_id != BlockCacheTraceHelper::kReservedGetId) { + std::string row_key = BlockCacheTraceHelper::ComputeRowKey(access); + if (get_key_info_map_.find(row_key) == get_key_info_map_.end()) { + get_key_info_map_[row_key].key_id = unique_get_key_id_; + get_key_id = unique_get_key_id_; + unique_get_key_id_++; + } + get_key_info_map_[row_key].AddAccess(access, access_sequence_number_); } - trace_end_timestamp_in_seconds_ = access.access_timestamp / kMicrosInSecond; if (compute_reuse_distance_) { // Add this block to all existing blocks. @@ -1108,6 +1457,8 @@ void BlockCacheTraceAnalyzer::RecordAccess( } } } + return WriteHumanReadableTraceRecord(access, block_access_info.block_id, + get_key_id); } Status BlockCacheTraceAnalyzer::Analyze() { @@ -1122,32 +1473,68 @@ Status BlockCacheTraceAnalyzer::Analyze() { if (!s.ok()) { return s; } + if (!human_readable_trace_file_path_.empty()) { + s = env_->NewWritableFile(human_readable_trace_file_path_, + &human_readable_trace_file_writer_, EnvOptions()); + if (!s.ok()) { + return s; + } + } uint64_t start = env_->NowMicros(); - uint64_t processed_records = 0; uint64_t time_interval = 0; while (s.ok()) { BlockCacheTraceRecord access; s = reader.ReadAccess(&access); if (!s.ok()) { - return s; + break; } - RecordAccess(access); + if (!mrc_only_) { + s = RecordAccess(access); + if (!s.ok()) { + break; + } + } + if (trace_start_timestamp_in_seconds_ == 0) { + trace_start_timestamp_in_seconds_ = + access.access_timestamp / kMicrosInSecond; + } + trace_end_timestamp_in_seconds_ = access.access_timestamp / kMicrosInSecond; + miss_ratio_stats_.UpdateMetrics(access.access_timestamp, + is_user_access(access.caller), + access.is_cache_hit == Boolean::kFalse); if (cache_simulator_) { cache_simulator_->Access(access); } - processed_records++; + access_sequence_number_++; uint64_t now = env_->NowMicros(); uint64_t duration = (now - start) / kMicrosInSecond; if (duration > 10 * time_interval) { + uint64_t trace_duration = + trace_end_timestamp_in_seconds_ - trace_start_timestamp_in_seconds_; fprintf(stdout, "Running for %" PRIu64 " seconds: Processed %" PRIu64 - " records/second\n", - duration, processed_records / duration); - processed_records = 0; + " records/second. Trace duration %" PRIu64 + " seconds. Observed miss ratio %.2f\n", + duration, duration > 0 ? access_sequence_number_ / duration : 0, + trace_duration, miss_ratio_stats_.miss_ratio()); time_interval++; } } - return Status::OK(); + if (human_readable_trace_file_writer_) { + human_readable_trace_file_writer_->Flush(); + human_readable_trace_file_writer_->Close(); + } + uint64_t now = env_->NowMicros(); + uint64_t duration = (now - start) / kMicrosInSecond; + uint64_t trace_duration = + trace_end_timestamp_in_seconds_ - trace_start_timestamp_in_seconds_; + fprintf(stdout, + "Running for %" PRIu64 " seconds: Processed %" PRIu64 + " records/second. Trace duration %" PRIu64 + " seconds. Observed miss ratio %.2f\n", + duration, duration > 0 ? access_sequence_number_ / duration : 0, + trace_duration, miss_ratio_stats_.miss_ratio()); + return s; } void BlockCacheTraceAnalyzer::PrintBlockSizeStats() const { @@ -1321,15 +1708,6 @@ void BlockCacheTraceAnalyzer::PrintAccessCountStats(bool user_access_only, "Top %" PRIu32 " access count blocks access_count=%" PRIu64 " %s\n", top_k, naccess_it->first, statistics.c_str()); - // if (block->referenced_data_size > block->block_size) { - // for (auto const& ref_key_it : block->key_num_access_map) { - // ParsedInternalKey internal_key; - // ParseInternalKey(ref_key_it.first, &internal_key); - // printf("######%lu %lu %d %s\n", block->referenced_data_size, - // block->block_size, internal_key.type, - // internal_key.user_key.ToString().c_str()); - // } - // } } } @@ -1696,16 +2074,32 @@ int block_cache_trace_analyzer_tool(int argc, char** argv) { exit(1); } } - BlockCacheTraceAnalyzer analyzer( - FLAGS_block_cache_trace_path, FLAGS_block_cache_analysis_result_dir, - !FLAGS_reuse_distance_labels.empty(), std::move(cache_simulator)); + BlockCacheTraceAnalyzer analyzer(FLAGS_block_cache_trace_path, + FLAGS_block_cache_analysis_result_dir, + FLAGS_human_readable_trace_file_path, + !FLAGS_reuse_distance_labels.empty(), + FLAGS_mrc_only, std::move(cache_simulator)); Status s = analyzer.Analyze(); - if (!s.IsIncomplete()) { + if (!s.IsIncomplete() && !s.ok()) { // Read all traces. fprintf(stderr, "Cannot process the trace %s\n", s.ToString().c_str()); exit(1); } fprintf(stdout, "Status: %s\n", s.ToString().c_str()); + analyzer.WriteMissRatioCurves(); + analyzer.WriteMissRatioTimeline(1); + analyzer.WriteMissRatioTimeline(kSecondInMinute); + analyzer.WriteMissRatioTimeline(kSecondInHour); + analyzer.WriteMissTimeline(1); + analyzer.WriteMissTimeline(kSecondInMinute); + analyzer.WriteMissTimeline(kSecondInHour); + + if (FLAGS_mrc_only) { + fprintf(stdout, + "Skipping the analysis statistics since the user wants to compute " + "MRC only"); + return 0; + } analyzer.PrintStatsSummary(); if (FLAGS_print_access_count_stats) { @@ -1727,7 +2121,6 @@ int block_cache_trace_analyzer_tool(int argc, char** argv) { analyzer.PrintDataBlockAccessStats(); } print_break_lines(/*num_break_lines=*/3); - analyzer.WriteMissRatioCurves(); if (!FLAGS_timeline_labels.empty()) { std::stringstream ss(FLAGS_timeline_labels); @@ -1819,6 +2212,18 @@ int block_cache_trace_analyzer_tool(int argc, char** argv) { analyzer.WriteGetSpatialLocality(label, buckets); } } + + if (!FLAGS_analyze_correlation_coefficients_labels.empty()) { + std::stringstream ss(FLAGS_analyze_correlation_coefficients_labels); + while (ss.good()) { + std::string label; + getline(ss, label, ','); + analyzer.WriteCorrelationFeatures( + label, FLAGS_analyze_correlation_coefficients_max_number_of_values); + } + analyzer.WriteCorrelationFeaturesForGet( + FLAGS_analyze_correlation_coefficients_max_number_of_values); + } return 0; } diff --git a/tools/block_cache_trace_analyzer.h b/tools/block_cache_trace_analyzer.h index 32a90342c..bc41ff468 100644 --- a/tools/block_cache_trace_analyzer.h +++ b/tools/block_cache_trace_analyzer.h @@ -16,8 +16,23 @@ #include "utilities/simulator_cache/cache_simulator.h" namespace rocksdb { + +// Statistics of a key refereneced by a Get. +struct GetKeyInfo { + uint64_t key_id = 0; + std::vector access_sequence_number_timeline; + std::vector access_timeline; + + void AddAccess(const BlockCacheTraceRecord& access, + uint64_t access_sequnce_number) { + access_sequence_number_timeline.push_back(access_sequnce_number); + access_timeline.push_back(access.access_timestamp); + } +}; + // Statistics of a block. struct BlockAccessInfo { + uint64_t block_id = 0; uint64_t num_accesses = 0; uint64_t block_size = 0; uint64_t first_access_time = 0; @@ -39,7 +54,16 @@ struct BlockAccessInfo { // Number of reuses grouped by reuse distance. std::map reuse_distance_count; - void AddAccess(const BlockCacheTraceRecord& access) { + // The access sequence numbers of this block. + std::vector access_sequence_number_timeline; + std::map> + caller_access_sequence__number_timeline; + // The access timestamp in microseconds of this block. + std::vector access_timeline; + std::map> caller_access_timeline; + + void AddAccess(const BlockCacheTraceRecord& access, + uint64_t access_sequnce_number) { if (block_size != 0 && access.block_size != 0) { assert(block_size == access.block_size); } @@ -57,6 +81,12 @@ struct BlockAccessInfo { const uint64_t timestamp_in_seconds = access.access_timestamp / kMicrosInSecond; caller_num_accesses_timeline[access.caller][timestamp_in_seconds] += 1; + // Populate the feature vectors. + access_sequence_number_timeline.push_back(access_sequnce_number); + caller_access_sequence__number_timeline[access.caller].push_back( + access_sequnce_number); + access_timeline.push_back(access.access_timestamp); + caller_access_timeline[access.caller].push_back(access.access_timestamp); if (BlockCacheTraceHelper::IsGetOrMultiGetOnDataBlock(access.block_type, access.caller)) { num_keys = access.num_keys_in_block; @@ -94,11 +124,23 @@ struct ColumnFamilyAccessInfoAggregate { std::map fd_aggregates_map; }; +struct Features { + std::vector elapsed_time_since_last_access; + std::vector num_accesses_since_last_access; + std::vector num_past_accesses; +}; + +struct Predictions { + std::vector elapsed_time_till_next_access; + std::vector num_accesses_till_next_access; +}; + class BlockCacheTraceAnalyzer { public: BlockCacheTraceAnalyzer( const std::string& trace_file_path, const std::string& output_dir, - bool compute_reuse_distance, + const std::string& human_readable_trace_file_path, + bool compute_reuse_distance, bool mrc_only, std::unique_ptr&& cache_simulator); ~BlockCacheTraceAnalyzer() = default; // No copy and move. @@ -184,6 +226,24 @@ class BlockCacheTraceAnalyzer { // "cache_name,num_shard_bits,capacity,miss_ratio,total_accesses". void WriteMissRatioCurves() const; + // Write miss ratio timeline of simulated cache configurations into several + // csv files, one per cache capacity saved in 'output_dir'. + // + // The file format is + // "time,label_1_access_per_second,label_2_access_per_second,...,label_N_access_per_second" + // where N is the number of unique cache names + // (cache_name+num_shard_bits+ghost_capacity). + void WriteMissRatioTimeline(uint64_t time_unit) const; + + // Write misses timeline of simulated cache configurations into several + // csv files, one per cache capacity saved in 'output_dir'. + // + // The file format is + // "time,label_1_access_per_second,label_2_access_per_second,...,label_N_access_per_second" + // where N is the number of unique cache names + // (cache_name+num_shard_bits+ghost_capacity). + void WriteMissTimeline(uint64_t time_unit) const; + // Write the access timeline into a csv file saved in 'output_dir'. // // The file is named "label_access_timeline".The file format is @@ -236,6 +296,11 @@ class BlockCacheTraceAnalyzer { const std::string& label_str, const std::vector& percent_buckets) const; + void WriteCorrelationFeatures(const std::string& label_str, + uint32_t max_number_of_values) const; + + void WriteCorrelationFeaturesForGet(uint32_t max_number_of_values) const; + const std::map& TEST_cf_aggregates_map() const { return cf_aggregates_map_; @@ -251,7 +316,7 @@ class BlockCacheTraceAnalyzer { void ComputeReuseDistance(BlockAccessInfo* info) const; - void RecordAccess(const BlockCacheTraceRecord& access); + Status RecordAccess(const BlockCacheTraceRecord& access); void UpdateReuseIntervalStats( const std::string& label, const std::vector& time_buckets, @@ -278,17 +343,41 @@ class BlockCacheTraceAnalyzer { const BlockAccessInfo& /*block_access_info*/)> block_callback) const; + void UpdateFeatureVectors( + const std::vector& access_sequence_number_timeline, + const std::vector& access_timeline, const std::string& label, + std::map* label_features, + std::map* label_predictions) const; + + void WriteCorrelationFeaturesToFile( + const std::string& label, + const std::map& label_features, + const std::map& label_predictions, + uint32_t max_number_of_values) const; + + Status WriteHumanReadableTraceRecord(const BlockCacheTraceRecord& access, + uint64_t block_id, uint64_t get_key_id); + rocksdb::Env* env_; const std::string trace_file_path_; const std::string output_dir_; + std::string human_readable_trace_file_path_; const bool compute_reuse_distance_; + const bool mrc_only_; BlockCacheTraceHeader header_; std::unique_ptr cache_simulator_; std::map cf_aggregates_map_; std::map block_info_map_; + std::unordered_map get_key_info_map_; + uint64_t access_sequence_number_ = 0; uint64_t trace_start_timestamp_in_seconds_ = 0; uint64_t trace_end_timestamp_in_seconds_ = 0; + MissRatioStats miss_ratio_stats_; + uint64_t unique_block_id_ = 1; + uint64_t unique_get_key_id_ = 1; + char trace_record_buffer_[1024 * 1024]; + std::unique_ptr human_readable_trace_file_writer_; }; int block_cache_trace_analyzer_tool(int argc, char** argv); diff --git a/tools/block_cache_trace_analyzer_test.cc b/tools/block_cache_trace_analyzer_test.cc index 45ef99eee..a028bf197 100644 --- a/tools/block_cache_trace_analyzer_test.cc +++ b/tools/block_cache_trace_analyzer_test.cc @@ -117,7 +117,8 @@ class BlockCacheTracerTest : public testing::Test { // Provide these fields for all block types. // The writer should only write these fields for data blocks and the // caller is either GET or MGET. - record.referenced_key = kRefKeyPrefix + std::to_string(key_id); + record.referenced_key = + kRefKeyPrefix + std::to_string(key_id) + std::string(8, 0); record.referenced_key_exist_in_block = Boolean::kTrue; record.num_keys_in_block = kNumKeysInBlock; ASSERT_OK(writer->WriteBlockAccess( @@ -179,7 +180,8 @@ class BlockCacheTracerTest : public testing::Test { "-analyze_get_spatial_locality_labels=" + analyze_get_spatial_locality_labels_, "-analyze_get_spatial_locality_buckets=" + - analyze_get_spatial_locality_buckets_}; + analyze_get_spatial_locality_buckets_, + "-analyze_correlation_coefficients_labels=all"}; char arg_buffer[kArgBufferSize]; char* argv[kMaxArgCount]; int argc = 0; @@ -236,9 +238,9 @@ TEST_F(BlockCacheTracerTest, BlockCacheAnalyzer) { RunBlockCacheTraceAnalyzer(); { // Validate the cache miss ratios. - const std::vector expected_capacities{1024, 1024 * 1024, - 1024 * 1024 * 1024}; - const std::string mrc_path = test_path_ + "/mrc"; + std::vector expected_capacities{1024, 1024 * 1024, + 1024 * 1024 * 1024}; + const std::string mrc_path = test_path_ + "/49_50_mrc"; std::ifstream infile(mrc_path); uint32_t config_index = 0; std::string line; @@ -266,6 +268,68 @@ TEST_F(BlockCacheTracerTest, BlockCacheAnalyzer) { ASSERT_EQ(expected_capacities.size(), config_index); infile.close(); ASSERT_OK(env_->DeleteFile(mrc_path)); + + const std::vector time_units{"1", "60", "3600"}; + expected_capacities.push_back(port::kMaxUint64); + for (auto const& expected_capacity : expected_capacities) { + for (auto const& time_unit : time_units) { + const std::string miss_ratio_timeline_path = + test_path_ + "/" + std::to_string(expected_capacity) + "_" + + time_unit + "_miss_ratio_timeline"; + std::ifstream mrt_file(miss_ratio_timeline_path); + // Read header. + ASSERT_TRUE(getline(mrt_file, line)); + ASSERT_TRUE(getline(mrt_file, line)); + std::stringstream ss(line); + bool read_header = false; + while (ss.good()) { + std::string substr; + getline(ss, substr, ','); + if (!read_header) { + if (expected_capacity == port::kMaxUint64) { + ASSERT_EQ("trace", substr); + } else { + ASSERT_EQ("lru-1-0", substr); + } + read_header = true; + continue; + } + ASSERT_DOUBLE_EQ(100.0, ParseDouble(substr)); + } + ASSERT_FALSE(getline(mrt_file, line)); + mrt_file.close(); + ASSERT_OK(env_->DeleteFile(miss_ratio_timeline_path)); + } + for (auto const& time_unit : time_units) { + const std::string miss_timeline_path = + test_path_ + "/" + std::to_string(expected_capacity) + "_" + + time_unit + "_miss_timeline"; + std::ifstream mt_file(miss_timeline_path); + // Read header. + ASSERT_TRUE(getline(mt_file, line)); + ASSERT_TRUE(getline(mt_file, line)); + std::stringstream ss(line); + uint32_t num_misses = 0; + while (ss.good()) { + std::string substr; + getline(ss, substr, ','); + if (num_misses == 0) { + if (expected_capacity == port::kMaxUint64) { + ASSERT_EQ("trace", substr); + } else { + ASSERT_EQ("lru-1-0", substr); + } + num_misses++; + continue; + } + num_misses += ParseInt(substr); + } + ASSERT_EQ(51, num_misses); + ASSERT_FALSE(getline(mt_file, line)); + mt_file.close(); + ASSERT_OK(env_->DeleteFile(miss_timeline_path)); + } + } } { // Validate the timeline csv files. @@ -543,7 +607,9 @@ TEST_F(BlockCacheTracerTest, MixedBlocks) { // Read blocks. BlockCacheTraceAnalyzer analyzer(trace_file_path_, /*output_miss_ratio_curve_path=*/"", + /*human_readable_trace_file_path=*/"", /*compute_reuse_distance=*/true, + /*mrc_only=*/false, /*simulator=*/nullptr); // The analyzer ends when it detects an incomplete access record. ASSERT_EQ(Status::Incomplete(""), analyzer.Analyze()); diff --git a/trace_replay/block_cache_tracer.cc b/trace_replay/block_cache_tracer.cc index 4f320ef2d..1eeb64ac8 100644 --- a/trace_replay/block_cache_tracer.cc +++ b/trace_replay/block_cache_tracer.cc @@ -6,6 +6,7 @@ #include "trace_replay/block_cache_tracer.h" #include "db/db_impl/db_impl.h" +#include "db/dbformat.h" #include "rocksdb/slice.h" #include "util/coding.h" #include "util/hash.h" @@ -54,6 +55,19 @@ bool BlockCacheTraceHelper::IsUserAccess(TableReaderCaller caller) { caller == TableReaderCaller::kUserVerifyChecksum; } +std::string BlockCacheTraceHelper::ComputeRowKey( + const BlockCacheTraceRecord& access) { + if (!IsGetOrMultiGet(access.caller)) { + return ""; + } + Slice key = ExtractUserKey(access.referenced_key); + uint64_t seq_no = access.get_from_user_specified_snapshot == Boolean::kFalse + ? 0 + : 1 + GetInternalKeySeqno(access.referenced_key); + return std::to_string(access.sst_fd_number) + "_" + key.ToString() + "_" + + std::to_string(seq_no); +} + BlockCacheTraceWriter::BlockCacheTraceWriter( Env* env, const TraceOptions& trace_options, std::unique_ptr&& trace_writer) diff --git a/trace_replay/block_cache_tracer.h b/trace_replay/block_cache_tracer.h index b1a258843..3863ca430 100644 --- a/trace_replay/block_cache_tracer.h +++ b/trace_replay/block_cache_tracer.h @@ -20,6 +20,7 @@ extern const uint64_t kMicrosInSecond; extern const uint64_t kSecondInMinute; extern const uint64_t kSecondInHour; +struct BlockCacheTraceRecord; class BlockCacheTraceHelper { public: @@ -27,7 +28,9 @@ class BlockCacheTraceHelper { TableReaderCaller caller); static bool IsGetOrMultiGet(TableReaderCaller caller); static bool IsUserAccess(TableReaderCaller caller); - + // Row key is a concatenation of the access's fd_number and the referenced + // user key. + static std::string ComputeRowKey(const BlockCacheTraceRecord& access); static const std::string kUnknownColumnFamilyName; static const uint64_t kReservedGetId; }; diff --git a/utilities/simulator_cache/cache_simulator.cc b/utilities/simulator_cache/cache_simulator.cc index 90433df11..06de4c119 100644 --- a/utilities/simulator_cache/cache_simulator.cc +++ b/utilities/simulator_cache/cache_simulator.cc @@ -4,13 +4,14 @@ // (found in the LICENSE.Apache file in the root directory). #include "utilities/simulator_cache/cache_simulator.h" +#include #include "db/dbformat.h" namespace rocksdb { namespace { const std::string kGhostCachePrefix = "ghost_"; -} +} // namespace GhostCache::GhostCache(std::shared_ptr sim_cache) : sim_cache_(sim_cache) {} @@ -22,7 +23,7 @@ bool GhostCache::Admit(const Slice& lookup_key) { return true; } sim_cache_->Insert(lookup_key, /*value=*/nullptr, lookup_key.size(), - /*deleter=*/nullptr, /*handle=*/nullptr); + /*deleter=*/nullptr); return false; } @@ -43,18 +44,27 @@ void CacheSimulator::Access(const BlockCacheTraceRecord& access) { sim_cache_->Release(handle); is_cache_miss = false; } else { - if (access.no_insert == Boolean::kFalse && admit) { + if (access.no_insert == Boolean::kFalse && admit && access.block_size > 0) { sim_cache_->Insert(access.block_key, /*value=*/nullptr, access.block_size, - /*deleter=*/nullptr, /*handle=*/nullptr); + /*deleter=*/nullptr); } } - UpdateMetrics(is_user_access, is_cache_miss); + miss_ratio_stats_.UpdateMetrics(access.access_timestamp, is_user_access, + is_cache_miss); } -void CacheSimulator::UpdateMetrics(bool is_user_access, bool is_cache_miss) { +void MissRatioStats::UpdateMetrics(uint64_t timestamp_in_ms, + bool is_user_access, bool is_cache_miss) { + uint64_t timestamp_in_seconds = timestamp_in_ms / kMicrosInSecond; + num_accesses_timeline_[timestamp_in_seconds] += 1; num_accesses_ += 1; + if (num_misses_timeline_.find(timestamp_in_seconds) == + num_misses_timeline_.end()) { + num_misses_timeline_[timestamp_in_seconds] = 0; + } if (is_cache_miss) { num_misses_ += 1; + num_misses_timeline_[timestamp_in_seconds] += 1; } if (is_user_access) { user_accesses_ += 1; @@ -76,8 +86,8 @@ Cache::Priority PrioritizedCacheSimulator::ComputeBlockPriority( void PrioritizedCacheSimulator::AccessKVPair( const Slice& key, uint64_t value_size, Cache::Priority priority, - bool no_insert, bool is_user_access, bool* is_cache_miss, bool* admitted, - bool update_metrics) { + const BlockCacheTraceRecord& access, bool no_insert, bool is_user_access, + bool* is_cache_miss, bool* admitted, bool update_metrics) { assert(is_cache_miss); assert(admitted); *is_cache_miss = true; @@ -90,11 +100,12 @@ void PrioritizedCacheSimulator::AccessKVPair( sim_cache_->Release(handle); *is_cache_miss = false; } else if (!no_insert && *admitted && value_size > 0) { - sim_cache_->Insert(key, /*value=*/nullptr, value_size, - /*deleter=*/nullptr, /*handle=*/nullptr, priority); + sim_cache_->Insert(key, /*value=*/nullptr, value_size, /*deleter=*/nullptr, + /*handle=*/nullptr, priority); } if (update_metrics) { - UpdateMetrics(is_user_access, *is_cache_miss); + miss_ratio_stats_.UpdateMetrics(access.access_timestamp, is_user_access, + *is_cache_miss); } } @@ -102,38 +113,28 @@ void PrioritizedCacheSimulator::Access(const BlockCacheTraceRecord& access) { bool is_cache_miss = true; bool admitted = true; AccessKVPair(access.block_key, access.block_size, - ComputeBlockPriority(access), access.no_insert, + ComputeBlockPriority(access), access, access.no_insert, BlockCacheTraceHelper::IsUserAccess(access.caller), &is_cache_miss, &admitted, /*update_metrics=*/true); } -std::string HybridRowBlockCacheSimulator::ComputeRowKey( - const BlockCacheTraceRecord& access) { - assert(access.get_id != BlockCacheTraceHelper::kReservedGetId); - Slice key = ExtractUserKey(access.referenced_key); - uint64_t seq_no = access.get_from_user_specified_snapshot == Boolean::kFalse - ? 0 - : 1 + GetInternalKeySeqno(access.referenced_key); - return std::to_string(access.sst_fd_number) + "_" + key.ToString() + "_" + - std::to_string(seq_no); -} - void HybridRowBlockCacheSimulator::Access(const BlockCacheTraceRecord& access) { - bool is_cache_miss = true; - bool admitted = true; // TODO (haoyu): We only support Get for now. We need to extend the tracing // for MultiGet, i.e., non-data block accesses must log all keys in a // MultiGet. + bool is_cache_miss = false; + bool admitted = false; if (access.caller == TableReaderCaller::kUserGet && access.get_id != BlockCacheTraceHelper::kReservedGetId) { // This is a Get/MultiGet request. - const std::string& row_key = ComputeRowKey(access); + const std::string& row_key = BlockCacheTraceHelper::ComputeRowKey(access); if (getid_getkeys_map_[access.get_id].find(row_key) == getid_getkeys_map_[access.get_id].end()) { // This is the first time that this key is accessed. Look up the key-value // pair first. Do not update the miss/accesses metrics here since it will // be updated later. AccessKVPair(row_key, access.referenced_data_size, Cache::Priority::HIGH, + access, /*no_insert=*/false, /*is_user_access=*/true, &is_cache_miss, &admitted, /*update_metrics=*/false); @@ -154,28 +155,31 @@ void HybridRowBlockCacheSimulator::Access(const BlockCacheTraceRecord& access) { // referenced key-value pair already. Thus, we treat these lookups as // hits. This is also to ensure the total number of accesses are the same // when comparing to other policies. - UpdateMetrics(/*is_user_access=*/true, /*is_cache_miss=*/false); + miss_ratio_stats_.UpdateMetrics(access.access_timestamp, + /*is_user_access=*/true, + /*is_cache_miss=*/false); return; } // The key-value pair observes a cache miss. We need to access its // index/filter/data blocks. AccessKVPair( access.block_key, access.block_type, ComputeBlockPriority(access), + access, /*no_insert=*/!insert_blocks_upon_row_kvpair_miss_ || access.no_insert, /*is_user_access=*/true, &is_cache_miss, &admitted, /*update_metrics=*/true); if (access.referenced_data_size > 0 && miss_inserted.second == InsertResult::ADMITTED) { - sim_cache_->Insert( - row_key, /*value=*/nullptr, access.referenced_data_size, - /*deleter=*/nullptr, /*handle=*/nullptr, Cache::Priority::HIGH); + sim_cache_->Insert(row_key, /*value=*/nullptr, + access.referenced_data_size, /*deleter=*/nullptr, + /*handle=*/nullptr, Cache::Priority::HIGH); getid_getkeys_map_[access.get_id][row_key] = std::make_pair(true, InsertResult::INSERTED); } return; } AccessKVPair(access.block_key, access.block_size, - ComputeBlockPriority(access), access.no_insert, + ComputeBlockPriority(access), access, access.no_insert, BlockCacheTraceHelper::IsUserAccess(access.caller), &is_cache_miss, &admitted, /*update_metrics=*/true); } diff --git a/utilities/simulator_cache/cache_simulator.h b/utilities/simulator_cache/cache_simulator.h index 829726886..3863fcf88 100644 --- a/utilities/simulator_cache/cache_simulator.h +++ b/utilities/simulator_cache/cache_simulator.h @@ -5,6 +5,9 @@ #pragma once +#include + +#include "cache/lru_cache.h" #include "trace_replay/block_cache_tracer.h" namespace rocksdb { @@ -29,6 +32,51 @@ struct CacheConfiguration { } }; +class MissRatioStats { + public: + void reset_counter() { + num_misses_ = 0; + num_accesses_ = 0; + user_accesses_ = 0; + user_misses_ = 0; + } + double miss_ratio() const { + if (num_accesses_ == 0) { + return -1; + } + return static_cast(num_misses_ * 100.0 / num_accesses_); + } + uint64_t total_accesses() const { return num_accesses_; } + + const std::map& num_accesses_timeline() const { + return num_accesses_timeline_; + } + + const std::map& num_misses_timeline() const { + return num_misses_timeline_; + } + + double user_miss_ratio() const { + if (user_accesses_ == 0) { + return -1; + } + return static_cast(user_misses_ * 100.0 / user_accesses_); + } + uint64_t user_accesses() const { return user_accesses_; } + + void UpdateMetrics(uint64_t timestamp_in_ms, bool is_user_access, + bool is_cache_miss); + + private: + uint64_t num_accesses_ = 0; + uint64_t num_misses_ = 0; + uint64_t user_accesses_ = 0; + uint64_t user_misses_ = 0; + + std::map num_accesses_timeline_; + std::map num_misses_timeline_; +}; + // A ghost cache admits an entry on its second access. class GhostCache { public: @@ -61,37 +109,15 @@ class CacheSimulator { CacheSimulator& operator=(CacheSimulator&&) = delete; virtual void Access(const BlockCacheTraceRecord& access); - void reset_counter() { - num_misses_ = 0; - num_accesses_ = 0; - user_accesses_ = 0; - user_misses_ = 0; - } - double miss_ratio() const { - if (num_accesses_ == 0) { - return -1; - } - return static_cast(num_misses_ * 100.0 / num_accesses_); - } - uint64_t total_accesses() const { return num_accesses_; } - double user_miss_ratio() const { - if (user_accesses_ == 0) { - return -1; - } - return static_cast(user_misses_ * 100.0 / user_accesses_); - } - uint64_t user_accesses() const { return user_accesses_; } + void reset_counter() { miss_ratio_stats_.reset_counter(); } + + const MissRatioStats& miss_ratio_stats() const { return miss_ratio_stats_; } protected: - void UpdateMetrics(bool is_user_access, bool is_cache_miss); - + MissRatioStats miss_ratio_stats_; std::unique_ptr ghost_cache_; std::shared_ptr sim_cache_; - uint64_t num_accesses_ = 0; - uint64_t num_misses_ = 0; - uint64_t user_accesses_ = 0; - uint64_t user_misses_ = 0; }; // A prioritized cache simulator that runs against a block cache trace. @@ -107,7 +133,8 @@ class PrioritizedCacheSimulator : public CacheSimulator { protected: // Access the key-value pair and returns true upon a cache miss. void AccessKVPair(const Slice& key, uint64_t value_size, - Cache::Priority priority, bool no_insert, + Cache::Priority priority, + const BlockCacheTraceRecord& access, bool no_insert, bool is_user_access, bool* is_cache_miss, bool* admitted, bool update_metrics); @@ -135,10 +162,6 @@ class HybridRowBlockCacheSimulator : public PrioritizedCacheSimulator { void Access(const BlockCacheTraceRecord& access) override; private: - // Row key is a concatenation of the access's fd_number and the referenced - // user key. - std::string ComputeRowKey(const BlockCacheTraceRecord& access); - enum InsertResult : char { INSERTED, ADMITTED, diff --git a/utilities/simulator_cache/cache_simulator_test.cc b/utilities/simulator_cache/cache_simulator_test.cc index f435785e6..dc3b8327e 100644 --- a/utilities/simulator_cache/cache_simulator_test.cc +++ b/utilities/simulator_cache/cache_simulator_test.cc @@ -94,21 +94,21 @@ TEST_F(CacheSimulatorTest, CacheSimulator) { new CacheSimulator(nullptr, sim_cache)); cache_simulator->Access(access); cache_simulator->Access(access); - ASSERT_EQ(2, cache_simulator->total_accesses()); - ASSERT_EQ(50, cache_simulator->miss_ratio()); - ASSERT_EQ(2, cache_simulator->user_accesses()); - ASSERT_EQ(50, cache_simulator->user_miss_ratio()); + ASSERT_EQ(2, cache_simulator->miss_ratio_stats().total_accesses()); + ASSERT_EQ(50, cache_simulator->miss_ratio_stats().miss_ratio()); + ASSERT_EQ(2, cache_simulator->miss_ratio_stats().user_accesses()); + ASSERT_EQ(50, cache_simulator->miss_ratio_stats().user_miss_ratio()); cache_simulator->Access(compaction_access); cache_simulator->Access(compaction_access); - ASSERT_EQ(4, cache_simulator->total_accesses()); - ASSERT_EQ(75, cache_simulator->miss_ratio()); - ASSERT_EQ(2, cache_simulator->user_accesses()); - ASSERT_EQ(50, cache_simulator->user_miss_ratio()); + ASSERT_EQ(4, cache_simulator->miss_ratio_stats().total_accesses()); + ASSERT_EQ(75, cache_simulator->miss_ratio_stats().miss_ratio()); + ASSERT_EQ(2, cache_simulator->miss_ratio_stats().user_accesses()); + ASSERT_EQ(50, cache_simulator->miss_ratio_stats().user_miss_ratio()); cache_simulator->reset_counter(); - ASSERT_EQ(0, cache_simulator->total_accesses()); - ASSERT_EQ(-1, cache_simulator->miss_ratio()); + ASSERT_EQ(0, cache_simulator->miss_ratio_stats().total_accesses()); + ASSERT_EQ(-1, cache_simulator->miss_ratio_stats().miss_ratio()); auto handle = sim_cache->Lookup(access.block_key); ASSERT_NE(nullptr, handle); sim_cache->Release(handle); @@ -129,9 +129,9 @@ TEST_F(CacheSimulatorTest, GhostCacheSimulator) { /*high_pri_pool_ratio=*/0))); cache_simulator->Access(access); cache_simulator->Access(access); - ASSERT_EQ(2, cache_simulator->total_accesses()); + ASSERT_EQ(2, cache_simulator->miss_ratio_stats().total_accesses()); // Both of them will be miss since we have a ghost cache. - ASSERT_EQ(100, cache_simulator->miss_ratio()); + ASSERT_EQ(100, cache_simulator->miss_ratio_stats().miss_ratio()); } TEST_F(CacheSimulatorTest, PrioritizedCacheSimulator) { @@ -144,8 +144,8 @@ TEST_F(CacheSimulatorTest, PrioritizedCacheSimulator) { new PrioritizedCacheSimulator(nullptr, sim_cache)); cache_simulator->Access(access); cache_simulator->Access(access); - ASSERT_EQ(2, cache_simulator->total_accesses()); - ASSERT_EQ(50, cache_simulator->miss_ratio()); + ASSERT_EQ(2, cache_simulator->miss_ratio_stats().total_accesses()); + ASSERT_EQ(50, cache_simulator->miss_ratio_stats().miss_ratio()); auto handle = sim_cache->Lookup(access.block_key); ASSERT_NE(nullptr, handle); @@ -166,9 +166,9 @@ TEST_F(CacheSimulatorTest, GhostPrioritizedCacheSimulator) { /*high_pri_pool_ratio=*/0))); cache_simulator->Access(access); cache_simulator->Access(access); - ASSERT_EQ(2, cache_simulator->total_accesses()); + ASSERT_EQ(2, cache_simulator->miss_ratio_stats().total_accesses()); // Both of them will be miss since we have a ghost cache. - ASSERT_EQ(100, cache_simulator->miss_ratio()); + ASSERT_EQ(100, cache_simulator->miss_ratio_stats().miss_ratio()); } TEST_F(CacheSimulatorTest, HybridRowBlockCacheSimulator) { @@ -200,10 +200,11 @@ TEST_F(CacheSimulatorTest, HybridRowBlockCacheSimulator) { cache_simulator->Access(first_get); block_id++; } - ASSERT_EQ(10, cache_simulator->total_accesses()); - ASSERT_EQ(100, cache_simulator->miss_ratio()); - ASSERT_EQ(10, cache_simulator->user_accesses()); - ASSERT_EQ(100, cache_simulator->user_miss_ratio()); + + ASSERT_EQ(10, cache_simulator->miss_ratio_stats().total_accesses()); + ASSERT_EQ(100, cache_simulator->miss_ratio_stats().miss_ratio()); + ASSERT_EQ(10, cache_simulator->miss_ratio_stats().user_accesses()); + ASSERT_EQ(100, cache_simulator->miss_ratio_stats().user_miss_ratio()); auto handle = sim_cache->Lookup( std::to_string(first_get.sst_fd_number) + "_" + ExtractUserKey(first_get.referenced_key).ToString() + "_" + @@ -225,10 +226,12 @@ TEST_F(CacheSimulatorTest, HybridRowBlockCacheSimulator) { cache_simulator->Access(second_get); block_id++; } - ASSERT_EQ(15, cache_simulator->total_accesses()); - ASSERT_EQ(66, static_cast(cache_simulator->miss_ratio())); - ASSERT_EQ(15, cache_simulator->user_accesses()); - ASSERT_EQ(66, static_cast(cache_simulator->user_miss_ratio())); + ASSERT_EQ(15, cache_simulator->miss_ratio_stats().total_accesses()); + ASSERT_EQ(66, static_cast( + cache_simulator->miss_ratio_stats().miss_ratio())); + ASSERT_EQ(15, cache_simulator->miss_ratio_stats().user_accesses()); + ASSERT_EQ(66, static_cast( + cache_simulator->miss_ratio_stats().user_miss_ratio())); handle = sim_cache->Lookup( std::to_string(second_get.sst_fd_number) + "_" + ExtractUserKey(second_get.referenced_key).ToString() + "_" + @@ -252,10 +255,12 @@ TEST_F(CacheSimulatorTest, HybridRowBlockCacheSimulator) { cache_simulator->Access(third_get); block_id++; } - ASSERT_EQ(20, cache_simulator->total_accesses()); - ASSERT_EQ(75, static_cast(cache_simulator->miss_ratio())); - ASSERT_EQ(20, cache_simulator->user_accesses()); - ASSERT_EQ(75, static_cast(cache_simulator->user_miss_ratio())); + ASSERT_EQ(20, cache_simulator->miss_ratio_stats().total_accesses()); + ASSERT_EQ(75, static_cast( + cache_simulator->miss_ratio_stats().miss_ratio())); + ASSERT_EQ(20, cache_simulator->miss_ratio_stats().user_accesses()); + ASSERT_EQ(75, static_cast( + cache_simulator->miss_ratio_stats().user_miss_ratio())); // Assert that the third key is not inserted into the cache. handle = sim_cache->Lookup(std::to_string(third_get.sst_fd_number) + "_" + third_get.referenced_key); @@ -318,19 +323,21 @@ TEST_F(CacheSimulatorTest, GhostHybridRowBlockCacheSimulator) { // Two get requests access the same key. cache_simulator->Access(first_get); cache_simulator->Access(second_get); - ASSERT_EQ(2, cache_simulator->total_accesses()); - ASSERT_EQ(100, cache_simulator->miss_ratio()); - ASSERT_EQ(2, cache_simulator->user_accesses()); - ASSERT_EQ(100, cache_simulator->user_miss_ratio()); + ASSERT_EQ(2, cache_simulator->miss_ratio_stats().total_accesses()); + ASSERT_EQ(100, cache_simulator->miss_ratio_stats().miss_ratio()); + ASSERT_EQ(2, cache_simulator->miss_ratio_stats().user_accesses()); + ASSERT_EQ(100, cache_simulator->miss_ratio_stats().user_miss_ratio()); // We insert the key-value pair upon the second get request. A third get // request should observe a hit. for (uint32_t i = 0; i < 10; i++) { cache_simulator->Access(third_get); } - ASSERT_EQ(12, cache_simulator->total_accesses()); - ASSERT_EQ(16, static_cast(cache_simulator->miss_ratio())); - ASSERT_EQ(12, cache_simulator->user_accesses()); - ASSERT_EQ(16, static_cast(cache_simulator->user_miss_ratio())); + ASSERT_EQ(12, cache_simulator->miss_ratio_stats().total_accesses()); + ASSERT_EQ(16, static_cast( + cache_simulator->miss_ratio_stats().miss_ratio())); + ASSERT_EQ(12, cache_simulator->miss_ratio_stats().user_accesses()); + ASSERT_EQ(16, static_cast( + cache_simulator->miss_ratio_stats().user_miss_ratio())); } } // namespace rocksdb