table_cache.cc 28 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744
  1. // Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
  2. // This source code is licensed under both the GPLv2 (found in the
  3. // COPYING file in the root directory) and Apache 2.0 License
  4. // (found in the LICENSE.Apache file in the root directory).
  5. //
  6. // Copyright (c) 2011 The LevelDB Authors. All rights reserved.
  7. // Use of this source code is governed by a BSD-style license that can be
  8. // found in the LICENSE file. See the AUTHORS file for names of contributors.
  9. #include "db/table_cache.h"
  10. #include "db/dbformat.h"
  11. #include "db/range_tombstone_fragmenter.h"
  12. #include "db/snapshot_impl.h"
  13. #include "db/version_edit.h"
  14. #include "file/file_util.h"
  15. #include "file/filename.h"
  16. #include "file/random_access_file_reader.h"
  17. #include "monitoring/perf_context_imp.h"
  18. #include "rocksdb/advanced_options.h"
  19. #include "rocksdb/statistics.h"
  20. #include "table/block_based/block_based_table_reader.h"
  21. #include "table/get_context.h"
  22. #include "table/internal_iterator.h"
  23. #include "table/iterator_wrapper.h"
  24. #include "table/multiget_context.h"
  25. #include "table/table_builder.h"
  26. #include "table/table_reader.h"
  27. #include "test_util/sync_point.h"
  28. #include "util/cast_util.h"
  29. #include "util/coding.h"
  30. #include "util/stop_watch.h"
  31. // Generate the regular and coroutine versions of some methods by
  32. // including table_cache_sync_and_async.h twice
  33. // Macros in the header will expand differently based on whether
  34. // WITH_COROUTINES or WITHOUT_COROUTINES is defined
  35. // clang-format off
  36. #define WITHOUT_COROUTINES
  37. #include "db/table_cache_sync_and_async.h"
  38. #undef WITHOUT_COROUTINES
  39. #define WITH_COROUTINES
  40. #include "db/table_cache_sync_and_async.h"
  41. #undef WITH_COROUTINES
  42. // clang-format on
  43. namespace ROCKSDB_NAMESPACE {
  44. namespace {
  45. static Slice GetSliceForFileNumber(const uint64_t* file_number) {
  46. return Slice(reinterpret_cast<const char*>(file_number),
  47. sizeof(*file_number));
  48. }
  49. void AppendVarint64(IterKey* key, uint64_t v) {
  50. char buf[10];
  51. auto ptr = EncodeVarint64(buf, v);
  52. key->TrimAppend(key->Size(), buf, ptr - buf);
  53. }
  54. } // anonymous namespace
  55. const int kLoadConcurency = 128;
  56. TableCache::TableCache(const ImmutableOptions& ioptions,
  57. const FileOptions* file_options, Cache* const cache,
  58. BlockCacheTracer* const block_cache_tracer,
  59. const std::shared_ptr<IOTracer>& io_tracer,
  60. const std::string& db_session_id)
  61. : ioptions_(ioptions),
  62. file_options_(*file_options),
  63. cache_(cache),
  64. immortal_tables_(false),
  65. block_cache_tracer_(block_cache_tracer),
  66. loader_mutex_(kLoadConcurency),
  67. io_tracer_(io_tracer),
  68. db_session_id_(db_session_id) {
  69. if (ioptions_.row_cache) {
  70. // If the same cache is shared by multiple instances, we need to
  71. // disambiguate its entries.
  72. PutVarint64(&row_cache_id_, ioptions_.row_cache->NewId());
  73. }
  74. }
  75. TableCache::~TableCache() = default;
  76. Status TableCache::GetTableReader(
  77. const ReadOptions& ro, const FileOptions& file_options,
  78. const InternalKeyComparator& internal_comparator,
  79. const FileMetaData& file_meta, bool sequential_mode,
  80. HistogramImpl* file_read_hist, std::unique_ptr<TableReader>* table_reader,
  81. const MutableCFOptions& mutable_cf_options, bool skip_filters, int level,
  82. bool prefetch_index_and_filter_in_cache,
  83. size_t max_file_size_for_l0_meta_pin, Temperature file_temperature) {
  84. std::string fname = TableFileName(
  85. ioptions_.cf_paths, file_meta.fd.GetNumber(), file_meta.fd.GetPathId());
  86. std::unique_ptr<FSRandomAccessFile> file;
  87. FileOptions fopts = file_options;
  88. fopts.temperature = file_temperature;
  89. Status s = PrepareIOFromReadOptions(ro, ioptions_.clock, fopts.io_options);
  90. TEST_SYNC_POINT_CALLBACK("TableCache::GetTableReader:BeforeOpenFile",
  91. const_cast<Status*>(&s));
  92. if (s.ok()) {
  93. s = ioptions_.fs->NewRandomAccessFile(fname, fopts, &file, nullptr);
  94. }
  95. if (s.ok()) {
  96. RecordTick(ioptions_.stats, NO_FILE_OPENS);
  97. } else if (s.IsPathNotFound()) {
  98. fname = Rocks2LevelTableFileName(fname);
  99. // If this file is also not found, we want to use the error message
  100. // that contains the table file name which is less confusing.
  101. Status temp_s =
  102. PrepareIOFromReadOptions(ro, ioptions_.clock, fopts.io_options);
  103. if (temp_s.ok()) {
  104. temp_s = ioptions_.fs->NewRandomAccessFile(fname, file_options, &file,
  105. nullptr);
  106. }
  107. if (temp_s.ok()) {
  108. RecordTick(ioptions_.stats, NO_FILE_OPENS);
  109. s = temp_s;
  110. }
  111. }
  112. if (s.ok()) {
  113. if (!sequential_mode && ioptions_.advise_random_on_open) {
  114. file->Hint(FSRandomAccessFile::kRandom);
  115. }
  116. if (ioptions_.default_temperature != Temperature::kUnknown &&
  117. file_temperature == Temperature::kUnknown) {
  118. file_temperature = ioptions_.default_temperature;
  119. }
  120. StopWatch sw(ioptions_.clock, ioptions_.stats, TABLE_OPEN_IO_MICROS);
  121. std::unique_ptr<RandomAccessFileReader> file_reader(
  122. new RandomAccessFileReader(std::move(file), fname, ioptions_.clock,
  123. io_tracer_, ioptions_.stats, SST_READ_MICROS,
  124. file_read_hist, ioptions_.rate_limiter.get(),
  125. ioptions_.listeners, file_temperature,
  126. level == ioptions_.num_levels - 1));
  127. UniqueId64x2 expected_unique_id;
  128. if (ioptions_.verify_sst_unique_id_in_manifest) {
  129. expected_unique_id = file_meta.unique_id;
  130. } else {
  131. expected_unique_id = kNullUniqueId64x2; // null ID == no verification
  132. }
  133. s = mutable_cf_options.table_factory->NewTableReader(
  134. ro,
  135. TableReaderOptions(
  136. ioptions_, mutable_cf_options.prefix_extractor,
  137. mutable_cf_options.compression_manager.get(), file_options,
  138. internal_comparator,
  139. mutable_cf_options.block_protection_bytes_per_key, skip_filters,
  140. immortal_tables_, false /* force_direct_prefetch */, level,
  141. block_cache_tracer_, max_file_size_for_l0_meta_pin, db_session_id_,
  142. file_meta.fd.GetNumber(), expected_unique_id,
  143. file_meta.fd.largest_seqno, file_meta.tail_size,
  144. file_meta.user_defined_timestamps_persisted),
  145. std::move(file_reader), file_meta.fd.GetFileSize(), table_reader,
  146. prefetch_index_and_filter_in_cache);
  147. TEST_SYNC_POINT("TableCache::GetTableReader:0");
  148. }
  149. return s;
  150. }
  151. Cache::Handle* TableCache::Lookup(Cache* cache, uint64_t file_number) {
  152. // NOTE: sharing same Cache with BlobFileCache
  153. Slice key = GetSliceForFileNumber(&file_number);
  154. return cache->Lookup(key);
  155. }
  156. Status TableCache::FindTable(
  157. const ReadOptions& ro, const FileOptions& file_options,
  158. const InternalKeyComparator& internal_comparator,
  159. const FileMetaData& file_meta, TypedHandle** handle,
  160. const MutableCFOptions& mutable_cf_options, const bool no_io,
  161. HistogramImpl* file_read_hist, bool skip_filters, int level,
  162. bool prefetch_index_and_filter_in_cache,
  163. size_t max_file_size_for_l0_meta_pin, Temperature file_temperature) {
  164. PERF_TIMER_GUARD_WITH_CLOCK(find_table_nanos, ioptions_.clock);
  165. uint64_t number = file_meta.fd.GetNumber();
  166. // NOTE: sharing same Cache with BlobFileCache
  167. Slice key = GetSliceForFileNumber(&number);
  168. *handle = cache_.Lookup(key);
  169. TEST_SYNC_POINT_CALLBACK("TableCache::FindTable:0",
  170. const_cast<bool*>(&no_io));
  171. if (*handle == nullptr) {
  172. if (no_io) {
  173. return Status::Incomplete("Table not found in table_cache, no_io is set");
  174. }
  175. MutexLock load_lock(&loader_mutex_.Get(key));
  176. // We check the cache again under loading mutex
  177. *handle = cache_.Lookup(key);
  178. if (*handle != nullptr) {
  179. return Status::OK();
  180. }
  181. std::unique_ptr<TableReader> table_reader;
  182. Status s = GetTableReader(ro, file_options, internal_comparator, file_meta,
  183. false /* sequential mode */, file_read_hist,
  184. &table_reader, mutable_cf_options, skip_filters,
  185. level, prefetch_index_and_filter_in_cache,
  186. max_file_size_for_l0_meta_pin, file_temperature);
  187. if (!s.ok()) {
  188. assert(table_reader == nullptr);
  189. RecordTick(ioptions_.stats, NO_FILE_ERRORS);
  190. // We do not cache error results so that if the error is transient,
  191. // or somebody repairs the file, we recover automatically.
  192. IGNORE_STATUS_IF_ERROR(s);
  193. } else {
  194. s = cache_.Insert(key, table_reader.get(), 1, handle);
  195. if (s.ok()) {
  196. // Release ownership of table reader.
  197. table_reader.release();
  198. }
  199. }
  200. return s;
  201. }
  202. return Status::OK();
  203. }
  204. InternalIterator* TableCache::NewIterator(
  205. const ReadOptions& options, const FileOptions& file_options,
  206. const InternalKeyComparator& icomparator, const FileMetaData& file_meta,
  207. RangeDelAggregator* range_del_agg,
  208. const MutableCFOptions& mutable_cf_options, TableReader** table_reader_ptr,
  209. HistogramImpl* file_read_hist, TableReaderCaller caller, Arena* arena,
  210. bool skip_filters, int level, size_t max_file_size_for_l0_meta_pin,
  211. const InternalKey* smallest_compaction_key,
  212. const InternalKey* largest_compaction_key, bool allow_unprepared_value,
  213. const SequenceNumber* read_seqno,
  214. std::unique_ptr<TruncatedRangeDelIterator>* range_del_iter) {
  215. PERF_TIMER_GUARD(new_table_iterator_nanos);
  216. Status s;
  217. TableReader* table_reader = nullptr;
  218. TypedHandle* handle = nullptr;
  219. if (table_reader_ptr != nullptr) {
  220. *table_reader_ptr = nullptr;
  221. }
  222. bool for_compaction = caller == TableReaderCaller::kCompaction;
  223. auto& fd = file_meta.fd;
  224. table_reader = fd.table_reader;
  225. if (table_reader == nullptr) {
  226. s = FindTable(options, file_options, icomparator, file_meta, &handle,
  227. mutable_cf_options,
  228. options.read_tier == kBlockCacheTier /* no_io */,
  229. file_read_hist, skip_filters, level,
  230. true /* prefetch_index_and_filter_in_cache */,
  231. max_file_size_for_l0_meta_pin, file_meta.temperature);
  232. if (s.ok()) {
  233. table_reader = cache_.Value(handle);
  234. }
  235. }
  236. InternalIterator* result = nullptr;
  237. if (s.ok()) {
  238. if (options.table_filter &&
  239. !options.table_filter(*table_reader->GetTableProperties())) {
  240. result = NewEmptyInternalIterator<Slice>(arena);
  241. } else {
  242. result = table_reader->NewIterator(
  243. options, mutable_cf_options.prefix_extractor.get(), arena,
  244. skip_filters, caller, file_options.compaction_readahead_size,
  245. allow_unprepared_value);
  246. }
  247. if (handle != nullptr) {
  248. cache_.RegisterReleaseAsCleanup(handle, *result);
  249. handle = nullptr; // prevent from releasing below
  250. }
  251. if (for_compaction) {
  252. table_reader->SetupForCompaction();
  253. }
  254. if (table_reader_ptr != nullptr) {
  255. *table_reader_ptr = table_reader;
  256. }
  257. }
  258. if (s.ok() && !options.ignore_range_deletions) {
  259. if (range_del_iter != nullptr) {
  260. auto new_range_del_iter =
  261. read_seqno ? table_reader->NewRangeTombstoneIterator(
  262. *read_seqno, options.timestamp)
  263. : table_reader->NewRangeTombstoneIterator(options);
  264. if (new_range_del_iter == nullptr || new_range_del_iter->empty()) {
  265. delete new_range_del_iter;
  266. *range_del_iter = nullptr;
  267. } else {
  268. *range_del_iter = std::make_unique<TruncatedRangeDelIterator>(
  269. std::unique_ptr<FragmentedRangeTombstoneIterator>(
  270. new_range_del_iter),
  271. &icomparator, &file_meta.smallest, &file_meta.largest);
  272. }
  273. }
  274. if (range_del_agg != nullptr) {
  275. if (range_del_agg->AddFile(fd.GetNumber())) {
  276. std::unique_ptr<FragmentedRangeTombstoneIterator> new_range_del_iter(
  277. static_cast<FragmentedRangeTombstoneIterator*>(
  278. table_reader->NewRangeTombstoneIterator(options)));
  279. if (new_range_del_iter != nullptr) {
  280. s = new_range_del_iter->status();
  281. }
  282. if (s.ok()) {
  283. const InternalKey* smallest = &file_meta.smallest;
  284. const InternalKey* largest = &file_meta.largest;
  285. if (smallest_compaction_key != nullptr) {
  286. smallest = smallest_compaction_key;
  287. }
  288. if (largest_compaction_key != nullptr) {
  289. largest = largest_compaction_key;
  290. }
  291. range_del_agg->AddTombstones(std::move(new_range_del_iter), smallest,
  292. largest);
  293. }
  294. }
  295. }
  296. }
  297. if (handle != nullptr) {
  298. cache_.Release(handle);
  299. }
  300. if (!s.ok()) {
  301. assert(result == nullptr);
  302. result = NewErrorInternalIterator<Slice>(s, arena);
  303. }
  304. return result;
  305. }
  306. Status TableCache::GetRangeTombstoneIterator(
  307. const ReadOptions& options,
  308. const InternalKeyComparator& internal_comparator,
  309. const FileMetaData& file_meta, const MutableCFOptions& mutable_cf_options,
  310. std::unique_ptr<FragmentedRangeTombstoneIterator>* out_iter) {
  311. assert(out_iter);
  312. const FileDescriptor& fd = file_meta.fd;
  313. Status s;
  314. TableReader* t = fd.table_reader;
  315. TypedHandle* handle = nullptr;
  316. if (t == nullptr) {
  317. s = FindTable(options, file_options_, internal_comparator, file_meta,
  318. &handle, mutable_cf_options);
  319. if (s.ok()) {
  320. t = cache_.Value(handle);
  321. }
  322. }
  323. if (s.ok()) {
  324. // Note: NewRangeTombstoneIterator could return nullptr
  325. out_iter->reset(t->NewRangeTombstoneIterator(options));
  326. }
  327. if (handle) {
  328. if (*out_iter) {
  329. cache_.RegisterReleaseAsCleanup(handle, **out_iter);
  330. } else {
  331. cache_.Release(handle);
  332. }
  333. }
  334. return s;
  335. }
  336. uint64_t TableCache::CreateRowCacheKeyPrefix(const ReadOptions& options,
  337. const FileDescriptor& fd,
  338. const Slice& internal_key,
  339. GetContext* get_context,
  340. IterKey& row_cache_key) {
  341. uint64_t fd_number = fd.GetNumber();
  342. // We use the user key as cache key instead of the internal key,
  343. // otherwise the whole cache would be invalidated every time the
  344. // sequence key increases. However, to support caching snapshot
  345. // reads, we append a sequence number (incremented by 1 to
  346. // distinguish from 0) other than internal_key seq no
  347. // to determine row cache entry visibility.
  348. // If the snapshot is larger than the largest seqno in the file,
  349. // all data should be exposed to the snapshot, so we treat it
  350. // the same as there is no snapshot. The exception is that if
  351. // a seq-checking callback is registered, some internal keys
  352. // may still be filtered out.
  353. uint64_t cache_entry_seq_no = 0;
  354. // Maybe we can include the whole file ifsnapshot == fd.largest_seqno.
  355. if (options.snapshot != nullptr &&
  356. (get_context->has_callback() ||
  357. static_cast_with_check<const SnapshotImpl>(options.snapshot)
  358. ->GetSequenceNumber() <= fd.largest_seqno)) {
  359. // We should consider to use options.snapshot->GetSequenceNumber()
  360. // instead of GetInternalKeySeqno(k), which will make the code
  361. // easier to understand.
  362. cache_entry_seq_no = 1 + GetInternalKeySeqno(internal_key);
  363. }
  364. // Compute row cache key.
  365. row_cache_key.TrimAppend(row_cache_key.Size(), row_cache_id_.data(),
  366. row_cache_id_.size());
  367. AppendVarint64(&row_cache_key, fd_number);
  368. AppendVarint64(&row_cache_key, cache_entry_seq_no);
  369. // Provide a sequence number for callback checking on cache hit.
  370. // As cache_entry_seq_no starts at 1, decrease it's value by 1 to get
  371. // a sequence number align with get context's logic.
  372. return cache_entry_seq_no == 0 ? 0 : cache_entry_seq_no - 1;
  373. }
  374. bool TableCache::GetFromRowCache(const Slice& user_key, IterKey& row_cache_key,
  375. size_t prefix_size, GetContext* get_context,
  376. Status* read_status, SequenceNumber seq_no) {
  377. bool found = false;
  378. row_cache_key.TrimAppend(prefix_size, user_key.data(), user_key.size());
  379. RowCacheInterface row_cache{ioptions_.row_cache.get()};
  380. if (auto row_handle = row_cache.Lookup(row_cache_key.GetUserKey())) {
  381. // Cleanable routine to release the cache entry
  382. Cleanable value_pinner;
  383. // If it comes here value is located on the cache.
  384. // found_row_cache_entry points to the value on cache,
  385. // and value_pinner has cleanup procedure for the cached entry.
  386. // After replayGetContextLog() returns, get_context.pinnable_slice_
  387. // will point to cache entry buffer (or a copy based on that) and
  388. // cleanup routine under value_pinner will be delegated to
  389. // get_context.pinnable_slice_. Cache entry is released when
  390. // get_context.pinnable_slice_ is reset.
  391. row_cache.RegisterReleaseAsCleanup(row_handle, value_pinner);
  392. // If row cache hit, knowing cache key is the same to row_cache_key,
  393. // can use row_cache_key's seq no to construct InternalKey.
  394. *read_status = replayGetContextLog(*row_cache.Value(row_handle), user_key,
  395. get_context, &value_pinner, seq_no);
  396. RecordTick(ioptions_.stats, ROW_CACHE_HIT);
  397. found = true;
  398. } else {
  399. RecordTick(ioptions_.stats, ROW_CACHE_MISS);
  400. }
  401. return found;
  402. }
  403. Status TableCache::Get(const ReadOptions& options,
  404. const InternalKeyComparator& internal_comparator,
  405. const FileMetaData& file_meta, const Slice& k,
  406. GetContext* get_context,
  407. const MutableCFOptions& mutable_cf_options,
  408. HistogramImpl* file_read_hist, bool skip_filters,
  409. int level, size_t max_file_size_for_l0_meta_pin) {
  410. auto& fd = file_meta.fd;
  411. std::string* row_cache_entry = nullptr;
  412. bool done = false;
  413. IterKey row_cache_key;
  414. std::string row_cache_entry_buffer;
  415. // Check row cache if enabled.
  416. // Reuse row_cache_key sequence number when row cache hits.
  417. Status s;
  418. if (ioptions_.row_cache && !get_context->NeedToReadSequence()) {
  419. auto user_key = ExtractUserKey(k);
  420. uint64_t cache_entry_seq_no =
  421. CreateRowCacheKeyPrefix(options, fd, k, get_context, row_cache_key);
  422. done = GetFromRowCache(user_key, row_cache_key, row_cache_key.Size(),
  423. get_context, &s, cache_entry_seq_no);
  424. if (!done) {
  425. row_cache_entry = &row_cache_entry_buffer;
  426. }
  427. }
  428. TableReader* t = fd.table_reader;
  429. TypedHandle* handle = nullptr;
  430. if (s.ok() && !done) {
  431. if (t == nullptr) {
  432. s = FindTable(options, file_options_, internal_comparator, file_meta,
  433. &handle, mutable_cf_options,
  434. options.read_tier == kBlockCacheTier /* no_io */,
  435. file_read_hist, skip_filters, level,
  436. true /* prefetch_index_and_filter_in_cache */,
  437. max_file_size_for_l0_meta_pin, file_meta.temperature);
  438. if (s.ok()) {
  439. t = cache_.Value(handle);
  440. }
  441. }
  442. SequenceNumber* max_covering_tombstone_seq =
  443. get_context->max_covering_tombstone_seq();
  444. if (s.ok() && max_covering_tombstone_seq != nullptr &&
  445. !options.ignore_range_deletions) {
  446. std::unique_ptr<FragmentedRangeTombstoneIterator> range_del_iter(
  447. t->NewRangeTombstoneIterator(options));
  448. if (range_del_iter != nullptr) {
  449. SequenceNumber seq =
  450. range_del_iter->MaxCoveringTombstoneSeqnum(ExtractUserKey(k));
  451. if (seq > *max_covering_tombstone_seq) {
  452. *max_covering_tombstone_seq = seq;
  453. if (get_context->NeedTimestamp()) {
  454. get_context->SetTimestampFromRangeTombstone(
  455. range_del_iter->timestamp());
  456. }
  457. }
  458. }
  459. }
  460. if (s.ok()) {
  461. get_context->SetReplayLog(row_cache_entry); // nullptr if no cache.
  462. s = t->Get(options, k, get_context,
  463. mutable_cf_options.prefix_extractor.get(), skip_filters);
  464. get_context->SetReplayLog(nullptr);
  465. } else if (options.read_tier == kBlockCacheTier && s.IsIncomplete()) {
  466. // Couldn't find table in cache and couldn't open it because of no_io.
  467. get_context->MarkKeyMayExist();
  468. done = true;
  469. }
  470. }
  471. // Put the replay log in row cache only if something was found.
  472. if (!done && s.ok() && row_cache_entry && !row_cache_entry->empty()) {
  473. RowCacheInterface row_cache{ioptions_.row_cache.get()};
  474. size_t charge = row_cache_entry->capacity() + sizeof(std::string);
  475. auto row_ptr = new std::string(std::move(*row_cache_entry));
  476. Status rcs = row_cache.Insert(row_cache_key.GetUserKey(), row_ptr, charge);
  477. if (!rcs.ok()) {
  478. // If row cache is full, it's OK to continue, but we keep ownership of
  479. // row_ptr.
  480. delete row_ptr;
  481. }
  482. }
  483. if (handle != nullptr) {
  484. cache_.Release(handle);
  485. }
  486. return s;
  487. }
  488. void TableCache::UpdateRangeTombstoneSeqnums(
  489. const ReadOptions& options, TableReader* t,
  490. MultiGetContext::Range& table_range) {
  491. std::unique_ptr<FragmentedRangeTombstoneIterator> range_del_iter(
  492. t->NewRangeTombstoneIterator(options));
  493. if (range_del_iter != nullptr) {
  494. for (auto iter = table_range.begin(); iter != table_range.end(); ++iter) {
  495. SequenceNumber* max_covering_tombstone_seq =
  496. iter->get_context->max_covering_tombstone_seq();
  497. SequenceNumber seq =
  498. range_del_iter->MaxCoveringTombstoneSeqnum(iter->ukey_with_ts);
  499. if (seq > *max_covering_tombstone_seq) {
  500. *max_covering_tombstone_seq = seq;
  501. if (iter->get_context->NeedTimestamp()) {
  502. iter->get_context->SetTimestampFromRangeTombstone(
  503. range_del_iter->timestamp());
  504. }
  505. }
  506. }
  507. }
  508. }
  509. Status TableCache::MultiGetFilter(
  510. const ReadOptions& options,
  511. const InternalKeyComparator& internal_comparator,
  512. const FileMetaData& file_meta, const MutableCFOptions& mutable_cf_options,
  513. HistogramImpl* file_read_hist, int level,
  514. MultiGetContext::Range* mget_range, TypedHandle** table_handle) {
  515. auto& fd = file_meta.fd;
  516. IterKey row_cache_key;
  517. std::string row_cache_entry_buffer;
  518. // Check if we need to use the row cache. If yes, then we cannot do the
  519. // filtering here, since the filtering needs to happen after the row cache
  520. // lookup.
  521. KeyContext& first_key = *mget_range->begin();
  522. if (ioptions_.row_cache && !first_key.get_context->NeedToReadSequence()) {
  523. return Status::NotSupported();
  524. }
  525. Status s;
  526. TableReader* t = fd.table_reader;
  527. TypedHandle* handle = nullptr;
  528. MultiGetContext::Range tombstone_range(*mget_range, mget_range->begin(),
  529. mget_range->end());
  530. if (t == nullptr) {
  531. s = FindTable(options, file_options_, internal_comparator, file_meta,
  532. &handle, mutable_cf_options,
  533. options.read_tier == kBlockCacheTier /* no_io */,
  534. file_read_hist,
  535. /*skip_filters=*/false, level,
  536. true /* prefetch_index_and_filter_in_cache */,
  537. /*max_file_size_for_l0_meta_pin=*/0, file_meta.temperature);
  538. if (s.ok()) {
  539. t = cache_.Value(handle);
  540. }
  541. *table_handle = handle;
  542. }
  543. if (s.ok()) {
  544. s = t->MultiGetFilter(options, mutable_cf_options.prefix_extractor.get(),
  545. mget_range);
  546. }
  547. if (s.ok() && !options.ignore_range_deletions) {
  548. // Update the range tombstone sequence numbers for the keys here
  549. // as TableCache::MultiGet may or may not be called, and even if it
  550. // is, it may be called with fewer keys in the rangedue to filtering.
  551. UpdateRangeTombstoneSeqnums(options, t, tombstone_range);
  552. }
  553. if (mget_range->empty() && handle) {
  554. cache_.Release(handle);
  555. *table_handle = nullptr;
  556. }
  557. return s;
  558. }
  559. Status TableCache::GetTableProperties(
  560. const FileOptions& file_options, const ReadOptions& read_options,
  561. const InternalKeyComparator& internal_comparator,
  562. const FileMetaData& file_meta,
  563. std::shared_ptr<const TableProperties>* properties,
  564. const MutableCFOptions& mutable_cf_options, bool no_io) {
  565. auto table_reader = file_meta.fd.table_reader;
  566. // table already been pre-loaded?
  567. if (table_reader) {
  568. *properties = table_reader->GetTableProperties();
  569. return Status::OK();
  570. }
  571. TypedHandle* table_handle = nullptr;
  572. Status s = FindTable(read_options, file_options, internal_comparator,
  573. file_meta, &table_handle, mutable_cf_options, no_io);
  574. if (!s.ok()) {
  575. return s;
  576. }
  577. assert(table_handle);
  578. auto table = cache_.Value(table_handle);
  579. *properties = table->GetTableProperties();
  580. cache_.Release(table_handle);
  581. return s;
  582. }
  583. Status TableCache::ApproximateKeyAnchors(
  584. const ReadOptions& ro, const InternalKeyComparator& internal_comparator,
  585. const FileMetaData& file_meta, const MutableCFOptions& mutable_cf_options,
  586. std::vector<TableReader::Anchor>& anchors) {
  587. Status s;
  588. TableReader* t = file_meta.fd.table_reader;
  589. TypedHandle* handle = nullptr;
  590. if (t == nullptr) {
  591. s = FindTable(ro, file_options_, internal_comparator, file_meta, &handle,
  592. mutable_cf_options);
  593. if (s.ok()) {
  594. t = cache_.Value(handle);
  595. }
  596. }
  597. if (s.ok() && t != nullptr) {
  598. s = t->ApproximateKeyAnchors(ro, anchors);
  599. }
  600. if (handle != nullptr) {
  601. cache_.Release(handle);
  602. }
  603. return s;
  604. }
  605. size_t TableCache::GetMemoryUsageByTableReader(
  606. const FileOptions& file_options, const ReadOptions& read_options,
  607. const InternalKeyComparator& internal_comparator,
  608. const FileMetaData& file_meta, const MutableCFOptions& mutable_cf_options) {
  609. auto table_reader = file_meta.fd.table_reader;
  610. // table already been pre-loaded?
  611. if (table_reader) {
  612. return table_reader->ApproximateMemoryUsage();
  613. }
  614. TypedHandle* table_handle = nullptr;
  615. Status s =
  616. FindTable(read_options, file_options, internal_comparator, file_meta,
  617. &table_handle, mutable_cf_options, true /* no_io */);
  618. if (!s.ok()) {
  619. return 0;
  620. }
  621. assert(table_handle);
  622. auto table = cache_.Value(table_handle);
  623. auto ret = table->ApproximateMemoryUsage();
  624. cache_.Release(table_handle);
  625. return ret;
  626. }
  627. void TableCache::Evict(Cache* cache, uint64_t file_number) {
  628. cache->Erase(GetSliceForFileNumber(&file_number));
  629. }
  630. uint64_t TableCache::ApproximateOffsetOf(
  631. const ReadOptions& read_options, const Slice& key,
  632. const FileMetaData& file_meta, TableReaderCaller caller,
  633. const InternalKeyComparator& internal_comparator,
  634. const MutableCFOptions& mutable_cf_options) {
  635. uint64_t result = 0;
  636. TableReader* table_reader = file_meta.fd.table_reader;
  637. TypedHandle* table_handle = nullptr;
  638. if (table_reader == nullptr) {
  639. Status s =
  640. FindTable(read_options, file_options_, internal_comparator, file_meta,
  641. &table_handle, mutable_cf_options, false /* no_io */);
  642. if (s.ok()) {
  643. table_reader = cache_.Value(table_handle);
  644. }
  645. }
  646. if (table_reader != nullptr) {
  647. result = table_reader->ApproximateOffsetOf(read_options, key, caller);
  648. }
  649. if (table_handle != nullptr) {
  650. cache_.Release(table_handle);
  651. }
  652. return result;
  653. }
  654. uint64_t TableCache::ApproximateSize(
  655. const ReadOptions& read_options, const Slice& start, const Slice& end,
  656. const FileMetaData& file_meta, TableReaderCaller caller,
  657. const InternalKeyComparator& internal_comparator,
  658. const MutableCFOptions& mutable_cf_options) {
  659. uint64_t result = 0;
  660. TableReader* table_reader = file_meta.fd.table_reader;
  661. TypedHandle* table_handle = nullptr;
  662. if (table_reader == nullptr) {
  663. Status s =
  664. FindTable(read_options, file_options_, internal_comparator, file_meta,
  665. &table_handle, mutable_cf_options, false /* no_io */);
  666. if (s.ok()) {
  667. table_reader = cache_.Value(table_handle);
  668. }
  669. }
  670. if (table_reader != nullptr) {
  671. result = table_reader->ApproximateSize(read_options, start, end, caller);
  672. }
  673. if (table_handle != nullptr) {
  674. cache_.Release(table_handle);
  675. }
  676. return result;
  677. }
  678. void TableCache::ReleaseObsolete(Cache* cache, uint64_t file_number,
  679. Cache::Handle* h,
  680. uint32_t uncache_aggressiveness) {
  681. CacheInterface typed_cache(cache);
  682. TypedHandle* table_handle = reinterpret_cast<TypedHandle*>(h);
  683. if (table_handle == nullptr) {
  684. table_handle = typed_cache.Lookup(GetSliceForFileNumber(&file_number));
  685. }
  686. if (table_handle != nullptr) {
  687. TableReader* table_reader = typed_cache.Value(table_handle);
  688. table_reader->MarkObsolete(uncache_aggressiveness);
  689. typed_cache.ReleaseAndEraseIfLastRef(table_handle);
  690. }
  691. }
  692. } // namespace ROCKSDB_NAMESPACE