1 //  Copyright (c) 2011-present, Facebook, Inc.  All rights reserved.
2 //  This source code is licensed under both the GPLv2 (found in the
3 //  COPYING file in the root directory) and Apache 2.0 License
4 //  (found in the LICENSE.Apache file in the root directory).
5 #include "table/meta_blocks.h"
6 
7 #include <map>
8 #include <string>
9 
10 #include "block_fetcher.h"
11 #include "db/table_properties_collector.h"
12 #include "file/random_access_file_reader.h"
13 #include "logging/logging.h"
14 #include "rocksdb/table.h"
15 #include "rocksdb/table_properties.h"
16 #include "table/block_based/block.h"
17 #include "table/format.h"
18 #include "table/internal_iterator.h"
19 #include "table/persistent_cache_helper.h"
20 #include "table/table_properties_internal.h"
21 #include "test_util/sync_point.h"
22 #include "util/coding.h"
23 
24 namespace ROCKSDB_NAMESPACE {
25 
MetaIndexBuilder()26 MetaIndexBuilder::MetaIndexBuilder()
27     : meta_index_block_(new BlockBuilder(1 /* restart interval */)) {}
28 
Add(const std::string & key,const BlockHandle & handle)29 void MetaIndexBuilder::Add(const std::string& key,
30                            const BlockHandle& handle) {
31   std::string handle_encoding;
32   handle.EncodeTo(&handle_encoding);
33   meta_block_handles_.insert({key, handle_encoding});
34 }
35 
Finish()36 Slice MetaIndexBuilder::Finish() {
37   for (const auto& metablock : meta_block_handles_) {
38     meta_index_block_->Add(metablock.first, metablock.second);
39   }
40   return meta_index_block_->Finish();
41 }
42 
43 // Property block will be read sequentially and cached in a heap located
44 // object, so there's no need for restart points. Thus we set the restart
45 // interval to infinity to save space.
PropertyBlockBuilder()46 PropertyBlockBuilder::PropertyBlockBuilder()
47     : properties_block_(
48           new BlockBuilder(port::kMaxInt32 /* restart interval */)) {}
49 
Add(const std::string & name,const std::string & val)50 void PropertyBlockBuilder::Add(const std::string& name,
51                                const std::string& val) {
52   props_.insert({name, val});
53 }
54 
Add(const std::string & name,uint64_t val)55 void PropertyBlockBuilder::Add(const std::string& name, uint64_t val) {
56   assert(props_.find(name) == props_.end());
57 
58   std::string dst;
59   PutVarint64(&dst, val);
60 
61   Add(name, dst);
62 }
63 
Add(const UserCollectedProperties & user_collected_properties)64 void PropertyBlockBuilder::Add(
65     const UserCollectedProperties& user_collected_properties) {
66   for (const auto& prop : user_collected_properties) {
67     Add(prop.first, prop.second);
68   }
69 }
70 
AddTableProperty(const TableProperties & props)71 void PropertyBlockBuilder::AddTableProperty(const TableProperties& props) {
72   TEST_SYNC_POINT_CALLBACK("PropertyBlockBuilder::AddTableProperty:Start",
73                            const_cast<TableProperties*>(&props));
74 
75   Add(TablePropertiesNames::kOriginalFileNumber, props.orig_file_number);
76   Add(TablePropertiesNames::kRawKeySize, props.raw_key_size);
77   Add(TablePropertiesNames::kRawValueSize, props.raw_value_size);
78   Add(TablePropertiesNames::kDataSize, props.data_size);
79   Add(TablePropertiesNames::kIndexSize, props.index_size);
80   if (props.index_partitions != 0) {
81     Add(TablePropertiesNames::kIndexPartitions, props.index_partitions);
82     Add(TablePropertiesNames::kTopLevelIndexSize, props.top_level_index_size);
83   }
84   Add(TablePropertiesNames::kIndexKeyIsUserKey, props.index_key_is_user_key);
85   Add(TablePropertiesNames::kIndexValueIsDeltaEncoded,
86       props.index_value_is_delta_encoded);
87   Add(TablePropertiesNames::kNumEntries, props.num_entries);
88   Add(TablePropertiesNames::kNumFilterEntries, props.num_filter_entries);
89   Add(TablePropertiesNames::kDeletedKeys, props.num_deletions);
90   Add(TablePropertiesNames::kMergeOperands, props.num_merge_operands);
91   Add(TablePropertiesNames::kNumRangeDeletions, props.num_range_deletions);
92   Add(TablePropertiesNames::kNumDataBlocks, props.num_data_blocks);
93   Add(TablePropertiesNames::kFilterSize, props.filter_size);
94   Add(TablePropertiesNames::kFormatVersion, props.format_version);
95   Add(TablePropertiesNames::kFixedKeyLen, props.fixed_key_len);
96   Add(TablePropertiesNames::kColumnFamilyId, props.column_family_id);
97   Add(TablePropertiesNames::kCreationTime, props.creation_time);
98   Add(TablePropertiesNames::kOldestKeyTime, props.oldest_key_time);
99   if (props.file_creation_time > 0) {
100     Add(TablePropertiesNames::kFileCreationTime, props.file_creation_time);
101   }
102   if (props.slow_compression_estimated_data_size > 0) {
103     Add(TablePropertiesNames::kSlowCompressionEstimatedDataSize,
104         props.slow_compression_estimated_data_size);
105   }
106   if (props.fast_compression_estimated_data_size > 0) {
107     Add(TablePropertiesNames::kFastCompressionEstimatedDataSize,
108         props.fast_compression_estimated_data_size);
109   }
110   if (!props.db_id.empty()) {
111     Add(TablePropertiesNames::kDbId, props.db_id);
112   }
113   if (!props.db_session_id.empty()) {
114     Add(TablePropertiesNames::kDbSessionId, props.db_session_id);
115   }
116   if (!props.db_host_id.empty()) {
117     Add(TablePropertiesNames::kDbHostId, props.db_host_id);
118   }
119 
120   if (!props.filter_policy_name.empty()) {
121     Add(TablePropertiesNames::kFilterPolicy, props.filter_policy_name);
122   }
123   if (!props.comparator_name.empty()) {
124     Add(TablePropertiesNames::kComparator, props.comparator_name);
125   }
126 
127   if (!props.merge_operator_name.empty()) {
128     Add(TablePropertiesNames::kMergeOperator, props.merge_operator_name);
129   }
130   if (!props.prefix_extractor_name.empty()) {
131     Add(TablePropertiesNames::kPrefixExtractorName,
132         props.prefix_extractor_name);
133   }
134   if (!props.property_collectors_names.empty()) {
135     Add(TablePropertiesNames::kPropertyCollectors,
136         props.property_collectors_names);
137   }
138   if (!props.column_family_name.empty()) {
139     Add(TablePropertiesNames::kColumnFamilyName, props.column_family_name);
140   }
141 
142   if (!props.compression_name.empty()) {
143     Add(TablePropertiesNames::kCompression, props.compression_name);
144   }
145   if (!props.compression_options.empty()) {
146     Add(TablePropertiesNames::kCompressionOptions, props.compression_options);
147   }
148 }
149 
Finish()150 Slice PropertyBlockBuilder::Finish() {
151   for (const auto& prop : props_) {
152     properties_block_->Add(prop.first, prop.second);
153   }
154 
155   return properties_block_->Finish();
156 }
157 
LogPropertiesCollectionError(Logger * info_log,const std::string & method,const std::string & name)158 void LogPropertiesCollectionError(Logger* info_log, const std::string& method,
159                                   const std::string& name) {
160   assert(method == "Add" || method == "Finish");
161 
162   std::string msg =
163     "Encountered error when calling TablePropertiesCollector::" +
164     method + "() with collector name: " + name;
165   ROCKS_LOG_ERROR(info_log, "%s", msg.c_str());
166 }
167 
NotifyCollectTableCollectorsOnAdd(const Slice & key,const Slice & value,uint64_t file_size,const std::vector<std::unique_ptr<IntTblPropCollector>> & collectors,Logger * info_log)168 bool NotifyCollectTableCollectorsOnAdd(
169     const Slice& key, const Slice& value, uint64_t file_size,
170     const std::vector<std::unique_ptr<IntTblPropCollector>>& collectors,
171     Logger* info_log) {
172   bool all_succeeded = true;
173   for (auto& collector : collectors) {
174     Status s = collector->InternalAdd(key, value, file_size);
175     all_succeeded = all_succeeded && s.ok();
176     if (!s.ok()) {
177       LogPropertiesCollectionError(info_log, "Add" /* method */,
178                                    collector->Name());
179     }
180   }
181   return all_succeeded;
182 }
183 
NotifyCollectTableCollectorsOnBlockAdd(const std::vector<std::unique_ptr<IntTblPropCollector>> & collectors,const uint64_t block_raw_bytes,const uint64_t block_compressed_bytes_fast,const uint64_t block_compressed_bytes_slow)184 void NotifyCollectTableCollectorsOnBlockAdd(
185     const std::vector<std::unique_ptr<IntTblPropCollector>>& collectors,
186     const uint64_t block_raw_bytes, const uint64_t block_compressed_bytes_fast,
187     const uint64_t block_compressed_bytes_slow) {
188   for (auto& collector : collectors) {
189     collector->BlockAdd(block_raw_bytes, block_compressed_bytes_fast,
190                         block_compressed_bytes_slow);
191   }
192 }
193 
NotifyCollectTableCollectorsOnFinish(const std::vector<std::unique_ptr<IntTblPropCollector>> & collectors,Logger * info_log,PropertyBlockBuilder * builder)194 bool NotifyCollectTableCollectorsOnFinish(
195     const std::vector<std::unique_ptr<IntTblPropCollector>>& collectors,
196     Logger* info_log, PropertyBlockBuilder* builder) {
197   bool all_succeeded = true;
198   for (auto& collector : collectors) {
199     UserCollectedProperties user_collected_properties;
200     Status s = collector->Finish(&user_collected_properties);
201 
202     all_succeeded = all_succeeded && s.ok();
203     if (!s.ok()) {
204       LogPropertiesCollectionError(info_log, "Finish" /* method */,
205                                    collector->Name());
206     } else {
207       builder->Add(user_collected_properties);
208     }
209   }
210 
211   return all_succeeded;
212 }
213 
ReadProperties(const ReadOptions & read_options,const Slice & handle_value,RandomAccessFileReader * file,FilePrefetchBuffer * prefetch_buffer,const Footer & footer,const ImmutableOptions & ioptions,TableProperties ** table_properties,bool verify_checksum,BlockHandle * ret_block_handle,CacheAllocationPtr * verification_buf,bool,MemoryAllocator * memory_allocator)214 Status ReadProperties(const ReadOptions& read_options,
215                       const Slice& handle_value, RandomAccessFileReader* file,
216                       FilePrefetchBuffer* prefetch_buffer, const Footer& footer,
217                       const ImmutableOptions& ioptions,
218                       TableProperties** table_properties, bool verify_checksum,
219                       BlockHandle* ret_block_handle,
220                       CacheAllocationPtr* verification_buf,
221                       bool /*compression_type_missing*/,
222                       MemoryAllocator* memory_allocator) {
223   assert(table_properties);
224 
225   Slice v = handle_value;
226   BlockHandle handle;
227   if (!handle.DecodeFrom(&v).ok()) {
228     return Status::InvalidArgument("Failed to decode properties block handle");
229   }
230 
231   BlockContents block_contents;
232   Status s;
233   // FIXME: should be a parameter for reading table properties to use persistent
234   // cache
235   PersistentCacheOptions cache_options;
236   ReadOptions ro = read_options;
237   ro.verify_checksums = verify_checksum;
238 
239   BlockFetcher block_fetcher(file, prefetch_buffer, footer, ro, handle,
240                              &block_contents, ioptions, false /* decompress */,
241                              false /*maybe_compressed*/, BlockType::kProperties,
242                              UncompressionDict::GetEmptyDict(), cache_options,
243                              memory_allocator);
244   s = block_fetcher.ReadBlockContents();
245   // property block is never compressed. Need to add uncompress logic if we are
246   // to compress it..
247 
248   if (!s.ok()) {
249     return s;
250   }
251 
252   Block properties_block(std::move(block_contents));
253   DataBlockIter iter;
254   properties_block.NewDataIterator(BytewiseComparator(),
255                                    kDisableGlobalSequenceNumber, &iter);
256 
257   auto new_table_properties = new TableProperties();
258   // All pre-defined properties of type uint64_t
259   std::unordered_map<std::string, uint64_t*> predefined_uint64_properties = {
260       {TablePropertiesNames::kOriginalFileNumber,
261        &new_table_properties->orig_file_number},
262       {TablePropertiesNames::kDataSize, &new_table_properties->data_size},
263       {TablePropertiesNames::kIndexSize, &new_table_properties->index_size},
264       {TablePropertiesNames::kIndexPartitions,
265        &new_table_properties->index_partitions},
266       {TablePropertiesNames::kTopLevelIndexSize,
267        &new_table_properties->top_level_index_size},
268       {TablePropertiesNames::kIndexKeyIsUserKey,
269        &new_table_properties->index_key_is_user_key},
270       {TablePropertiesNames::kIndexValueIsDeltaEncoded,
271        &new_table_properties->index_value_is_delta_encoded},
272       {TablePropertiesNames::kFilterSize, &new_table_properties->filter_size},
273       {TablePropertiesNames::kRawKeySize, &new_table_properties->raw_key_size},
274       {TablePropertiesNames::kRawValueSize,
275        &new_table_properties->raw_value_size},
276       {TablePropertiesNames::kNumDataBlocks,
277        &new_table_properties->num_data_blocks},
278       {TablePropertiesNames::kNumEntries, &new_table_properties->num_entries},
279       {TablePropertiesNames::kNumFilterEntries,
280        &new_table_properties->num_filter_entries},
281       {TablePropertiesNames::kDeletedKeys,
282        &new_table_properties->num_deletions},
283       {TablePropertiesNames::kMergeOperands,
284        &new_table_properties->num_merge_operands},
285       {TablePropertiesNames::kNumRangeDeletions,
286        &new_table_properties->num_range_deletions},
287       {TablePropertiesNames::kFormatVersion,
288        &new_table_properties->format_version},
289       {TablePropertiesNames::kFixedKeyLen,
290        &new_table_properties->fixed_key_len},
291       {TablePropertiesNames::kColumnFamilyId,
292        &new_table_properties->column_family_id},
293       {TablePropertiesNames::kCreationTime,
294        &new_table_properties->creation_time},
295       {TablePropertiesNames::kOldestKeyTime,
296        &new_table_properties->oldest_key_time},
297       {TablePropertiesNames::kFileCreationTime,
298        &new_table_properties->file_creation_time},
299       {TablePropertiesNames::kSlowCompressionEstimatedDataSize,
300        &new_table_properties->slow_compression_estimated_data_size},
301       {TablePropertiesNames::kFastCompressionEstimatedDataSize,
302        &new_table_properties->fast_compression_estimated_data_size},
303   };
304 
305   std::string last_key;
306   for (iter.SeekToFirstOrReport(); iter.Valid(); iter.NextOrReport()) {
307     s = iter.status();
308     if (!s.ok()) {
309       break;
310     }
311 
312     auto key = iter.key().ToString();
313     // properties block should be strictly sorted with no duplicate key.
314     if (!last_key.empty() &&
315         BytewiseComparator()->Compare(key, last_key) <= 0) {
316       s = Status::Corruption("properties unsorted");
317       break;
318     }
319     last_key = key;
320 
321     auto raw_val = iter.value();
322     auto pos = predefined_uint64_properties.find(key);
323 
324     new_table_properties->properties_offsets.insert(
325         {key, handle.offset() + iter.ValueOffset()});
326 
327     if (pos != predefined_uint64_properties.end()) {
328       if (key == TablePropertiesNames::kDeletedKeys ||
329           key == TablePropertiesNames::kMergeOperands) {
330         // Insert in user-collected properties for API backwards compatibility
331         new_table_properties->user_collected_properties.insert(
332             {key, raw_val.ToString()});
333       }
334       // handle predefined rocksdb properties
335       uint64_t val;
336       if (!GetVarint64(&raw_val, &val)) {
337         // skip malformed value
338         auto error_msg =
339           "Detect malformed value in properties meta-block:"
340           "\tkey: " + key + "\tval: " + raw_val.ToString();
341         ROCKS_LOG_ERROR(ioptions.logger, "%s", error_msg.c_str());
342         continue;
343       }
344       *(pos->second) = val;
345     } else if (key == TablePropertiesNames::kDbId) {
346       new_table_properties->db_id = raw_val.ToString();
347     } else if (key == TablePropertiesNames::kDbSessionId) {
348       new_table_properties->db_session_id = raw_val.ToString();
349     } else if (key == TablePropertiesNames::kDbHostId) {
350       new_table_properties->db_host_id = raw_val.ToString();
351     } else if (key == TablePropertiesNames::kFilterPolicy) {
352       new_table_properties->filter_policy_name = raw_val.ToString();
353     } else if (key == TablePropertiesNames::kColumnFamilyName) {
354       new_table_properties->column_family_name = raw_val.ToString();
355     } else if (key == TablePropertiesNames::kComparator) {
356       new_table_properties->comparator_name = raw_val.ToString();
357     } else if (key == TablePropertiesNames::kMergeOperator) {
358       new_table_properties->merge_operator_name = raw_val.ToString();
359     } else if (key == TablePropertiesNames::kPrefixExtractorName) {
360       new_table_properties->prefix_extractor_name = raw_val.ToString();
361     } else if (key == TablePropertiesNames::kPropertyCollectors) {
362       new_table_properties->property_collectors_names = raw_val.ToString();
363     } else if (key == TablePropertiesNames::kCompression) {
364       new_table_properties->compression_name = raw_val.ToString();
365     } else if (key == TablePropertiesNames::kCompressionOptions) {
366       new_table_properties->compression_options = raw_val.ToString();
367     } else {
368       // handle user-collected properties
369       new_table_properties->user_collected_properties.insert(
370           {key, raw_val.ToString()});
371     }
372   }
373   if (s.ok()) {
374     *table_properties = new_table_properties;
375     if (ret_block_handle != nullptr) {
376       *ret_block_handle = handle;
377     }
378     if (verification_buf != nullptr) {
379       size_t len = static_cast<size_t>(handle.size() + kBlockTrailerSize);
380       *verification_buf =
381           ROCKSDB_NAMESPACE::AllocateBlock(len, memory_allocator);
382       if (verification_buf->get() != nullptr) {
383         memcpy(verification_buf->get(), block_contents.data.data(), len);
384       }
385     }
386   } else {
387     delete new_table_properties;
388   }
389 
390   return s;
391 }
392 
ReadTableProperties(RandomAccessFileReader * file,uint64_t file_size,uint64_t table_magic_number,const ImmutableOptions & ioptions,TableProperties ** properties,bool compression_type_missing,MemoryAllocator * memory_allocator,FilePrefetchBuffer * prefetch_buffer)393 Status ReadTableProperties(RandomAccessFileReader* file, uint64_t file_size,
394                            uint64_t table_magic_number,
395                            const ImmutableOptions& ioptions,
396                            TableProperties** properties,
397                            bool compression_type_missing,
398                            MemoryAllocator* memory_allocator,
399                            FilePrefetchBuffer* prefetch_buffer) {
400   // -- Read metaindex block
401   Footer footer;
402   IOOptions opts;
403   auto s = ReadFooterFromFile(opts, file, prefetch_buffer, file_size, &footer,
404                               table_magic_number);
405   if (!s.ok()) {
406     return s;
407   }
408 
409   auto metaindex_handle = footer.metaindex_handle();
410   BlockContents metaindex_contents;
411   ReadOptions read_options;
412   read_options.verify_checksums = false;
413   PersistentCacheOptions cache_options;
414 
415   BlockFetcher block_fetcher(
416       file, prefetch_buffer, footer, read_options, metaindex_handle,
417       &metaindex_contents, ioptions, false /* decompress */,
418       false /*maybe_compressed*/, BlockType::kMetaIndex,
419       UncompressionDict::GetEmptyDict(), cache_options, memory_allocator);
420   s = block_fetcher.ReadBlockContents();
421   if (!s.ok()) {
422     return s;
423   }
424   // property blocks are never compressed. Need to add uncompress logic if we
425   // are to compress it.
426   Block metaindex_block(std::move(metaindex_contents));
427   std::unique_ptr<InternalIterator> meta_iter(metaindex_block.NewDataIterator(
428       BytewiseComparator(), kDisableGlobalSequenceNumber));
429 
430   // -- Read property block
431   bool found_properties_block = true;
432   s = SeekToPropertiesBlock(meta_iter.get(), &found_properties_block);
433   if (!s.ok()) {
434     return s;
435   }
436 
437   TableProperties table_properties;
438   if (found_properties_block == true) {
439     s = ReadProperties(
440         read_options, meta_iter->value(), file, prefetch_buffer, footer,
441         ioptions, properties, false /* verify_checksum */,
442         nullptr /* ret_block_hanel */, nullptr /* ret_block_contents */,
443         compression_type_missing, memory_allocator);
444   } else {
445     s = Status::NotFound();
446   }
447 
448   return s;
449 }
450 
FindMetaBlock(InternalIterator * meta_index_iter,const std::string & meta_block_name,BlockHandle * block_handle)451 Status FindMetaBlock(InternalIterator* meta_index_iter,
452                      const std::string& meta_block_name,
453                      BlockHandle* block_handle) {
454   meta_index_iter->Seek(meta_block_name);
455   if (meta_index_iter->status().ok() && meta_index_iter->Valid() &&
456       meta_index_iter->key() == meta_block_name) {
457     Slice v = meta_index_iter->value();
458     return block_handle->DecodeFrom(&v);
459   } else {
460     return Status::Corruption("Cannot find the meta block", meta_block_name);
461   }
462 }
463 
FindMetaBlock(RandomAccessFileReader * file,uint64_t file_size,uint64_t table_magic_number,const ImmutableOptions & ioptions,const std::string & meta_block_name,BlockHandle * block_handle,bool,MemoryAllocator * memory_allocator)464 Status FindMetaBlock(RandomAccessFileReader* file, uint64_t file_size,
465                      uint64_t table_magic_number,
466                      const ImmutableOptions& ioptions,
467                      const std::string& meta_block_name,
468                      BlockHandle* block_handle,
469                      bool /*compression_type_missing*/,
470                      MemoryAllocator* memory_allocator) {
471   Footer footer;
472   IOOptions opts;
473   auto s = ReadFooterFromFile(opts, file, nullptr /* prefetch_buffer */,
474                               file_size, &footer, table_magic_number);
475   if (!s.ok()) {
476     return s;
477   }
478 
479   auto metaindex_handle = footer.metaindex_handle();
480   BlockContents metaindex_contents;
481   ReadOptions read_options;
482   read_options.verify_checksums = false;
483   PersistentCacheOptions cache_options;
484   BlockFetcher block_fetcher(
485       file, nullptr /* prefetch_buffer */, footer, read_options,
486       metaindex_handle, &metaindex_contents, ioptions,
487       false /* do decompression */, false /*maybe_compressed*/,
488       BlockType::kMetaIndex, UncompressionDict::GetEmptyDict(), cache_options,
489       memory_allocator);
490   s = block_fetcher.ReadBlockContents();
491   if (!s.ok()) {
492     return s;
493   }
494   // meta blocks are never compressed. Need to add uncompress logic if we are to
495   // compress it.
496   Block metaindex_block(std::move(metaindex_contents));
497 
498   std::unique_ptr<InternalIterator> meta_iter;
499   meta_iter.reset(metaindex_block.NewDataIterator(
500       BytewiseComparator(), kDisableGlobalSequenceNumber));
501 
502   return FindMetaBlock(meta_iter.get(), meta_block_name, block_handle);
503 }
504 
ReadMetaBlock(RandomAccessFileReader * file,FilePrefetchBuffer * prefetch_buffer,uint64_t file_size,uint64_t table_magic_number,const ImmutableOptions & ioptions,const std::string & meta_block_name,BlockType block_type,BlockContents * contents,bool,MemoryAllocator * memory_allocator)505 Status ReadMetaBlock(RandomAccessFileReader* file,
506                      FilePrefetchBuffer* prefetch_buffer, uint64_t file_size,
507                      uint64_t table_magic_number,
508                      const ImmutableOptions& ioptions,
509                      const std::string& meta_block_name, BlockType block_type,
510                      BlockContents* contents, bool /*compression_type_missing*/,
511                      MemoryAllocator* memory_allocator) {
512   Status status;
513   Footer footer;
514   IOOptions opts;
515   status = ReadFooterFromFile(opts, file, prefetch_buffer, file_size, &footer,
516                               table_magic_number);
517   if (!status.ok()) {
518     return status;
519   }
520 
521   // Reading metaindex block
522   auto metaindex_handle = footer.metaindex_handle();
523   BlockContents metaindex_contents;
524   ReadOptions read_options;
525   read_options.verify_checksums = false;
526   PersistentCacheOptions cache_options;
527 
528   BlockFetcher block_fetcher(
529       file, prefetch_buffer, footer, read_options, metaindex_handle,
530       &metaindex_contents, ioptions, false /* decompress */,
531       false /*maybe_compressed*/, BlockType::kMetaIndex,
532       UncompressionDict::GetEmptyDict(), cache_options, memory_allocator);
533   status = block_fetcher.ReadBlockContents();
534   if (!status.ok()) {
535     return status;
536   }
537   // meta block is never compressed. Need to add uncompress logic if we are to
538   // compress it.
539 
540   // Finding metablock
541   Block metaindex_block(std::move(metaindex_contents));
542 
543   std::unique_ptr<InternalIterator> meta_iter;
544   meta_iter.reset(metaindex_block.NewDataIterator(
545       BytewiseComparator(), kDisableGlobalSequenceNumber));
546 
547   BlockHandle block_handle;
548   status = FindMetaBlock(meta_iter.get(), meta_block_name, &block_handle);
549 
550   if (!status.ok()) {
551     return status;
552   }
553 
554   // Reading metablock
555   BlockFetcher block_fetcher2(
556       file, prefetch_buffer, footer, read_options, block_handle, contents,
557       ioptions, false /* decompress */, false /*maybe_compressed*/, block_type,
558       UncompressionDict::GetEmptyDict(), cache_options, memory_allocator);
559   return block_fetcher2.ReadBlockContents();
560 }
561 
562 }  // namespace ROCKSDB_NAMESPACE
563