table_cache.cc 25 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668
  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/filename.h"
  15. #include "file/random_access_file_reader.h"
  16. #include "monitoring/perf_context_imp.h"
  17. #include "rocksdb/statistics.h"
  18. #include "table/block_based/block_based_table_reader.h"
  19. #include "table/get_context.h"
  20. #include "table/internal_iterator.h"
  21. #include "table/iterator_wrapper.h"
  22. #include "table/multiget_context.h"
  23. #include "table/table_builder.h"
  24. #include "table/table_reader.h"
  25. #include "test_util/sync_point.h"
  26. #include "util/cast_util.h"
  27. #include "util/coding.h"
  28. #include "util/stop_watch.h"
  29. namespace ROCKSDB_NAMESPACE {
  30. namespace {
  31. template <class T>
  32. static void DeleteEntry(const Slice& /*key*/, void* value) {
  33. T* typed_value = reinterpret_cast<T*>(value);
  34. delete typed_value;
  35. }
  36. static void UnrefEntry(void* arg1, void* arg2) {
  37. Cache* cache = reinterpret_cast<Cache*>(arg1);
  38. Cache::Handle* h = reinterpret_cast<Cache::Handle*>(arg2);
  39. cache->Release(h);
  40. }
  41. static Slice GetSliceForFileNumber(const uint64_t* file_number) {
  42. return Slice(reinterpret_cast<const char*>(file_number),
  43. sizeof(*file_number));
  44. }
  45. #ifndef ROCKSDB_LITE
  46. void AppendVarint64(IterKey* key, uint64_t v) {
  47. char buf[10];
  48. auto ptr = EncodeVarint64(buf, v);
  49. key->TrimAppend(key->Size(), buf, ptr - buf);
  50. }
  51. #endif // ROCKSDB_LITE
  52. } // namespace
  53. TableCache::TableCache(const ImmutableCFOptions& ioptions,
  54. const FileOptions& file_options, Cache* const cache,
  55. BlockCacheTracer* const block_cache_tracer)
  56. : ioptions_(ioptions),
  57. file_options_(file_options),
  58. cache_(cache),
  59. immortal_tables_(false),
  60. block_cache_tracer_(block_cache_tracer) {
  61. if (ioptions_.row_cache) {
  62. // If the same cache is shared by multiple instances, we need to
  63. // disambiguate its entries.
  64. PutVarint64(&row_cache_id_, ioptions_.row_cache->NewId());
  65. }
  66. }
  67. TableCache::~TableCache() {
  68. }
  69. TableReader* TableCache::GetTableReaderFromHandle(Cache::Handle* handle) {
  70. return reinterpret_cast<TableReader*>(cache_->Value(handle));
  71. }
  72. void TableCache::ReleaseHandle(Cache::Handle* handle) {
  73. cache_->Release(handle);
  74. }
  75. Status TableCache::GetTableReader(
  76. const FileOptions& file_options,
  77. const InternalKeyComparator& internal_comparator, const FileDescriptor& fd,
  78. bool sequential_mode, bool record_read_stats, HistogramImpl* file_read_hist,
  79. std::unique_ptr<TableReader>* table_reader,
  80. const SliceTransform* prefix_extractor, bool skip_filters, int level,
  81. bool prefetch_index_and_filter_in_cache) {
  82. std::string fname =
  83. TableFileName(ioptions_.cf_paths, fd.GetNumber(), fd.GetPathId());
  84. std::unique_ptr<FSRandomAccessFile> file;
  85. Status s = ioptions_.fs->NewRandomAccessFile(fname, file_options, &file,
  86. nullptr);
  87. RecordTick(ioptions_.statistics, NO_FILE_OPENS);
  88. if (s.IsPathNotFound()) {
  89. fname = Rocks2LevelTableFileName(fname);
  90. s = ioptions_.fs->NewRandomAccessFile(fname, file_options, &file, nullptr);
  91. RecordTick(ioptions_.statistics, NO_FILE_OPENS);
  92. }
  93. if (s.ok()) {
  94. if (!sequential_mode && ioptions_.advise_random_on_open) {
  95. file->Hint(FSRandomAccessFile::kRandom);
  96. }
  97. StopWatch sw(ioptions_.env, ioptions_.statistics, TABLE_OPEN_IO_MICROS);
  98. std::unique_ptr<RandomAccessFileReader> file_reader(
  99. new RandomAccessFileReader(
  100. std::move(file), fname, ioptions_.env,
  101. record_read_stats ? ioptions_.statistics : nullptr, SST_READ_MICROS,
  102. file_read_hist, ioptions_.rate_limiter, ioptions_.listeners));
  103. s = ioptions_.table_factory->NewTableReader(
  104. TableReaderOptions(ioptions_, prefix_extractor, file_options,
  105. internal_comparator, skip_filters, immortal_tables_,
  106. level, fd.largest_seqno, block_cache_tracer_),
  107. std::move(file_reader), fd.GetFileSize(), table_reader,
  108. prefetch_index_and_filter_in_cache);
  109. TEST_SYNC_POINT("TableCache::GetTableReader:0");
  110. }
  111. return s;
  112. }
  113. void TableCache::EraseHandle(const FileDescriptor& fd, Cache::Handle* handle) {
  114. ReleaseHandle(handle);
  115. uint64_t number = fd.GetNumber();
  116. Slice key = GetSliceForFileNumber(&number);
  117. cache_->Erase(key);
  118. }
  119. Status TableCache::FindTable(const FileOptions& file_options,
  120. const InternalKeyComparator& internal_comparator,
  121. const FileDescriptor& fd, Cache::Handle** handle,
  122. const SliceTransform* prefix_extractor,
  123. const bool no_io, bool record_read_stats,
  124. HistogramImpl* file_read_hist, bool skip_filters,
  125. int level,
  126. bool prefetch_index_and_filter_in_cache) {
  127. PERF_TIMER_GUARD_WITH_ENV(find_table_nanos, ioptions_.env);
  128. Status s;
  129. uint64_t number = fd.GetNumber();
  130. Slice key = GetSliceForFileNumber(&number);
  131. *handle = cache_->Lookup(key);
  132. TEST_SYNC_POINT_CALLBACK("TableCache::FindTable:0",
  133. const_cast<bool*>(&no_io));
  134. if (*handle == nullptr) {
  135. if (no_io) { // Don't do IO and return a not-found status
  136. return Status::Incomplete("Table not found in table_cache, no_io is set");
  137. }
  138. std::unique_ptr<TableReader> table_reader;
  139. s = GetTableReader(file_options, internal_comparator, fd,
  140. false /* sequential mode */, record_read_stats,
  141. file_read_hist, &table_reader, prefix_extractor,
  142. skip_filters, level, prefetch_index_and_filter_in_cache);
  143. if (!s.ok()) {
  144. assert(table_reader == nullptr);
  145. RecordTick(ioptions_.statistics, NO_FILE_ERRORS);
  146. // We do not cache error results so that if the error is transient,
  147. // or somebody repairs the file, we recover automatically.
  148. } else {
  149. s = cache_->Insert(key, table_reader.get(), 1, &DeleteEntry<TableReader>,
  150. handle);
  151. if (s.ok()) {
  152. // Release ownership of table reader.
  153. table_reader.release();
  154. }
  155. }
  156. }
  157. return s;
  158. }
  159. InternalIterator* TableCache::NewIterator(
  160. const ReadOptions& options, const FileOptions& file_options,
  161. const InternalKeyComparator& icomparator, const FileMetaData& file_meta,
  162. RangeDelAggregator* range_del_agg, const SliceTransform* prefix_extractor,
  163. TableReader** table_reader_ptr, HistogramImpl* file_read_hist,
  164. TableReaderCaller caller, Arena* arena, bool skip_filters, int level,
  165. const InternalKey* smallest_compaction_key,
  166. const InternalKey* largest_compaction_key) {
  167. PERF_TIMER_GUARD(new_table_iterator_nanos);
  168. Status s;
  169. TableReader* table_reader = nullptr;
  170. Cache::Handle* handle = nullptr;
  171. if (table_reader_ptr != nullptr) {
  172. *table_reader_ptr = nullptr;
  173. }
  174. bool for_compaction = caller == TableReaderCaller::kCompaction;
  175. auto& fd = file_meta.fd;
  176. table_reader = fd.table_reader;
  177. if (table_reader == nullptr) {
  178. s = FindTable(file_options, icomparator, fd, &handle, prefix_extractor,
  179. options.read_tier == kBlockCacheTier /* no_io */,
  180. !for_compaction /* record_read_stats */, file_read_hist,
  181. skip_filters, level);
  182. if (s.ok()) {
  183. table_reader = GetTableReaderFromHandle(handle);
  184. }
  185. }
  186. InternalIterator* result = nullptr;
  187. if (s.ok()) {
  188. if (options.table_filter &&
  189. !options.table_filter(*table_reader->GetTableProperties())) {
  190. result = NewEmptyInternalIterator<Slice>(arena);
  191. } else {
  192. result = table_reader->NewIterator(options, prefix_extractor, arena,
  193. skip_filters, caller,
  194. file_options.compaction_readahead_size);
  195. }
  196. if (handle != nullptr) {
  197. result->RegisterCleanup(&UnrefEntry, cache_, handle);
  198. handle = nullptr; // prevent from releasing below
  199. }
  200. if (for_compaction) {
  201. table_reader->SetupForCompaction();
  202. }
  203. if (table_reader_ptr != nullptr) {
  204. *table_reader_ptr = table_reader;
  205. }
  206. }
  207. if (s.ok() && range_del_agg != nullptr && !options.ignore_range_deletions) {
  208. if (range_del_agg->AddFile(fd.GetNumber())) {
  209. std::unique_ptr<FragmentedRangeTombstoneIterator> range_del_iter(
  210. static_cast<FragmentedRangeTombstoneIterator*>(
  211. table_reader->NewRangeTombstoneIterator(options)));
  212. if (range_del_iter != nullptr) {
  213. s = range_del_iter->status();
  214. }
  215. if (s.ok()) {
  216. const InternalKey* smallest = &file_meta.smallest;
  217. const InternalKey* largest = &file_meta.largest;
  218. if (smallest_compaction_key != nullptr) {
  219. smallest = smallest_compaction_key;
  220. }
  221. if (largest_compaction_key != nullptr) {
  222. largest = largest_compaction_key;
  223. }
  224. range_del_agg->AddTombstones(std::move(range_del_iter), smallest,
  225. largest);
  226. }
  227. }
  228. }
  229. if (handle != nullptr) {
  230. ReleaseHandle(handle);
  231. }
  232. if (!s.ok()) {
  233. assert(result == nullptr);
  234. result = NewErrorInternalIterator<Slice>(s, arena);
  235. }
  236. return result;
  237. }
  238. Status TableCache::GetRangeTombstoneIterator(
  239. const ReadOptions& options,
  240. const InternalKeyComparator& internal_comparator,
  241. const FileMetaData& file_meta,
  242. std::unique_ptr<FragmentedRangeTombstoneIterator>* out_iter) {
  243. const FileDescriptor& fd = file_meta.fd;
  244. Status s;
  245. TableReader* t = fd.table_reader;
  246. Cache::Handle* handle = nullptr;
  247. if (t == nullptr) {
  248. s = FindTable(file_options_, internal_comparator, fd, &handle);
  249. if (s.ok()) {
  250. t = GetTableReaderFromHandle(handle);
  251. }
  252. }
  253. if (s.ok()) {
  254. out_iter->reset(t->NewRangeTombstoneIterator(options));
  255. assert(out_iter);
  256. }
  257. return s;
  258. }
  259. #ifndef ROCKSDB_LITE
  260. void TableCache::CreateRowCacheKeyPrefix(const ReadOptions& options,
  261. const FileDescriptor& fd,
  262. const Slice& internal_key,
  263. GetContext* get_context,
  264. IterKey& row_cache_key) {
  265. uint64_t fd_number = fd.GetNumber();
  266. // We use the user key as cache key instead of the internal key,
  267. // otherwise the whole cache would be invalidated every time the
  268. // sequence key increases. However, to support caching snapshot
  269. // reads, we append the sequence number (incremented by 1 to
  270. // distinguish from 0) only in this case.
  271. // If the snapshot is larger than the largest seqno in the file,
  272. // all data should be exposed to the snapshot, so we treat it
  273. // the same as there is no snapshot. The exception is that if
  274. // a seq-checking callback is registered, some internal keys
  275. // may still be filtered out.
  276. uint64_t seq_no = 0;
  277. // Maybe we can include the whole file ifsnapshot == fd.largest_seqno.
  278. if (options.snapshot != nullptr &&
  279. (get_context->has_callback() ||
  280. static_cast_with_check<const SnapshotImpl, const Snapshot>(
  281. options.snapshot)
  282. ->GetSequenceNumber() <= fd.largest_seqno)) {
  283. // We should consider to use options.snapshot->GetSequenceNumber()
  284. // instead of GetInternalKeySeqno(k), which will make the code
  285. // easier to understand.
  286. seq_no = 1 + GetInternalKeySeqno(internal_key);
  287. }
  288. // Compute row cache key.
  289. row_cache_key.TrimAppend(row_cache_key.Size(), row_cache_id_.data(),
  290. row_cache_id_.size());
  291. AppendVarint64(&row_cache_key, fd_number);
  292. AppendVarint64(&row_cache_key, seq_no);
  293. }
  294. bool TableCache::GetFromRowCache(const Slice& user_key, IterKey& row_cache_key,
  295. size_t prefix_size, GetContext* get_context) {
  296. bool found = false;
  297. row_cache_key.TrimAppend(prefix_size, user_key.data(), user_key.size());
  298. if (auto row_handle =
  299. ioptions_.row_cache->Lookup(row_cache_key.GetUserKey())) {
  300. // Cleanable routine to release the cache entry
  301. Cleanable value_pinner;
  302. auto release_cache_entry_func = [](void* cache_to_clean,
  303. void* cache_handle) {
  304. ((Cache*)cache_to_clean)->Release((Cache::Handle*)cache_handle);
  305. };
  306. auto found_row_cache_entry =
  307. static_cast<const std::string*>(ioptions_.row_cache->Value(row_handle));
  308. // If it comes here value is located on the cache.
  309. // found_row_cache_entry points to the value on cache,
  310. // and value_pinner has cleanup procedure for the cached entry.
  311. // After replayGetContextLog() returns, get_context.pinnable_slice_
  312. // will point to cache entry buffer (or a copy based on that) and
  313. // cleanup routine under value_pinner will be delegated to
  314. // get_context.pinnable_slice_. Cache entry is released when
  315. // get_context.pinnable_slice_ is reset.
  316. value_pinner.RegisterCleanup(release_cache_entry_func,
  317. ioptions_.row_cache.get(), row_handle);
  318. replayGetContextLog(*found_row_cache_entry, user_key, get_context,
  319. &value_pinner);
  320. RecordTick(ioptions_.statistics, ROW_CACHE_HIT);
  321. found = true;
  322. } else {
  323. RecordTick(ioptions_.statistics, ROW_CACHE_MISS);
  324. }
  325. return found;
  326. }
  327. #endif // ROCKSDB_LITE
  328. Status TableCache::Get(const ReadOptions& options,
  329. const InternalKeyComparator& internal_comparator,
  330. const FileMetaData& file_meta, const Slice& k,
  331. GetContext* get_context,
  332. const SliceTransform* prefix_extractor,
  333. HistogramImpl* file_read_hist, bool skip_filters,
  334. int level) {
  335. auto& fd = file_meta.fd;
  336. std::string* row_cache_entry = nullptr;
  337. bool done = false;
  338. #ifndef ROCKSDB_LITE
  339. IterKey row_cache_key;
  340. std::string row_cache_entry_buffer;
  341. // Check row cache if enabled. Since row cache does not currently store
  342. // sequence numbers, we cannot use it if we need to fetch the sequence.
  343. if (ioptions_.row_cache && !get_context->NeedToReadSequence()) {
  344. auto user_key = ExtractUserKey(k);
  345. CreateRowCacheKeyPrefix(options, fd, k, get_context, row_cache_key);
  346. done = GetFromRowCache(user_key, row_cache_key, row_cache_key.Size(),
  347. get_context);
  348. if (!done) {
  349. row_cache_entry = &row_cache_entry_buffer;
  350. }
  351. }
  352. #endif // ROCKSDB_LITE
  353. Status s;
  354. TableReader* t = fd.table_reader;
  355. Cache::Handle* handle = nullptr;
  356. if (!done && s.ok()) {
  357. if (t == nullptr) {
  358. s = FindTable(
  359. file_options_, internal_comparator, fd, &handle, prefix_extractor,
  360. options.read_tier == kBlockCacheTier /* no_io */,
  361. true /* record_read_stats */, file_read_hist, skip_filters, level);
  362. if (s.ok()) {
  363. t = GetTableReaderFromHandle(handle);
  364. }
  365. }
  366. SequenceNumber* max_covering_tombstone_seq =
  367. get_context->max_covering_tombstone_seq();
  368. if (s.ok() && max_covering_tombstone_seq != nullptr &&
  369. !options.ignore_range_deletions) {
  370. std::unique_ptr<FragmentedRangeTombstoneIterator> range_del_iter(
  371. t->NewRangeTombstoneIterator(options));
  372. if (range_del_iter != nullptr) {
  373. *max_covering_tombstone_seq = std::max(
  374. *max_covering_tombstone_seq,
  375. range_del_iter->MaxCoveringTombstoneSeqnum(ExtractUserKey(k)));
  376. }
  377. }
  378. if (s.ok()) {
  379. get_context->SetReplayLog(row_cache_entry); // nullptr if no cache.
  380. s = t->Get(options, k, get_context, prefix_extractor, skip_filters);
  381. get_context->SetReplayLog(nullptr);
  382. } else if (options.read_tier == kBlockCacheTier && s.IsIncomplete()) {
  383. // Couldn't find Table in cache but treat as kFound if no_io set
  384. get_context->MarkKeyMayExist();
  385. s = Status::OK();
  386. done = true;
  387. }
  388. }
  389. #ifndef ROCKSDB_LITE
  390. // Put the replay log in row cache only if something was found.
  391. if (!done && s.ok() && row_cache_entry && !row_cache_entry->empty()) {
  392. size_t charge =
  393. row_cache_key.Size() + row_cache_entry->size() + sizeof(std::string);
  394. void* row_ptr = new std::string(std::move(*row_cache_entry));
  395. ioptions_.row_cache->Insert(row_cache_key.GetUserKey(), row_ptr, charge,
  396. &DeleteEntry<std::string>);
  397. }
  398. #endif // ROCKSDB_LITE
  399. if (handle != nullptr) {
  400. ReleaseHandle(handle);
  401. }
  402. return s;
  403. }
  404. // Batched version of TableCache::MultiGet.
  405. Status TableCache::MultiGet(const ReadOptions& options,
  406. const InternalKeyComparator& internal_comparator,
  407. const FileMetaData& file_meta,
  408. const MultiGetContext::Range* mget_range,
  409. const SliceTransform* prefix_extractor,
  410. HistogramImpl* file_read_hist, bool skip_filters,
  411. int level) {
  412. auto& fd = file_meta.fd;
  413. Status s;
  414. TableReader* t = fd.table_reader;
  415. Cache::Handle* handle = nullptr;
  416. MultiGetRange table_range(*mget_range, mget_range->begin(),
  417. mget_range->end());
  418. #ifndef ROCKSDB_LITE
  419. autovector<std::string, MultiGetContext::MAX_BATCH_SIZE> row_cache_entries;
  420. IterKey row_cache_key;
  421. size_t row_cache_key_prefix_size = 0;
  422. KeyContext& first_key = *table_range.begin();
  423. bool lookup_row_cache =
  424. ioptions_.row_cache && !first_key.get_context->NeedToReadSequence();
  425. // Check row cache if enabled. Since row cache does not currently store
  426. // sequence numbers, we cannot use it if we need to fetch the sequence.
  427. if (lookup_row_cache) {
  428. GetContext* first_context = first_key.get_context;
  429. CreateRowCacheKeyPrefix(options, fd, first_key.ikey, first_context,
  430. row_cache_key);
  431. row_cache_key_prefix_size = row_cache_key.Size();
  432. for (auto miter = table_range.begin(); miter != table_range.end();
  433. ++miter) {
  434. const Slice& user_key = miter->ukey;
  435. ;
  436. GetContext* get_context = miter->get_context;
  437. if (GetFromRowCache(user_key, row_cache_key, row_cache_key_prefix_size,
  438. get_context)) {
  439. table_range.SkipKey(miter);
  440. } else {
  441. row_cache_entries.emplace_back();
  442. get_context->SetReplayLog(&(row_cache_entries.back()));
  443. }
  444. }
  445. }
  446. #endif // ROCKSDB_LITE
  447. // Check that table_range is not empty. Its possible all keys may have been
  448. // found in the row cache and thus the range may now be empty
  449. if (s.ok() && !table_range.empty()) {
  450. if (t == nullptr) {
  451. s = FindTable(
  452. file_options_, internal_comparator, fd, &handle, prefix_extractor,
  453. options.read_tier == kBlockCacheTier /* no_io */,
  454. true /* record_read_stats */, file_read_hist, skip_filters, level);
  455. TEST_SYNC_POINT_CALLBACK("TableCache::MultiGet:FindTable", &s);
  456. if (s.ok()) {
  457. t = GetTableReaderFromHandle(handle);
  458. assert(t);
  459. }
  460. }
  461. if (s.ok() && !options.ignore_range_deletions) {
  462. std::unique_ptr<FragmentedRangeTombstoneIterator> range_del_iter(
  463. t->NewRangeTombstoneIterator(options));
  464. if (range_del_iter != nullptr) {
  465. for (auto iter = table_range.begin(); iter != table_range.end();
  466. ++iter) {
  467. SequenceNumber* max_covering_tombstone_seq =
  468. iter->get_context->max_covering_tombstone_seq();
  469. *max_covering_tombstone_seq =
  470. std::max(*max_covering_tombstone_seq,
  471. range_del_iter->MaxCoveringTombstoneSeqnum(iter->ukey));
  472. }
  473. }
  474. }
  475. if (s.ok()) {
  476. t->MultiGet(options, &table_range, prefix_extractor, skip_filters);
  477. } else if (options.read_tier == kBlockCacheTier && s.IsIncomplete()) {
  478. for (auto iter = table_range.begin(); iter != table_range.end(); ++iter) {
  479. Status* status = iter->s;
  480. if (status->IsIncomplete()) {
  481. // Couldn't find Table in cache but treat as kFound if no_io set
  482. iter->get_context->MarkKeyMayExist();
  483. s = Status::OK();
  484. }
  485. }
  486. }
  487. }
  488. #ifndef ROCKSDB_LITE
  489. if (lookup_row_cache) {
  490. size_t row_idx = 0;
  491. for (auto miter = table_range.begin(); miter != table_range.end();
  492. ++miter) {
  493. std::string& row_cache_entry = row_cache_entries[row_idx++];
  494. const Slice& user_key = miter->ukey;
  495. ;
  496. GetContext* get_context = miter->get_context;
  497. get_context->SetReplayLog(nullptr);
  498. // Compute row cache key.
  499. row_cache_key.TrimAppend(row_cache_key_prefix_size, user_key.data(),
  500. user_key.size());
  501. // Put the replay log in row cache only if something was found.
  502. if (s.ok() && !row_cache_entry.empty()) {
  503. size_t charge =
  504. row_cache_key.Size() + row_cache_entry.size() + sizeof(std::string);
  505. void* row_ptr = new std::string(std::move(row_cache_entry));
  506. ioptions_.row_cache->Insert(row_cache_key.GetUserKey(), row_ptr, charge,
  507. &DeleteEntry<std::string>);
  508. }
  509. }
  510. }
  511. #endif // ROCKSDB_LITE
  512. if (handle != nullptr) {
  513. ReleaseHandle(handle);
  514. }
  515. return s;
  516. }
  517. Status TableCache::GetTableProperties(
  518. const FileOptions& file_options,
  519. const InternalKeyComparator& internal_comparator, const FileDescriptor& fd,
  520. std::shared_ptr<const TableProperties>* properties,
  521. const SliceTransform* prefix_extractor, bool no_io) {
  522. Status s;
  523. auto table_reader = fd.table_reader;
  524. // table already been pre-loaded?
  525. if (table_reader) {
  526. *properties = table_reader->GetTableProperties();
  527. return s;
  528. }
  529. Cache::Handle* table_handle = nullptr;
  530. s = FindTable(file_options, internal_comparator, fd, &table_handle,
  531. prefix_extractor, no_io);
  532. if (!s.ok()) {
  533. return s;
  534. }
  535. assert(table_handle);
  536. auto table = GetTableReaderFromHandle(table_handle);
  537. *properties = table->GetTableProperties();
  538. ReleaseHandle(table_handle);
  539. return s;
  540. }
  541. size_t TableCache::GetMemoryUsageByTableReader(
  542. const FileOptions& file_options,
  543. const InternalKeyComparator& internal_comparator, const FileDescriptor& fd,
  544. const SliceTransform* prefix_extractor) {
  545. Status s;
  546. auto table_reader = fd.table_reader;
  547. // table already been pre-loaded?
  548. if (table_reader) {
  549. return table_reader->ApproximateMemoryUsage();
  550. }
  551. Cache::Handle* table_handle = nullptr;
  552. s = FindTable(file_options, internal_comparator, fd, &table_handle,
  553. prefix_extractor, true);
  554. if (!s.ok()) {
  555. return 0;
  556. }
  557. assert(table_handle);
  558. auto table = GetTableReaderFromHandle(table_handle);
  559. auto ret = table->ApproximateMemoryUsage();
  560. ReleaseHandle(table_handle);
  561. return ret;
  562. }
  563. void TableCache::Evict(Cache* cache, uint64_t file_number) {
  564. cache->Erase(GetSliceForFileNumber(&file_number));
  565. }
  566. uint64_t TableCache::ApproximateOffsetOf(
  567. const Slice& key, const FileDescriptor& fd, TableReaderCaller caller,
  568. const InternalKeyComparator& internal_comparator,
  569. const SliceTransform* prefix_extractor) {
  570. uint64_t result = 0;
  571. TableReader* table_reader = fd.table_reader;
  572. Cache::Handle* table_handle = nullptr;
  573. if (table_reader == nullptr) {
  574. const bool for_compaction = (caller == TableReaderCaller::kCompaction);
  575. Status s = FindTable(file_options_, internal_comparator, fd, &table_handle,
  576. prefix_extractor, false /* no_io */,
  577. !for_compaction /* record_read_stats */);
  578. if (s.ok()) {
  579. table_reader = GetTableReaderFromHandle(table_handle);
  580. }
  581. }
  582. if (table_reader != nullptr) {
  583. result = table_reader->ApproximateOffsetOf(key, caller);
  584. }
  585. if (table_handle != nullptr) {
  586. ReleaseHandle(table_handle);
  587. }
  588. return result;
  589. }
  590. uint64_t TableCache::ApproximateSize(
  591. const Slice& start, const Slice& end, const FileDescriptor& fd,
  592. TableReaderCaller caller, const InternalKeyComparator& internal_comparator,
  593. const SliceTransform* prefix_extractor) {
  594. uint64_t result = 0;
  595. TableReader* table_reader = fd.table_reader;
  596. Cache::Handle* table_handle = nullptr;
  597. if (table_reader == nullptr) {
  598. const bool for_compaction = (caller == TableReaderCaller::kCompaction);
  599. Status s = FindTable(file_options_, internal_comparator, fd, &table_handle,
  600. prefix_extractor, false /* no_io */,
  601. !for_compaction /* record_read_stats */);
  602. if (s.ok()) {
  603. table_reader = GetTableReaderFromHandle(table_handle);
  604. }
  605. }
  606. if (table_reader != nullptr) {
  607. result = table_reader->ApproximateSize(start, end, caller);
  608. }
  609. if (table_handle != nullptr) {
  610. ReleaseHandle(table_handle);
  611. }
  612. return result;
  613. }
  614. } // namespace ROCKSDB_NAMESPACE