format.cc 28 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766
  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. //
  6. // Copyright (c) 2011 The LevelDB Authors. All rights reserved.
  7. // Use of this source code is governed by a BSD-style license that can be
  8. // found in the LICENSE file. See the AUTHORS file for names of contributors.
  9. #include "table/format.h"
  10. #include <cinttypes>
  11. #include <cstdint>
  12. #include <string>
  13. #include "block_fetcher.h"
  14. #include "file/random_access_file_reader.h"
  15. #include "memory/memory_allocator_impl.h"
  16. #include "monitoring/perf_context_imp.h"
  17. #include "monitoring/statistics_impl.h"
  18. #include "options/options_helper.h"
  19. #include "port/likely.h"
  20. #include "rocksdb/env.h"
  21. #include "rocksdb/options.h"
  22. #include "rocksdb/table.h"
  23. #include "table/block_based/block.h"
  24. #include "table/block_based/block_based_table_reader.h"
  25. #include "table/persistent_cache_helper.h"
  26. #include "unique_id_impl.h"
  27. #include "util/cast_util.h"
  28. #include "util/coding.h"
  29. #include "util/compression.h"
  30. #include "util/crc32c.h"
  31. #include "util/hash.h"
  32. #include "util/stop_watch.h"
  33. #include "util/string_util.h"
  34. #include "util/xxhash.h"
  35. namespace ROCKSDB_NAMESPACE {
  36. const char* kHostnameForDbHostId = "__hostname__";
  37. bool ShouldReportDetailedTime(Env* env, Statistics* stats) {
  38. return env != nullptr && stats != nullptr &&
  39. stats->get_stats_level() > kExceptDetailedTimers;
  40. }
  41. void BlockHandle::EncodeTo(std::string* dst) const {
  42. // Sanity check that all fields have been set
  43. assert(offset_ != ~uint64_t{0});
  44. assert(size_ != ~uint64_t{0});
  45. PutVarint64Varint64(dst, offset_, size_);
  46. }
  47. char* BlockHandle::EncodeTo(char* dst) const {
  48. // Sanity check that all fields have been set
  49. assert(offset_ != ~uint64_t{0});
  50. assert(size_ != ~uint64_t{0});
  51. char* cur = EncodeVarint64(dst, offset_);
  52. cur = EncodeVarint64(cur, size_);
  53. return cur;
  54. }
  55. Status BlockHandle::DecodeFrom(Slice* input) {
  56. if (GetVarint64(input, &offset_) && GetVarint64(input, &size_)) {
  57. return Status::OK();
  58. } else {
  59. // reset in case failure after partially decoding
  60. offset_ = 0;
  61. size_ = 0;
  62. return Status::Corruption("bad block handle");
  63. }
  64. }
  65. Status BlockHandle::DecodeSizeFrom(uint64_t _offset, Slice* input) {
  66. if (GetVarint64(input, &size_)) {
  67. offset_ = _offset;
  68. return Status::OK();
  69. } else {
  70. // reset in case failure after partially decoding
  71. offset_ = 0;
  72. size_ = 0;
  73. return Status::Corruption("bad block handle");
  74. }
  75. }
  76. // Return a string that contains the copy of handle.
  77. std::string BlockHandle::ToString(bool hex) const {
  78. std::string handle_str;
  79. EncodeTo(&handle_str);
  80. if (hex) {
  81. return Slice(handle_str).ToString(true);
  82. } else {
  83. return handle_str;
  84. }
  85. }
  86. const BlockHandle BlockHandle::kNullBlockHandle(0, 0);
  87. void IndexValue::EncodeTo(std::string* dst, bool have_first_key,
  88. const BlockHandle* previous_handle) const {
  89. if (previous_handle) {
  90. // WART: this is specific to Block-based table
  91. assert(handle.offset() == previous_handle->offset() +
  92. previous_handle->size() +
  93. BlockBasedTable::kBlockTrailerSize);
  94. PutVarsignedint64(dst, handle.size() - previous_handle->size());
  95. } else {
  96. handle.EncodeTo(dst);
  97. }
  98. assert(dst->size() != 0);
  99. if (have_first_key) {
  100. PutLengthPrefixedSlice(dst, first_internal_key);
  101. }
  102. }
  103. Status IndexValue::DecodeFrom(Slice* input, bool have_first_key,
  104. const BlockHandle* previous_handle) {
  105. if (previous_handle) {
  106. int64_t delta;
  107. if (!GetVarsignedint64(input, &delta)) {
  108. return Status::Corruption("bad delta-encoded index value");
  109. }
  110. // WART: this is specific to Block-based table
  111. handle = BlockHandle(previous_handle->offset() + previous_handle->size() +
  112. BlockBasedTable::kBlockTrailerSize,
  113. previous_handle->size() + delta);
  114. } else {
  115. Status s = handle.DecodeFrom(input);
  116. if (!s.ok()) {
  117. return s;
  118. }
  119. }
  120. if (!have_first_key) {
  121. first_internal_key = Slice();
  122. } else if (!GetLengthPrefixedSlice(input, &first_internal_key)) {
  123. return Status::Corruption("bad first key in block info");
  124. }
  125. return Status::OK();
  126. }
  127. std::string IndexValue::ToString(bool hex, bool have_first_key) const {
  128. std::string s;
  129. EncodeTo(&s, have_first_key, nullptr);
  130. if (hex) {
  131. return Slice(s).ToString(true);
  132. } else {
  133. return s;
  134. }
  135. }
  136. namespace {
  137. inline bool IsLegacyFooterFormat(uint64_t magic_number) {
  138. return magic_number == kLegacyBlockBasedTableMagicNumber ||
  139. magic_number == kLegacyPlainTableMagicNumber;
  140. }
  141. inline uint64_t UpconvertLegacyFooterFormat(uint64_t magic_number) {
  142. if (magic_number == kLegacyBlockBasedTableMagicNumber) {
  143. return kBlockBasedTableMagicNumber;
  144. }
  145. if (magic_number == kLegacyPlainTableMagicNumber) {
  146. return kPlainTableMagicNumber;
  147. }
  148. assert(false);
  149. return magic_number;
  150. }
  151. inline uint64_t DownconvertToLegacyFooterFormat(uint64_t magic_number) {
  152. if (magic_number == kBlockBasedTableMagicNumber) {
  153. return kLegacyBlockBasedTableMagicNumber;
  154. }
  155. if (magic_number == kPlainTableMagicNumber) {
  156. return kLegacyPlainTableMagicNumber;
  157. }
  158. assert(false);
  159. return magic_number;
  160. }
  161. inline uint8_t BlockTrailerSizeForMagicNumber(uint64_t magic_number) {
  162. if (magic_number == kBlockBasedTableMagicNumber ||
  163. magic_number == kLegacyBlockBasedTableMagicNumber) {
  164. return static_cast<uint8_t>(BlockBasedTable::kBlockTrailerSize);
  165. } else {
  166. return 0;
  167. }
  168. }
  169. // Footer format, in three parts:
  170. // * Part1
  171. // -> format_version == 0 (inferred from legacy magic number)
  172. // <empty> (0 bytes)
  173. // -> format_version >= 1
  174. // checksum type (char, 1 byte)
  175. // * Part2
  176. // -> format_version <= 5
  177. // metaindex handle (varint64 offset, varint64 size)
  178. // index handle (varint64 offset, varint64 size)
  179. // <zero padding> for part2 size = 2 * BlockHandle::kMaxEncodedLength = 40
  180. // - This padding is unchecked/ignored
  181. // -> format_version >= 6
  182. // extended magic number (4 bytes) = 0x3e 0x00 0x7a 0x00
  183. // - Also surely invalid (size 0) handles if interpreted as older version
  184. // - (Helps ensure a corrupted format_version doesn't get us far with no
  185. // footer checksum.)
  186. // footer_checksum (uint32LE, 4 bytes)
  187. // - Checksum of above checksum type of whole footer, with this field
  188. // set to all zeros.
  189. // base_context_checksum (uint32LE, 4 bytes)
  190. // metaindex block size (uint32LE, 4 bytes)
  191. // - Assumed to be immediately before footer, < 4GB
  192. // <zero padding> (24 bytes, reserved for future use)
  193. // - Brings part2 size also to 40 bytes
  194. // - Checked that last eight bytes == 0, so reserved for a future
  195. // incompatible feature (but under format_version=6)
  196. // * Part3
  197. // -> format_version == 0 (inferred from legacy magic number)
  198. // legacy magic number (8 bytes)
  199. // -> format_version >= 1 (inferred from NOT legacy magic number)
  200. // format_version (uint32LE, 4 bytes), also called "footer version"
  201. // newer magic number (8 bytes)
  202. const std::array<char, 4> kExtendedMagic{{0x3e, 0x00, 0x7a, 0x00}};
  203. constexpr size_t kFooterPart2Size = 2 * BlockHandle::kMaxEncodedLength;
  204. } // namespace
  205. Status FooterBuilder::Build(uint64_t magic_number, uint32_t format_version,
  206. uint64_t footer_offset, ChecksumType checksum_type,
  207. const BlockHandle& metaindex_handle,
  208. const BlockHandle& index_handle,
  209. uint32_t base_context_checksum) {
  210. assert(magic_number != Footer::kNullTableMagicNumber);
  211. assert(IsSupportedFormatVersion(format_version) ||
  212. TEST_AllowUnsupportedFormatVersion());
  213. char* part2;
  214. char* part3;
  215. if (format_version > 0) {
  216. slice_ = Slice(data_.data(), Footer::kNewVersionsEncodedLength);
  217. // Generate parts 1 and 3
  218. char* cur = data_.data();
  219. // Part 1
  220. *(cur++) = checksum_type;
  221. // Part 2
  222. part2 = cur;
  223. // Skip over part 2 for now
  224. cur += kFooterPart2Size;
  225. // Part 3
  226. part3 = cur;
  227. EncodeFixed32(cur, format_version);
  228. cur += 4;
  229. EncodeFixed64(cur, magic_number);
  230. assert(cur + 8 == slice_.data() + slice_.size());
  231. } else {
  232. slice_ = Slice(data_.data(), Footer::kVersion0EncodedLength);
  233. // Legacy SST files use kCRC32c checksum but it's not stored in footer.
  234. assert(checksum_type == kNoChecksum || checksum_type == kCRC32c);
  235. // Generate part 3 (part 1 empty, skip part 2 for now)
  236. part2 = data_.data();
  237. part3 = part2 + kFooterPart2Size;
  238. char* cur = part3;
  239. // Use legacy magic numbers to indicate format_version=0, for
  240. // compatibility. No other cases should use format_version=0.
  241. EncodeFixed64(cur, DownconvertToLegacyFooterFormat(magic_number));
  242. assert(cur + 8 == slice_.data() + slice_.size());
  243. }
  244. if (format_version >= 6) {
  245. if (BlockTrailerSizeForMagicNumber(magic_number) != 0) {
  246. // base context checksum required for table formats with block checksums
  247. assert(base_context_checksum != 0);
  248. assert(ChecksumModifierForContext(base_context_checksum, 0) != 0);
  249. } else {
  250. // base context checksum not used
  251. assert(base_context_checksum == 0);
  252. assert(ChecksumModifierForContext(base_context_checksum, 0) == 0);
  253. }
  254. // Start populating Part 2
  255. char* cur = data_.data() + /* part 1 size */ 1;
  256. // Set extended magic of part2
  257. std::copy(kExtendedMagic.begin(), kExtendedMagic.end(), cur);
  258. cur += kExtendedMagic.size();
  259. // Fill checksum data with zeros (for later computing checksum)
  260. char* checksum_data = cur;
  261. EncodeFixed32(cur, 0);
  262. cur += 4;
  263. // Save base context checksum
  264. EncodeFixed32(cur, base_context_checksum);
  265. cur += 4;
  266. // Compute and save metaindex size
  267. uint32_t metaindex_size = static_cast<uint32_t>(metaindex_handle.size());
  268. if (metaindex_size != metaindex_handle.size()) {
  269. return Status::NotSupported("Metaindex block size > 4GB");
  270. }
  271. // Metaindex must be adjacent to footer
  272. assert(metaindex_size == 0 ||
  273. metaindex_handle.offset() + metaindex_handle.size() ==
  274. footer_offset - BlockTrailerSizeForMagicNumber(magic_number));
  275. EncodeFixed32(cur, metaindex_size);
  276. cur += 4;
  277. // Zero pad remainder (for future use)
  278. std::fill_n(cur, 24U, char{0});
  279. assert(cur + 24 == part3);
  280. // Compute checksum, add context
  281. uint32_t checksum = ComputeBuiltinChecksum(
  282. checksum_type, data_.data(), Footer::kNewVersionsEncodedLength);
  283. checksum +=
  284. ChecksumModifierForContext(base_context_checksum, footer_offset);
  285. // Store it
  286. EncodeFixed32(checksum_data, checksum);
  287. } else {
  288. // Base context checksum not used
  289. assert(!FormatVersionUsesContextChecksum(format_version));
  290. // Should be left empty
  291. assert(base_context_checksum == 0);
  292. assert(ChecksumModifierForContext(base_context_checksum, 0) == 0);
  293. // Populate all of part 2
  294. char* cur = part2;
  295. cur = metaindex_handle.EncodeTo(cur);
  296. cur = index_handle.EncodeTo(cur);
  297. // Zero pad remainder
  298. std::fill(cur, part3, char{0});
  299. }
  300. return Status::OK();
  301. }
  302. Status Footer::DecodeFrom(Slice input, uint64_t input_offset,
  303. uint64_t enforce_table_magic_number) {
  304. // Only decode to unused Footer
  305. assert(table_magic_number_ == kNullTableMagicNumber);
  306. assert(input != nullptr);
  307. assert(input.size() >= kMinEncodedLength);
  308. const char* magic_ptr = input.data() + input.size() - kMagicNumberLengthByte;
  309. uint64_t magic = DecodeFixed64(magic_ptr);
  310. // We check for legacy formats here and silently upconvert them
  311. bool legacy = IsLegacyFooterFormat(magic);
  312. if (legacy) {
  313. magic = UpconvertLegacyFooterFormat(magic);
  314. }
  315. if (enforce_table_magic_number != 0 && enforce_table_magic_number != magic) {
  316. return Status::Corruption("Bad table magic number: expected " +
  317. std::to_string(enforce_table_magic_number) +
  318. ", found " + std::to_string(magic));
  319. }
  320. table_magic_number_ = magic;
  321. block_trailer_size_ = BlockTrailerSizeForMagicNumber(magic);
  322. // Parse Part3
  323. const char* part3_ptr = magic_ptr;
  324. uint32_t computed_checksum = 0;
  325. uint64_t footer_offset = 0;
  326. if (legacy) {
  327. // The size is already asserted to be at least kMinEncodedLength
  328. // at the beginning of the function
  329. input.remove_prefix(input.size() - kVersion0EncodedLength);
  330. format_version_ = 0 /* legacy */;
  331. checksum_type_ = kCRC32c;
  332. } else {
  333. part3_ptr = magic_ptr - 4;
  334. format_version_ = DecodeFixed32(part3_ptr);
  335. if (UNLIKELY(!IsSupportedFormatVersion(format_version_) &&
  336. !TEST_AllowUnsupportedFormatVersion())) {
  337. return Status::Corruption("Corrupt or unsupported format_version: " +
  338. std::to_string(format_version_));
  339. }
  340. // All known format versions >= 1 occupy exactly this many bytes.
  341. if (UNLIKELY(input.size() < kNewVersionsEncodedLength)) {
  342. return Status::Corruption("Input is too short to be an SST file");
  343. }
  344. uint64_t adjustment = input.size() - kNewVersionsEncodedLength;
  345. input.remove_prefix(adjustment);
  346. footer_offset = input_offset + adjustment;
  347. // Parse Part1
  348. char chksum = input.data()[0];
  349. checksum_type_ = lossless_cast<ChecksumType>(chksum);
  350. if (UNLIKELY(!IsSupportedChecksumType(checksum_type()))) {
  351. return Status::Corruption("Corrupt or unsupported checksum type: " +
  352. std::to_string(lossless_cast<uint8_t>(chksum)));
  353. }
  354. // This is the most convenient place to compute the checksum
  355. if (checksum_type_ != kNoChecksum && format_version_ >= 6) {
  356. std::array<char, kNewVersionsEncodedLength> copy_without_checksum;
  357. std::copy_n(input.data(), kNewVersionsEncodedLength,
  358. copy_without_checksum.data());
  359. EncodeFixed32(&copy_without_checksum[5], 0); // Clear embedded checksum
  360. computed_checksum =
  361. ComputeBuiltinChecksum(checksum_type(), copy_without_checksum.data(),
  362. kNewVersionsEncodedLength);
  363. }
  364. // Consume checksum type field
  365. input.remove_prefix(1);
  366. }
  367. // Parse Part2
  368. if (format_version_ >= 6) {
  369. Slice ext_magic(input.data(), 4);
  370. if (UNLIKELY(ext_magic.compare(Slice(kExtendedMagic.data(),
  371. kExtendedMagic.size())) != 0)) {
  372. return Status::Corruption("Bad extended magic number: 0x" +
  373. ext_magic.ToString(/*hex*/ true));
  374. }
  375. input.remove_prefix(4);
  376. uint32_t stored_checksum = 0, metaindex_size = 0;
  377. bool success;
  378. success = GetFixed32(&input, &stored_checksum);
  379. assert(success);
  380. success = GetFixed32(&input, &base_context_checksum_);
  381. assert(success);
  382. if (UNLIKELY(ChecksumModifierForContext(base_context_checksum_, 0) == 0)) {
  383. return Status::Corruption("Invalid base context checksum");
  384. }
  385. computed_checksum +=
  386. ChecksumModifierForContext(base_context_checksum_, footer_offset);
  387. if (UNLIKELY(computed_checksum != stored_checksum)) {
  388. return Status::Corruption("Footer at " + std::to_string(footer_offset) +
  389. " checksum mismatch");
  390. }
  391. success = GetFixed32(&input, &metaindex_size);
  392. assert(success);
  393. (void)success;
  394. uint64_t metaindex_end = footer_offset - GetBlockTrailerSize();
  395. metaindex_handle_ =
  396. BlockHandle(metaindex_end - metaindex_size, metaindex_size);
  397. // Mark unpopulated
  398. index_handle_ = BlockHandle::NullBlockHandle();
  399. // 16 bytes of unchecked reserved padding
  400. input.remove_prefix(16U);
  401. // 8 bytes of checked reserved padding (expected to be zero unless using a
  402. // future feature).
  403. uint64_t reserved = 0;
  404. success = GetFixed64(&input, &reserved);
  405. assert(success);
  406. if (UNLIKELY(reserved != 0)) {
  407. return Status::NotSupported(
  408. "File uses a future feature not supported in this version");
  409. }
  410. // End of part 2
  411. assert(input.data() == part3_ptr);
  412. } else {
  413. // format_version_ < 6
  414. Status result = metaindex_handle_.DecodeFrom(&input);
  415. if (result.ok()) {
  416. result = index_handle_.DecodeFrom(&input);
  417. }
  418. if (!result.ok()) {
  419. return result;
  420. }
  421. // Padding in part2 is ignored
  422. }
  423. return Status::OK();
  424. }
  425. std::string Footer::ToString() const {
  426. std::string result;
  427. result.reserve(1024);
  428. result.append("metaindex handle: " + metaindex_handle_.ToString() +
  429. " offset: " + std::to_string(metaindex_handle_.offset()) +
  430. " size: " + std::to_string(metaindex_handle_.size()) + "\n ");
  431. result.append("index handle: " + index_handle_.ToString() +
  432. " offset: " + std::to_string(index_handle_.offset()) +
  433. " size: " + std::to_string(index_handle_.size()) + "\n ");
  434. result.append("table_magic_number: " + std::to_string(table_magic_number_) +
  435. "\n ");
  436. if (!IsLegacyFooterFormat(table_magic_number_)) {
  437. result.append("format version: " + std::to_string(format_version_) + "\n");
  438. }
  439. return result;
  440. }
  441. bool& TEST_AllowUnsupportedFormatVersion() {
  442. static bool allow = false;
  443. return allow;
  444. }
  445. static Status ReadFooterFromFileInternal(
  446. const IOOptions& opts, RandomAccessFileReader* file, FileSystem& fs,
  447. FilePrefetchBuffer* prefetch_buffer, uint64_t expected_file_size,
  448. Footer* footer, uint64_t enforce_table_magic_number) {
  449. uint64_t file_size_from_file_system = 0;
  450. Status s;
  451. // Prefer the more efficient FSRandomAccessFile::GetFileSize when available
  452. s = file->file()->GetFileSize(&file_size_from_file_system);
  453. if (!s.ok()) {
  454. // Fall back on FileSystem::GetFileSize on failure
  455. s = fs.GetFileSize(file->file_name(), IOOptions(),
  456. &file_size_from_file_system, nullptr);
  457. if (!s.ok()) {
  458. return s;
  459. }
  460. }
  461. if (expected_file_size != file_size_from_file_system) {
  462. // When file is opened during DB Open, the expected file size is from
  463. // manifest. Otherwise it is not guaranteed.
  464. return Status::Corruption("Sst file size mismatch between expected " +
  465. std::to_string(expected_file_size) +
  466. " and file system " +
  467. std::to_string(file_size_from_file_system) +
  468. " sstable: " + file->file_name());
  469. }
  470. if (expected_file_size < Footer::kMinEncodedLength) {
  471. return Status::Corruption("file is too short (" +
  472. std::to_string(expected_file_size) +
  473. " bytes) to be an "
  474. "sstable: " +
  475. file->file_name());
  476. }
  477. std::array<char, Footer::kMaxEncodedLength + 1> footer_buf;
  478. AlignedBuf internal_buf;
  479. Slice footer_input;
  480. uint64_t read_offset = (expected_file_size > Footer::kMaxEncodedLength)
  481. ? expected_file_size - Footer::kMaxEncodedLength
  482. : 0;
  483. // TODO: Need to pass appropriate deadline to TryReadFromCache(). Right now,
  484. // there is no readahead for point lookups, so TryReadFromCache will fail if
  485. // the required data is not in the prefetch buffer. Once deadline is enabled
  486. // for iterator, TryReadFromCache might do a readahead. Revisit to see if we
  487. // need to pass a timeout at that point
  488. // TODO: rate limit footer reads.
  489. if (prefetch_buffer == nullptr ||
  490. !prefetch_buffer->TryReadFromCache(opts, file, read_offset,
  491. Footer::kMaxEncodedLength,
  492. &footer_input, nullptr)) {
  493. if (file->use_direct_io()) {
  494. s = file->Read(opts, read_offset, Footer::kMaxEncodedLength,
  495. &footer_input, nullptr, &internal_buf);
  496. } else {
  497. s = file->Read(opts, read_offset, Footer::kMaxEncodedLength,
  498. &footer_input, footer_buf.data(), nullptr);
  499. }
  500. if (!s.ok()) {
  501. return s;
  502. }
  503. }
  504. TEST_SYNC_POINT_CALLBACK("ReadFooterFromFileInternal:0", &footer_input);
  505. // Check that we actually read the whole footer from the file.
  506. if (footer_input.size() < Footer::kMinEncodedLength) {
  507. return Status::Corruption(
  508. "The number of bytes read for Footer input " +
  509. std::to_string(footer_input.size()) +
  510. " is smaller than minimum footer encoded length: " +
  511. std::to_string(Footer::kMinEncodedLength) + " for file " +
  512. file->file_name() + "\n");
  513. }
  514. s = footer->DecodeFrom(footer_input, read_offset, enforce_table_magic_number);
  515. if (!s.ok()) {
  516. s = Status::CopyAppendMessage(s, " in ", file->file_name());
  517. return s;
  518. }
  519. return Status::OK();
  520. }
  521. Status ReadFooterFromFile(const IOOptions& opts, RandomAccessFileReader* file,
  522. FileSystem& fs, FilePrefetchBuffer* prefetch_buffer,
  523. uint64_t expected_file_size, Footer* footer,
  524. uint64_t enforce_table_magic_number,
  525. Statistics* stats) {
  526. Status s = ReadFooterFromFileInternal(opts, file, fs, prefetch_buffer,
  527. expected_file_size, footer,
  528. enforce_table_magic_number);
  529. if (s.IsCorruption() &&
  530. CheckFSFeatureSupport(&fs, FSSupportedOps::kVerifyAndReconstructRead)) {
  531. IOOptions new_opts = opts;
  532. new_opts.verify_and_reconstruct_read = true;
  533. footer->Reset();
  534. s = ReadFooterFromFileInternal(new_opts, file, fs,
  535. /*prefetch_buffer=*/nullptr,
  536. expected_file_size, footer,
  537. enforce_table_magic_number);
  538. RecordTick(stats, FILE_READ_CORRUPTION_RETRY_COUNT);
  539. if (s.ok()) {
  540. RecordTick(stats, FILE_READ_CORRUPTION_RETRY_SUCCESS_COUNT);
  541. }
  542. }
  543. return s;
  544. }
  545. namespace {
  546. // Custom handling for the last byte of a block, to avoid invoking streaming
  547. // API to get an effective block checksum. This function is its own inverse
  548. // because it uses xor.
  549. inline uint32_t ModifyChecksumForLastByte(uint32_t checksum, char last_byte) {
  550. // This strategy bears some resemblance to extending a CRC checksum by one
  551. // more byte, except we don't need to re-mix the input checksum as long as
  552. // we do this step only once (per checksum).
  553. const uint32_t kRandomPrime = 0x6b9083d9;
  554. return checksum ^ lossless_cast<uint8_t>(last_byte) * kRandomPrime;
  555. }
  556. } // namespace
  557. uint32_t ComputeBuiltinChecksum(ChecksumType type, const char* data,
  558. size_t data_size) {
  559. switch (type) {
  560. case kCRC32c:
  561. return crc32c::Mask(crc32c::Value(data, data_size));
  562. case kxxHash:
  563. return XXH32(data, data_size, /*seed*/ 0);
  564. case kxxHash64:
  565. return Lower32of64(XXH64(data, data_size, /*seed*/ 0));
  566. case kXXH3: {
  567. if (data_size == 0) {
  568. // Special case because of special handling for last byte, not
  569. // present in this case. Can be any value different from other
  570. // small input size checksums.
  571. return 0;
  572. } else {
  573. // See corresponding code in ComputeBuiltinChecksumWithLastByte
  574. uint32_t v = Lower32of64(XXH3_64bits(data, data_size - 1));
  575. return ModifyChecksumForLastByte(v, data[data_size - 1]);
  576. }
  577. }
  578. default: // including kNoChecksum
  579. return 0;
  580. }
  581. }
  582. uint32_t ComputeBuiltinChecksumWithLastByte(ChecksumType type, const char* data,
  583. size_t data_size, char last_byte) {
  584. switch (type) {
  585. case kCRC32c: {
  586. uint32_t crc = crc32c::Value(data, data_size);
  587. // Extend to cover last byte (compression type)
  588. crc = crc32c::Extend(crc, &last_byte, 1);
  589. return crc32c::Mask(crc);
  590. }
  591. case kxxHash: {
  592. XXH32_state_t* const state = XXH32_createState();
  593. XXH32_reset(state, 0);
  594. XXH32_update(state, data, data_size);
  595. // Extend to cover last byte (compression type)
  596. XXH32_update(state, &last_byte, 1);
  597. uint32_t v = XXH32_digest(state);
  598. XXH32_freeState(state);
  599. return v;
  600. }
  601. case kxxHash64: {
  602. XXH64_state_t* const state = XXH64_createState();
  603. XXH64_reset(state, 0);
  604. XXH64_update(state, data, data_size);
  605. // Extend to cover last byte (compression type)
  606. XXH64_update(state, &last_byte, 1);
  607. uint32_t v = Lower32of64(XXH64_digest(state));
  608. XXH64_freeState(state);
  609. return v;
  610. }
  611. case kXXH3: {
  612. // XXH3 is a complicated hash function that is extremely fast on
  613. // contiguous input, but that makes its streaming support rather
  614. // complex. It is worth custom handling of the last byte (`type`)
  615. // in order to avoid allocating a large state object and bringing
  616. // that code complexity into CPU working set.
  617. uint32_t v = Lower32of64(XXH3_64bits(data, data_size));
  618. return ModifyChecksumForLastByte(v, last_byte);
  619. }
  620. default: // including kNoChecksum
  621. return 0;
  622. }
  623. }
  624. Status DecompressBlockData(Decompressor::Args& args, Decompressor& decompressor,
  625. BlockContents* out_contents,
  626. const ImmutableOptions& ioptions,
  627. MemoryAllocator* allocator) {
  628. assert(args.compression_type != kNoCompression && "Invalid compression type");
  629. StopWatchNano timer(ioptions.clock,
  630. ShouldReportDetailedTime(ioptions.env, ioptions.stats));
  631. Status s = decompressor.ExtractUncompressedSize(args);
  632. if (UNLIKELY(!s.ok())) {
  633. return s;
  634. }
  635. CacheAllocationPtr ubuf = AllocateBlock(args.uncompressed_size, allocator);
  636. s = decompressor.DecompressBlock(args, ubuf.get());
  637. if (UNLIKELY(!s.ok())) {
  638. return s;
  639. }
  640. *out_contents = BlockContents(std::move(ubuf), args.uncompressed_size);
  641. if (ShouldReportDetailedTime(ioptions.env, ioptions.stats)) {
  642. RecordTimeToHistogram(ioptions.stats, DECOMPRESSION_TIMES_NANOS,
  643. timer.ElapsedNanos());
  644. }
  645. RecordTick(ioptions.stats, BYTES_DECOMPRESSED_FROM,
  646. args.compressed_data.size());
  647. RecordTick(ioptions.stats, BYTES_DECOMPRESSED_TO, out_contents->data.size());
  648. RecordTick(ioptions.stats, NUMBER_BLOCK_DECOMPRESSED);
  649. TEST_SYNC_POINT_CALLBACK("DecompressBlockData:TamperWithReturnValue",
  650. static_cast<void*>(&s));
  651. TEST_SYNC_POINT_CALLBACK("DecompressBlockData:TamperWithDecompressionOutput",
  652. static_cast<void*>(out_contents));
  653. return s;
  654. }
  655. Status DecompressBlockData(const char* data, size_t size, CompressionType type,
  656. Decompressor& decompressor,
  657. BlockContents* out_contents,
  658. const ImmutableOptions& ioptions,
  659. MemoryAllocator* allocator,
  660. Decompressor::ManagedWorkingArea* working_area) {
  661. Decompressor::Args args;
  662. args.compressed_data = Slice(data, size);
  663. args.compression_type = type;
  664. args.working_area = working_area;
  665. return DecompressBlockData(args, decompressor, out_contents, ioptions,
  666. allocator);
  667. }
  668. Status DecompressSerializedBlock(const char* data, size_t size,
  669. CompressionType type,
  670. Decompressor& decompressor,
  671. BlockContents* out_contents,
  672. const ImmutableOptions& ioptions,
  673. MemoryAllocator* allocator) {
  674. assert(data[size] != kNoCompression);
  675. assert(data[size] == static_cast<char>(type));
  676. return DecompressBlockData(data, size, type, decompressor, out_contents,
  677. ioptions, allocator);
  678. }
  679. Status DecompressSerializedBlock(Decompressor::Args& args,
  680. Decompressor& decompressor,
  681. BlockContents* out_contents,
  682. const ImmutableOptions& ioptions,
  683. MemoryAllocator* allocator) {
  684. assert(args.compressed_data.data()[args.compressed_data.size()] !=
  685. kNoCompression);
  686. assert(args.compressed_data.data()[args.compressed_data.size()] ==
  687. static_cast<char>(args.compression_type));
  688. return DecompressBlockData(args, decompressor, out_contents, ioptions,
  689. allocator);
  690. }
  691. // Replace the contents of db_host_id with the actual hostname, if db_host_id
  692. // matches the keyword kHostnameForDbHostId
  693. Status ReifyDbHostIdProperty(Env* env, std::string* db_host_id) {
  694. assert(db_host_id);
  695. if (*db_host_id == kHostnameForDbHostId) {
  696. Status s = env->GetHostNameString(db_host_id);
  697. if (!s.ok()) {
  698. db_host_id->clear();
  699. }
  700. return s;
  701. }
  702. return Status::OK();
  703. }
  704. } // namespace ROCKSDB_NAMESPACE