block_cache_trace_analyzer.h 16 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393
  1. // Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
  2. // This source code is licensed under both the GPLv2 (found in the
  3. // COPYING file in the root directory) and Apache 2.0 License
  4. // (found in the LICENSE.Apache file in the root directory).
  5. #pragma once
  6. #include <map>
  7. #include <set>
  8. #include <vector>
  9. #include "db/dbformat.h"
  10. #include "rocksdb/env.h"
  11. #include "rocksdb/utilities/sim_cache.h"
  12. #include "trace_replay/block_cache_tracer.h"
  13. #include "utilities/simulator_cache/cache_simulator.h"
  14. namespace ROCKSDB_NAMESPACE {
  15. // Statistics of a key refereneced by a Get.
  16. struct GetKeyInfo {
  17. uint64_t key_id = 0;
  18. std::vector<uint64_t> access_sequence_number_timeline;
  19. std::vector<uint64_t> access_timeline;
  20. void AddAccess(const BlockCacheTraceRecord& access,
  21. uint64_t access_sequnce_number) {
  22. access_sequence_number_timeline.push_back(access_sequnce_number);
  23. access_timeline.push_back(access.access_timestamp);
  24. }
  25. };
  26. // Statistics of a block.
  27. struct BlockAccessInfo {
  28. uint64_t block_id = 0;
  29. uint64_t table_id = 0;
  30. uint64_t block_offset = 0;
  31. uint64_t num_accesses = 0;
  32. uint64_t block_size = 0;
  33. uint64_t first_access_time = 0;
  34. uint64_t last_access_time = 0;
  35. uint64_t num_keys = 0;
  36. std::map<std::string, std::map<TableReaderCaller, uint64_t>>
  37. key_num_access_map; // for keys exist in this block.
  38. std::map<std::string, std::map<TableReaderCaller, uint64_t>>
  39. non_exist_key_num_access_map; // for keys do not exist in this block.
  40. uint64_t num_referenced_key_exist_in_block = 0;
  41. uint64_t referenced_data_size = 0;
  42. std::map<TableReaderCaller, uint64_t> caller_num_access_map;
  43. // caller:timestamp:number_of_accesses. The granularity of the timestamp is
  44. // seconds.
  45. std::map<TableReaderCaller, std::map<uint64_t, uint64_t>>
  46. caller_num_accesses_timeline;
  47. // Unique blocks since the last access.
  48. std::set<std::string> unique_blocks_since_last_access;
  49. // Number of reuses grouped by reuse distance.
  50. std::map<uint64_t, uint64_t> reuse_distance_count;
  51. // The access sequence numbers of this block.
  52. std::vector<uint64_t> access_sequence_number_timeline;
  53. std::map<TableReaderCaller, std::vector<uint64_t>>
  54. caller_access_sequence__number_timeline;
  55. // The access timestamp in microseconds of this block.
  56. std::vector<uint64_t> access_timeline;
  57. std::map<TableReaderCaller, std::vector<uint64_t>> caller_access_timeline;
  58. void AddAccess(const BlockCacheTraceRecord& access,
  59. uint64_t access_sequnce_number) {
  60. if (block_size != 0 && access.block_size != 0) {
  61. assert(block_size == access.block_size);
  62. }
  63. if (num_keys != 0 && access.num_keys_in_block != 0) {
  64. assert(num_keys == access.num_keys_in_block);
  65. }
  66. if (first_access_time == 0) {
  67. first_access_time = access.access_timestamp;
  68. }
  69. table_id = BlockCacheTraceHelper::GetTableId(access);
  70. block_offset = BlockCacheTraceHelper::GetBlockOffsetInFile(access);
  71. last_access_time = access.access_timestamp;
  72. block_size = access.block_size;
  73. caller_num_access_map[access.caller]++;
  74. num_accesses++;
  75. // access.access_timestamp is in microsecond.
  76. const uint64_t timestamp_in_seconds =
  77. access.access_timestamp / kMicrosInSecond;
  78. caller_num_accesses_timeline[access.caller][timestamp_in_seconds] += 1;
  79. // Populate the feature vectors.
  80. access_sequence_number_timeline.push_back(access_sequnce_number);
  81. caller_access_sequence__number_timeline[access.caller].push_back(
  82. access_sequnce_number);
  83. access_timeline.push_back(access.access_timestamp);
  84. caller_access_timeline[access.caller].push_back(access.access_timestamp);
  85. if (BlockCacheTraceHelper::IsGetOrMultiGetOnDataBlock(access.block_type,
  86. access.caller)) {
  87. num_keys = access.num_keys_in_block;
  88. if (access.referenced_key_exist_in_block == Boolean::kTrue) {
  89. if (key_num_access_map.find(access.referenced_key) ==
  90. key_num_access_map.end()) {
  91. referenced_data_size += access.referenced_data_size;
  92. }
  93. key_num_access_map[access.referenced_key][access.caller]++;
  94. num_referenced_key_exist_in_block++;
  95. if (referenced_data_size > block_size && block_size != 0) {
  96. ParsedInternalKey internal_key;
  97. ParseInternalKey(access.referenced_key, &internal_key);
  98. }
  99. } else {
  100. non_exist_key_num_access_map[access.referenced_key][access.caller]++;
  101. }
  102. }
  103. }
  104. };
  105. // Aggregates stats of a block given a block type.
  106. struct BlockTypeAccessInfoAggregate {
  107. std::map<std::string, BlockAccessInfo> block_access_info_map;
  108. };
  109. // Aggregates BlockTypeAggregate given a SST file.
  110. struct SSTFileAccessInfoAggregate {
  111. uint32_t level;
  112. std::map<TraceType, BlockTypeAccessInfoAggregate> block_type_aggregates_map;
  113. };
  114. // Aggregates SSTFileAggregate given a column family.
  115. struct ColumnFamilyAccessInfoAggregate {
  116. std::map<uint64_t, SSTFileAccessInfoAggregate> fd_aggregates_map;
  117. };
  118. struct Features {
  119. std::vector<uint64_t> elapsed_time_since_last_access;
  120. std::vector<uint64_t> num_accesses_since_last_access;
  121. std::vector<uint64_t> num_past_accesses;
  122. };
  123. struct Predictions {
  124. std::vector<uint64_t> elapsed_time_till_next_access;
  125. std::vector<uint64_t> num_accesses_till_next_access;
  126. };
  127. class BlockCacheTraceAnalyzer {
  128. public:
  129. BlockCacheTraceAnalyzer(
  130. const std::string& trace_file_path, const std::string& output_dir,
  131. const std::string& human_readable_trace_file_path,
  132. bool compute_reuse_distance, bool mrc_only,
  133. bool is_human_readable_trace_file,
  134. std::unique_ptr<BlockCacheTraceSimulator>&& cache_simulator);
  135. ~BlockCacheTraceAnalyzer() = default;
  136. // No copy and move.
  137. BlockCacheTraceAnalyzer(const BlockCacheTraceAnalyzer&) = delete;
  138. BlockCacheTraceAnalyzer& operator=(const BlockCacheTraceAnalyzer&) = delete;
  139. BlockCacheTraceAnalyzer(BlockCacheTraceAnalyzer&&) = delete;
  140. BlockCacheTraceAnalyzer& operator=(BlockCacheTraceAnalyzer&&) = delete;
  141. // Read all access records in the given trace_file, maintains the stats of
  142. // a block, and aggregates the information by block type, sst file, and column
  143. // family. Subsequently, the caller may call Print* functions to print
  144. // statistics.
  145. Status Analyze();
  146. // Print a summary of statistics of the trace, e.g.,
  147. // Number of files: 2 Number of blocks: 50 Number of accesses: 50
  148. // Number of Index blocks: 10
  149. // Number of Filter blocks: 10
  150. // Number of Data blocks: 10
  151. // Number of UncompressionDict blocks: 10
  152. // Number of RangeDeletion blocks: 10
  153. // ***************************************************************
  154. // Caller Get: Number of accesses 10
  155. // Caller Get: Number of accesses per level break down
  156. // Level 0: Number of accesses: 10
  157. // Caller Get: Number of accesses per block type break down
  158. // Block Type Index: Number of accesses: 2
  159. // Block Type Filter: Number of accesses: 2
  160. // Block Type Data: Number of accesses: 2
  161. // Block Type UncompressionDict: Number of accesses: 2
  162. // Block Type RangeDeletion: Number of accesses: 2
  163. void PrintStatsSummary() const;
  164. // Print block size distribution and the distribution break down by block type
  165. // and column family.
  166. void PrintBlockSizeStats() const;
  167. // Print access count distribution and the distribution break down by block
  168. // type and column family.
  169. void PrintAccessCountStats(bool user_access_only, uint32_t bottom_k,
  170. uint32_t top_k) const;
  171. // Print data block accesses by user Get and Multi-Get.
  172. // It prints out 1) A histogram on the percentage of keys accessed in a data
  173. // block break down by if a referenced key exists in the data block andthe
  174. // histogram break down by column family. 2) A histogram on the percentage of
  175. // accesses on keys exist in a data block and its break down by column family.
  176. void PrintDataBlockAccessStats() const;
  177. // Write the percentage of accesses break down by column family into a csv
  178. // file saved in 'output_dir'.
  179. //
  180. // The file is named "percentage_of_accesses_summary". The file format is
  181. // caller,cf_0,cf_1,...,cf_n where the cf_i is the column family name found in
  182. // the trace.
  183. void WritePercentAccessSummaryStats() const;
  184. // Write the percentage of accesses for the given caller break down by column
  185. // family, level, and block type into a csv file saved in 'output_dir'.
  186. //
  187. // It generates two files: 1) caller_level_percentage_of_accesses_summary and
  188. // 2) caller_bt_percentage_of_accesses_summary which break down by the level
  189. // and block type, respectively. The file format is
  190. // level/bt,cf_0,cf_1,...,cf_n where cf_i is the column family name found in
  191. // the trace.
  192. void WriteDetailedPercentAccessSummaryStats(TableReaderCaller caller) const;
  193. // Write the access count summary into a csv file saved in 'output_dir'.
  194. // It groups blocks by their access count.
  195. //
  196. // It generates two files: 1) cf_access_count_summary and 2)
  197. // bt_access_count_summary which break down the access count by column family
  198. // and block type, respectively. The file format is
  199. // cf/bt,bucket_0,bucket_1,...,bucket_N.
  200. void WriteAccessCountSummaryStats(
  201. const std::vector<uint64_t>& access_count_buckets,
  202. bool user_access_only) const;
  203. // Write miss ratio curves of simulated cache configurations into a csv file
  204. // named "mrc" saved in 'output_dir'.
  205. //
  206. // The file format is
  207. // "cache_name,num_shard_bits,capacity,miss_ratio,total_accesses".
  208. void WriteMissRatioCurves() const;
  209. // Write miss ratio timeline of simulated cache configurations into several
  210. // csv files, one per cache capacity saved in 'output_dir'.
  211. //
  212. // The file format is
  213. // "time,label_1_access_per_second,label_2_access_per_second,...,label_N_access_per_second"
  214. // where N is the number of unique cache names
  215. // (cache_name+num_shard_bits+ghost_capacity).
  216. void WriteMissRatioTimeline(uint64_t time_unit) const;
  217. // Write misses timeline of simulated cache configurations into several
  218. // csv files, one per cache capacity saved in 'output_dir'.
  219. //
  220. // The file format is
  221. // "time,label_1_access_per_second,label_2_access_per_second,...,label_N_access_per_second"
  222. // where N is the number of unique cache names
  223. // (cache_name+num_shard_bits+ghost_capacity).
  224. void WriteMissTimeline(uint64_t time_unit) const;
  225. // Write the access timeline into a csv file saved in 'output_dir'.
  226. //
  227. // The file is named "label_access_timeline".The file format is
  228. // "time,label_1_access_per_second,label_2_access_per_second,...,label_N_access_per_second"
  229. // where N is the number of unique labels found in the trace.
  230. void WriteAccessTimeline(const std::string& label, uint64_t time_unit,
  231. bool user_access_only) const;
  232. // Write the reuse distance into a csv file saved in 'output_dir'. Reuse
  233. // distance is defined as the cumulated size of unique blocks read between two
  234. // consective accesses on the same block.
  235. //
  236. // The file is named "label_reuse_distance". The file format is
  237. // bucket,label_1,label_2,...,label_N.
  238. void WriteReuseDistance(const std::string& label_str,
  239. const std::vector<uint64_t>& distance_buckets) const;
  240. // Write the reuse interval into a csv file saved in 'output_dir'. Reuse
  241. // interval is defined as the time between two consecutive accesses on the
  242. // same block.
  243. //
  244. // The file is named "label_reuse_interval". The file format is
  245. // bucket,label_1,label_2,...,label_N.
  246. void WriteReuseInterval(const std::string& label_str,
  247. const std::vector<uint64_t>& time_buckets) const;
  248. // Write the reuse lifetime into a csv file saved in 'output_dir'. Reuse
  249. // lifetime is defined as the time interval between the first access of a
  250. // block and its last access.
  251. //
  252. // The file is named "label_reuse_lifetime". The file format is
  253. // bucket,label_1,label_2,...,label_N.
  254. void WriteReuseLifetime(const std::string& label_str,
  255. const std::vector<uint64_t>& time_buckets) const;
  256. // Write the reuse timeline into a csv file saved in 'output_dir'.
  257. //
  258. // The file is named
  259. // "block_type_user_access_only_reuse_window_reuse_timeline". The file format
  260. // is start_time,0,1,...,N where N equals trace_duration / reuse_window.
  261. void WriteBlockReuseTimeline(const uint64_t reuse_window, bool user_access_only,
  262. TraceType block_type) const;
  263. // Write the Get spatical locality into csv files saved in 'output_dir'.
  264. //
  265. // It generates three csv files. label_percent_ref_keys,
  266. // label_percent_accesses_on_ref_keys, and
  267. // label_percent_data_size_on_ref_keys.
  268. void WriteGetSpatialLocality(
  269. const std::string& label_str,
  270. const std::vector<uint64_t>& percent_buckets) const;
  271. void WriteCorrelationFeatures(const std::string& label_str,
  272. uint32_t max_number_of_values) const;
  273. void WriteCorrelationFeaturesForGet(uint32_t max_number_of_values) const;
  274. void WriteSkewness(const std::string& label_str,
  275. const std::vector<uint64_t>& percent_buckets,
  276. TraceType target_block_type) const;
  277. const std::map<std::string, ColumnFamilyAccessInfoAggregate>&
  278. TEST_cf_aggregates_map() const {
  279. return cf_aggregates_map_;
  280. }
  281. private:
  282. std::set<std::string> ParseLabelStr(const std::string& label_str) const;
  283. std::string BuildLabel(const std::set<std::string>& labels,
  284. const std::string& cf_name, uint64_t fd,
  285. uint32_t level, TraceType type,
  286. TableReaderCaller caller, uint64_t block_key,
  287. const BlockAccessInfo& block) const;
  288. void ComputeReuseDistance(BlockAccessInfo* info) const;
  289. Status RecordAccess(const BlockCacheTraceRecord& access);
  290. void UpdateReuseIntervalStats(
  291. const std::string& label, const std::vector<uint64_t>& time_buckets,
  292. const std::map<uint64_t, uint64_t> timeline,
  293. std::map<std::string, std::map<uint64_t, uint64_t>>*
  294. label_time_num_reuses,
  295. uint64_t* total_num_reuses) const;
  296. std::string OutputPercentAccessStats(
  297. uint64_t total_accesses,
  298. const std::map<std::string, uint64_t>& cf_access_count) const;
  299. void WriteStatsToFile(
  300. const std::string& label_str, const std::vector<uint64_t>& time_buckets,
  301. const std::string& filename_suffix,
  302. const std::map<std::string, std::map<uint64_t, uint64_t>>& label_data,
  303. uint64_t ntotal) const;
  304. void TraverseBlocks(
  305. std::function<void(const std::string& /*cf_name*/, uint64_t /*fd*/,
  306. uint32_t /*level*/, TraceType /*block_type*/,
  307. const std::string& /*block_key*/,
  308. uint64_t /*block_key_id*/,
  309. const BlockAccessInfo& /*block_access_info*/)>
  310. block_callback,
  311. std::set<std::string>* labels = nullptr) const;
  312. void UpdateFeatureVectors(
  313. const std::vector<uint64_t>& access_sequence_number_timeline,
  314. const std::vector<uint64_t>& access_timeline, const std::string& label,
  315. std::map<std::string, Features>* label_features,
  316. std::map<std::string, Predictions>* label_predictions) const;
  317. void WriteCorrelationFeaturesToFile(
  318. const std::string& label,
  319. const std::map<std::string, Features>& label_features,
  320. const std::map<std::string, Predictions>& label_predictions,
  321. uint32_t max_number_of_values) const;
  322. ROCKSDB_NAMESPACE::Env* env_;
  323. const std::string trace_file_path_;
  324. const std::string output_dir_;
  325. std::string human_readable_trace_file_path_;
  326. const bool compute_reuse_distance_;
  327. const bool mrc_only_;
  328. const bool is_human_readable_trace_file_;
  329. BlockCacheTraceHeader header_;
  330. std::unique_ptr<BlockCacheTraceSimulator> cache_simulator_;
  331. std::map<std::string, ColumnFamilyAccessInfoAggregate> cf_aggregates_map_;
  332. std::map<std::string, BlockAccessInfo*> block_info_map_;
  333. std::unordered_map<std::string, GetKeyInfo> get_key_info_map_;
  334. uint64_t access_sequence_number_ = 0;
  335. uint64_t trace_start_timestamp_in_seconds_ = 0;
  336. uint64_t trace_end_timestamp_in_seconds_ = 0;
  337. MissRatioStats miss_ratio_stats_;
  338. uint64_t unique_block_id_ = 1;
  339. uint64_t unique_get_key_id_ = 1;
  340. BlockCacheHumanReadableTraceWriter human_readable_trace_writer_;
  341. };
  342. int block_cache_trace_analyzer_tool(int argc, char** argv);
  343. } // namespace ROCKSDB_NAMESPACE