meta_blocks.cc 27 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660
  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 "logging/logging.h"
  12. #include "rocksdb/options.h"
  13. #include "rocksdb/table.h"
  14. #include "rocksdb/table_properties.h"
  15. #include "table/block_based/block.h"
  16. #include "table/block_based/reader_common.h"
  17. #include "table/format.h"
  18. #include "table/internal_iterator.h"
  19. #include "table/persistent_cache_helper.h"
  20. #include "table/sst_file_writer_collectors.h"
  21. #include "table/table_properties_internal.h"
  22. #include "test_util/sync_point.h"
  23. #include "util/coding.h"
  24. namespace ROCKSDB_NAMESPACE {
  25. const std::string kPropertiesBlockName = "rocksdb.properties";
  26. // NB: only used with format_version >= 6
  27. const std::string kIndexBlockName = "rocksdb.index";
  28. // Old property block name for backward compatibility
  29. const std::string kPropertiesBlockOldName = "rocksdb.stats";
  30. const std::string kCompressionDictBlockName = "rocksdb.compression_dict";
  31. const std::string kRangeDelBlockName = "rocksdb.range_del";
  32. MetaIndexBuilder::MetaIndexBuilder()
  33. : meta_index_block_(new BlockBuilder(1 /* restart interval */)) {}
  34. void MetaIndexBuilder::Add(const std::string& key, const BlockHandle& handle) {
  35. std::string handle_encoding;
  36. handle.EncodeTo(&handle_encoding);
  37. meta_block_handles_.insert({key, handle_encoding});
  38. }
  39. Slice MetaIndexBuilder::Finish() {
  40. for (const auto& metablock : meta_block_handles_) {
  41. meta_index_block_->Add(metablock.first, metablock.second);
  42. }
  43. return meta_index_block_->Finish();
  44. }
  45. // Property block will be read sequentially and cached in a heap located
  46. // object, so there's no need for restart points. Thus we set the restart
  47. // interval to infinity to save space.
  48. PropertyBlockBuilder::PropertyBlockBuilder()
  49. : properties_block_(new BlockBuilder(
  50. std::numeric_limits<int32_t>::max() /* restart interval */)) {}
  51. void PropertyBlockBuilder::Add(const std::string& name,
  52. const std::string& val) {
  53. assert(props_.find(name) == props_.end());
  54. props_.insert({name, val});
  55. }
  56. void PropertyBlockBuilder::Add(const std::string& name, uint64_t val) {
  57. std::string dst;
  58. PutVarint64(&dst, val);
  59. Add(name, dst);
  60. }
  61. void PropertyBlockBuilder::Add(
  62. const UserCollectedProperties& user_collected_properties) {
  63. for (const auto& prop : user_collected_properties) {
  64. Add(prop.first, prop.second);
  65. }
  66. }
  67. void PropertyBlockBuilder::AddTableProperty(const TableProperties& props) {
  68. TEST_SYNC_POINT_CALLBACK("PropertyBlockBuilder::AddTableProperty:Start",
  69. const_cast<TableProperties*>(&props));
  70. Add(TablePropertiesNames::kOriginalFileNumber, props.orig_file_number);
  71. Add(TablePropertiesNames::kRawKeySize, props.raw_key_size);
  72. Add(TablePropertiesNames::kRawValueSize, props.raw_value_size);
  73. Add(TablePropertiesNames::kDataSize, props.data_size);
  74. Add(TablePropertiesNames::kIndexSize, props.index_size);
  75. if (props.index_partitions != 0) {
  76. Add(TablePropertiesNames::kIndexPartitions, props.index_partitions);
  77. Add(TablePropertiesNames::kTopLevelIndexSize, props.top_level_index_size);
  78. }
  79. Add(TablePropertiesNames::kIndexKeyIsUserKey, props.index_key_is_user_key);
  80. Add(TablePropertiesNames::kIndexValueIsDeltaEncoded,
  81. props.index_value_is_delta_encoded);
  82. Add(TablePropertiesNames::kNumEntries, props.num_entries);
  83. Add(TablePropertiesNames::kNumFilterEntries, props.num_filter_entries);
  84. Add(TablePropertiesNames::kDeletedKeys, props.num_deletions);
  85. Add(TablePropertiesNames::kMergeOperands, props.num_merge_operands);
  86. Add(TablePropertiesNames::kNumRangeDeletions, props.num_range_deletions);
  87. Add(TablePropertiesNames::kNumDataBlocks, props.num_data_blocks);
  88. Add(TablePropertiesNames::kFilterSize, props.filter_size);
  89. Add(TablePropertiesNames::kFormatVersion, props.format_version);
  90. Add(TablePropertiesNames::kFixedKeyLen, props.fixed_key_len);
  91. Add(TablePropertiesNames::kColumnFamilyId, props.column_family_id);
  92. Add(TablePropertiesNames::kCreationTime, props.creation_time);
  93. Add(TablePropertiesNames::kOldestKeyTime, props.oldest_key_time);
  94. Add(TablePropertiesNames::kNewestKeyTime, props.newest_key_time);
  95. if (props.file_creation_time > 0) {
  96. Add(TablePropertiesNames::kFileCreationTime, props.file_creation_time);
  97. }
  98. if (props.slow_compression_estimated_data_size > 0) {
  99. Add(TablePropertiesNames::kSlowCompressionEstimatedDataSize,
  100. props.slow_compression_estimated_data_size);
  101. }
  102. if (props.fast_compression_estimated_data_size > 0) {
  103. Add(TablePropertiesNames::kFastCompressionEstimatedDataSize,
  104. props.fast_compression_estimated_data_size);
  105. }
  106. Add(TablePropertiesNames::kTailStartOffset, props.tail_start_offset);
  107. if (props.user_defined_timestamps_persisted == 0) {
  108. Add(TablePropertiesNames::kUserDefinedTimestampsPersisted,
  109. props.user_defined_timestamps_persisted);
  110. }
  111. if (!props.db_id.empty()) {
  112. Add(TablePropertiesNames::kDbId, props.db_id);
  113. }
  114. if (!props.db_session_id.empty()) {
  115. Add(TablePropertiesNames::kDbSessionId, props.db_session_id);
  116. }
  117. if (!props.db_host_id.empty()) {
  118. Add(TablePropertiesNames::kDbHostId, props.db_host_id);
  119. }
  120. if (!props.filter_policy_name.empty()) {
  121. Add(TablePropertiesNames::kFilterPolicy, props.filter_policy_name);
  122. }
  123. if (!props.comparator_name.empty()) {
  124. Add(TablePropertiesNames::kComparator, props.comparator_name);
  125. }
  126. if (!props.merge_operator_name.empty()) {
  127. Add(TablePropertiesNames::kMergeOperator, props.merge_operator_name);
  128. }
  129. if (!props.prefix_extractor_name.empty()) {
  130. Add(TablePropertiesNames::kPrefixExtractorName,
  131. props.prefix_extractor_name);
  132. }
  133. if (!props.property_collectors_names.empty()) {
  134. Add(TablePropertiesNames::kPropertyCollectors,
  135. props.property_collectors_names);
  136. }
  137. if (!props.column_family_name.empty()) {
  138. Add(TablePropertiesNames::kColumnFamilyName, props.column_family_name);
  139. }
  140. if (!props.compression_name.empty()) {
  141. Add(TablePropertiesNames::kCompression, props.compression_name);
  142. }
  143. if (!props.compression_options.empty()) {
  144. Add(TablePropertiesNames::kCompressionOptions, props.compression_options);
  145. }
  146. if (!props.seqno_to_time_mapping.empty()) {
  147. Add(TablePropertiesNames::kSequenceNumberTimeMapping,
  148. props.seqno_to_time_mapping);
  149. }
  150. if (props.key_largest_seqno != UINT64_MAX) {
  151. Add(TablePropertiesNames::kKeyLargestSeqno, props.key_largest_seqno);
  152. }
  153. if (props.key_smallest_seqno != UINT64_MAX) {
  154. Add(TablePropertiesNames::kKeySmallestSeqno, props.key_smallest_seqno);
  155. }
  156. }
  157. Slice PropertyBlockBuilder::Finish() {
  158. for (const auto& prop : props_) {
  159. assert(last_prop_added_to_block_.empty() ||
  160. comparator_->Compare(prop.first, last_prop_added_to_block_) > 0);
  161. properties_block_->Add(prop.first, prop.second);
  162. #ifndef NDEBUG
  163. last_prop_added_to_block_ = prop.first;
  164. #endif /* !NDEBUG */
  165. }
  166. return properties_block_->Finish();
  167. }
  168. void LogPropertiesCollectionError(Logger* info_log, const std::string& method,
  169. const std::string& name) {
  170. assert(method == "Add" || method == "Finish");
  171. std::string msg =
  172. "Encountered error when calling TablePropertiesCollector::" + method +
  173. "() with collector name: " + name;
  174. ROCKS_LOG_ERROR(info_log, "%s", msg.c_str());
  175. }
  176. bool NotifyCollectTableCollectorsOnAdd(
  177. const Slice& key, const Slice& value, uint64_t file_size,
  178. const std::vector<std::unique_ptr<InternalTblPropColl>>& collectors,
  179. Logger* info_log) {
  180. bool all_succeeded = true;
  181. for (auto& collector : collectors) {
  182. Status s = collector->InternalAdd(key, value, file_size);
  183. all_succeeded = all_succeeded && s.ok();
  184. if (!s.ok()) {
  185. LogPropertiesCollectionError(info_log, "Add" /* method */,
  186. collector->Name());
  187. }
  188. }
  189. return all_succeeded;
  190. }
  191. void NotifyCollectTableCollectorsOnBlockAdd(
  192. const std::vector<std::unique_ptr<InternalTblPropColl>>& collectors,
  193. const uint64_t block_uncomp_bytes,
  194. const uint64_t block_compressed_bytes_fast,
  195. const uint64_t block_compressed_bytes_slow) {
  196. for (auto& collector : collectors) {
  197. collector->BlockAdd(block_uncomp_bytes, block_compressed_bytes_fast,
  198. block_compressed_bytes_slow);
  199. }
  200. }
  201. bool NotifyCollectTableCollectorsOnFinish(
  202. const std::vector<std::unique_ptr<InternalTblPropColl>>& collectors,
  203. Logger* info_log, PropertyBlockBuilder* builder,
  204. UserCollectedProperties& user_collected_properties,
  205. UserCollectedProperties& readable_properties) {
  206. bool all_succeeded = true;
  207. for (auto& collector : collectors) {
  208. UserCollectedProperties user_properties;
  209. Status s = collector->Finish(&user_properties);
  210. if (s.ok()) {
  211. for (const auto& prop : collector->GetReadableProperties()) {
  212. readable_properties.insert(prop);
  213. }
  214. #ifndef NDEBUG
  215. // Check different user properties collectors are not adding properties of
  216. // the same name.
  217. for (const auto& pair : user_properties) {
  218. assert(user_collected_properties.find(pair.first) ==
  219. user_collected_properties.end());
  220. }
  221. #endif /* !NDEBUG */
  222. user_collected_properties.merge(user_properties);
  223. } else {
  224. LogPropertiesCollectionError(info_log, "Finish" /* method */,
  225. collector->Name());
  226. if (all_succeeded) {
  227. all_succeeded = false;
  228. }
  229. }
  230. }
  231. builder->Add(user_collected_properties);
  232. return all_succeeded;
  233. }
  234. Status ParsePropertiesBlock(
  235. const ImmutableOptions& ioptions, uint64_t offset, Block& properties_block,
  236. std::unique_ptr<TableProperties>& new_table_properties) {
  237. std::unique_ptr<MetaBlockIter> iter(properties_block.NewMetaIterator());
  238. // All pre-defined properties of type uint64_t
  239. std::unordered_map<std::string, uint64_t*> predefined_uint64_properties = {
  240. {TablePropertiesNames::kOriginalFileNumber,
  241. &new_table_properties->orig_file_number},
  242. {TablePropertiesNames::kDataSize, &new_table_properties->data_size},
  243. {TablePropertiesNames::kIndexSize, &new_table_properties->index_size},
  244. {TablePropertiesNames::kIndexPartitions,
  245. &new_table_properties->index_partitions},
  246. {TablePropertiesNames::kTopLevelIndexSize,
  247. &new_table_properties->top_level_index_size},
  248. {TablePropertiesNames::kIndexKeyIsUserKey,
  249. &new_table_properties->index_key_is_user_key},
  250. {TablePropertiesNames::kIndexValueIsDeltaEncoded,
  251. &new_table_properties->index_value_is_delta_encoded},
  252. {TablePropertiesNames::kFilterSize, &new_table_properties->filter_size},
  253. {TablePropertiesNames::kRawKeySize, &new_table_properties->raw_key_size},
  254. {TablePropertiesNames::kRawValueSize,
  255. &new_table_properties->raw_value_size},
  256. {TablePropertiesNames::kNumDataBlocks,
  257. &new_table_properties->num_data_blocks},
  258. {TablePropertiesNames::kNumEntries, &new_table_properties->num_entries},
  259. {TablePropertiesNames::kNumFilterEntries,
  260. &new_table_properties->num_filter_entries},
  261. {TablePropertiesNames::kDeletedKeys,
  262. &new_table_properties->num_deletions},
  263. {TablePropertiesNames::kMergeOperands,
  264. &new_table_properties->num_merge_operands},
  265. {TablePropertiesNames::kNumRangeDeletions,
  266. &new_table_properties->num_range_deletions},
  267. {TablePropertiesNames::kFormatVersion,
  268. &new_table_properties->format_version},
  269. {TablePropertiesNames::kFixedKeyLen,
  270. &new_table_properties->fixed_key_len},
  271. {TablePropertiesNames::kColumnFamilyId,
  272. &new_table_properties->column_family_id},
  273. {TablePropertiesNames::kCreationTime,
  274. &new_table_properties->creation_time},
  275. {TablePropertiesNames::kOldestKeyTime,
  276. &new_table_properties->oldest_key_time},
  277. {TablePropertiesNames::kNewestKeyTime,
  278. &new_table_properties->newest_key_time},
  279. {TablePropertiesNames::kFileCreationTime,
  280. &new_table_properties->file_creation_time},
  281. {TablePropertiesNames::kSlowCompressionEstimatedDataSize,
  282. &new_table_properties->slow_compression_estimated_data_size},
  283. {TablePropertiesNames::kFastCompressionEstimatedDataSize,
  284. &new_table_properties->fast_compression_estimated_data_size},
  285. {TablePropertiesNames::kTailStartOffset,
  286. &new_table_properties->tail_start_offset},
  287. {TablePropertiesNames::kUserDefinedTimestampsPersisted,
  288. &new_table_properties->user_defined_timestamps_persisted},
  289. {TablePropertiesNames::kKeyLargestSeqno,
  290. &new_table_properties->key_largest_seqno},
  291. {TablePropertiesNames::kKeySmallestSeqno,
  292. &new_table_properties->key_smallest_seqno},
  293. };
  294. Status s;
  295. std::string last_key;
  296. for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {
  297. s = iter->status();
  298. if (!s.ok()) {
  299. break;
  300. }
  301. auto key = iter->key().ToString();
  302. // properties block should be strictly sorted with no duplicate key.
  303. if (!last_key.empty() &&
  304. BytewiseComparator()->Compare(key, last_key) <= 0) {
  305. s = Status::Corruption("properties unsorted");
  306. break;
  307. }
  308. last_key = key;
  309. auto raw_val = iter->value();
  310. auto pos = predefined_uint64_properties.find(key);
  311. if (key == ExternalSstFilePropertyNames::kGlobalSeqno) {
  312. new_table_properties->external_sst_file_global_seqno_offset =
  313. offset + iter->ValueOffset();
  314. }
  315. if (pos != predefined_uint64_properties.end()) {
  316. if (key == TablePropertiesNames::kDeletedKeys ||
  317. key == TablePropertiesNames::kMergeOperands) {
  318. // Insert in user-collected properties for API backwards compatibility
  319. new_table_properties->user_collected_properties.insert(
  320. {key, raw_val.ToString()});
  321. }
  322. // handle predefined rocksdb properties
  323. uint64_t val;
  324. if (!GetVarint64(&raw_val, &val)) {
  325. // skip malformed value
  326. auto error_msg =
  327. "Detect malformed value in properties meta-block:"
  328. "\tkey: " +
  329. key + "\tval: " + raw_val.ToString();
  330. ROCKS_LOG_ERROR(ioptions.logger, "%s", error_msg.c_str());
  331. continue;
  332. }
  333. *(pos->second) = val;
  334. } else if (key == TablePropertiesNames::kDbId) {
  335. new_table_properties->db_id = raw_val.ToString();
  336. } else if (key == TablePropertiesNames::kDbSessionId) {
  337. new_table_properties->db_session_id = raw_val.ToString();
  338. } else if (key == TablePropertiesNames::kDbHostId) {
  339. new_table_properties->db_host_id = raw_val.ToString();
  340. } else if (key == TablePropertiesNames::kFilterPolicy) {
  341. new_table_properties->filter_policy_name = raw_val.ToString();
  342. } else if (key == TablePropertiesNames::kColumnFamilyName) {
  343. new_table_properties->column_family_name = raw_val.ToString();
  344. } else if (key == TablePropertiesNames::kComparator) {
  345. new_table_properties->comparator_name = raw_val.ToString();
  346. } else if (key == TablePropertiesNames::kMergeOperator) {
  347. new_table_properties->merge_operator_name = raw_val.ToString();
  348. } else if (key == TablePropertiesNames::kPrefixExtractorName) {
  349. new_table_properties->prefix_extractor_name = raw_val.ToString();
  350. } else if (key == TablePropertiesNames::kPropertyCollectors) {
  351. new_table_properties->property_collectors_names = raw_val.ToString();
  352. } else if (key == TablePropertiesNames::kCompression) {
  353. new_table_properties->compression_name = raw_val.ToString();
  354. } else if (key == TablePropertiesNames::kCompressionOptions) {
  355. new_table_properties->compression_options = raw_val.ToString();
  356. } else if (key == TablePropertiesNames::kSequenceNumberTimeMapping) {
  357. new_table_properties->seqno_to_time_mapping = raw_val.ToString();
  358. } else {
  359. // handle user-collected properties
  360. new_table_properties->user_collected_properties.insert(
  361. {key, raw_val.ToString()});
  362. }
  363. }
  364. return s;
  365. }
  366. // FIXME: should be a parameter for reading table properties to use persistent
  367. // cache?
  368. Status ReadTablePropertiesHelper(
  369. const ReadOptions& ro, const BlockHandle& handle,
  370. RandomAccessFileReader* file, FilePrefetchBuffer* prefetch_buffer,
  371. const Footer& footer, const ImmutableOptions& ioptions,
  372. std::unique_ptr<TableProperties>* table_properties,
  373. MemoryAllocator* memory_allocator) {
  374. assert(table_properties);
  375. Status s;
  376. bool retry = false;
  377. while (true) {
  378. BlockContents block_contents;
  379. size_t len = handle.size() + footer.GetBlockTrailerSize();
  380. // If this is an external SST file ingested with write_global_seqno set to
  381. // true, then we expect the checksum mismatch because checksum was written
  382. // by SstFileWriter, but its global seqno in the properties block may have
  383. // been changed during ingestion. For this reason, we initially read
  384. // and process without checksum verification, then later try checksum
  385. // verification so that if it fails, we can copy to a temporary buffer with
  386. // global seqno set to its original value, i.e. 0, and attempt checksum
  387. // verification again.
  388. if (!retry) {
  389. ReadOptions modified_ro = ro;
  390. modified_ro.verify_checksums = false;
  391. BlockFetcher block_fetcher(
  392. file, prefetch_buffer, footer, modified_ro, handle, &block_contents,
  393. ioptions, false /* decompress */, false /*maybe_compressed*/,
  394. BlockType::kProperties, nullptr /*decompressor*/,
  395. PersistentCacheOptions::kEmpty, memory_allocator);
  396. s = block_fetcher.ReadBlockContents();
  397. if (!s.ok()) {
  398. return s;
  399. }
  400. assert(block_fetcher.GetBlockSizeWithTrailer() == len);
  401. TEST_SYNC_POINT_CALLBACK("ReadTablePropertiesHelper:0",
  402. &block_contents.data);
  403. } else {
  404. assert(s.IsCorruption());
  405. // If retrying, use a stronger file system read to check and correct
  406. // data corruption
  407. IOOptions opts;
  408. IODebugContext dbg;
  409. if (PrepareIOFromReadOptions(ro, ioptions.clock, opts, &dbg) !=
  410. IOStatus::OK()) {
  411. return s;
  412. }
  413. opts.verify_and_reconstruct_read = true;
  414. std::unique_ptr<char[]> data(new char[len]);
  415. Slice result;
  416. IOStatus io_s = file->Read(opts, handle.offset(), len, &result,
  417. data.get(), nullptr, &dbg);
  418. RecordTick(ioptions.stats, FILE_READ_CORRUPTION_RETRY_COUNT);
  419. if (!io_s.ok()) {
  420. ROCKS_LOG_INFO(ioptions.info_log,
  421. "Reading properties block failed - %s",
  422. io_s.ToString().c_str());
  423. // Return the original corruption error as that's more serious
  424. return s;
  425. }
  426. if (result.size() < len) {
  427. return Status::Corruption("Reading properties block failed - " +
  428. std::to_string(result.size()) +
  429. " bytes read");
  430. }
  431. RecordTick(ioptions.stats, FILE_READ_CORRUPTION_RETRY_SUCCESS_COUNT);
  432. block_contents = BlockContents(std::move(data), handle.size());
  433. }
  434. uint64_t block_size = block_contents.data.size();
  435. Block properties_block(std::move(block_contents));
  436. std::unique_ptr<TableProperties> new_table_properties{new TableProperties};
  437. s = ParsePropertiesBlock(ioptions, handle.offset(), properties_block,
  438. new_table_properties);
  439. // Modified version of BlockFetcher checksum verification
  440. // (See write_global_seqno comment above)
  441. if (s.ok() && footer.GetBlockTrailerSize() > 0) {
  442. s = VerifyBlockChecksum(footer, properties_block.data(), block_size,
  443. file->file_name(), handle.offset());
  444. if (s.IsCorruption()) {
  445. if (new_table_properties->external_sst_file_global_seqno_offset != 0) {
  446. std::string tmp_buf(properties_block.data(), len);
  447. uint64_t global_seqno_offset =
  448. new_table_properties->external_sst_file_global_seqno_offset -
  449. handle.offset();
  450. EncodeFixed64(&tmp_buf[static_cast<size_t>(global_seqno_offset)], 0);
  451. s = VerifyBlockChecksum(footer, tmp_buf.data(), block_size,
  452. file->file_name(), handle.offset());
  453. }
  454. }
  455. }
  456. // If we detected a corruption and the file system supports verification
  457. // and reconstruction, retry the read
  458. if (s.IsCorruption() && !retry &&
  459. CheckFSFeatureSupport(ioptions.fs.get(),
  460. FSSupportedOps::kVerifyAndReconstructRead)) {
  461. retry = true;
  462. } else {
  463. if (s.ok()) {
  464. *table_properties = std::move(new_table_properties);
  465. }
  466. break;
  467. }
  468. }
  469. return s;
  470. }
  471. Status ReadTableProperties(RandomAccessFileReader* file, uint64_t file_size,
  472. uint64_t table_magic_number,
  473. const ImmutableOptions& ioptions,
  474. const ReadOptions& read_options,
  475. std::unique_ptr<TableProperties>* properties,
  476. MemoryAllocator* memory_allocator,
  477. FilePrefetchBuffer* prefetch_buffer) {
  478. BlockHandle block_handle;
  479. Footer footer;
  480. Status s =
  481. FindMetaBlockInFile(file, file_size, table_magic_number, ioptions,
  482. read_options, kPropertiesBlockName, &block_handle,
  483. memory_allocator, prefetch_buffer, &footer);
  484. if (!s.ok()) {
  485. return s;
  486. }
  487. if (!block_handle.IsNull()) {
  488. s = ReadTablePropertiesHelper(read_options, block_handle, file,
  489. prefetch_buffer, footer, ioptions, properties,
  490. memory_allocator);
  491. } else {
  492. s = Status::NotFound();
  493. }
  494. return s;
  495. }
  496. Status FindOptionalMetaBlock(InternalIterator* meta_index_iter,
  497. const std::string& meta_block_name,
  498. BlockHandle* block_handle) {
  499. assert(block_handle != nullptr);
  500. meta_index_iter->Seek(meta_block_name);
  501. if (meta_index_iter->status().ok()) {
  502. if (meta_index_iter->Valid() && meta_index_iter->key() == meta_block_name) {
  503. Slice v = meta_index_iter->value();
  504. return block_handle->DecodeFrom(&v);
  505. } else if (meta_block_name == kPropertiesBlockName) {
  506. // Have to try old name for compatibility
  507. meta_index_iter->Seek(kPropertiesBlockOldName);
  508. if (meta_index_iter->status().ok() && meta_index_iter->Valid() &&
  509. meta_index_iter->key() == kPropertiesBlockOldName) {
  510. Slice v = meta_index_iter->value();
  511. return block_handle->DecodeFrom(&v);
  512. }
  513. }
  514. }
  515. // else
  516. *block_handle = BlockHandle::NullBlockHandle();
  517. return meta_index_iter->status();
  518. }
  519. Status FindMetaBlock(InternalIterator* meta_index_iter,
  520. const std::string& meta_block_name,
  521. BlockHandle* block_handle) {
  522. Status s =
  523. FindOptionalMetaBlock(meta_index_iter, meta_block_name, block_handle);
  524. if (s.ok() && block_handle->IsNull()) {
  525. return Status::Corruption("Cannot find the meta block", meta_block_name);
  526. } else {
  527. return s;
  528. }
  529. }
  530. Status ReadMetaIndexBlockInFile(RandomAccessFileReader* file,
  531. uint64_t file_size, uint64_t table_magic_number,
  532. const ImmutableOptions& ioptions,
  533. const ReadOptions& read_options,
  534. BlockContents* metaindex_contents,
  535. MemoryAllocator* memory_allocator,
  536. FilePrefetchBuffer* prefetch_buffer,
  537. Footer* footer_out) {
  538. Footer footer;
  539. IOOptions opts;
  540. IODebugContext dbg;
  541. Status s;
  542. s = file->PrepareIOOptions(read_options, opts, &dbg);
  543. if (!s.ok()) {
  544. return s;
  545. }
  546. s = ReadFooterFromFile(opts, file, *ioptions.fs, prefetch_buffer, file_size,
  547. &footer, table_magic_number, ioptions.stats);
  548. if (!s.ok()) {
  549. return s;
  550. }
  551. if (footer_out) {
  552. *footer_out = footer;
  553. }
  554. auto metaindex_handle = footer.metaindex_handle();
  555. return BlockFetcher(file, prefetch_buffer, footer, read_options,
  556. metaindex_handle, metaindex_contents, ioptions,
  557. false /* do decompression */, false /*maybe_compressed*/,
  558. BlockType::kMetaIndex, nullptr /*decompressor*/,
  559. PersistentCacheOptions::kEmpty, memory_allocator)
  560. .ReadBlockContents();
  561. }
  562. Status FindMetaBlockInFile(
  563. RandomAccessFileReader* file, uint64_t file_size,
  564. uint64_t table_magic_number, const ImmutableOptions& ioptions,
  565. const ReadOptions& read_options, const std::string& meta_block_name,
  566. BlockHandle* block_handle, MemoryAllocator* memory_allocator,
  567. FilePrefetchBuffer* prefetch_buffer, Footer* footer_out) {
  568. BlockContents metaindex_contents;
  569. auto s = ReadMetaIndexBlockInFile(
  570. file, file_size, table_magic_number, ioptions, read_options,
  571. &metaindex_contents, memory_allocator, prefetch_buffer, footer_out);
  572. if (!s.ok()) {
  573. return s;
  574. }
  575. // meta blocks are never compressed. Need to add uncompress logic if we are to
  576. // compress it.
  577. Block metaindex_block(std::move(metaindex_contents));
  578. std::unique_ptr<InternalIterator> meta_iter;
  579. meta_iter.reset(metaindex_block.NewMetaIterator());
  580. return FindMetaBlock(meta_iter.get(), meta_block_name, block_handle);
  581. }
  582. Status ReadMetaBlock(RandomAccessFileReader* file,
  583. FilePrefetchBuffer* prefetch_buffer, uint64_t file_size,
  584. uint64_t table_magic_number,
  585. const ImmutableOptions& ioptions,
  586. const ReadOptions& read_options,
  587. const std::string& meta_block_name, BlockType block_type,
  588. BlockContents* contents,
  589. MemoryAllocator* memory_allocator) {
  590. // TableProperties requires special handling because of checksum issues.
  591. // Call ReadTableProperties instead for that case.
  592. assert(block_type != BlockType::kProperties);
  593. BlockHandle block_handle;
  594. Footer footer;
  595. Status status =
  596. FindMetaBlockInFile(file, file_size, table_magic_number, ioptions,
  597. read_options, meta_block_name, &block_handle,
  598. memory_allocator, prefetch_buffer, &footer);
  599. if (!status.ok()) {
  600. return status;
  601. }
  602. return BlockFetcher(file, prefetch_buffer, footer, read_options, block_handle,
  603. contents, ioptions, false /* decompress */,
  604. false /*maybe_compressed*/, block_type,
  605. nullptr /*decompressor*/, PersistentCacheOptions::kEmpty,
  606. memory_allocator)
  607. .ReadBlockContents();
  608. }
  609. } // namespace ROCKSDB_NAMESPACE