table_properties.cc 24 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558
  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. #include "rocksdb/table_properties.h"
  6. #include "db/seqno_to_time_mapping.h"
  7. #include "port/malloc.h"
  8. #include "port/port.h"
  9. #include "rocksdb/env.h"
  10. #include "rocksdb/unique_id.h"
  11. #include "rocksdb/utilities/options_type.h"
  12. #include "table/table_properties_internal.h"
  13. #include "table/unique_id_impl.h"
  14. #include "util/random.h"
  15. #include "util/string_util.h"
  16. namespace ROCKSDB_NAMESPACE {
  17. const uint32_t TablePropertiesCollectorFactory::Context::kUnknownColumnFamily =
  18. std::numeric_limits<int32_t>::max();
  19. namespace {
  20. void AppendProperty(std::string& props, const std::string& key,
  21. const std::string& value, const std::string& prop_delim,
  22. const std::string& kv_delim) {
  23. props.append(key);
  24. props.append(kv_delim);
  25. props.append(value);
  26. props.append(prop_delim);
  27. }
  28. template <class TValue>
  29. void AppendProperty(std::string& props, const std::string& key,
  30. const TValue& value, const std::string& prop_delim,
  31. const std::string& kv_delim) {
  32. AppendProperty(props, key, std::to_string(value), prop_delim, kv_delim);
  33. }
  34. } // namespace
  35. std::string TableProperties::ToString(const std::string& prop_delim,
  36. const std::string& kv_delim) const {
  37. std::string result;
  38. result.reserve(1024);
  39. // Basic Info
  40. AppendProperty(result, "# data blocks", num_data_blocks, prop_delim,
  41. kv_delim);
  42. AppendProperty(result, "# entries", num_entries, prop_delim, kv_delim);
  43. AppendProperty(result, "# deletions", num_deletions, prop_delim, kv_delim);
  44. AppendProperty(result, "# merge operands", num_merge_operands, prop_delim,
  45. kv_delim);
  46. AppendProperty(result, "# range deletions", num_range_deletions, prop_delim,
  47. kv_delim);
  48. AppendProperty(result, "raw key size", raw_key_size, prop_delim, kv_delim);
  49. AppendProperty(result, "raw average key size",
  50. num_entries != 0 ? 1.0 * raw_key_size / num_entries : 0.0,
  51. prop_delim, kv_delim);
  52. AppendProperty(result, "raw value size", raw_value_size, prop_delim,
  53. kv_delim);
  54. AppendProperty(result, "raw average value size",
  55. num_entries != 0 ? 1.0 * raw_value_size / num_entries : 0.0,
  56. prop_delim, kv_delim);
  57. AppendProperty(result, "data block size", data_size, prop_delim, kv_delim);
  58. AppendProperty(result, "data uncompressed size", uncompressed_data_size,
  59. prop_delim, kv_delim);
  60. char index_block_size_str[80];
  61. snprintf(index_block_size_str, sizeof(index_block_size_str),
  62. "index block size (user-key? %d, delta-value? %d)",
  63. static_cast<int>(index_key_is_user_key),
  64. static_cast<int>(index_value_is_delta_encoded));
  65. AppendProperty(result, index_block_size_str, index_size, prop_delim,
  66. kv_delim);
  67. if (index_partitions != 0) {
  68. AppendProperty(result, "# index partitions", index_partitions, prop_delim,
  69. kv_delim);
  70. AppendProperty(result, "top-level index size", top_level_index_size,
  71. prop_delim, kv_delim);
  72. }
  73. AppendProperty(result, "filter block size", filter_size, prop_delim,
  74. kv_delim);
  75. AppendProperty(result, "# entries for filter", num_filter_entries, prop_delim,
  76. kv_delim);
  77. AppendProperty(result, "(estimated) table size",
  78. data_size + index_size + filter_size, prop_delim, kv_delim);
  79. AppendProperty(
  80. result, "filter policy name",
  81. filter_policy_name.empty() ? std::string("N/A") : filter_policy_name,
  82. prop_delim, kv_delim);
  83. AppendProperty(result, "prefix extractor name",
  84. prefix_extractor_name.empty() ? std::string("N/A")
  85. : prefix_extractor_name,
  86. prop_delim, kv_delim);
  87. AppendProperty(result, "column family ID",
  88. column_family_id ==
  89. ROCKSDB_NAMESPACE::TablePropertiesCollectorFactory::
  90. Context::kUnknownColumnFamily
  91. ? std::string("N/A")
  92. : std::to_string(column_family_id),
  93. prop_delim, kv_delim);
  94. AppendProperty(
  95. result, "column family name",
  96. column_family_name.empty() ? std::string("N/A") : column_family_name,
  97. prop_delim, kv_delim);
  98. AppendProperty(result, "comparator name",
  99. comparator_name.empty() ? std::string("N/A") : comparator_name,
  100. prop_delim, kv_delim);
  101. AppendProperty(result, "user defined timestamps persisted",
  102. user_defined_timestamps_persisted ? std::string("true")
  103. : std::string("false"),
  104. prop_delim, kv_delim);
  105. AppendProperty(result, "largest sequence number in file", key_largest_seqno,
  106. prop_delim, kv_delim);
  107. AppendProperty(result, "smallest sequence number in file", key_smallest_seqno,
  108. prop_delim, kv_delim);
  109. AppendProperty(
  110. result, "merge operator name",
  111. merge_operator_name.empty() ? std::string("N/A") : merge_operator_name,
  112. prop_delim, kv_delim);
  113. AppendProperty(result, "property collectors names",
  114. property_collectors_names.empty() ? std::string("N/A")
  115. : property_collectors_names,
  116. prop_delim, kv_delim);
  117. AppendProperty(
  118. result, "SST file compression algo",
  119. compression_name.empty() ? std::string("N/A") : compression_name,
  120. prop_delim, kv_delim);
  121. AppendProperty(
  122. result, "SST file compression options",
  123. compression_options.empty() ? std::string("N/A") : compression_options,
  124. prop_delim, kv_delim);
  125. AppendProperty(result, "creation time", creation_time, prop_delim, kv_delim);
  126. AppendProperty(result, "time stamp of earliest key", oldest_key_time,
  127. prop_delim, kv_delim);
  128. AppendProperty(result, "time stamp of newest key", newest_key_time,
  129. prop_delim, kv_delim);
  130. AppendProperty(result, "file creation time", file_creation_time, prop_delim,
  131. kv_delim);
  132. AppendProperty(result, "slow compression estimated data size",
  133. slow_compression_estimated_data_size, prop_delim, kv_delim);
  134. AppendProperty(result, "fast compression estimated data size",
  135. fast_compression_estimated_data_size, prop_delim, kv_delim);
  136. // DB identity and DB session ID
  137. AppendProperty(result, "DB identity", db_id, prop_delim, kv_delim);
  138. AppendProperty(result, "DB session identity", db_session_id, prop_delim,
  139. kv_delim);
  140. AppendProperty(result, "DB host id", db_host_id, prop_delim, kv_delim);
  141. AppendProperty(result, "original file number", orig_file_number, prop_delim,
  142. kv_delim);
  143. // Unique ID, when available
  144. std::string id;
  145. Status s = GetUniqueIdFromTableProperties(*this, &id);
  146. AppendProperty(result, "unique ID",
  147. s.ok() ? UniqueIdToHumanString(id) : "N/A", prop_delim,
  148. kv_delim);
  149. SeqnoToTimeMapping seq_time_mapping;
  150. s = seq_time_mapping.DecodeFrom(seqno_to_time_mapping);
  151. AppendProperty(result, "Sequence number to time mapping",
  152. s.ok() ? seq_time_mapping.ToHumanString() : "N/A", prop_delim,
  153. kv_delim);
  154. return result;
  155. }
  156. void TableProperties::Add(const TableProperties& tp) {
  157. data_size += tp.data_size;
  158. uncompressed_data_size += tp.uncompressed_data_size;
  159. index_size += tp.index_size;
  160. index_partitions += tp.index_partitions;
  161. top_level_index_size += tp.top_level_index_size;
  162. index_key_is_user_key += tp.index_key_is_user_key;
  163. index_value_is_delta_encoded += tp.index_value_is_delta_encoded;
  164. filter_size += tp.filter_size;
  165. raw_key_size += tp.raw_key_size;
  166. raw_value_size += tp.raw_value_size;
  167. num_data_blocks += tp.num_data_blocks;
  168. num_entries += tp.num_entries;
  169. num_filter_entries += tp.num_filter_entries;
  170. num_deletions += tp.num_deletions;
  171. num_merge_operands += tp.num_merge_operands;
  172. num_range_deletions += tp.num_range_deletions;
  173. slow_compression_estimated_data_size +=
  174. tp.slow_compression_estimated_data_size;
  175. fast_compression_estimated_data_size +=
  176. tp.fast_compression_estimated_data_size;
  177. }
  178. std::map<std::string, uint64_t>
  179. TableProperties::GetAggregatablePropertiesAsMap() const {
  180. std::map<std::string, uint64_t> rv;
  181. rv["data_size"] = data_size;
  182. rv["uncompressed_data_size"] = uncompressed_data_size;
  183. rv["index_size"] = index_size;
  184. rv["index_partitions"] = index_partitions;
  185. rv["top_level_index_size"] = top_level_index_size;
  186. rv["filter_size"] = filter_size;
  187. rv["raw_key_size"] = raw_key_size;
  188. rv["raw_value_size"] = raw_value_size;
  189. rv["num_data_blocks"] = num_data_blocks;
  190. rv["num_entries"] = num_entries;
  191. rv["num_filter_entries"] = num_filter_entries;
  192. rv["num_deletions"] = num_deletions;
  193. rv["num_merge_operands"] = num_merge_operands;
  194. rv["num_range_deletions"] = num_range_deletions;
  195. rv["slow_compression_estimated_data_size"] =
  196. slow_compression_estimated_data_size;
  197. rv["fast_compression_estimated_data_size"] =
  198. fast_compression_estimated_data_size;
  199. return rv;
  200. }
  201. // WARNING: manual update to this function is needed
  202. // whenever a new string property is added to TableProperties
  203. // to reduce approximation error.
  204. //
  205. // TODO: eliminate the need of manually updating this function
  206. // for new string properties
  207. std::size_t TableProperties::ApproximateMemoryUsage() const {
  208. std::size_t usage = 0;
  209. #ifdef ROCKSDB_MALLOC_USABLE_SIZE
  210. usage += malloc_usable_size((void*)this);
  211. #else
  212. usage += sizeof(*this);
  213. #endif // ROCKSDB_MALLOC_USABLE_SIZE
  214. std::size_t string_props_mem_usage =
  215. db_id.size() + db_session_id.size() + db_host_id.size() +
  216. column_family_name.size() + filter_policy_name.size() +
  217. comparator_name.size() + merge_operator_name.size() +
  218. prefix_extractor_name.size() + property_collectors_names.size() +
  219. compression_name.size() + compression_options.size();
  220. usage += string_props_mem_usage;
  221. for (auto iter = user_collected_properties.begin();
  222. iter != user_collected_properties.end(); ++iter) {
  223. usage += (iter->first.size() + iter->second.size());
  224. }
  225. return usage;
  226. }
  227. const std::string TablePropertiesNames::kDbId = "rocksdb.creating.db.identity";
  228. const std::string TablePropertiesNames::kDbSessionId =
  229. "rocksdb.creating.session.identity";
  230. const std::string TablePropertiesNames::kDbHostId =
  231. "rocksdb.creating.host.identity";
  232. const std::string TablePropertiesNames::kOriginalFileNumber =
  233. "rocksdb.original.file.number";
  234. const std::string TablePropertiesNames::kDataSize = "rocksdb.data.size";
  235. const std::string TablePropertiesNames::kIndexSize = "rocksdb.index.size";
  236. const std::string TablePropertiesNames::kIndexPartitions =
  237. "rocksdb.index.partitions";
  238. const std::string TablePropertiesNames::kTopLevelIndexSize =
  239. "rocksdb.top-level.index.size";
  240. const std::string TablePropertiesNames::kIndexKeyIsUserKey =
  241. "rocksdb.index.key.is.user.key";
  242. const std::string TablePropertiesNames::kIndexValueIsDeltaEncoded =
  243. "rocksdb.index.value.is.delta.encoded";
  244. const std::string TablePropertiesNames::kFilterSize = "rocksdb.filter.size";
  245. const std::string TablePropertiesNames::kRawKeySize = "rocksdb.raw.key.size";
  246. const std::string TablePropertiesNames::kRawValueSize =
  247. "rocksdb.raw.value.size";
  248. const std::string TablePropertiesNames::kNumDataBlocks =
  249. "rocksdb.num.data.blocks";
  250. const std::string TablePropertiesNames::kNumEntries = "rocksdb.num.entries";
  251. const std::string TablePropertiesNames::kNumFilterEntries =
  252. "rocksdb.num.filter_entries";
  253. const std::string TablePropertiesNames::kDeletedKeys = "rocksdb.deleted.keys";
  254. const std::string TablePropertiesNames::kMergeOperands =
  255. "rocksdb.merge.operands";
  256. const std::string TablePropertiesNames::kNumRangeDeletions =
  257. "rocksdb.num.range-deletions";
  258. const std::string TablePropertiesNames::kFilterPolicy = "rocksdb.filter.policy";
  259. const std::string TablePropertiesNames::kFormatVersion =
  260. "rocksdb.format.version";
  261. const std::string TablePropertiesNames::kFixedKeyLen =
  262. "rocksdb.fixed.key.length";
  263. const std::string TablePropertiesNames::kColumnFamilyId =
  264. "rocksdb.column.family.id";
  265. const std::string TablePropertiesNames::kColumnFamilyName =
  266. "rocksdb.column.family.name";
  267. const std::string TablePropertiesNames::kComparator = "rocksdb.comparator";
  268. const std::string TablePropertiesNames::kMergeOperator =
  269. "rocksdb.merge.operator";
  270. const std::string TablePropertiesNames::kPrefixExtractorName =
  271. "rocksdb.prefix.extractor.name";
  272. const std::string TablePropertiesNames::kPropertyCollectors =
  273. "rocksdb.property.collectors";
  274. const std::string TablePropertiesNames::kCompression = "rocksdb.compression";
  275. const std::string TablePropertiesNames::kCompressionOptions =
  276. "rocksdb.compression_options";
  277. const std::string TablePropertiesNames::kCreationTime = "rocksdb.creation.time";
  278. const std::string TablePropertiesNames::kOldestKeyTime =
  279. "rocksdb.oldest.key.time";
  280. const std::string TablePropertiesNames::kNewestKeyTime =
  281. "rocksdb.newest.key.time";
  282. const std::string TablePropertiesNames::kFileCreationTime =
  283. "rocksdb.file.creation.time";
  284. const std::string TablePropertiesNames::kSlowCompressionEstimatedDataSize =
  285. "rocksdb.sample_for_compression.slow.data.size";
  286. const std::string TablePropertiesNames::kFastCompressionEstimatedDataSize =
  287. "rocksdb.sample_for_compression.fast.data.size";
  288. const std::string TablePropertiesNames::kSequenceNumberTimeMapping =
  289. "rocksdb.seqno.time.map";
  290. const std::string TablePropertiesNames::kTailStartOffset =
  291. "rocksdb.tail.start.offset";
  292. const std::string TablePropertiesNames::kUserDefinedTimestampsPersisted =
  293. "rocksdb.user.defined.timestamps.persisted";
  294. const std::string TablePropertiesNames::kKeyLargestSeqno =
  295. "rocksdb.key.largest.seqno";
  296. const std::string TablePropertiesNames::kKeySmallestSeqno =
  297. "rocksdb.key.smallest.seqno";
  298. static std::unordered_map<std::string, OptionTypeInfo>
  299. table_properties_type_info = {
  300. {"orig_file_number",
  301. {offsetof(struct TableProperties, orig_file_number),
  302. OptionType::kUInt64T, OptionVerificationType::kNormal,
  303. OptionTypeFlags::kNone}},
  304. {"data_size",
  305. {offsetof(struct TableProperties, data_size), OptionType::kUInt64T,
  306. OptionVerificationType::kNormal, OptionTypeFlags::kNone}},
  307. {"uncompressed_data_size",
  308. {offsetof(struct TableProperties, uncompressed_data_size),
  309. OptionType::kUInt64T, OptionVerificationType::kNormal,
  310. OptionTypeFlags::kNone}},
  311. {"index_size",
  312. {offsetof(struct TableProperties, index_size), OptionType::kUInt64T,
  313. OptionVerificationType::kNormal, OptionTypeFlags::kNone}},
  314. {"index_partitions",
  315. {offsetof(struct TableProperties, index_partitions),
  316. OptionType::kUInt64T, OptionVerificationType::kNormal,
  317. OptionTypeFlags::kNone}},
  318. {"top_level_index_size",
  319. {offsetof(struct TableProperties, top_level_index_size),
  320. OptionType::kUInt64T, OptionVerificationType::kNormal,
  321. OptionTypeFlags::kNone}},
  322. {"index_key_is_user_key",
  323. {offsetof(struct TableProperties, index_key_is_user_key),
  324. OptionType::kUInt64T, OptionVerificationType::kNormal,
  325. OptionTypeFlags::kNone}},
  326. {"index_value_is_delta_encoded",
  327. {offsetof(struct TableProperties, index_value_is_delta_encoded),
  328. OptionType::kUInt64T, OptionVerificationType::kNormal,
  329. OptionTypeFlags::kNone}},
  330. {"filter_size",
  331. {offsetof(struct TableProperties, filter_size), OptionType::kUInt64T,
  332. OptionVerificationType::kNormal, OptionTypeFlags::kNone}},
  333. {"raw_key_size",
  334. {offsetof(struct TableProperties, raw_key_size), OptionType::kUInt64T,
  335. OptionVerificationType::kNormal, OptionTypeFlags::kNone}},
  336. {"raw_value_size",
  337. {offsetof(struct TableProperties, raw_value_size),
  338. OptionType::kUInt64T, OptionVerificationType::kNormal,
  339. OptionTypeFlags::kNone}},
  340. {"num_data_blocks",
  341. {offsetof(struct TableProperties, num_data_blocks),
  342. OptionType::kUInt64T, OptionVerificationType::kNormal,
  343. OptionTypeFlags::kNone}},
  344. {"num_entries",
  345. {offsetof(struct TableProperties, num_entries), OptionType::kUInt64T,
  346. OptionVerificationType::kNormal, OptionTypeFlags::kNone}},
  347. {"num_filter_entries",
  348. {offsetof(struct TableProperties, num_filter_entries),
  349. OptionType::kUInt64T, OptionVerificationType::kNormal,
  350. OptionTypeFlags::kNone}},
  351. {"num_deletions",
  352. {offsetof(struct TableProperties, num_deletions), OptionType::kUInt64T,
  353. OptionVerificationType::kNormal, OptionTypeFlags::kNone}},
  354. {"num_merge_operands",
  355. {offsetof(struct TableProperties, num_merge_operands),
  356. OptionType::kUInt64T, OptionVerificationType::kNormal,
  357. OptionTypeFlags::kNone}},
  358. {"num_range_deletions",
  359. {offsetof(struct TableProperties, num_range_deletions),
  360. OptionType::kUInt64T, OptionVerificationType::kNormal,
  361. OptionTypeFlags::kNone}},
  362. {"format_version",
  363. {offsetof(struct TableProperties, format_version),
  364. OptionType::kUInt64T, OptionVerificationType::kNormal,
  365. OptionTypeFlags::kNone}},
  366. {"fixed_key_len",
  367. {offsetof(struct TableProperties, fixed_key_len), OptionType::kUInt64T,
  368. OptionVerificationType::kNormal, OptionTypeFlags::kNone}},
  369. {"column_family_id",
  370. {offsetof(struct TableProperties, column_family_id),
  371. OptionType::kUInt64T, OptionVerificationType::kNormal,
  372. OptionTypeFlags::kNone}},
  373. {"creation_time",
  374. {offsetof(struct TableProperties, creation_time), OptionType::kUInt64T,
  375. OptionVerificationType::kNormal, OptionTypeFlags::kNone}},
  376. {"oldest_key_time",
  377. {offsetof(struct TableProperties, oldest_key_time),
  378. OptionType::kUInt64T, OptionVerificationType::kNormal,
  379. OptionTypeFlags::kNone}},
  380. {"newest_key_time",
  381. {offsetof(struct TableProperties, newest_key_time),
  382. OptionType::kUInt64T, OptionVerificationType::kNormal,
  383. OptionTypeFlags::kNone}},
  384. {"file_creation_time",
  385. {offsetof(struct TableProperties, file_creation_time),
  386. OptionType::kUInt64T, OptionVerificationType::kNormal,
  387. OptionTypeFlags::kNone}},
  388. {"slow_compression_estimated_data_size",
  389. {offsetof(struct TableProperties,
  390. slow_compression_estimated_data_size),
  391. OptionType::kUInt64T, OptionVerificationType::kNormal,
  392. OptionTypeFlags::kNone}},
  393. {"fast_compression_estimated_data_size",
  394. {offsetof(struct TableProperties,
  395. fast_compression_estimated_data_size),
  396. OptionType::kUInt64T, OptionVerificationType::kNormal,
  397. OptionTypeFlags::kNone}},
  398. {"external_sst_file_global_seqno_offset",
  399. {offsetof(struct TableProperties,
  400. external_sst_file_global_seqno_offset),
  401. OptionType::kUInt64T, OptionVerificationType::kNormal,
  402. OptionTypeFlags::kNone}},
  403. {"tail_start_offset",
  404. {offsetof(struct TableProperties, tail_start_offset),
  405. OptionType::kUInt64T, OptionVerificationType::kNormal,
  406. OptionTypeFlags::kNone}},
  407. {"user_defined_timestamps_persisted",
  408. {offsetof(struct TableProperties, user_defined_timestamps_persisted),
  409. OptionType::kUInt64T, OptionVerificationType::kNormal,
  410. OptionTypeFlags::kNone}},
  411. {"key_largest_seqno",
  412. {offsetof(struct TableProperties, key_largest_seqno),
  413. OptionType::kUInt64T, OptionVerificationType::kNormal,
  414. OptionTypeFlags::kNone}},
  415. {"key_smallest_seqno",
  416. {offsetof(struct TableProperties, key_smallest_seqno),
  417. OptionType::kUInt64T, OptionVerificationType::kNormal,
  418. OptionTypeFlags::kNone}},
  419. {"db_id",
  420. {offsetof(struct TableProperties, db_id), OptionType::kEncodedString}},
  421. {"db_session_id",
  422. {offsetof(struct TableProperties, db_session_id),
  423. OptionType::kEncodedString}},
  424. {"db_host_id",
  425. {offsetof(struct TableProperties, db_host_id),
  426. OptionType::kEncodedString}},
  427. {"column_family_name",
  428. {offsetof(struct TableProperties, column_family_name),
  429. OptionType::kEncodedString}},
  430. {"filter_policy_name",
  431. {offsetof(struct TableProperties, filter_policy_name),
  432. OptionType::kEncodedString}},
  433. {"comparator_name",
  434. {offsetof(struct TableProperties, comparator_name),
  435. OptionType::kEncodedString}},
  436. {"merge_operator_name",
  437. {offsetof(struct TableProperties, merge_operator_name),
  438. OptionType::kEncodedString}},
  439. {"prefix_extractor_name",
  440. {offsetof(struct TableProperties, prefix_extractor_name),
  441. OptionType::kEncodedString}},
  442. {"property_collectors_names",
  443. {offsetof(struct TableProperties, property_collectors_names),
  444. OptionType::kEncodedString}},
  445. {"compression_name",
  446. {offsetof(struct TableProperties, compression_name),
  447. OptionType::kEncodedString}},
  448. {"compression_options",
  449. {offsetof(struct TableProperties, compression_options),
  450. OptionType::kEncodedString}},
  451. {"seqno_to_time_mapping",
  452. {offsetof(struct TableProperties, seqno_to_time_mapping),
  453. OptionType::kEncodedString}},
  454. {"user_collected_properties",
  455. OptionTypeInfo::StringMap(
  456. offsetof(struct TableProperties, user_collected_properties),
  457. OptionVerificationType::kNormal, OptionTypeFlags::kNone)},
  458. {"readable_properties",
  459. OptionTypeInfo::StringMap(
  460. offsetof(struct TableProperties, readable_properties),
  461. OptionVerificationType::kNormal, OptionTypeFlags::kNone)},
  462. };
  463. Status TableProperties::Serialize(const ConfigOptions& opts,
  464. std::string* output) const {
  465. return OptionTypeInfo::SerializeType(opts, table_properties_type_info, this,
  466. output);
  467. }
  468. Status TableProperties::Parse(const ConfigOptions& opts,
  469. const std::string& serialized,
  470. TableProperties* table_properties) {
  471. return OptionTypeInfo::ParseType(opts, serialized, table_properties_type_info,
  472. table_properties);
  473. }
  474. bool TableProperties::AreEqual(const ConfigOptions& opts,
  475. const TableProperties* other_table_properties,
  476. std::string* mismatch) const {
  477. return OptionTypeInfo::TypesAreEqual(opts, table_properties_type_info, this,
  478. other_table_properties, mismatch);
  479. }
  480. #ifndef NDEBUG
  481. // WARNING: TEST_SetRandomTableProperties assumes the following layout of
  482. // TableProperties
  483. //
  484. // struct TableProperties {
  485. // int64_t orig_file_number = 0;
  486. // ...
  487. // ... int64_t properties only
  488. // ...
  489. // std::string db_id;
  490. // ...
  491. // ... std::string properties only
  492. // ...
  493. // std::string compression_options;
  494. // UserCollectedProperties user_collected_properties;
  495. // ...
  496. // ... Other extra properties: non-int64_t/non-std::string properties only
  497. // ...
  498. // }
  499. void TEST_SetRandomTableProperties(TableProperties* props) {
  500. Random* r = Random::GetTLSInstance();
  501. uint64_t* pu = &props->orig_file_number;
  502. assert(static_cast<void*>(pu) == static_cast<void*>(props));
  503. std::string* ps = &props->db_id;
  504. const uint64_t* const pu_end = reinterpret_cast<const uint64_t*>(ps);
  505. // Use the last string property's address instead of
  506. // the first extra property (e.g `user_collected_properties`)'s address
  507. // in the for-loop to avoid advancing pointer to pointing to
  508. // potential non-zero padding bytes between these two addresses due to
  509. // user_collected_properties's alignment requirement
  510. const std::string* const ps_end_inclusive = &props->compression_options;
  511. for (; pu < pu_end; ++pu) {
  512. *pu = r->Next64();
  513. }
  514. assert(static_cast<void*>(pu) == static_cast<void*>(ps));
  515. for (; ps <= ps_end_inclusive; ++ps) {
  516. *ps = r->RandomBinaryString(13);
  517. }
  518. }
  519. #endif
  520. } // namespace ROCKSDB_NAMESPACE