meta_blocks.cc 20 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525
  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 "table/meta_blocks.h"
  6. #include <map>
  7. #include <string>
  8. #include "block_fetcher.h"
  9. #include "db/table_properties_collector.h"
  10. #include "file/random_access_file_reader.h"
  11. #include "rocksdb/table.h"
  12. #include "rocksdb/table_properties.h"
  13. #include "table/block_based/block.h"
  14. #include "table/format.h"
  15. #include "table/internal_iterator.h"
  16. #include "table/persistent_cache_helper.h"
  17. #include "table/table_properties_internal.h"
  18. #include "test_util/sync_point.h"
  19. #include "util/coding.h"
  20. namespace ROCKSDB_NAMESPACE {
  21. MetaIndexBuilder::MetaIndexBuilder()
  22. : meta_index_block_(new BlockBuilder(1 /* restart interval */)) {}
  23. void MetaIndexBuilder::Add(const std::string& key,
  24. const BlockHandle& handle) {
  25. std::string handle_encoding;
  26. handle.EncodeTo(&handle_encoding);
  27. meta_block_handles_.insert({key, handle_encoding});
  28. }
  29. Slice MetaIndexBuilder::Finish() {
  30. for (const auto& metablock : meta_block_handles_) {
  31. meta_index_block_->Add(metablock.first, metablock.second);
  32. }
  33. return meta_index_block_->Finish();
  34. }
  35. // Property block will be read sequentially and cached in a heap located
  36. // object, so there's no need for restart points. Thus we set the restart
  37. // interval to infinity to save space.
  38. PropertyBlockBuilder::PropertyBlockBuilder()
  39. : properties_block_(
  40. new BlockBuilder(port::kMaxInt32 /* restart interval */)) {}
  41. void PropertyBlockBuilder::Add(const std::string& name,
  42. const std::string& val) {
  43. props_.insert({name, val});
  44. }
  45. void PropertyBlockBuilder::Add(const std::string& name, uint64_t val) {
  46. assert(props_.find(name) == props_.end());
  47. std::string dst;
  48. PutVarint64(&dst, val);
  49. Add(name, dst);
  50. }
  51. void PropertyBlockBuilder::Add(
  52. const UserCollectedProperties& user_collected_properties) {
  53. for (const auto& prop : user_collected_properties) {
  54. Add(prop.first, prop.second);
  55. }
  56. }
  57. void PropertyBlockBuilder::AddTableProperty(const TableProperties& props) {
  58. TEST_SYNC_POINT_CALLBACK("PropertyBlockBuilder::AddTableProperty:Start",
  59. const_cast<TableProperties*>(&props));
  60. Add(TablePropertiesNames::kRawKeySize, props.raw_key_size);
  61. Add(TablePropertiesNames::kRawValueSize, props.raw_value_size);
  62. Add(TablePropertiesNames::kDataSize, props.data_size);
  63. Add(TablePropertiesNames::kIndexSize, props.index_size);
  64. if (props.index_partitions != 0) {
  65. Add(TablePropertiesNames::kIndexPartitions, props.index_partitions);
  66. Add(TablePropertiesNames::kTopLevelIndexSize, props.top_level_index_size);
  67. }
  68. Add(TablePropertiesNames::kIndexKeyIsUserKey, props.index_key_is_user_key);
  69. Add(TablePropertiesNames::kIndexValueIsDeltaEncoded,
  70. props.index_value_is_delta_encoded);
  71. Add(TablePropertiesNames::kNumEntries, props.num_entries);
  72. Add(TablePropertiesNames::kDeletedKeys, props.num_deletions);
  73. Add(TablePropertiesNames::kMergeOperands, props.num_merge_operands);
  74. Add(TablePropertiesNames::kNumRangeDeletions, props.num_range_deletions);
  75. Add(TablePropertiesNames::kNumDataBlocks, props.num_data_blocks);
  76. Add(TablePropertiesNames::kFilterSize, props.filter_size);
  77. Add(TablePropertiesNames::kFormatVersion, props.format_version);
  78. Add(TablePropertiesNames::kFixedKeyLen, props.fixed_key_len);
  79. Add(TablePropertiesNames::kColumnFamilyId, props.column_family_id);
  80. Add(TablePropertiesNames::kCreationTime, props.creation_time);
  81. Add(TablePropertiesNames::kOldestKeyTime, props.oldest_key_time);
  82. if (props.file_creation_time > 0) {
  83. Add(TablePropertiesNames::kFileCreationTime, props.file_creation_time);
  84. }
  85. if (!props.filter_policy_name.empty()) {
  86. Add(TablePropertiesNames::kFilterPolicy, props.filter_policy_name);
  87. }
  88. if (!props.comparator_name.empty()) {
  89. Add(TablePropertiesNames::kComparator, props.comparator_name);
  90. }
  91. if (!props.merge_operator_name.empty()) {
  92. Add(TablePropertiesNames::kMergeOperator, props.merge_operator_name);
  93. }
  94. if (!props.prefix_extractor_name.empty()) {
  95. Add(TablePropertiesNames::kPrefixExtractorName,
  96. props.prefix_extractor_name);
  97. }
  98. if (!props.property_collectors_names.empty()) {
  99. Add(TablePropertiesNames::kPropertyCollectors,
  100. props.property_collectors_names);
  101. }
  102. if (!props.column_family_name.empty()) {
  103. Add(TablePropertiesNames::kColumnFamilyName, props.column_family_name);
  104. }
  105. if (!props.compression_name.empty()) {
  106. Add(TablePropertiesNames::kCompression, props.compression_name);
  107. }
  108. if (!props.compression_options.empty()) {
  109. Add(TablePropertiesNames::kCompressionOptions, props.compression_options);
  110. }
  111. }
  112. Slice PropertyBlockBuilder::Finish() {
  113. for (const auto& prop : props_) {
  114. properties_block_->Add(prop.first, prop.second);
  115. }
  116. return properties_block_->Finish();
  117. }
  118. void LogPropertiesCollectionError(
  119. Logger* info_log, const std::string& method, const std::string& name) {
  120. assert(method == "Add" || method == "Finish");
  121. std::string msg =
  122. "Encountered error when calling TablePropertiesCollector::" +
  123. method + "() with collector name: " + name;
  124. ROCKS_LOG_ERROR(info_log, "%s", msg.c_str());
  125. }
  126. bool NotifyCollectTableCollectorsOnAdd(
  127. const Slice& key, const Slice& value, uint64_t file_size,
  128. const std::vector<std::unique_ptr<IntTblPropCollector>>& collectors,
  129. Logger* info_log) {
  130. bool all_succeeded = true;
  131. for (auto& collector : collectors) {
  132. Status s = collector->InternalAdd(key, value, file_size);
  133. all_succeeded = all_succeeded && s.ok();
  134. if (!s.ok()) {
  135. LogPropertiesCollectionError(info_log, "Add" /* method */,
  136. collector->Name());
  137. }
  138. }
  139. return all_succeeded;
  140. }
  141. void NotifyCollectTableCollectorsOnBlockAdd(
  142. const std::vector<std::unique_ptr<IntTblPropCollector>>& collectors,
  143. const uint64_t blockRawBytes, const uint64_t blockCompressedBytesFast,
  144. const uint64_t blockCompressedBytesSlow) {
  145. for (auto& collector : collectors) {
  146. collector->BlockAdd(blockRawBytes, blockCompressedBytesFast,
  147. blockCompressedBytesSlow);
  148. }
  149. }
  150. bool NotifyCollectTableCollectorsOnFinish(
  151. const std::vector<std::unique_ptr<IntTblPropCollector>>& collectors,
  152. Logger* info_log, PropertyBlockBuilder* builder) {
  153. bool all_succeeded = true;
  154. for (auto& collector : collectors) {
  155. UserCollectedProperties user_collected_properties;
  156. Status s = collector->Finish(&user_collected_properties);
  157. all_succeeded = all_succeeded && s.ok();
  158. if (!s.ok()) {
  159. LogPropertiesCollectionError(info_log, "Finish" /* method */,
  160. collector->Name());
  161. } else {
  162. builder->Add(user_collected_properties);
  163. }
  164. }
  165. return all_succeeded;
  166. }
  167. Status ReadProperties(const Slice& handle_value, RandomAccessFileReader* file,
  168. FilePrefetchBuffer* prefetch_buffer, const Footer& footer,
  169. const ImmutableCFOptions& ioptions,
  170. TableProperties** table_properties, bool verify_checksum,
  171. BlockHandle* ret_block_handle,
  172. CacheAllocationPtr* verification_buf,
  173. bool /*compression_type_missing*/,
  174. MemoryAllocator* memory_allocator) {
  175. assert(table_properties);
  176. Slice v = handle_value;
  177. BlockHandle handle;
  178. if (!handle.DecodeFrom(&v).ok()) {
  179. return Status::InvalidArgument("Failed to decode properties block handle");
  180. }
  181. BlockContents block_contents;
  182. ReadOptions read_options;
  183. read_options.verify_checksums = verify_checksum;
  184. Status s;
  185. PersistentCacheOptions cache_options;
  186. BlockFetcher block_fetcher(
  187. file, prefetch_buffer, footer, read_options, handle, &block_contents,
  188. ioptions, false /* decompress */, false /*maybe_compressed*/,
  189. BlockType::kProperties, UncompressionDict::GetEmptyDict(), cache_options,
  190. memory_allocator);
  191. s = block_fetcher.ReadBlockContents();
  192. // property block is never compressed. Need to add uncompress logic if we are
  193. // to compress it..
  194. if (!s.ok()) {
  195. return s;
  196. }
  197. Block properties_block(std::move(block_contents),
  198. kDisableGlobalSequenceNumber);
  199. DataBlockIter iter;
  200. properties_block.NewDataIterator(BytewiseComparator(), BytewiseComparator(),
  201. &iter);
  202. auto new_table_properties = new TableProperties();
  203. // All pre-defined properties of type uint64_t
  204. std::unordered_map<std::string, uint64_t*> predefined_uint64_properties = {
  205. {TablePropertiesNames::kDataSize, &new_table_properties->data_size},
  206. {TablePropertiesNames::kIndexSize, &new_table_properties->index_size},
  207. {TablePropertiesNames::kIndexPartitions,
  208. &new_table_properties->index_partitions},
  209. {TablePropertiesNames::kTopLevelIndexSize,
  210. &new_table_properties->top_level_index_size},
  211. {TablePropertiesNames::kIndexKeyIsUserKey,
  212. &new_table_properties->index_key_is_user_key},
  213. {TablePropertiesNames::kIndexValueIsDeltaEncoded,
  214. &new_table_properties->index_value_is_delta_encoded},
  215. {TablePropertiesNames::kFilterSize, &new_table_properties->filter_size},
  216. {TablePropertiesNames::kRawKeySize, &new_table_properties->raw_key_size},
  217. {TablePropertiesNames::kRawValueSize,
  218. &new_table_properties->raw_value_size},
  219. {TablePropertiesNames::kNumDataBlocks,
  220. &new_table_properties->num_data_blocks},
  221. {TablePropertiesNames::kNumEntries, &new_table_properties->num_entries},
  222. {TablePropertiesNames::kDeletedKeys,
  223. &new_table_properties->num_deletions},
  224. {TablePropertiesNames::kMergeOperands,
  225. &new_table_properties->num_merge_operands},
  226. {TablePropertiesNames::kNumRangeDeletions,
  227. &new_table_properties->num_range_deletions},
  228. {TablePropertiesNames::kFormatVersion,
  229. &new_table_properties->format_version},
  230. {TablePropertiesNames::kFixedKeyLen,
  231. &new_table_properties->fixed_key_len},
  232. {TablePropertiesNames::kColumnFamilyId,
  233. &new_table_properties->column_family_id},
  234. {TablePropertiesNames::kCreationTime,
  235. &new_table_properties->creation_time},
  236. {TablePropertiesNames::kOldestKeyTime,
  237. &new_table_properties->oldest_key_time},
  238. {TablePropertiesNames::kFileCreationTime,
  239. &new_table_properties->file_creation_time},
  240. };
  241. std::string last_key;
  242. for (iter.SeekToFirstOrReport(); iter.Valid(); iter.NextOrReport()) {
  243. s = iter.status();
  244. if (!s.ok()) {
  245. break;
  246. }
  247. auto key = iter.key().ToString();
  248. // properties block should be strictly sorted with no duplicate key.
  249. if (!last_key.empty() &&
  250. BytewiseComparator()->Compare(key, last_key) <= 0) {
  251. s = Status::Corruption("properties unsorted");
  252. break;
  253. }
  254. last_key = key;
  255. auto raw_val = iter.value();
  256. auto pos = predefined_uint64_properties.find(key);
  257. new_table_properties->properties_offsets.insert(
  258. {key, handle.offset() + iter.ValueOffset()});
  259. if (pos != predefined_uint64_properties.end()) {
  260. if (key == TablePropertiesNames::kDeletedKeys ||
  261. key == TablePropertiesNames::kMergeOperands) {
  262. // Insert in user-collected properties for API backwards compatibility
  263. new_table_properties->user_collected_properties.insert(
  264. {key, raw_val.ToString()});
  265. }
  266. // handle predefined rocksdb properties
  267. uint64_t val;
  268. if (!GetVarint64(&raw_val, &val)) {
  269. // skip malformed value
  270. auto error_msg =
  271. "Detect malformed value in properties meta-block:"
  272. "\tkey: " + key + "\tval: " + raw_val.ToString();
  273. ROCKS_LOG_ERROR(ioptions.info_log, "%s", error_msg.c_str());
  274. continue;
  275. }
  276. *(pos->second) = val;
  277. } else if (key == TablePropertiesNames::kFilterPolicy) {
  278. new_table_properties->filter_policy_name = raw_val.ToString();
  279. } else if (key == TablePropertiesNames::kColumnFamilyName) {
  280. new_table_properties->column_family_name = raw_val.ToString();
  281. } else if (key == TablePropertiesNames::kComparator) {
  282. new_table_properties->comparator_name = raw_val.ToString();
  283. } else if (key == TablePropertiesNames::kMergeOperator) {
  284. new_table_properties->merge_operator_name = raw_val.ToString();
  285. } else if (key == TablePropertiesNames::kPrefixExtractorName) {
  286. new_table_properties->prefix_extractor_name = raw_val.ToString();
  287. } else if (key == TablePropertiesNames::kPropertyCollectors) {
  288. new_table_properties->property_collectors_names = raw_val.ToString();
  289. } else if (key == TablePropertiesNames::kCompression) {
  290. new_table_properties->compression_name = raw_val.ToString();
  291. } else if (key == TablePropertiesNames::kCompressionOptions) {
  292. new_table_properties->compression_options = raw_val.ToString();
  293. } else {
  294. // handle user-collected properties
  295. new_table_properties->user_collected_properties.insert(
  296. {key, raw_val.ToString()});
  297. }
  298. }
  299. if (s.ok()) {
  300. *table_properties = new_table_properties;
  301. if (ret_block_handle != nullptr) {
  302. *ret_block_handle = handle;
  303. }
  304. if (verification_buf != nullptr) {
  305. size_t len = static_cast<size_t>(handle.size() + kBlockTrailerSize);
  306. *verification_buf =
  307. ROCKSDB_NAMESPACE::AllocateBlock(len, memory_allocator);
  308. if (verification_buf->get() != nullptr) {
  309. memcpy(verification_buf->get(), block_contents.data.data(), len);
  310. }
  311. }
  312. } else {
  313. delete new_table_properties;
  314. }
  315. return s;
  316. }
  317. Status ReadTableProperties(RandomAccessFileReader* file, uint64_t file_size,
  318. uint64_t table_magic_number,
  319. const ImmutableCFOptions& ioptions,
  320. TableProperties** properties,
  321. bool compression_type_missing,
  322. MemoryAllocator* memory_allocator) {
  323. // -- Read metaindex block
  324. Footer footer;
  325. auto s = ReadFooterFromFile(file, nullptr /* prefetch_buffer */, file_size,
  326. &footer, table_magic_number);
  327. if (!s.ok()) {
  328. return s;
  329. }
  330. auto metaindex_handle = footer.metaindex_handle();
  331. BlockContents metaindex_contents;
  332. ReadOptions read_options;
  333. read_options.verify_checksums = false;
  334. PersistentCacheOptions cache_options;
  335. BlockFetcher block_fetcher(
  336. file, nullptr /* prefetch_buffer */, footer, read_options,
  337. metaindex_handle, &metaindex_contents, ioptions, false /* decompress */,
  338. false /*maybe_compressed*/, BlockType::kMetaIndex,
  339. UncompressionDict::GetEmptyDict(), cache_options, memory_allocator);
  340. s = block_fetcher.ReadBlockContents();
  341. if (!s.ok()) {
  342. return s;
  343. }
  344. // property blocks are never compressed. Need to add uncompress logic if we
  345. // are to compress it.
  346. Block metaindex_block(std::move(metaindex_contents),
  347. kDisableGlobalSequenceNumber);
  348. std::unique_ptr<InternalIterator> meta_iter(metaindex_block.NewDataIterator(
  349. BytewiseComparator(), BytewiseComparator()));
  350. // -- Read property block
  351. bool found_properties_block = true;
  352. s = SeekToPropertiesBlock(meta_iter.get(), &found_properties_block);
  353. if (!s.ok()) {
  354. return s;
  355. }
  356. TableProperties table_properties;
  357. if (found_properties_block == true) {
  358. s = ReadProperties(
  359. meta_iter->value(), file, nullptr /* prefetch_buffer */, footer,
  360. ioptions, properties, false /* verify_checksum */,
  361. nullptr /* ret_block_hanel */, nullptr /* ret_block_contents */,
  362. compression_type_missing, memory_allocator);
  363. } else {
  364. s = Status::NotFound();
  365. }
  366. return s;
  367. }
  368. Status FindMetaBlock(InternalIterator* meta_index_iter,
  369. const std::string& meta_block_name,
  370. BlockHandle* block_handle) {
  371. meta_index_iter->Seek(meta_block_name);
  372. if (meta_index_iter->status().ok() && meta_index_iter->Valid() &&
  373. meta_index_iter->key() == meta_block_name) {
  374. Slice v = meta_index_iter->value();
  375. return block_handle->DecodeFrom(&v);
  376. } else {
  377. return Status::Corruption("Cannot find the meta block", meta_block_name);
  378. }
  379. }
  380. Status FindMetaBlock(RandomAccessFileReader* file, uint64_t file_size,
  381. uint64_t table_magic_number,
  382. const ImmutableCFOptions& ioptions,
  383. const std::string& meta_block_name,
  384. BlockHandle* block_handle,
  385. bool /*compression_type_missing*/,
  386. MemoryAllocator* memory_allocator) {
  387. Footer footer;
  388. auto s = ReadFooterFromFile(file, nullptr /* prefetch_buffer */, file_size,
  389. &footer, table_magic_number);
  390. if (!s.ok()) {
  391. return s;
  392. }
  393. auto metaindex_handle = footer.metaindex_handle();
  394. BlockContents metaindex_contents;
  395. ReadOptions read_options;
  396. read_options.verify_checksums = false;
  397. PersistentCacheOptions cache_options;
  398. BlockFetcher block_fetcher(
  399. file, nullptr /* prefetch_buffer */, footer, read_options,
  400. metaindex_handle, &metaindex_contents, ioptions,
  401. false /* do decompression */, false /*maybe_compressed*/,
  402. BlockType::kMetaIndex, UncompressionDict::GetEmptyDict(), cache_options,
  403. memory_allocator);
  404. s = block_fetcher.ReadBlockContents();
  405. if (!s.ok()) {
  406. return s;
  407. }
  408. // meta blocks are never compressed. Need to add uncompress logic if we are to
  409. // compress it.
  410. Block metaindex_block(std::move(metaindex_contents),
  411. kDisableGlobalSequenceNumber);
  412. std::unique_ptr<InternalIterator> meta_iter;
  413. meta_iter.reset(metaindex_block.NewDataIterator(BytewiseComparator(),
  414. BytewiseComparator()));
  415. return FindMetaBlock(meta_iter.get(), meta_block_name, block_handle);
  416. }
  417. Status ReadMetaBlock(RandomAccessFileReader* file,
  418. FilePrefetchBuffer* prefetch_buffer, uint64_t file_size,
  419. uint64_t table_magic_number,
  420. const ImmutableCFOptions& ioptions,
  421. const std::string& meta_block_name, BlockType block_type,
  422. BlockContents* contents, bool /*compression_type_missing*/,
  423. MemoryAllocator* memory_allocator) {
  424. Status status;
  425. Footer footer;
  426. status = ReadFooterFromFile(file, prefetch_buffer, file_size, &footer,
  427. table_magic_number);
  428. if (!status.ok()) {
  429. return status;
  430. }
  431. // Reading metaindex block
  432. auto metaindex_handle = footer.metaindex_handle();
  433. BlockContents metaindex_contents;
  434. ReadOptions read_options;
  435. read_options.verify_checksums = false;
  436. PersistentCacheOptions cache_options;
  437. BlockFetcher block_fetcher(
  438. file, prefetch_buffer, footer, read_options, metaindex_handle,
  439. &metaindex_contents, ioptions, false /* decompress */,
  440. false /*maybe_compressed*/, BlockType::kMetaIndex,
  441. UncompressionDict::GetEmptyDict(), cache_options, memory_allocator);
  442. status = block_fetcher.ReadBlockContents();
  443. if (!status.ok()) {
  444. return status;
  445. }
  446. // meta block is never compressed. Need to add uncompress logic if we are to
  447. // compress it.
  448. // Finding metablock
  449. Block metaindex_block(std::move(metaindex_contents),
  450. kDisableGlobalSequenceNumber);
  451. std::unique_ptr<InternalIterator> meta_iter;
  452. meta_iter.reset(metaindex_block.NewDataIterator(BytewiseComparator(),
  453. BytewiseComparator()));
  454. BlockHandle block_handle;
  455. status = FindMetaBlock(meta_iter.get(), meta_block_name, &block_handle);
  456. if (!status.ok()) {
  457. return status;
  458. }
  459. // Reading metablock
  460. BlockFetcher block_fetcher2(
  461. file, prefetch_buffer, footer, read_options, block_handle, contents,
  462. ioptions, false /* decompress */, false /*maybe_compressed*/, block_type,
  463. UncompressionDict::GetEmptyDict(), cache_options, memory_allocator);
  464. return block_fetcher2.ReadBlockContents();
  465. }
  466. } // namespace ROCKSDB_NAMESPACE