]> git.proxmox.com Git - ceph.git/blobdiff - ceph/src/rocksdb/table/meta_blocks.cc
update ceph source to reef 18.1.2
[ceph.git] / ceph / src / rocksdb / table / meta_blocks.cc
index 4c41b7f8602cb0969e5cf8043076db0bde3b0ccc..78a62359db8dc8998a8912dca0d9c3512053a990 100644 (file)
 #include "block_fetcher.h"
 #include "db/table_properties_collector.h"
 #include "file/random_access_file_reader.h"
+#include "logging/logging.h"
+#include "rocksdb/options.h"
 #include "rocksdb/table.h"
 #include "rocksdb/table_properties.h"
 #include "table/block_based/block.h"
+#include "table/block_based/reader_common.h"
 #include "table/format.h"
 #include "table/internal_iterator.h"
 #include "table/persistent_cache_helper.h"
+#include "table/sst_file_writer_collectors.h"
 #include "table/table_properties_internal.h"
 #include "test_util/sync_point.h"
 #include "util/coding.h"
 
 namespace ROCKSDB_NAMESPACE {
 
+const std::string kPropertiesBlockName = "rocksdb.properties";
+// Old property block name for backward compatibility
+const std::string kPropertiesBlockOldName = "rocksdb.stats";
+const std::string kCompressionDictBlockName = "rocksdb.compression_dict";
+const std::string kRangeDelBlockName = "rocksdb.range_del";
+
 MetaIndexBuilder::MetaIndexBuilder()
     : meta_index_block_(new BlockBuilder(1 /* restart interval */)) {}
 
-void MetaIndexBuilder::Add(const std::string& key,
-                           const BlockHandle& handle) {
+void MetaIndexBuilder::Add(const std::string& key, const BlockHandle& handle) {
   std::string handle_encoding;
   handle.EncodeTo(&handle_encoding);
   meta_block_handles_.insert({key, handle_encoding});
@@ -43,8 +52,8 @@ Slice MetaIndexBuilder::Finish() {
 // object, so there's no need for restart points. Thus we set the restart
 // interval to infinity to save space.
 PropertyBlockBuilder::PropertyBlockBuilder()
-    : properties_block_(
-          new BlockBuilder(port::kMaxInt32 /* restart interval */)) {}
+    : properties_block_(new BlockBuilder(
+          std::numeric_limits<int32_t>::max() /* restart interval */)) {}
 
 void PropertyBlockBuilder::Add(const std::string& name,
                                const std::string& val) {
@@ -71,6 +80,7 @@ void PropertyBlockBuilder::AddTableProperty(const TableProperties& props) {
   TEST_SYNC_POINT_CALLBACK("PropertyBlockBuilder::AddTableProperty:Start",
                            const_cast<TableProperties*>(&props));
 
+  Add(TablePropertiesNames::kOriginalFileNumber, props.orig_file_number);
   Add(TablePropertiesNames::kRawKeySize, props.raw_key_size);
   Add(TablePropertiesNames::kRawValueSize, props.raw_value_size);
   Add(TablePropertiesNames::kDataSize, props.data_size);
@@ -83,6 +93,7 @@ void PropertyBlockBuilder::AddTableProperty(const TableProperties& props) {
   Add(TablePropertiesNames::kIndexValueIsDeltaEncoded,
       props.index_value_is_delta_encoded);
   Add(TablePropertiesNames::kNumEntries, props.num_entries);
+  Add(TablePropertiesNames::kNumFilterEntries, props.num_filter_entries);
   Add(TablePropertiesNames::kDeletedKeys, props.num_deletions);
   Add(TablePropertiesNames::kMergeOperands, props.num_merge_operands);
   Add(TablePropertiesNames::kNumRangeDeletions, props.num_range_deletions);
@@ -96,6 +107,14 @@ void PropertyBlockBuilder::AddTableProperty(const TableProperties& props) {
   if (props.file_creation_time > 0) {
     Add(TablePropertiesNames::kFileCreationTime, props.file_creation_time);
   }
+  if (props.slow_compression_estimated_data_size > 0) {
+    Add(TablePropertiesNames::kSlowCompressionEstimatedDataSize,
+        props.slow_compression_estimated_data_size);
+  }
+  if (props.fast_compression_estimated_data_size > 0) {
+    Add(TablePropertiesNames::kFastCompressionEstimatedDataSize,
+        props.fast_compression_estimated_data_size);
+  }
   if (!props.db_id.empty()) {
     Add(TablePropertiesNames::kDbId, props.db_id);
   }
@@ -134,6 +153,10 @@ void PropertyBlockBuilder::AddTableProperty(const TableProperties& props) {
   if (!props.compression_options.empty()) {
     Add(TablePropertiesNames::kCompressionOptions, props.compression_options);
   }
+  if (!props.seqno_to_time_mapping.empty()) {
+    Add(TablePropertiesNames::kSequenceNumberTimeMapping,
+        props.seqno_to_time_mapping);
+  }
 }
 
 Slice PropertyBlockBuilder::Finish() {
@@ -144,13 +167,13 @@ Slice PropertyBlockBuilder::Finish() {
   return properties_block_->Finish();
 }
 
-void LogPropertiesCollectionError(
-    Logger* info_log, const std::string& method, const std::string& name) {
+void LogPropertiesCollectionError(Logger* info_log, const std::string& method,
+                                  const std::string& name) {
   assert(method == "Add" || method == "Finish");
 
   std::string msg =
-    "Encountered error when calling TablePropertiesCollector::" +
-    method + "() with collector name: " + name;
+      "Encountered error when calling TablePropertiesCollector::" + method +
+      "() with collector name: " + name;
   ROCKS_LOG_ERROR(info_log, "%s", msg.c_str());
 }
 
@@ -172,11 +195,12 @@ bool NotifyCollectTableCollectorsOnAdd(
 
 void NotifyCollectTableCollectorsOnBlockAdd(
     const std::vector<std::unique_ptr<IntTblPropCollector>>& collectors,
-    const uint64_t blockRawBytes, const uint64_t blockCompressedBytesFast,
-    const uint64_t blockCompressedBytesSlow) {
+    const uint64_t block_uncomp_bytes,
+    const uint64_t block_compressed_bytes_fast,
+    const uint64_t block_compressed_bytes_slow) {
   for (auto& collector : collectors) {
-    collector->BlockAdd(blockRawBytes, blockCompressedBytesFast,
-                        blockCompressedBytesSlow);
+    collector->BlockAdd(block_uncomp_bytes, block_compressed_bytes_fast,
+                        block_compressed_bytes_slow);
   }
 }
 
@@ -200,50 +224,48 @@ bool NotifyCollectTableCollectorsOnFinish(
   return all_succeeded;
 }
 
-Status ReadProperties(const ReadOptions& read_options,
-                      const Slice& handle_value, RandomAccessFileReader* file,
-                      FilePrefetchBuffer* prefetch_buffer, const Footer& footer,
-                      const ImmutableCFOptions& ioptions,
-                      TableProperties** table_properties, bool verify_checksum,
-                      BlockHandle* ret_block_handle,
-                      CacheAllocationPtr* verification_buf,
-                      bool /*compression_type_missing*/,
-                      MemoryAllocator* memory_allocator) {
+// FIXME: should be a parameter for reading table properties to use persistent
+// cache?
+Status ReadTablePropertiesHelper(
+    const ReadOptions& ro, const BlockHandle& handle,
+    RandomAccessFileReader* file, FilePrefetchBuffer* prefetch_buffer,
+    const Footer& footer, const ImmutableOptions& ioptions,
+    std::unique_ptr<TableProperties>* table_properties,
+    MemoryAllocator* memory_allocator) {
   assert(table_properties);
 
-  Slice v = handle_value;
-  BlockHandle handle;
-  if (!handle.DecodeFrom(&v).ok()) {
-    return Status::InvalidArgument("Failed to decode properties block handle");
-  }
-
+  // If this is an external SST file ingested with write_global_seqno set to
+  // true, then we expect the checksum mismatch because checksum was written
+  // by SstFileWriter, but its global seqno in the properties block may have
+  // been changed during ingestion. For this reason, we initially read
+  // and process without checksum verification, then later try checksum
+  // verification so that if it fails, we can copy to a temporary buffer with
+  // global seqno set to its original value, i.e. 0, and attempt checksum
+  // verification again.
+  ReadOptions modified_ro = ro;
+  modified_ro.verify_checksums = false;
   BlockContents block_contents;
-  Status s;
-  PersistentCacheOptions cache_options;
-  ReadOptions ro = read_options;
-  ro.verify_checksums = verify_checksum;
-
-  BlockFetcher block_fetcher(file, prefetch_buffer, footer, ro, handle,
+  BlockFetcher block_fetcher(file, prefetch_buffer, footer, modified_ro, handle,
                              &block_contents, ioptions, false /* decompress */,
                              false /*maybe_compressed*/, BlockType::kProperties,
-                             UncompressionDict::GetEmptyDict(), cache_options,
-                             memory_allocator);
-  s = block_fetcher.ReadBlockContents();
-  // property block is never compressed. Need to add uncompress logic if we are
-  // to compress it..
-
+                             UncompressionDict::GetEmptyDict(),
+                             PersistentCacheOptions::kEmpty, memory_allocator);
+  Status s = block_fetcher.ReadBlockContents();
   if (!s.ok()) {
     return s;
   }
 
+  // Unfortunately, Block::size() might not equal block_contents.data.size(),
+  // and Block hides block_contents
+  uint64_t block_size = block_contents.data.size();
   Block properties_block(std::move(block_contents));
-  DataBlockIter iter;
-  properties_block.NewDataIterator(BytewiseComparator(),
-                                   kDisableGlobalSequenceNumber, &iter);
+  std::unique_ptr<MetaBlockIter> iter(properties_block.NewMetaIterator());
 
-  auto new_table_properties = new TableProperties();
+  std::unique_ptr<TableProperties> new_table_properties{new TableProperties};
   // All pre-defined properties of type uint64_t
   std::unordered_map<std::string, uint64_t*> predefined_uint64_properties = {
+      {TablePropertiesNames::kOriginalFileNumber,
+       &new_table_properties->orig_file_number},
       {TablePropertiesNames::kDataSize, &new_table_properties->data_size},
       {TablePropertiesNames::kIndexSize, &new_table_properties->index_size},
       {TablePropertiesNames::kIndexPartitions,
@@ -261,6 +283,8 @@ Status ReadProperties(const ReadOptions& read_options,
       {TablePropertiesNames::kNumDataBlocks,
        &new_table_properties->num_data_blocks},
       {TablePropertiesNames::kNumEntries, &new_table_properties->num_entries},
+      {TablePropertiesNames::kNumFilterEntries,
+       &new_table_properties->num_filter_entries},
       {TablePropertiesNames::kDeletedKeys,
        &new_table_properties->num_deletions},
       {TablePropertiesNames::kMergeOperands,
@@ -279,16 +303,20 @@ Status ReadProperties(const ReadOptions& read_options,
        &new_table_properties->oldest_key_time},
       {TablePropertiesNames::kFileCreationTime,
        &new_table_properties->file_creation_time},
+      {TablePropertiesNames::kSlowCompressionEstimatedDataSize,
+       &new_table_properties->slow_compression_estimated_data_size},
+      {TablePropertiesNames::kFastCompressionEstimatedDataSize,
+       &new_table_properties->fast_compression_estimated_data_size},
   };
 
   std::string last_key;
-  for (iter.SeekToFirstOrReport(); iter.Valid(); iter.NextOrReport()) {
-    s = iter.status();
+  for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {
+    s = iter->status();
     if (!s.ok()) {
       break;
     }
 
-    auto key = iter.key().ToString();
+    auto key = iter->key().ToString();
     // properties block should be strictly sorted with no duplicate key.
     if (!last_key.empty() &&
         BytewiseComparator()->Compare(key, last_key) <= 0) {
@@ -297,11 +325,13 @@ Status ReadProperties(const ReadOptions& read_options,
     }
     last_key = key;
 
-    auto raw_val = iter.value();
+    auto raw_val = iter->value();
     auto pos = predefined_uint64_properties.find(key);
 
-    new_table_properties->properties_offsets.insert(
-        {key, handle.offset() + iter.ValueOffset()});
+    if (key == ExternalSstFilePropertyNames::kGlobalSeqno) {
+      new_table_properties->external_sst_file_global_seqno_offset =
+          handle.offset() + iter->ValueOffset();
+    }
 
     if (pos != predefined_uint64_properties.end()) {
       if (key == TablePropertiesNames::kDeletedKeys ||
@@ -315,9 +345,10 @@ Status ReadProperties(const ReadOptions& read_options,
       if (!GetVarint64(&raw_val, &val)) {
         // skip malformed value
         auto error_msg =
-          "Detect malformed value in properties meta-block:"
-          "\tkey: " + key + "\tval: " + raw_val.ToString();
-        ROCKS_LOG_ERROR(ioptions.info_log, "%s", error_msg.c_str());
+            "Detect malformed value in properties meta-block:"
+            "\tkey: " +
+            key + "\tval: " + raw_val.ToString();
+        ROCKS_LOG_ERROR(ioptions.logger, "%s", error_msg.c_str());
         continue;
       }
       *(pos->second) = val;
@@ -343,27 +374,36 @@ Status ReadProperties(const ReadOptions& read_options,
       new_table_properties->compression_name = raw_val.ToString();
     } else if (key == TablePropertiesNames::kCompressionOptions) {
       new_table_properties->compression_options = raw_val.ToString();
+    } else if (key == TablePropertiesNames::kSequenceNumberTimeMapping) {
+      new_table_properties->seqno_to_time_mapping = raw_val.ToString();
     } else {
       // handle user-collected properties
       new_table_properties->user_collected_properties.insert(
           {key, raw_val.ToString()});
     }
   }
-  if (s.ok()) {
-    *table_properties = new_table_properties;
-    if (ret_block_handle != nullptr) {
-      *ret_block_handle = handle;
-    }
-    if (verification_buf != nullptr) {
-      size_t len = static_cast<size_t>(handle.size() + kBlockTrailerSize);
-      *verification_buf =
-          ROCKSDB_NAMESPACE::AllocateBlock(len, memory_allocator);
-      if (verification_buf->get() != nullptr) {
-        memcpy(verification_buf->get(), block_contents.data.data(), len);
+
+  // 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());
+    if (s.IsCorruption()) {
+      if (new_table_properties->external_sst_file_global_seqno_offset != 0) {
+        std::string tmp_buf(properties_block.data(),
+                            block_fetcher.GetBlockSizeWithTrailer());
+        uint64_t global_seqno_offset =
+            new_table_properties->external_sst_file_global_seqno_offset -
+            handle.offset();
+        EncodeFixed64(&tmp_buf[static_cast<size_t>(global_seqno_offset)], 0);
+        s = VerifyBlockChecksum(footer.checksum_type(), tmp_buf.data(),
+                                block_size, file->file_name(), handle.offset());
       }
     }
-  } else {
-    delete new_table_properties;
+  }
+
+  if (s.ok()) {
+    *table_properties = std::move(new_table_properties);
   }
 
   return s;
@@ -371,102 +411,104 @@ Status ReadProperties(const ReadOptions& read_options,
 
 Status ReadTableProperties(RandomAccessFileReader* file, uint64_t file_size,
                            uint64_t table_magic_number,
-                           const ImmutableCFOptions& ioptions,
-                           TableProperties** properties,
-                           bool compression_type_missing,
+                           const ImmutableOptions& ioptions,
+                           std::unique_ptr<TableProperties>* properties,
                            MemoryAllocator* memory_allocator,
                            FilePrefetchBuffer* prefetch_buffer) {
-  // -- Read metaindex block
+  BlockHandle block_handle;
   Footer footer;
-  IOOptions opts;
-  auto s = ReadFooterFromFile(opts, file, prefetch_buffer, file_size, &footer,
-                              table_magic_number);
-  if (!s.ok()) {
-    return s;
-  }
-
-  auto metaindex_handle = footer.metaindex_handle();
-  BlockContents metaindex_contents;
-  ReadOptions read_options;
-  read_options.verify_checksums = false;
-  PersistentCacheOptions cache_options;
-
-  BlockFetcher block_fetcher(
-      file, prefetch_buffer, footer, read_options, metaindex_handle,
-      &metaindex_contents, ioptions, false /* decompress */,
-      false /*maybe_compressed*/, BlockType::kMetaIndex,
-      UncompressionDict::GetEmptyDict(), cache_options, memory_allocator);
-  s = block_fetcher.ReadBlockContents();
-  if (!s.ok()) {
-    return s;
-  }
-  // property blocks are never compressed. Need to add uncompress logic if we
-  // are to compress it.
-  Block metaindex_block(std::move(metaindex_contents));
-  std::unique_ptr<InternalIterator> meta_iter(metaindex_block.NewDataIterator(
-      BytewiseComparator(), kDisableGlobalSequenceNumber));
-
-  // -- Read property block
-  bool found_properties_block = true;
-  s = SeekToPropertiesBlock(meta_iter.get(), &found_properties_block);
+  Status s = FindMetaBlockInFile(file, file_size, table_magic_number, ioptions,
+                                 kPropertiesBlockName, &block_handle,
+                                 memory_allocator, prefetch_buffer, &footer);
   if (!s.ok()) {
     return s;
   }
 
-  TableProperties table_properties;
-  if (found_properties_block == true) {
-    s = ReadProperties(
-        read_options, meta_iter->value(), file, prefetch_buffer, footer,
-        ioptions, properties, false /* verify_checksum */,
-        nullptr /* ret_block_hanel */, nullptr /* ret_block_contents */,
-        compression_type_missing, memory_allocator);
+  if (!block_handle.IsNull()) {
+    s = ReadTablePropertiesHelper(ReadOptions(), block_handle, file,
+                                  prefetch_buffer, footer, ioptions, properties,
+                                  memory_allocator);
   } else {
     s = Status::NotFound();
   }
-
   return s;
 }
 
+Status FindOptionalMetaBlock(InternalIterator* meta_index_iter,
+                             const std::string& meta_block_name,
+                             BlockHandle* block_handle) {
+  assert(block_handle != nullptr);
+  meta_index_iter->Seek(meta_block_name);
+  if (meta_index_iter->status().ok()) {
+    if (meta_index_iter->Valid() && meta_index_iter->key() == meta_block_name) {
+      Slice v = meta_index_iter->value();
+      return block_handle->DecodeFrom(&v);
+    } else if (meta_block_name == kPropertiesBlockName) {
+      // Have to try old name for compatibility
+      meta_index_iter->Seek(kPropertiesBlockOldName);
+      if (meta_index_iter->status().ok() && meta_index_iter->Valid() &&
+          meta_index_iter->key() == kPropertiesBlockOldName) {
+        Slice v = meta_index_iter->value();
+        return block_handle->DecodeFrom(&v);
+      }
+    }
+  }
+  // else
+  *block_handle = BlockHandle::NullBlockHandle();
+  return meta_index_iter->status();
+}
+
 Status FindMetaBlock(InternalIterator* meta_index_iter,
                      const std::string& meta_block_name,
                      BlockHandle* block_handle) {
-  meta_index_iter->Seek(meta_block_name);
-  if (meta_index_iter->status().ok() && meta_index_iter->Valid() &&
-      meta_index_iter->key() == meta_block_name) {
-    Slice v = meta_index_iter->value();
-    return block_handle->DecodeFrom(&v);
-  } else {
+  Status s =
+      FindOptionalMetaBlock(meta_index_iter, meta_block_name, block_handle);
+  if (s.ok() && block_handle->IsNull()) {
     return Status::Corruption("Cannot find the meta block", meta_block_name);
+  } else {
+    return s;
   }
 }
 
-Status FindMetaBlock(RandomAccessFileReader* file, uint64_t file_size,
-                     uint64_t table_magic_number,
-                     const ImmutableCFOptions& ioptions,
-                     const std::string& meta_block_name,
-                     BlockHandle* block_handle,
-                     bool /*compression_type_missing*/,
-                     MemoryAllocator* memory_allocator) {
+Status ReadMetaIndexBlockInFile(RandomAccessFileReader* file,
+                                uint64_t file_size, uint64_t table_magic_number,
+                                const ImmutableOptions& ioptions,
+                                BlockContents* metaindex_contents,
+                                MemoryAllocator* memory_allocator,
+                                FilePrefetchBuffer* prefetch_buffer,
+                                Footer* footer_out) {
   Footer footer;
   IOOptions opts;
-  auto s = ReadFooterFromFile(opts, file, nullptr /* prefetch_buffer */,
-                              file_size, &footer, table_magic_number);
+  auto s = ReadFooterFromFile(opts, file, prefetch_buffer, file_size, &footer,
+                              table_magic_number);
   if (!s.ok()) {
     return s;
   }
+  if (footer_out) {
+    *footer_out = footer;
+  }
 
   auto metaindex_handle = footer.metaindex_handle();
+  return BlockFetcher(file, prefetch_buffer, footer, ReadOptions(),
+                      metaindex_handle, metaindex_contents, ioptions,
+                      false /* do decompression */, false /*maybe_compressed*/,
+                      BlockType::kMetaIndex, UncompressionDict::GetEmptyDict(),
+                      PersistentCacheOptions::kEmpty, memory_allocator)
+      .ReadBlockContents();
+}
+
+Status FindMetaBlockInFile(RandomAccessFileReader* file, uint64_t file_size,
+                           uint64_t table_magic_number,
+                           const ImmutableOptions& ioptions,
+                           const std::string& meta_block_name,
+                           BlockHandle* block_handle,
+                           MemoryAllocator* memory_allocator,
+                           FilePrefetchBuffer* prefetch_buffer,
+                           Footer* footer_out) {
   BlockContents metaindex_contents;
-  ReadOptions read_options;
-  read_options.verify_checksums = false;
-  PersistentCacheOptions cache_options;
-  BlockFetcher block_fetcher(
-      file, nullptr /* prefetch_buffer */, footer, read_options,
-      metaindex_handle, &metaindex_contents, ioptions,
-      false /* do decompression */, false /*maybe_compressed*/,
-      BlockType::kMetaIndex, UncompressionDict::GetEmptyDict(), cache_options,
-      memory_allocator);
-  s = block_fetcher.ReadBlockContents();
+  auto s = ReadMetaIndexBlockInFile(
+      file, file_size, table_magic_number, ioptions, &metaindex_contents,
+      memory_allocator, prefetch_buffer, footer_out);
   if (!s.ok()) {
     return s;
   }
@@ -475,8 +517,7 @@ Status FindMetaBlock(RandomAccessFileReader* file, uint64_t file_size,
   Block metaindex_block(std::move(metaindex_contents));
 
   std::unique_ptr<InternalIterator> meta_iter;
-  meta_iter.reset(metaindex_block.NewDataIterator(
-      BytewiseComparator(), kDisableGlobalSequenceNumber));
+  meta_iter.reset(metaindex_block.NewMetaIterator());
 
   return FindMetaBlock(meta_iter.get(), meta_block_name, block_handle);
 }
@@ -484,58 +525,29 @@ Status FindMetaBlock(RandomAccessFileReader* file, uint64_t file_size,
 Status ReadMetaBlock(RandomAccessFileReader* file,
                      FilePrefetchBuffer* prefetch_buffer, uint64_t file_size,
                      uint64_t table_magic_number,
-                     const ImmutableCFOptions& ioptions,
+                     const ImmutableOptions& ioptions,
                      const std::string& meta_block_name, BlockType block_type,
-                     BlockContents* contents, bool /*compression_type_missing*/,
+                     BlockContents* contents,
                      MemoryAllocator* memory_allocator) {
-  Status status;
-  Footer footer;
-  IOOptions opts;
-  status = ReadFooterFromFile(opts, file, prefetch_buffer, file_size, &footer,
-                              table_magic_number);
-  if (!status.ok()) {
-    return status;
-  }
-
-  // Reading metaindex block
-  auto metaindex_handle = footer.metaindex_handle();
-  BlockContents metaindex_contents;
-  ReadOptions read_options;
-  read_options.verify_checksums = false;
-  PersistentCacheOptions cache_options;
-
-  BlockFetcher block_fetcher(
-      file, prefetch_buffer, footer, read_options, metaindex_handle,
-      &metaindex_contents, ioptions, false /* decompress */,
-      false /*maybe_compressed*/, BlockType::kMetaIndex,
-      UncompressionDict::GetEmptyDict(), cache_options, memory_allocator);
-  status = block_fetcher.ReadBlockContents();
-  if (!status.ok()) {
-    return status;
-  }
-  // meta block is never compressed. Need to add uncompress logic if we are to
-  // compress it.
-
-  // Finding metablock
-  Block metaindex_block(std::move(metaindex_contents));
-
-  std::unique_ptr<InternalIterator> meta_iter;
-  meta_iter.reset(metaindex_block.NewDataIterator(
-      BytewiseComparator(), kDisableGlobalSequenceNumber));
+  // TableProperties requires special handling because of checksum issues.
+  // Call ReadTableProperties instead for that case.
+  assert(block_type != BlockType::kProperties);
 
   BlockHandle block_handle;
-  status = FindMetaBlock(meta_iter.get(), meta_block_name, &block_handle);
-
+  Footer footer;
+  Status status = FindMetaBlockInFile(
+      file, file_size, table_magic_number, ioptions, meta_block_name,
+      &block_handle, memory_allocator, prefetch_buffer, &footer);
   if (!status.ok()) {
     return status;
   }
 
-  // Reading metablock
-  BlockFetcher block_fetcher2(
-      file, prefetch_buffer, footer, read_options, block_handle, contents,
-      ioptions, false /* decompress */, false /*maybe_compressed*/, block_type,
-      UncompressionDict::GetEmptyDict(), cache_options, memory_allocator);
-  return block_fetcher2.ReadBlockContents();
+  return BlockFetcher(file, prefetch_buffer, footer, ReadOptions(),
+                      block_handle, contents, ioptions, false /* decompress */,
+                      false /*maybe_compressed*/, block_type,
+                      UncompressionDict::GetEmptyDict(),
+                      PersistentCacheOptions::kEmpty, memory_allocator)
+      .ReadBlockContents();
 }
 
 }  // namespace ROCKSDB_NAMESPACE