diff --git a/db/db_impl/db_impl.cc b/db/db_impl/db_impl.cc index 4433c310ea..5dee778532 100644 --- a/db/db_impl/db_impl.cc +++ b/db/db_impl/db_impl.cc @@ -5824,6 +5824,7 @@ Status DBImpl::VerifyChecksumInternal(const ReadOptions& read_options, return s; } } + // FIXME? What does it mean if read_options.verify_checksums == false? // TODO: simplify using GetRefedColumnFamilySet? std::vector cfd_list; diff --git a/db/db_properties_test.cc b/db/db_properties_test.cc index d689686c3d..a7faa64147 100644 --- a/db/db_properties_test.cc +++ b/db/db_properties_test.cc @@ -2348,6 +2348,9 @@ TEST_F(DBPropertiesTest, TableMetaIndexKeys) { EXPECT_EQ("rocksdb.hashindex.prefixes", PopMetaIndexKey(meta_iter.get())); } + if (bbto->format_version >= 6) { + EXPECT_EQ("rocksdb.index", PopMetaIndexKey(meta_iter.get())); + } } EXPECT_EQ("rocksdb.properties", PopMetaIndexKey(meta_iter.get())); EXPECT_EQ("NOT_FOUND", PopMetaIndexKey(meta_iter.get())); diff --git a/env/env_test.cc b/env/env_test.cc index 4462b95b81..fb23bae130 100644 --- a/env/env_test.cc +++ b/env/env_test.cc @@ -3054,7 +3054,7 @@ TEST_F(EnvTest, PortGenerateRfcUuid) { VerifyRfcUuids(t.ids); } -// Test the atomic, linear generation of GenerateRawUuid +// Test the atomic, linear generation of GenerateRawUniqueId TEST_F(EnvTest, GenerateRawUniqueId) { struct MyStressTest : public NoDuplicateMiniStressTest { diff --git a/include/rocksdb/table.h b/include/rocksdb/table.h index 6e8f60577a..d19a95fa8e 100644 --- a/include/rocksdb/table.h +++ b/include/rocksdb/table.h @@ -47,7 +47,10 @@ struct EnvOptions; // Types of checksums to use for checking integrity of logical blocks within // files. All checksums currently use 32 bits of checking power (1 in 4B -// chance of failing to detect random corruption). +// chance of failing to detect random corruption). Traditionally, the actual +// checking power can be far from ideal if the corruption is due to misplaced +// data (e.g. physical blocks out of order in a file, or from another file), +// which is fixed in format_version=6 (see below). enum ChecksumType : char { kNoChecksum = 0x0, kCRC32c = 0x1, @@ -512,6 +515,9 @@ struct BlockBasedTableOptions { // 5 -- Can be read by RocksDB's versions since 6.6.0. Full and partitioned // filters use a generally faster and more accurate Bloom filter // implementation, with a different schema. + // 6 -- Modified the file footer and checksum matching so that SST data + // misplaced within or between files is as likely to fail checksum + // verification as random corruption. Also checksum-protects SST footer. uint32_t format_version = 5; // Store index blocks on disk in compressed format. Changing this option to diff --git a/table/block_based/block_based_table_builder.cc b/table/block_based/block_based_table_builder.cc index d3c70536da..26e071abb0 100644 --- a/table/block_based/block_based_table_builder.cc +++ b/table/block_based/block_based_table_builder.cc @@ -263,7 +263,9 @@ class BlockBasedTableBuilder::BlockBasedTablePropertiesCollector struct BlockBasedTableBuilder::Rep { const ImmutableOptions ioptions; - const MutableCFOptions moptions; + // BEGIN from MutableCFOptions + std::shared_ptr prefix_extractor; + // END from MutableCFOptions const BlockBasedTableOptions table_options; const InternalKeyComparator& internal_comparator; // Size in bytes for the user-defined timestamps. @@ -361,6 +363,9 @@ struct BlockBasedTableBuilder::Rep { // all blocks after data blocks till the end of the SST file. uint64_t tail_size; + // See class Footer + uint32_t base_context_checksum; + uint64_t get_offset() { return offset.load(std::memory_order_relaxed); } void set_offset(uint64_t o) { offset.store(o, std::memory_order_relaxed); } @@ -389,6 +394,12 @@ struct BlockBasedTableBuilder::Rep { // to false, and this is ensured by io_status_mutex, so no special memory // order for io_status_ok is required. if (io_status_ok.load(std::memory_order_relaxed)) { +#ifdef ROCKSDB_ASSERT_STATUS_CHECKED // Avoid unnecessary lock acquisition + auto ios = CopyIOStatus(); + ios.PermitUncheckedError(); + // Assume no races in unit tests + assert(ios.ok()); +#endif // ROCKSDB_ASSERT_STATUS_CHECKED return IOStatus::OK(); } else { return CopyIOStatus(); @@ -429,7 +440,7 @@ struct BlockBasedTableBuilder::Rep { Rep(const BlockBasedTableOptions& table_opt, const TableBuilderOptions& tbo, WritableFileWriter* f) : ioptions(tbo.ioptions), - moptions(tbo.moptions), + prefix_extractor(tbo.moptions.prefix_extractor), table_options(table_opt), internal_comparator(tbo.internal_comparator), ts_sz(tbo.internal_comparator.user_comparator()->timestamp_size()), @@ -456,7 +467,7 @@ struct BlockBasedTableBuilder::Rep { BlockBasedTableOptions::kDataBlockBinarySearch /* index_type */, 0.75 /* data_block_hash_table_util_ratio */, ts_sz, persist_user_defined_timestamps), - internal_prefix_transform(tbo.moptions.prefix_extractor.get()), + internal_prefix_transform(prefix_extractor.get()), compression_type(tbo.compression_type), sample_for_compression(tbo.moptions.sample_for_compression), compressible_input_data_bytes(0), @@ -557,7 +568,7 @@ struct BlockBasedTableBuilder::Rep { } filter_builder.reset(CreateFilterBlockBuilder( - ioptions, moptions, filter_context, + ioptions, tbo.moptions, filter_context, use_delta_encoding_for_index_values, p_index_builder_, ts_sz, persist_user_defined_timestamps)); } @@ -573,7 +584,7 @@ struct BlockBasedTableBuilder::Rep { table_properties_collectors.emplace_back( new BlockBasedTablePropertiesCollector( table_options.index_type, table_options.whole_key_filtering, - moptions.prefix_extractor != nullptr)); + prefix_extractor != nullptr)); if (ts_sz > 0 && persist_user_defined_timestamps) { table_properties_collectors.emplace_back( new TimestampTablePropertiesCollector( @@ -597,6 +608,17 @@ struct BlockBasedTableBuilder::Rep { if (!ReifyDbHostIdProperty(ioptions.env, &props.db_host_id).ok()) { ROCKS_LOG_INFO(ioptions.logger, "db_host_id property will not be set"); } + + if (FormatVersionUsesContextChecksum(table_options.format_version)) { + // Must be non-zero and semi- or quasi-random + // TODO: ideally guaranteed different for related files (e.g. use file + // number and db_session, for benefit of SstFileWriter) + do { + base_context_checksum = Random::GetTLSInstance()->Next(); + } while (UNLIKELY(base_context_checksum == 0)); + } else { + base_context_checksum = 0; + } } Rep(const Rep&) = delete; @@ -1285,7 +1307,8 @@ void BlockBasedTableBuilder::WriteMaybeCompressedBlock( bool is_data_block = block_type == BlockType::kData; // Old, misleading name of this function: WriteRawBlock StopWatch sw(r->ioptions.clock, r->ioptions.stats, WRITE_RAW_BLOCK_MICROS); - handle->set_offset(r->get_offset()); + const uint64_t offset = r->get_offset(); + handle->set_offset(offset); handle->set_size(block_contents.size()); assert(status().ok()); assert(io_status().ok()); @@ -1307,6 +1330,7 @@ void BlockBasedTableBuilder::WriteMaybeCompressedBlock( uint32_t checksum = ComputeBuiltinChecksumWithLastByte( r->table_options.checksum, block_contents.data(), block_contents.size(), /*last_byte*/ comp_type); + checksum += ChecksumModifierForContext(r->base_context_checksum, offset); if (block_type == BlockType::kFilter) { Status s = r->filter_builder->MaybePostVerifyFilter(block_contents); @@ -1600,6 +1624,11 @@ void BlockBasedTableBuilder::WriteIndexBlock( // The last index_block_handle will be for the partition index block } } + // If success and need to record in metaindex rather than footer... + if (!FormatVersionUsesIndexHandleInFooter( + rep_->table_options.format_version)) { + meta_index_builder->Add(kIndexBlockName, *index_block_handle); + } } void BlockBasedTableBuilder::WritePropertiesBlock( @@ -1625,9 +1654,7 @@ void BlockBasedTableBuilder::WritePropertiesBlock( rep_->props.compression_options = CompressionOptionsToString(rep_->compression_opts); rep_->props.prefix_extractor_name = - rep_->moptions.prefix_extractor != nullptr - ? rep_->moptions.prefix_extractor->AsString() - : "nullptr"; + rep_->prefix_extractor ? rep_->prefix_extractor->AsString() : "nullptr"; std::string property_collectors_names = "["; for (size_t i = 0; i < rep_->ioptions.table_properties_collector_factories.size(); ++i) { @@ -1746,16 +1773,20 @@ void BlockBasedTableBuilder::WriteRangeDelBlock( void BlockBasedTableBuilder::WriteFooter(BlockHandle& metaindex_block_handle, BlockHandle& index_block_handle) { + assert(ok()); Rep* r = rep_; // this is guaranteed by BlockBasedTableBuilder's constructor assert(r->table_options.checksum == kCRC32c || r->table_options.format_version != 0); - assert(ok()); - FooterBuilder footer; - footer.Build(kBlockBasedTableMagicNumber, r->table_options.format_version, - r->get_offset(), r->table_options.checksum, - metaindex_block_handle, index_block_handle); + Status s = footer.Build(kBlockBasedTableMagicNumber, + r->table_options.format_version, r->get_offset(), + r->table_options.checksum, metaindex_block_handle, + index_block_handle, r->base_context_checksum); + if (!s.ok()) { + r->SetStatus(s); + return; + } IOStatus ios = r->file->Append(footer.GetSlice()); if (ios.ok()) { r->set_offset(r->get_offset() + footer.GetSlice().size()); @@ -1970,10 +2001,14 @@ Status BlockBasedTableBuilder::Finish() { WriteFooter(metaindex_block_handle, index_block_handle); } r->state = Rep::State::kClosed; - r->SetStatus(r->CopyIOStatus()); - Status ret_status = r->CopyStatus(); - assert(!ret_status.ok() || io_status().ok()); r->tail_size = r->offset - r->props.tail_start_offset; + + Status ret_status = r->CopyStatus(); + IOStatus ios = r->GetIOStatus(); + if (!ios.ok() && ret_status.ok()) { + // Let io_status supersede ok status (otherwise status takes precedennce) + ret_status = ios; + } return ret_status; } @@ -1983,8 +2018,10 @@ void BlockBasedTableBuilder::Abandon() { StopParallelCompression(); } rep_->state = Rep::State::kClosed; +#ifdef ROCKSDB_ASSERT_STATUS_CHECKED // Avoid unnecessary lock acquisition rep_->CopyStatus().PermitUncheckedError(); rep_->CopyIOStatus().PermitUncheckedError(); +#endif // ROCKSDB_ASSERT_STATUS_CHECKED } uint64_t BlockBasedTableBuilder::NumEntries() const { diff --git a/table/block_based/block_based_table_reader.cc b/table/block_based/block_based_table_reader.cc index 35c6f46f6b..1fc477842f 100644 --- a/table/block_based/block_based_table_reader.cc +++ b/table/block_based/block_based_table_reader.cc @@ -771,6 +771,7 @@ Status BlockBasedTable::Open( if (!s.ok()) { return s; } + rep->verify_checksum_set_on_open = ro.verify_checksums; s = new_table->PrefetchIndexAndFilterBlocks( ro, prefetch_buffer.get(), metaindex_iter.get(), new_table.get(), prefetch_all, table_options, level, file_size, @@ -2454,6 +2455,10 @@ BlockType BlockBasedTable::GetBlockTypeForMetaBlockByName( return BlockType::kHashIndexMetadata; } + if (meta_block_name == kIndexBlockName) { + return BlockType::kIndex; + } + if (meta_block_name.starts_with(kObsoleteFilterBlockPrefix)) { // Obsolete but possible in old files return BlockType::kInvalid; @@ -2474,6 +2479,9 @@ Status BlockBasedTable::VerifyChecksumInMetaBlocks( BlockHandle handle; Slice input = index_iter->value(); s = handle.DecodeFrom(&input); + if (!s.ok()) { + break; + } BlockContents contents; const Slice meta_block_name = index_iter->key(); if (meta_block_name == kPropertiesBlockName) { @@ -2484,7 +2492,13 @@ Status BlockBasedTable::VerifyChecksumInMetaBlocks( nullptr /* prefetch_buffer */, rep_->footer, rep_->ioptions, &table_properties, nullptr /* memory_allocator */); + } else if (rep_->verify_checksum_set_on_open && + meta_block_name == kIndexBlockName) { + // WART: For now, to maintain similar I/O behavior as before + // format_version=6, we skip verifying index block checksum--but only + // if it was checked on open. } else { + // FIXME? Need to verify checksums of index and filter partitions? s = BlockFetcher( rep_->file.get(), nullptr /* prefetch buffer */, rep_->footer, read_options, handle, &contents, rep_->ioptions, @@ -2542,6 +2556,15 @@ Status BlockBasedTable::CreateIndexReader( InternalIterator* meta_iter, bool use_cache, bool prefetch, bool pin, BlockCacheLookupContext* lookup_context, std::unique_ptr* index_reader) { + if (FormatVersionUsesIndexHandleInFooter(rep_->footer.format_version())) { + rep_->index_handle = rep_->footer.index_handle(); + } else { + Status s = FindMetaBlock(meta_iter, kIndexBlockName, &rep_->index_handle); + if (!s.ok()) { + return s; + } + } + switch (rep_->index_type) { case BlockBasedTableOptions::kTwoLevelIndexSearch: { return PartitionIndexReader::Create(this, ro, prefetch_buffer, use_cache, @@ -2709,7 +2732,7 @@ bool BlockBasedTable::TEST_FilterBlockInCache() const { bool BlockBasedTable::TEST_IndexBlockInCache() const { assert(rep_ != nullptr); - return TEST_BlockInCache(rep_->footer.index_handle()); + return TEST_BlockInCache(rep_->index_handle); } Status BlockBasedTable::GetKVPairsFromDataBlocks( diff --git a/table/block_based/block_based_table_reader.h b/table/block_based/block_based_table_reader.h index d0686a4bce..0b5fe1cb80 100644 --- a/table/block_based/block_based_table_reader.h +++ b/table/block_based/block_based_table_reader.h @@ -594,6 +594,7 @@ struct BlockBasedTable::Rep { BlockHandle compression_dict_handle; std::shared_ptr table_properties; + BlockHandle index_handle; BlockBasedTableOptions::IndexType index_type; bool whole_key_filtering; bool prefix_filtering; @@ -637,6 +638,12 @@ struct BlockBasedTable::Rep { bool index_key_includes_seq = true; bool index_value_is_full = true; + // Whether block checksums in metadata blocks were verified on open. + // This is only to mostly maintain current dubious behavior of VerifyChecksum + // with respect to index blocks, but only when the checksum was previously + // verified. + bool verify_checksum_set_on_open = false; + const bool immortal_table; // Whether the user key contains user-defined timestamps. If this is false and // the running user comparator has a non-zero timestamp size, a min timestamp diff --git a/table/block_based/block_based_table_reader_sync_and_async.h b/table/block_based/block_based_table_reader_sync_and_async.h index eb4bf5bed4..43af02fadf 100644 --- a/table/block_based/block_based_table_reader_sync_and_async.h +++ b/table/block_based/block_based_table_reader_sync_and_async.h @@ -222,9 +222,8 @@ DEFINE_SYNC_AND_ASYNC(void, BlockBasedTable::RetrieveMultipleBlocks) // begin address of each read request, we need to add the offset // in each read request. Checksum is stored in the block trailer, // beyond the payload size. - s = VerifyBlockChecksum(footer.checksum_type(), data + req_offset, - handle.size(), rep_->file->file_name(), - handle.offset()); + s = VerifyBlockChecksum(footer, data + req_offset, handle.size(), + rep_->file->file_name(), handle.offset()); TEST_SYNC_POINT_CALLBACK("RetrieveMultipleBlocks:VerifyChecksum", &s); } } else if (!use_shared_buffer) { diff --git a/table/block_based/index_reader_common.cc b/table/block_based/index_reader_common.cc index 828200299d..e68be2a10a 100644 --- a/table/block_based/index_reader_common.cc +++ b/table/block_based/index_reader_common.cc @@ -26,7 +26,7 @@ Status BlockBasedTable::IndexReaderCommon::ReadIndexBlock( assert(rep != nullptr); const Status s = table->RetrieveBlock( - prefetch_buffer, read_options, rep->footer.index_handle(), + prefetch_buffer, read_options, rep->index_handle, UncompressionDict::GetEmptyDict(), &index_block->As(), get_context, lookup_context, /* for_compaction */ false, use_cache, /* async_read */ false); diff --git a/table/block_based/reader_common.cc b/table/block_based/reader_common.cc index 0ff43e9b4e..7d0c97c717 100644 --- a/table/block_based/reader_common.cc +++ b/table/block_based/reader_common.cc @@ -23,10 +23,14 @@ void ForceReleaseCachedEntry(void* arg, void* h) { } // WART: this is specific to block-based table -Status VerifyBlockChecksum(ChecksumType type, const char* data, +Status VerifyBlockChecksum(const Footer& footer, const char* data, size_t block_size, const std::string& file_name, uint64_t offset) { PERF_TIMER_GUARD(block_checksum_time); + + assert(footer.GetBlockTrailerSize() == 5); + ChecksumType type = footer.checksum_type(); + // After block_size bytes is compression type (1 byte), which is part of // the checksummed section. size_t len = block_size + 1; @@ -34,6 +38,13 @@ Status VerifyBlockChecksum(ChecksumType type, const char* data, uint32_t stored = DecodeFixed32(data + len); uint32_t computed = ComputeBuiltinChecksum(type, data, len); + + // Unapply context to 'stored' rather than apply to 'computed, for people + // who might look for reference crc value in error message + uint32_t modifier = + ChecksumModifierForContext(footer.base_context_checksum(), offset); + stored -= modifier; + if (stored == computed) { return Status::OK(); } else { @@ -43,8 +54,9 @@ Status VerifyBlockChecksum(ChecksumType type, const char* data, computed = crc32c::Unmask(computed); } return Status::Corruption( - "block checksum mismatch: stored = " + std::to_string(stored) + - ", computed = " + std::to_string(computed) + + "block checksum mismatch: stored" + + std::string(modifier ? "(context removed)" : "") + " = " + + std::to_string(stored) + ", computed = " + std::to_string(computed) + ", type = " + std::to_string(type) + " in " + file_name + " offset " + std::to_string(offset) + " size " + std::to_string(block_size)); } diff --git a/table/block_based/reader_common.h b/table/block_based/reader_common.h index 102802afec..08c2a756bb 100644 --- a/table/block_based/reader_common.h +++ b/table/block_based/reader_common.h @@ -12,6 +12,8 @@ #include "rocksdb/table.h" namespace ROCKSDB_NAMESPACE { +class Footer; + // Release the cached entry and decrement its ref count. extern void ForceReleaseCachedEntry(void* arg, void* h); @@ -22,12 +24,13 @@ inline MemoryAllocator* GetMemoryAllocator( : nullptr; } -// Assumes block has a trailer as in format.h. file_name and offset provided -// for generating a diagnostic message in returned status. +// Assumes block has a trailer past `data + block_size` as in format.h. +// `file_name` provided for generating diagnostic message in returned status. +// `offset` might be required for proper verification (also used for message). // // Returns Status::OK() on checksum match, or Status::Corruption() on checksum // mismatch. -extern Status VerifyBlockChecksum(ChecksumType type, const char* data, +extern Status VerifyBlockChecksum(const Footer& footer, const char* data, size_t block_size, const std::string& file_name, uint64_t offset); diff --git a/table/block_fetcher.cc b/table/block_fetcher.cc index b2fa6d4b52..412ac4bde1 100644 --- a/table/block_fetcher.cc +++ b/table/block_fetcher.cc @@ -33,9 +33,9 @@ inline void BlockFetcher::ProcessTrailerIfPresent() { if (footer_.GetBlockTrailerSize() > 0) { assert(footer_.GetBlockTrailerSize() == BlockBasedTable::kBlockTrailerSize); if (read_options_.verify_checksums) { - io_status_ = status_to_io_status(VerifyBlockChecksum( - footer_.checksum_type(), slice_.data(), block_size_, - file_->file_name(), handle_.offset())); + io_status_ = status_to_io_status( + VerifyBlockChecksum(footer_, slice_.data(), block_size_, + file_->file_name(), handle_.offset())); RecordTick(ioptions_.stats, BLOCK_CHECKSUM_COMPUTE_COUNT); if (!io_status_.ok()) { assert(io_status_.IsCorruption()); diff --git a/table/block_fetcher_test.cc b/table/block_fetcher_test.cc index c2f6552cc4..18109811d5 100644 --- a/table/block_fetcher_test.cc +++ b/table/block_fetcher_test.cc @@ -107,6 +107,9 @@ class BlockFetcherTest : public testing::Test { Footer footer; ReadFooter(file.get(), &footer); const BlockHandle& index_handle = footer.index_handle(); + // FIXME: index handle will need to come from metaindex for + // format_version >= 6 when that becomes the default + ASSERT_FALSE(index_handle.IsNull()); CompressionType compression_type; FetchBlock(file.get(), index_handle, BlockType::kIndex, diff --git a/table/cuckoo/cuckoo_table_builder.cc b/table/cuckoo/cuckoo_table_builder.cc index 7ca72365fd..0cf6834af8 100644 --- a/table/cuckoo/cuckoo_table_builder.cc +++ b/table/cuckoo/cuckoo_table_builder.cc @@ -403,8 +403,12 @@ Status CuckooTableBuilder::Finish() { } FooterBuilder footer; - footer.Build(kCuckooTableMagicNumber, /* format_version */ 1, offset, - kNoChecksum, meta_index_block_handle); + Status s = footer.Build(kCuckooTableMagicNumber, /* format_version */ 1, + offset, kNoChecksum, meta_index_block_handle); + if (!s.ok()) { + status_ = s; + return status_; + } io_status_ = file_->Append(footer.GetSlice()); status_ = io_status_; return status_; diff --git a/table/format.cc b/table/format.cc index dc077bd452..8825384f0b 100644 --- a/table/format.cc +++ b/table/format.cc @@ -10,6 +10,7 @@ #include "table/format.h" #include +#include #include #include "block_fetcher.h" @@ -18,12 +19,14 @@ #include "monitoring/perf_context_imp.h" #include "monitoring/statistics_impl.h" #include "options/options_helper.h" +#include "port/likely.h" #include "rocksdb/env.h" #include "rocksdb/options.h" #include "rocksdb/table.h" #include "table/block_based/block.h" #include "table/block_based/block_based_table_reader.h" #include "table/persistent_cache_helper.h" +#include "unique_id_impl.h" #include "util/cast_util.h" #include "util/coding.h" #include "util/compression.h" @@ -195,25 +198,41 @@ inline uint8_t BlockTrailerSizeForMagicNumber(uint64_t magic_number) { // -> format_version >= 1 // checksum type (char, 1 byte) // * Part2 +// -> format_version <= 5 // metaindex handle (varint64 offset, varint64 size) // index handle (varint64 offset, varint64 size) // for part2 size = 2 * BlockHandle::kMaxEncodedLength = 40 +// - This padding is unchecked/ignored +// -> format_version >= 6 +// extended magic number (4 bytes) = 0x3e 0x00 0x7a 0x00 +// - Also surely invalid (size 0) handles if interpreted as older version +// - (Helps ensure a corrupted format_version doesn't get us far with no +// footer checksum.) +// footer_checksum (uint32LE, 4 bytes) +// - Checksum of above checksum type of whole footer, with this field +// set to all zeros. +// base_context_checksum (uint32LE, 4 bytes) +// metaindex block size (uint32LE, 4 bytes) +// - Assumed to be immediately before footer, < 4GB +// (24 bytes, reserved for future use) +// - Brings part2 size also to 40 bytes +// - Checked that last eight bytes == 0, so reserved for a future +// incompatible feature (but under format_version=6) // * Part3 // -> format_version == 0 (inferred from legacy magic number) // legacy magic number (8 bytes) // -> format_version >= 1 (inferred from NOT legacy magic number) // format_version (uint32LE, 4 bytes), also called "footer version" // newer magic number (8 bytes) - +const std::array kExtendedMagic{{0x3e, 0x00, 0x7a, 0x00}}; constexpr size_t kFooterPart2Size = 2 * BlockHandle::kMaxEncodedLength; } // namespace -void FooterBuilder::Build(uint64_t magic_number, uint32_t format_version, - uint64_t footer_offset, ChecksumType checksum_type, - const BlockHandle& metaindex_handle, - const BlockHandle& index_handle) { - (void)footer_offset; // Future use - +Status FooterBuilder::Build(uint64_t magic_number, uint32_t format_version, + uint64_t footer_offset, ChecksumType checksum_type, + const BlockHandle& metaindex_handle, + const BlockHandle& index_handle, + uint32_t base_context_checksum) { assert(magic_number != Footer::kNullTableMagicNumber); assert(IsSupportedFormatVersion(format_version)); @@ -249,19 +268,71 @@ void FooterBuilder::Build(uint64_t magic_number, uint32_t format_version, assert(cur + 8 == slice_.data() + slice_.size()); } - { + if (format_version >= 6) { + if (BlockTrailerSizeForMagicNumber(magic_number) != 0) { + // base context checksum required for table formats with block checksums + assert(base_context_checksum != 0); + assert(ChecksumModifierForContext(base_context_checksum, 0) != 0); + } else { + // base context checksum not used + assert(base_context_checksum == 0); + assert(ChecksumModifierForContext(base_context_checksum, 0) == 0); + } + + // Start populating Part 2 + char* cur = data_.data() + /* part 1 size */ 1; + // Set extended magic of part2 + std::copy(kExtendedMagic.begin(), kExtendedMagic.end(), cur); + cur += kExtendedMagic.size(); + // Fill checksum data with zeros (for later computing checksum) + char* checksum_data = cur; + EncodeFixed32(cur, 0); + cur += 4; + // Save base context checksum + EncodeFixed32(cur, base_context_checksum); + cur += 4; + // Compute and save metaindex size + uint32_t metaindex_size = static_cast(metaindex_handle.size()); + if (metaindex_size != metaindex_handle.size()) { + return Status::NotSupported("Metaindex block size > 4GB"); + } + // Metaindex must be adjacent to footer + assert(metaindex_size == 0 || + metaindex_handle.offset() + metaindex_handle.size() == + footer_offset - BlockTrailerSizeForMagicNumber(magic_number)); + EncodeFixed32(cur, metaindex_size); + cur += 4; + + // Zero pad remainder (for future use) + std::fill_n(cur, 24U, char{0}); + assert(cur + 24 == part3); + + // Compute checksum, add context + uint32_t checksum = ComputeBuiltinChecksum( + checksum_type, data_.data(), Footer::kNewVersionsEncodedLength); + checksum += + ChecksumModifierForContext(base_context_checksum, footer_offset); + // Store it + EncodeFixed32(checksum_data, checksum); + } else { + // Base context checksum not used + assert(!FormatVersionUsesContextChecksum(format_version)); + // Should be left empty + assert(base_context_checksum == 0); + assert(ChecksumModifierForContext(base_context_checksum, 0) == 0); + + // Populate all of part 2 char* cur = part2; cur = metaindex_handle.EncodeTo(cur); cur = index_handle.EncodeTo(cur); // Zero pad remainder std::fill(cur, part3, char{0}); } + return Status::OK(); } Status Footer::DecodeFrom(Slice input, uint64_t input_offset, uint64_t enforce_table_magic_number) { - (void)input_offset; // Future use - // Only decode to unused Footer assert(table_magic_number_ == kNullTableMagicNumber); assert(input != nullptr); @@ -284,6 +355,9 @@ Status Footer::DecodeFrom(Slice input, uint64_t input_offset, block_trailer_size_ = BlockTrailerSizeForMagicNumber(magic); // Parse Part3 + const char* part3_ptr = magic_ptr; + uint32_t computed_checksum = 0; + uint64_t footer_offset = 0; if (legacy) { // The size is already asserted to be at least kMinEncodedLength // at the beginning of the function @@ -291,37 +365,101 @@ Status Footer::DecodeFrom(Slice input, uint64_t input_offset, format_version_ = 0 /* legacy */; checksum_type_ = kCRC32c; } else { - const char* part3_ptr = magic_ptr - 4; + part3_ptr = magic_ptr - 4; format_version_ = DecodeFixed32(part3_ptr); - if (!IsSupportedFormatVersion(format_version_)) { + if (UNLIKELY(!IsSupportedFormatVersion(format_version_))) { return Status::Corruption("Corrupt or unsupported format_version: " + std::to_string(format_version_)); } // All known format versions >= 1 occupy exactly this many bytes. - if (input.size() < kNewVersionsEncodedLength) { + if (UNLIKELY(input.size() < kNewVersionsEncodedLength)) { return Status::Corruption("Input is too short to be an SST file"); } uint64_t adjustment = input.size() - kNewVersionsEncodedLength; input.remove_prefix(adjustment); + footer_offset = input_offset + adjustment; // Parse Part1 char chksum = input.data()[0]; checksum_type_ = lossless_cast(chksum); - if (!IsSupportedChecksumType(checksum_type())) { + if (UNLIKELY(!IsSupportedChecksumType(checksum_type()))) { return Status::Corruption("Corrupt or unsupported checksum type: " + std::to_string(lossless_cast(chksum))); } + // This is the most convenient place to compute the checksum + if (checksum_type_ != kNoChecksum && format_version_ >= 6) { + std::array copy_without_checksum; + std::copy_n(input.data(), kNewVersionsEncodedLength, + ©_without_checksum[0]); + EncodeFixed32(©_without_checksum[5], 0); // Clear embedded checksum + computed_checksum = + ComputeBuiltinChecksum(checksum_type(), copy_without_checksum.data(), + kNewVersionsEncodedLength); + } // Consume checksum type field input.remove_prefix(1); } // Parse Part2 - Status result = metaindex_handle_.DecodeFrom(&input); - if (result.ok()) { - result = index_handle_.DecodeFrom(&input); + if (format_version_ >= 6) { + Slice ext_magic(input.data(), 4); + if (UNLIKELY(ext_magic.compare(Slice(kExtendedMagic.data(), + kExtendedMagic.size())) != 0)) { + return Status::Corruption("Bad extended magic number: 0x" + + ext_magic.ToString(/*hex*/ true)); + } + input.remove_prefix(4); + uint32_t stored_checksum = 0, metaindex_size = 0; + bool success; + success = GetFixed32(&input, &stored_checksum); + assert(success); + success = GetFixed32(&input, &base_context_checksum_); + assert(success); + if (UNLIKELY(ChecksumModifierForContext(base_context_checksum_, 0) == 0)) { + return Status::Corruption("Invalid base context checksum"); + } + computed_checksum += + ChecksumModifierForContext(base_context_checksum_, footer_offset); + if (UNLIKELY(computed_checksum != stored_checksum)) { + return Status::Corruption("Footer at " + std::to_string(footer_offset) + + " checksum mismatch"); + } + success = GetFixed32(&input, &metaindex_size); + assert(success); + (void)success; + uint64_t metaindex_end = footer_offset - GetBlockTrailerSize(); + metaindex_handle_ = + BlockHandle(metaindex_end - metaindex_size, metaindex_size); + + // Mark unpopulated + index_handle_ = BlockHandle::NullBlockHandle(); + + // 16 bytes of unchecked reserved padding + input.remove_prefix(16U); + + // 8 bytes of checked reserved padding (expected to be zero unless using a + // future feature). + uint64_t reserved = 0; + success = GetFixed64(&input, &reserved); + assert(success); + if (UNLIKELY(reserved != 0)) { + return Status::NotSupported( + "File uses a future feature not supported in this version"); + } + // End of part 2 + assert(input.data() == part3_ptr); + } else { + // format_version_ < 6 + Status result = metaindex_handle_.DecodeFrom(&input); + if (result.ok()) { + result = index_handle_.DecodeFrom(&input); + } + if (!result.ok()) { + return result; + } + // Padding in part2 is ignored } - return result; - // Padding in part2 is ignored + return Status::OK(); } std::string Footer::ToString() const { diff --git a/table/format.h b/table/format.h index c375165bf6..73675381ed 100644 --- a/table/format.h +++ b/table/format.h @@ -111,6 +111,40 @@ struct IndexValue { std::string ToString(bool hex, bool have_first_key) const; }; +// Given a file's base_context_checksum and an offset of a block within that +// file, choose a 32-bit value that is as unique as possible. This value will +// be added to the standard checksum to get a checksum "with context," or can +// be subtracted to "remove" context. Returns zero (no modifier) if feature is +// disabled with base_context_checksum == 0. +inline uint32_t ChecksumModifierForContext(uint32_t base_context_checksum, + uint64_t offset) { + // To disable on base_context_checksum == 0, we could write + // `if (base_context_checksum == 0) return 0;` but benchmarking shows + // measurable performance penalty vs. this: compute the modifier + // unconditionally and use an "all or nothing" bit mask to enable + // or disable. + uint32_t all_or_nothing = uint32_t{0} - (base_context_checksum != 0); + + // Desired properties: + // (call this function f(b, o) where b = base and o = offset) + // 1. Fast + // 2. f(b1, o) == f(b2, o) iff b1 == b2 + // (Perfectly preserve base entropy) + // 3. f(b, o1) == f(b, o2) only if o1 == o2 or |o1-o2| >= 4 billion + // (Guaranteed uniqueness for nearby offsets) + // 3. f(b, o + j * 2**32) == f(b, o + k * 2**32) only if j == k + // (Upper bits matter, and *aligned* misplacement fails check) + // 4. f(b1, o) == f(b2, o + x) then preferably not + // f(b1, o + y) == f(b2, o + x + y) + // (Avoid linearly correlated matches) + // 5. f(b, o) == 0 depends on both b and o + // (No predictable overlap with non-context checksums) + uint32_t modifier = + base_context_checksum ^ (Lower32of64(offset) + Upper32of64(offset)); + + return modifier & all_or_nothing; +} + inline uint32_t GetCompressFormatForVersion(uint32_t format_version) { // As of format_version 2, we encode compressed block with // compress_format_version == 2. Before that, the version is 1. @@ -118,18 +152,27 @@ inline uint32_t GetCompressFormatForVersion(uint32_t format_version) { return format_version >= 2 ? 2 : 1; } -constexpr uint32_t kLatestFormatVersion = 5; +constexpr uint32_t kLatestFormatVersion = 6; inline bool IsSupportedFormatVersion(uint32_t version) { return version <= kLatestFormatVersion; } +// Same as having a unique id in footer. +inline bool FormatVersionUsesContextChecksum(uint32_t version) { + return version >= 6; +} + +inline bool FormatVersionUsesIndexHandleInFooter(uint32_t version) { + return version < 6; +} + // Footer encapsulates the fixed information stored at the tail end of every // SST file. In general, it should only include things that cannot go // elsewhere under the metaindex block. For example, checksum_type is // required for verifying metaindex block checksum (when applicable), but -// index block handle can easily go in metaindex block (possible future). -// See also FooterBuilder below. +// index block handle can easily go in metaindex block. See also FooterBuilder +// below. class Footer { public: // Create empty. Populate using DecodeFrom. @@ -137,7 +180,7 @@ class Footer { // Deserialize a footer (populate fields) from `input` and check for various // corruptions. `input_offset` is the offset within the target file of - // `input` buffer (future use). + // `input` buffer, which is needed for verifying format_version >= 6 footer. // If enforce_table_magic_number != 0, will return corruption if table magic // number is not equal to enforce_table_magic_number. Status DecodeFrom(Slice input, uint64_t input_offset, @@ -152,13 +195,17 @@ class Footer { // BBTO::format_version.) uint32_t format_version() const { return format_version_; } + // See ChecksumModifierForContext() + uint32_t base_context_checksum() const { return base_context_checksum_; } + // Block handle for metaindex block. const BlockHandle& metaindex_handle() const { return metaindex_handle_; } // Block handle for (top-level) index block. + // TODO? remove from this struct and only read on decode for legacy cases const BlockHandle& index_handle() const { return index_handle_; } - // Checksum type used in the file. + // Checksum type used in the file, including footer for format version >= 6. ChecksumType checksum_type() const { return static_cast(checksum_type_); } @@ -198,6 +245,7 @@ class Footer { uint64_t table_magic_number_ = kNullTableMagicNumber; uint32_t format_version_ = kInvalidFormatVersion; + uint32_t base_context_checksum_ = 0; BlockHandle metaindex_handle_; BlockHandle index_handle_; int checksum_type_ = kInvalidChecksumType; @@ -219,11 +267,16 @@ class FooterBuilder { // * footer_offset is the file offset where the footer will be written // (for future use). // * checksum_type is for formats using block checksums. - // * index_handle is optional for some kinds of SST files. - void Build(uint64_t table_magic_number, uint32_t format_version, - uint64_t footer_offset, ChecksumType checksum_type, - const BlockHandle& metaindex_handle, - const BlockHandle& index_handle = BlockHandle::NullBlockHandle()); + // * index_handle is optional for some SST kinds and (for caller convenience) + // ignored when format_version >= 6. (Must be added to metaindex in that + // case.) + // * unique_id must be specified if format_vesion >= 6 and SST uses block + // checksums with context. Otherwise, auto-generated if format_vesion >= 6. + Status Build(uint64_t table_magic_number, uint32_t format_version, + uint64_t footer_offset, ChecksumType checksum_type, + const BlockHandle& metaindex_handle, + const BlockHandle& index_handle = BlockHandle::NullBlockHandle(), + uint32_t base_context_checksum = 0); // After Builder, get a Slice for the serialized Footer, backed by this // FooterBuilder. diff --git a/table/meta_blocks.cc b/table/meta_blocks.cc index cf756cfff0..b82b5962f6 100644 --- a/table/meta_blocks.cc +++ b/table/meta_blocks.cc @@ -27,6 +27,8 @@ namespace ROCKSDB_NAMESPACE { const std::string kPropertiesBlockName = "rocksdb.properties"; +// NB: only used with format_version >= 6 +const std::string kIndexBlockName = "rocksdb.index"; // Old property block name for backward compatibility const std::string kPropertiesBlockOldName = "rocksdb.stats"; const std::string kCompressionDictBlockName = "rocksdb.compression_dict"; @@ -395,8 +397,8 @@ Status ReadTablePropertiesHelper( // Modified version of BlockFetcher checksum verification // (See write_global_seqno comment above) if (s.ok() && footer.GetBlockTrailerSize() > 0) { - s = VerifyBlockChecksum(footer.checksum_type(), properties_block.data(), - block_size, file->file_name(), handle.offset()); + s = VerifyBlockChecksum(footer, properties_block.data(), block_size, + file->file_name(), handle.offset()); if (s.IsCorruption()) { if (new_table_properties->external_sst_file_global_seqno_offset != 0) { std::string tmp_buf(properties_block.data(), @@ -405,8 +407,8 @@ Status ReadTablePropertiesHelper( new_table_properties->external_sst_file_global_seqno_offset - handle.offset(); EncodeFixed64(&tmp_buf[static_cast(global_seqno_offset)], 0); - s = VerifyBlockChecksum(footer.checksum_type(), tmp_buf.data(), - block_size, file->file_name(), handle.offset()); + s = VerifyBlockChecksum(footer, tmp_buf.data(), block_size, + file->file_name(), handle.offset()); } } } diff --git a/table/meta_blocks.h b/table/meta_blocks.h index 962a316388..1ed9cf27f1 100644 --- a/table/meta_blocks.h +++ b/table/meta_blocks.h @@ -32,6 +32,7 @@ struct TableProperties; // Meta block names for metaindex extern const std::string kPropertiesBlockName; +extern const std::string kIndexBlockName; extern const std::string kPropertiesBlockOldName; extern const std::string kCompressionDictBlockName; extern const std::string kRangeDelBlockName; diff --git a/table/plain/plain_table_builder.cc b/table/plain/plain_table_builder.cc index 126098a868..ffa811c3ca 100644 --- a/table/plain/plain_table_builder.cc +++ b/table/plain/plain_table_builder.cc @@ -274,10 +274,11 @@ Status PlainTableBuilder::Finish() { // -- Write property block BlockHandle property_block_handle; - IOStatus s = WriteBlock(property_block_builder.Finish(), file_, &offset_, + io_status_ = WriteBlock(property_block_builder.Finish(), file_, &offset_, &property_block_handle); - if (!s.ok()) { - return static_cast(s); + if (!io_status_.ok()) { + status_ = io_status_; + return status_; } meta_index_builer.Add(kPropertiesBlockName, property_block_handle); @@ -293,8 +294,12 @@ Status PlainTableBuilder::Finish() { // Write Footer // no need to write out new footer if we're using default checksum FooterBuilder footer; - footer.Build(kPlainTableMagicNumber, /* format_version */ 0, offset_, - kNoChecksum, metaindex_block_handle); + Status s = footer.Build(kPlainTableMagicNumber, /* format_version */ 0, + offset_, kNoChecksum, metaindex_block_handle); + if (!s.ok()) { + status_ = s; + return status_; + } io_status_ = file_->Append(footer.GetSlice()); if (io_status_.ok()) { offset_ += footer.GetSlice().size(); diff --git a/table/table_test.cc b/table/table_test.cc index 9610ba7672..5b7e4682f1 100644 --- a/table/table_test.cc +++ b/table/table_test.cc @@ -4472,11 +4472,12 @@ TEST(TableTest, FooterTests) { BlockHandle index(data_size + 5, index_size); BlockHandle meta_index(data_size + index_size + 2 * 5, metaindex_size); uint64_t footer_offset = data_size + metaindex_size + index_size + 3 * 5; + uint32_t base_context_checksum = 123456789; { // legacy block based FooterBuilder footer; - footer.Build(kBlockBasedTableMagicNumber, /* format_version */ 0, - footer_offset, kCRC32c, meta_index, index); + ASSERT_OK(footer.Build(kBlockBasedTableMagicNumber, /* format_version */ 0, + footer_offset, kCRC32c, meta_index, index)); Footer decoded_footer; ASSERT_OK(decoded_footer.DecodeFrom(footer.GetSlice(), footer_offset)); ASSERT_EQ(decoded_footer.table_magic_number(), kBlockBasedTableMagicNumber); @@ -4486,6 +4487,7 @@ TEST(TableTest, FooterTests) { ASSERT_EQ(decoded_footer.index_handle().offset(), index.offset()); ASSERT_EQ(decoded_footer.index_handle().size(), index.size()); ASSERT_EQ(decoded_footer.format_version(), 0U); + ASSERT_EQ(decoded_footer.base_context_checksum(), 0U); ASSERT_EQ(decoded_footer.GetBlockTrailerSize(), 5U); // Ensure serialized with legacy magic ASSERT_EQ( @@ -4495,9 +4497,11 @@ TEST(TableTest, FooterTests) { // block based, various checksums, various versions for (auto t : GetSupportedChecksums()) { for (uint32_t fv = 1; IsSupportedFormatVersion(fv); ++fv) { + uint32_t maybe_bcc = + FormatVersionUsesContextChecksum(fv) ? base_context_checksum : 0U; FooterBuilder footer; - footer.Build(kBlockBasedTableMagicNumber, fv, footer_offset, t, - meta_index, index); + ASSERT_OK(footer.Build(kBlockBasedTableMagicNumber, fv, footer_offset, t, + meta_index, index, maybe_bcc)); Footer decoded_footer; ASSERT_OK(decoded_footer.DecodeFrom(footer.GetSlice(), footer_offset)); ASSERT_EQ(decoded_footer.table_magic_number(), @@ -4506,18 +4510,44 @@ TEST(TableTest, FooterTests) { ASSERT_EQ(decoded_footer.metaindex_handle().offset(), meta_index.offset()); ASSERT_EQ(decoded_footer.metaindex_handle().size(), meta_index.size()); - ASSERT_EQ(decoded_footer.index_handle().offset(), index.offset()); - ASSERT_EQ(decoded_footer.index_handle().size(), index.size()); + if (FormatVersionUsesIndexHandleInFooter(fv)) { + ASSERT_EQ(decoded_footer.index_handle().offset(), index.offset()); + ASSERT_EQ(decoded_footer.index_handle().size(), index.size()); + } ASSERT_EQ(decoded_footer.format_version(), fv); ASSERT_EQ(decoded_footer.GetBlockTrailerSize(), 5U); + + if (FormatVersionUsesContextChecksum(fv)) { + ASSERT_EQ(decoded_footer.base_context_checksum(), + base_context_checksum); + + // Bad offset should fail footer checksum + decoded_footer = Footer(); + ASSERT_NOK( + decoded_footer.DecodeFrom(footer.GetSlice(), footer_offset - 1)); + } else { + ASSERT_EQ(decoded_footer.base_context_checksum(), 0U); + } + + // Too big metaindex size should also fail encoding only in new footer + uint64_t big_metaindex_size = 0x100000007U; + uint64_t big_footer_offset = + data_size + big_metaindex_size + index_size + 3 * 5; + BlockHandle big_metaindex = + BlockHandle(data_size + index_size + 2 * 5, big_metaindex_size); + ASSERT_NE(footer + .Build(kBlockBasedTableMagicNumber, fv, big_footer_offset, + t, big_metaindex, index, maybe_bcc) + .ok(), + FormatVersionUsesContextChecksum(fv)); } } { // legacy plain table FooterBuilder footer; - footer.Build(kPlainTableMagicNumber, /* format_version */ 0, footer_offset, - kNoChecksum, meta_index); + ASSERT_OK(footer.Build(kPlainTableMagicNumber, /* format_version */ 0, + footer_offset, kNoChecksum, meta_index)); Footer decoded_footer; ASSERT_OK(decoded_footer.DecodeFrom(footer.GetSlice(), footer_offset)); ASSERT_EQ(decoded_footer.table_magic_number(), kPlainTableMagicNumber); @@ -4536,8 +4566,8 @@ TEST(TableTest, FooterTests) { { // xxhash plain table (not currently used) FooterBuilder footer; - footer.Build(kPlainTableMagicNumber, /* format_version */ 1, footer_offset, - kxxHash, meta_index); + ASSERT_OK(footer.Build(kPlainTableMagicNumber, /* format_version */ 1, + footer_offset, kxxHash, meta_index)); Footer decoded_footer; ASSERT_OK(decoded_footer.DecodeFrom(footer.GetSlice(), footer_offset)); ASSERT_EQ(decoded_footer.table_magic_number(), kPlainTableMagicNumber); @@ -5211,9 +5241,13 @@ TEST_P(BlockBasedTableTest, PropertiesMetaBlockLast) { } } ASSERT_EQ(kPropertiesBlockName, key_at_max_offset); - // index handle is stored in footer rather than metaindex block, so need - // separate logic to verify it comes before properties block. - ASSERT_GT(max_offset, footer.index_handle().offset()); + if (FormatVersionUsesIndexHandleInFooter(footer.format_version())) { + // If index handle is stored in footer rather than metaindex block, + // need separate logic to verify it comes before properties block. + ASSERT_GT(max_offset, footer.index_handle().offset()); + } else { + ASSERT_TRUE(footer.index_handle().IsNull()); + } c.ResetTableReader(); } diff --git a/test_util/testutil.cc b/test_util/testutil.cc index b128b797a8..1e771f4fd1 100644 --- a/test_util/testutil.cc +++ b/test_util/testutil.cc @@ -39,7 +39,10 @@ namespace test { const uint32_t kDefaultFormatVersion = BlockBasedTableOptions().format_version; const std::set kFooterFormatVersionsToTest{ + // Non-legacy, before big footer changes 5U, + // After big footer changes + 6U, // In case any interesting future changes kDefaultFormatVersion, kLatestFormatVersion, diff --git a/tools/db_crashtest.py b/tools/db_crashtest.py index 1a3821aa1f..01b71136a3 100644 --- a/tools/db_crashtest.py +++ b/tools/db_crashtest.py @@ -134,7 +134,7 @@ default_params = { "verify_checksum": 1, "write_buffer_size": 4 * 1024 * 1024, "writepercent": 35, - "format_version": lambda: random.choice([2, 3, 4, 5, 5]), + "format_version": lambda: random.choice([2, 3, 4, 5, 6, 6]), "index_block_restart_interval": lambda: random.choice(range(1, 16)), "use_multiget": lambda: random.randint(0, 1), "use_get_entity": lambda: random.choice([0] * 7 + [1]), diff --git a/tools/sst_dump_test.cc b/tools/sst_dump_test.cc index 29d11d4da3..481c4b7220 100644 --- a/tools/sst_dump_test.cc +++ b/tools/sst_dump_test.cc @@ -468,4 +468,3 @@ int main(int argc, char** argv) { RegisterCustomObjects(argc, argv); return RUN_ALL_TESTS(); } - diff --git a/unreleased_history/new_features/1_context_checksum.md b/unreleased_history/new_features/1_context_checksum.md new file mode 100644 index 0000000000..303613cada --- /dev/null +++ b/unreleased_history/new_features/1_context_checksum.md @@ -0,0 +1 @@ +* Added enhanced data integrity checking on SST files with new format_version=6. Performance impact is very small or negligible. Previously if SST data was misplaced or re-arranged by the storage layer, it could pass block checksum with higher than 1 in 4 billion probability. With format_version=6, block checksums depend on what file they are in and location within the file. This way, misplaced SST data is no more likely to pass checksum verification than randomly corrupted data. Also in format_version=6, SST footers are checksum-protected.