format.cc 17 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465
  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 <string>
  12. #include "block_fetcher.h"
  13. #include "file/random_access_file_reader.h"
  14. #include "logging/logging.h"
  15. #include "memory/memory_allocator.h"
  16. #include "monitoring/perf_context_imp.h"
  17. #include "monitoring/statistics.h"
  18. #include "rocksdb/env.h"
  19. #include "table/block_based/block.h"
  20. #include "table/block_based/block_based_table_reader.h"
  21. #include "table/persistent_cache_helper.h"
  22. #include "util/coding.h"
  23. #include "util/compression.h"
  24. #include "util/crc32c.h"
  25. #include "util/stop_watch.h"
  26. #include "util/string_util.h"
  27. namespace ROCKSDB_NAMESPACE {
  28. extern const uint64_t kLegacyBlockBasedTableMagicNumber;
  29. extern const uint64_t kBlockBasedTableMagicNumber;
  30. #ifndef ROCKSDB_LITE
  31. extern const uint64_t kLegacyPlainTableMagicNumber;
  32. extern const uint64_t kPlainTableMagicNumber;
  33. #else
  34. // ROCKSDB_LITE doesn't have plain table
  35. const uint64_t kLegacyPlainTableMagicNumber = 0;
  36. const uint64_t kPlainTableMagicNumber = 0;
  37. #endif
  38. bool ShouldReportDetailedTime(Env* env, Statistics* stats) {
  39. return env != nullptr && stats != nullptr &&
  40. stats->get_stats_level() > kExceptDetailedTimers;
  41. }
  42. void BlockHandle::EncodeTo(std::string* dst) const {
  43. // Sanity check that all fields have been set
  44. assert(offset_ != ~static_cast<uint64_t>(0));
  45. assert(size_ != ~static_cast<uint64_t>(0));
  46. PutVarint64Varint64(dst, offset_, size_);
  47. }
  48. Status BlockHandle::DecodeFrom(Slice* input) {
  49. if (GetVarint64(input, &offset_) && GetVarint64(input, &size_)) {
  50. return Status::OK();
  51. } else {
  52. // reset in case failure after partially decoding
  53. offset_ = 0;
  54. size_ = 0;
  55. return Status::Corruption("bad block handle");
  56. }
  57. }
  58. Status BlockHandle::DecodeSizeFrom(uint64_t _offset, Slice* input) {
  59. if (GetVarint64(input, &size_)) {
  60. offset_ = _offset;
  61. return Status::OK();
  62. } else {
  63. // reset in case failure after partially decoding
  64. offset_ = 0;
  65. size_ = 0;
  66. return Status::Corruption("bad block handle");
  67. }
  68. }
  69. // Return a string that contains the copy of handle.
  70. std::string BlockHandle::ToString(bool hex) const {
  71. std::string handle_str;
  72. EncodeTo(&handle_str);
  73. if (hex) {
  74. return Slice(handle_str).ToString(true);
  75. } else {
  76. return handle_str;
  77. }
  78. }
  79. const BlockHandle BlockHandle::kNullBlockHandle(0, 0);
  80. void IndexValue::EncodeTo(std::string* dst, bool have_first_key,
  81. const BlockHandle* previous_handle) const {
  82. if (previous_handle) {
  83. assert(handle.offset() == previous_handle->offset() +
  84. previous_handle->size() + kBlockTrailerSize);
  85. PutVarsignedint64(dst, handle.size() - previous_handle->size());
  86. } else {
  87. handle.EncodeTo(dst);
  88. }
  89. assert(dst->size() != 0);
  90. if (have_first_key) {
  91. PutLengthPrefixedSlice(dst, first_internal_key);
  92. }
  93. }
  94. Status IndexValue::DecodeFrom(Slice* input, bool have_first_key,
  95. const BlockHandle* previous_handle) {
  96. if (previous_handle) {
  97. int64_t delta;
  98. if (!GetVarsignedint64(input, &delta)) {
  99. return Status::Corruption("bad delta-encoded index value");
  100. }
  101. handle = BlockHandle(
  102. previous_handle->offset() + previous_handle->size() + kBlockTrailerSize,
  103. previous_handle->size() + delta);
  104. } else {
  105. Status s = handle.DecodeFrom(input);
  106. if (!s.ok()) {
  107. return s;
  108. }
  109. }
  110. if (!have_first_key) {
  111. first_internal_key = Slice();
  112. } else if (!GetLengthPrefixedSlice(input, &first_internal_key)) {
  113. return Status::Corruption("bad first key in block info");
  114. }
  115. return Status::OK();
  116. }
  117. std::string IndexValue::ToString(bool hex, bool have_first_key) const {
  118. std::string s;
  119. EncodeTo(&s, have_first_key, nullptr);
  120. if (hex) {
  121. return Slice(s).ToString(true);
  122. } else {
  123. return s;
  124. }
  125. }
  126. namespace {
  127. inline bool IsLegacyFooterFormat(uint64_t magic_number) {
  128. return magic_number == kLegacyBlockBasedTableMagicNumber ||
  129. magic_number == kLegacyPlainTableMagicNumber;
  130. }
  131. inline uint64_t UpconvertLegacyFooterFormat(uint64_t magic_number) {
  132. if (magic_number == kLegacyBlockBasedTableMagicNumber) {
  133. return kBlockBasedTableMagicNumber;
  134. }
  135. if (magic_number == kLegacyPlainTableMagicNumber) {
  136. return kPlainTableMagicNumber;
  137. }
  138. assert(false);
  139. return 0;
  140. }
  141. } // namespace
  142. // legacy footer format:
  143. // metaindex handle (varint64 offset, varint64 size)
  144. // index handle (varint64 offset, varint64 size)
  145. // <padding> to make the total size 2 * BlockHandle::kMaxEncodedLength
  146. // table_magic_number (8 bytes)
  147. // new footer format:
  148. // checksum type (char, 1 byte)
  149. // metaindex handle (varint64 offset, varint64 size)
  150. // index handle (varint64 offset, varint64 size)
  151. // <padding> to make the total size 2 * BlockHandle::kMaxEncodedLength + 1
  152. // footer version (4 bytes)
  153. // table_magic_number (8 bytes)
  154. void Footer::EncodeTo(std::string* dst) const {
  155. assert(HasInitializedTableMagicNumber());
  156. if (IsLegacyFooterFormat(table_magic_number())) {
  157. // has to be default checksum with legacy footer
  158. assert(checksum_ == kCRC32c);
  159. const size_t original_size = dst->size();
  160. metaindex_handle_.EncodeTo(dst);
  161. index_handle_.EncodeTo(dst);
  162. dst->resize(original_size + 2 * BlockHandle::kMaxEncodedLength); // Padding
  163. PutFixed32(dst, static_cast<uint32_t>(table_magic_number() & 0xffffffffu));
  164. PutFixed32(dst, static_cast<uint32_t>(table_magic_number() >> 32));
  165. assert(dst->size() == original_size + kVersion0EncodedLength);
  166. } else {
  167. const size_t original_size = dst->size();
  168. dst->push_back(static_cast<char>(checksum_));
  169. metaindex_handle_.EncodeTo(dst);
  170. index_handle_.EncodeTo(dst);
  171. dst->resize(original_size + kNewVersionsEncodedLength - 12); // Padding
  172. PutFixed32(dst, version());
  173. PutFixed32(dst, static_cast<uint32_t>(table_magic_number() & 0xffffffffu));
  174. PutFixed32(dst, static_cast<uint32_t>(table_magic_number() >> 32));
  175. assert(dst->size() == original_size + kNewVersionsEncodedLength);
  176. }
  177. }
  178. Footer::Footer(uint64_t _table_magic_number, uint32_t _version)
  179. : version_(_version),
  180. checksum_(kCRC32c),
  181. table_magic_number_(_table_magic_number) {
  182. // This should be guaranteed by constructor callers
  183. assert(!IsLegacyFooterFormat(_table_magic_number) || version_ == 0);
  184. }
  185. Status Footer::DecodeFrom(Slice* input) {
  186. assert(!HasInitializedTableMagicNumber());
  187. assert(input != nullptr);
  188. assert(input->size() >= kMinEncodedLength);
  189. const char* magic_ptr =
  190. input->data() + input->size() - kMagicNumberLengthByte;
  191. const uint32_t magic_lo = DecodeFixed32(magic_ptr);
  192. const uint32_t magic_hi = DecodeFixed32(magic_ptr + 4);
  193. uint64_t magic = ((static_cast<uint64_t>(magic_hi) << 32) |
  194. (static_cast<uint64_t>(magic_lo)));
  195. // We check for legacy formats here and silently upconvert them
  196. bool legacy = IsLegacyFooterFormat(magic);
  197. if (legacy) {
  198. magic = UpconvertLegacyFooterFormat(magic);
  199. }
  200. set_table_magic_number(magic);
  201. if (legacy) {
  202. // The size is already asserted to be at least kMinEncodedLength
  203. // at the beginning of the function
  204. input->remove_prefix(input->size() - kVersion0EncodedLength);
  205. version_ = 0 /* legacy */;
  206. checksum_ = kCRC32c;
  207. } else {
  208. version_ = DecodeFixed32(magic_ptr - 4);
  209. // Footer version 1 and higher will always occupy exactly this many bytes.
  210. // It consists of the checksum type, two block handles, padding,
  211. // a version number, and a magic number
  212. if (input->size() < kNewVersionsEncodedLength) {
  213. return Status::Corruption("input is too short to be an sstable");
  214. } else {
  215. input->remove_prefix(input->size() - kNewVersionsEncodedLength);
  216. }
  217. uint32_t chksum;
  218. if (!GetVarint32(input, &chksum)) {
  219. return Status::Corruption("bad checksum type");
  220. }
  221. checksum_ = static_cast<ChecksumType>(chksum);
  222. }
  223. Status result = metaindex_handle_.DecodeFrom(input);
  224. if (result.ok()) {
  225. result = index_handle_.DecodeFrom(input);
  226. }
  227. if (result.ok()) {
  228. // We skip over any leftover data (just padding for now) in "input"
  229. const char* end = magic_ptr + kMagicNumberLengthByte;
  230. *input = Slice(end, input->data() + input->size() - end);
  231. }
  232. return result;
  233. }
  234. std::string Footer::ToString() const {
  235. std::string result;
  236. result.reserve(1024);
  237. bool legacy = IsLegacyFooterFormat(table_magic_number_);
  238. if (legacy) {
  239. result.append("metaindex handle: " + metaindex_handle_.ToString() + "\n ");
  240. result.append("index handle: " + index_handle_.ToString() + "\n ");
  241. result.append("table_magic_number: " +
  242. ROCKSDB_NAMESPACE::ToString(table_magic_number_) + "\n ");
  243. } else {
  244. result.append("checksum: " + ROCKSDB_NAMESPACE::ToString(checksum_) +
  245. "\n ");
  246. result.append("metaindex handle: " + metaindex_handle_.ToString() + "\n ");
  247. result.append("index handle: " + index_handle_.ToString() + "\n ");
  248. result.append("footer version: " + ROCKSDB_NAMESPACE::ToString(version_) +
  249. "\n ");
  250. result.append("table_magic_number: " +
  251. ROCKSDB_NAMESPACE::ToString(table_magic_number_) + "\n ");
  252. }
  253. return result;
  254. }
  255. Status ReadFooterFromFile(RandomAccessFileReader* file,
  256. FilePrefetchBuffer* prefetch_buffer,
  257. uint64_t file_size, Footer* footer,
  258. uint64_t enforce_table_magic_number) {
  259. if (file_size < Footer::kMinEncodedLength) {
  260. return Status::Corruption("file is too short (" + ToString(file_size) +
  261. " bytes) to be an "
  262. "sstable: " +
  263. file->file_name());
  264. }
  265. char footer_space[Footer::kMaxEncodedLength];
  266. Slice footer_input;
  267. size_t read_offset =
  268. (file_size > Footer::kMaxEncodedLength)
  269. ? static_cast<size_t>(file_size - Footer::kMaxEncodedLength)
  270. : 0;
  271. Status s;
  272. if (prefetch_buffer == nullptr ||
  273. !prefetch_buffer->TryReadFromCache(read_offset, Footer::kMaxEncodedLength,
  274. &footer_input)) {
  275. s = file->Read(read_offset, Footer::kMaxEncodedLength, &footer_input,
  276. footer_space);
  277. if (!s.ok()) return s;
  278. }
  279. // Check that we actually read the whole footer from the file. It may be
  280. // that size isn't correct.
  281. if (footer_input.size() < Footer::kMinEncodedLength) {
  282. return Status::Corruption("file is too short (" + ToString(file_size) +
  283. " bytes) to be an "
  284. "sstable" +
  285. file->file_name());
  286. }
  287. s = footer->DecodeFrom(&footer_input);
  288. if (!s.ok()) {
  289. return s;
  290. }
  291. if (enforce_table_magic_number != 0 &&
  292. enforce_table_magic_number != footer->table_magic_number()) {
  293. return Status::Corruption(
  294. "Bad table magic number: expected " +
  295. ToString(enforce_table_magic_number) + ", found " +
  296. ToString(footer->table_magic_number()) + " in " + file->file_name());
  297. }
  298. return Status::OK();
  299. }
  300. Status UncompressBlockContentsForCompressionType(
  301. const UncompressionInfo& uncompression_info, const char* data, size_t n,
  302. BlockContents* contents, uint32_t format_version,
  303. const ImmutableCFOptions& ioptions, MemoryAllocator* allocator) {
  304. CacheAllocationPtr ubuf;
  305. assert(uncompression_info.type() != kNoCompression &&
  306. "Invalid compression type");
  307. StopWatchNano timer(ioptions.env, ShouldReportDetailedTime(
  308. ioptions.env, ioptions.statistics));
  309. int decompress_size = 0;
  310. switch (uncompression_info.type()) {
  311. case kSnappyCompression: {
  312. size_t ulength = 0;
  313. static char snappy_corrupt_msg[] =
  314. "Snappy not supported or corrupted Snappy compressed block contents";
  315. if (!Snappy_GetUncompressedLength(data, n, &ulength)) {
  316. return Status::Corruption(snappy_corrupt_msg);
  317. }
  318. ubuf = AllocateBlock(ulength, allocator);
  319. if (!Snappy_Uncompress(data, n, ubuf.get())) {
  320. return Status::Corruption(snappy_corrupt_msg);
  321. }
  322. *contents = BlockContents(std::move(ubuf), ulength);
  323. break;
  324. }
  325. case kZlibCompression:
  326. ubuf = Zlib_Uncompress(
  327. uncompression_info, data, n, &decompress_size,
  328. GetCompressFormatForVersion(kZlibCompression, format_version),
  329. allocator);
  330. if (!ubuf) {
  331. static char zlib_corrupt_msg[] =
  332. "Zlib not supported or corrupted Zlib compressed block contents";
  333. return Status::Corruption(zlib_corrupt_msg);
  334. }
  335. *contents = BlockContents(std::move(ubuf), decompress_size);
  336. break;
  337. case kBZip2Compression:
  338. ubuf = BZip2_Uncompress(
  339. data, n, &decompress_size,
  340. GetCompressFormatForVersion(kBZip2Compression, format_version),
  341. allocator);
  342. if (!ubuf) {
  343. static char bzip2_corrupt_msg[] =
  344. "Bzip2 not supported or corrupted Bzip2 compressed block contents";
  345. return Status::Corruption(bzip2_corrupt_msg);
  346. }
  347. *contents = BlockContents(std::move(ubuf), decompress_size);
  348. break;
  349. case kLZ4Compression:
  350. ubuf = LZ4_Uncompress(
  351. uncompression_info, data, n, &decompress_size,
  352. GetCompressFormatForVersion(kLZ4Compression, format_version),
  353. allocator);
  354. if (!ubuf) {
  355. static char lz4_corrupt_msg[] =
  356. "LZ4 not supported or corrupted LZ4 compressed block contents";
  357. return Status::Corruption(lz4_corrupt_msg);
  358. }
  359. *contents = BlockContents(std::move(ubuf), decompress_size);
  360. break;
  361. case kLZ4HCCompression:
  362. ubuf = LZ4_Uncompress(
  363. uncompression_info, data, n, &decompress_size,
  364. GetCompressFormatForVersion(kLZ4HCCompression, format_version),
  365. allocator);
  366. if (!ubuf) {
  367. static char lz4hc_corrupt_msg[] =
  368. "LZ4HC not supported or corrupted LZ4HC compressed block contents";
  369. return Status::Corruption(lz4hc_corrupt_msg);
  370. }
  371. *contents = BlockContents(std::move(ubuf), decompress_size);
  372. break;
  373. case kXpressCompression:
  374. // XPRESS allocates memory internally, thus no support for custom
  375. // allocator.
  376. ubuf.reset(XPRESS_Uncompress(data, n, &decompress_size));
  377. if (!ubuf) {
  378. static char xpress_corrupt_msg[] =
  379. "XPRESS not supported or corrupted XPRESS compressed block "
  380. "contents";
  381. return Status::Corruption(xpress_corrupt_msg);
  382. }
  383. *contents = BlockContents(std::move(ubuf), decompress_size);
  384. break;
  385. case kZSTD:
  386. case kZSTDNotFinalCompression:
  387. ubuf = ZSTD_Uncompress(uncompression_info, data, n, &decompress_size,
  388. allocator);
  389. if (!ubuf) {
  390. static char zstd_corrupt_msg[] =
  391. "ZSTD not supported or corrupted ZSTD compressed block contents";
  392. return Status::Corruption(zstd_corrupt_msg);
  393. }
  394. *contents = BlockContents(std::move(ubuf), decompress_size);
  395. break;
  396. default:
  397. return Status::Corruption("bad block type");
  398. }
  399. if (ShouldReportDetailedTime(ioptions.env, ioptions.statistics)) {
  400. RecordTimeToHistogram(ioptions.statistics, DECOMPRESSION_TIMES_NANOS,
  401. timer.ElapsedNanos());
  402. }
  403. RecordTimeToHistogram(ioptions.statistics, BYTES_DECOMPRESSED,
  404. contents->data.size());
  405. RecordTick(ioptions.statistics, NUMBER_BLOCK_DECOMPRESSED);
  406. return Status::OK();
  407. }
  408. //
  409. // The 'data' points to the raw block contents that was read in from file.
  410. // This method allocates a new heap buffer and the raw block
  411. // contents are uncompresed into this buffer. This
  412. // buffer is returned via 'result' and it is upto the caller to
  413. // free this buffer.
  414. // format_version is the block format as defined in include/rocksdb/table.h
  415. Status UncompressBlockContents(const UncompressionInfo& uncompression_info,
  416. const char* data, size_t n,
  417. BlockContents* contents, uint32_t format_version,
  418. const ImmutableCFOptions& ioptions,
  419. MemoryAllocator* allocator) {
  420. assert(data[n] != kNoCompression);
  421. assert(data[n] == uncompression_info.type());
  422. return UncompressBlockContentsForCompressionType(uncompression_info, data, n,
  423. contents, format_version,
  424. ioptions, allocator);
  425. }
  426. } // namespace ROCKSDB_NAMESPACE