plain_table_reader.cc 26 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781
  1. // Copyright (c) Facebook, Inc. and its affiliates. All Rights Reserved.
  2. // Copyright (c) 2011 The LevelDB Authors. All rights reserved.
  3. // Use of this source code is governed by a BSD-style license that can be
  4. // found in the LICENSE file. See the AUTHORS file for names of contributors.
  5. #include "table/plain/plain_table_reader.h"
  6. #include <string>
  7. #include <vector>
  8. #include "db/dbformat.h"
  9. #include "memory/arena.h"
  10. #include "monitoring/histogram.h"
  11. #include "monitoring/perf_context_imp.h"
  12. #include "rocksdb/cache.h"
  13. #include "rocksdb/comparator.h"
  14. #include "rocksdb/env.h"
  15. #include "rocksdb/filter_policy.h"
  16. #include "rocksdb/options.h"
  17. #include "rocksdb/statistics.h"
  18. #include "table/block_based/block.h"
  19. #include "table/block_based/filter_block.h"
  20. #include "table/format.h"
  21. #include "table/get_context.h"
  22. #include "table/internal_iterator.h"
  23. #include "table/meta_blocks.h"
  24. #include "table/plain/plain_table_bloom.h"
  25. #include "table/plain/plain_table_factory.h"
  26. #include "table/plain/plain_table_key_coding.h"
  27. #include "table/two_level_iterator.h"
  28. #include "util/coding.h"
  29. #include "util/dynamic_bloom.h"
  30. #include "util/hash.h"
  31. #include "util/stop_watch.h"
  32. #include "util/string_util.h"
  33. namespace ROCKSDB_NAMESPACE {
  34. namespace {
  35. // Safely getting a uint32_t element from a char array, where, starting from
  36. // `base`, every 4 bytes are considered as an fixed 32 bit integer.
  37. inline uint32_t GetFixed32Element(const char* base, size_t offset) {
  38. return DecodeFixed32(base + offset * sizeof(uint32_t));
  39. }
  40. } // namespace
  41. // Iterator to iterate IndexedTable
  42. class PlainTableIterator : public InternalIterator {
  43. public:
  44. explicit PlainTableIterator(PlainTableReader* table, bool use_prefix_seek);
  45. // No copying allowed
  46. PlainTableIterator(const PlainTableIterator&) = delete;
  47. void operator=(const Iterator&) = delete;
  48. ~PlainTableIterator() override;
  49. bool Valid() const override;
  50. void SeekToFirst() override;
  51. void SeekToLast() override;
  52. void Seek(const Slice& target) override;
  53. void SeekForPrev(const Slice& target) override;
  54. void Next() override;
  55. void Prev() override;
  56. Slice key() const override;
  57. Slice value() const override;
  58. Status status() const override;
  59. private:
  60. PlainTableReader* table_;
  61. PlainTableKeyDecoder decoder_;
  62. bool use_prefix_seek_;
  63. uint32_t offset_;
  64. uint32_t next_offset_;
  65. Slice key_;
  66. Slice value_;
  67. Status status_;
  68. };
  69. PlainTableReader::PlainTableReader(
  70. const ImmutableOptions& ioptions,
  71. std::unique_ptr<RandomAccessFileReader>&& file,
  72. const EnvOptions& storage_options, const InternalKeyComparator& icomparator,
  73. EncodingType encoding_type, uint64_t file_size,
  74. const TableProperties* table_properties,
  75. const SliceTransform* prefix_extractor)
  76. : internal_comparator_(icomparator),
  77. encoding_type_(encoding_type),
  78. full_scan_mode_(false),
  79. user_key_len_(static_cast<uint32_t>(table_properties->fixed_key_len)),
  80. prefix_extractor_(prefix_extractor),
  81. enable_bloom_(false),
  82. bloom_(6),
  83. file_info_(std::move(file), storage_options,
  84. static_cast<uint32_t>(table_properties->data_size)),
  85. ioptions_(ioptions),
  86. file_size_(file_size),
  87. table_properties_(nullptr) {}
  88. PlainTableReader::~PlainTableReader() {
  89. // Should fix?
  90. status_.PermitUncheckedError();
  91. }
  92. Status PlainTableReader::Open(
  93. const ImmutableOptions& ioptions, const EnvOptions& env_options,
  94. const InternalKeyComparator& internal_comparator,
  95. std::unique_ptr<RandomAccessFileReader>&& file, uint64_t file_size,
  96. std::unique_ptr<TableReader>* table_reader, const int bloom_bits_per_key,
  97. double hash_table_ratio, size_t index_sparseness, size_t huge_page_tlb_size,
  98. bool full_scan_mode, const bool immortal_table,
  99. const SliceTransform* prefix_extractor) {
  100. if (file_size > PlainTableIndex::kMaxFileSize) {
  101. return Status::NotSupported("File size " + std::to_string(file_size) +
  102. " exceeds PlainTableReader max file size " +
  103. std::to_string(PlainTableIndex::kMaxFileSize));
  104. }
  105. std::unique_ptr<TableProperties> props;
  106. // TODO: plumb Env::IOActivity, Env::IOPriority
  107. const ReadOptions read_options;
  108. auto s = ReadTableProperties(file.get(), file_size, kPlainTableMagicNumber,
  109. ioptions, read_options, &props);
  110. if (!s.ok()) {
  111. return s;
  112. }
  113. assert(hash_table_ratio >= 0.0);
  114. auto& user_props = props->user_collected_properties;
  115. auto prefix_extractor_in_file = props->prefix_extractor_name;
  116. if (!full_scan_mode &&
  117. !prefix_extractor_in_file.empty() /* old version sst file*/
  118. && prefix_extractor_in_file != "nullptr") {
  119. if (!prefix_extractor) {
  120. return Status::InvalidArgument(
  121. "Prefix extractor is missing when opening a PlainTable built "
  122. "using a prefix extractor");
  123. } else if (prefix_extractor_in_file != prefix_extractor->AsString()) {
  124. return Status::InvalidArgument(
  125. "Prefix extractor given doesn't match the one used to build "
  126. "PlainTable");
  127. }
  128. }
  129. EncodingType encoding_type = kPlain;
  130. auto encoding_type_prop =
  131. user_props.find(PlainTablePropertyNames::kEncodingType);
  132. if (encoding_type_prop != user_props.end()) {
  133. encoding_type = static_cast<EncodingType>(
  134. DecodeFixed32(encoding_type_prop->second.c_str()));
  135. }
  136. std::unique_ptr<PlainTableReader> new_reader(new PlainTableReader(
  137. ioptions, std::move(file), env_options, internal_comparator,
  138. encoding_type, file_size, props.get(), prefix_extractor));
  139. s = new_reader->MmapDataIfNeeded();
  140. if (!s.ok()) {
  141. return s;
  142. }
  143. if (!full_scan_mode) {
  144. s = new_reader->PopulateIndex(props.get(), bloom_bits_per_key,
  145. hash_table_ratio, index_sparseness,
  146. huge_page_tlb_size);
  147. if (!s.ok()) {
  148. return s;
  149. }
  150. } else {
  151. // Flag to indicate it is a full scan mode so that none of the indexes
  152. // can be used.
  153. new_reader->full_scan_mode_ = true;
  154. }
  155. // PopulateIndex can add to the props, so don't store them until now
  156. new_reader->table_properties_ = std::move(props);
  157. if (immortal_table && new_reader->file_info_.is_mmap_mode) {
  158. new_reader->dummy_cleanable_.reset(new Cleanable());
  159. }
  160. *table_reader = std::move(new_reader);
  161. return s;
  162. }
  163. void PlainTableReader::SetupForCompaction() {}
  164. InternalIterator* PlainTableReader::NewIterator(
  165. const ReadOptions& options, const SliceTransform* /* prefix_extractor */,
  166. Arena* arena, bool /*skip_filters*/, TableReaderCaller /*caller*/,
  167. size_t /*compaction_readahead_size*/, bool /* allow_unprepared_value */) {
  168. // Not necessarily used here, but make sure this has been initialized
  169. assert(table_properties_);
  170. // Auto prefix mode is not implemented in PlainTable.
  171. bool use_prefix_seek =
  172. !IsTotalOrderMode() &&
  173. (options.prefix_same_as_start ||
  174. (!options.total_order_seek && !options.auto_prefix_mode));
  175. if (arena == nullptr) {
  176. return new PlainTableIterator(this, use_prefix_seek);
  177. } else {
  178. auto mem = arena->AllocateAligned(sizeof(PlainTableIterator));
  179. return new (mem) PlainTableIterator(this, use_prefix_seek);
  180. }
  181. }
  182. Status PlainTableReader::PopulateIndexRecordList(
  183. PlainTableIndexBuilder* index_builder,
  184. std::vector<uint32_t>* prefix_hashes) {
  185. Slice prev_key_prefix_slice;
  186. std::string prev_key_prefix_buf;
  187. uint32_t pos = data_start_offset_;
  188. bool is_first_record = true;
  189. Slice key_prefix_slice;
  190. PlainTableKeyDecoder decoder(&file_info_, encoding_type_, user_key_len_,
  191. prefix_extractor_);
  192. while (pos < file_info_.data_end_offset) {
  193. uint32_t key_offset = pos;
  194. ParsedInternalKey key;
  195. Slice value_slice;
  196. bool seekable = false;
  197. Status s = Next(&decoder, &pos, &key, nullptr, &value_slice, &seekable);
  198. if (!s.ok()) {
  199. return s;
  200. }
  201. key_prefix_slice = GetPrefix(key);
  202. if (enable_bloom_) {
  203. bloom_.AddHash(GetSliceHash(key.user_key));
  204. } else {
  205. if (is_first_record || prev_key_prefix_slice != key_prefix_slice) {
  206. if (!is_first_record) {
  207. prefix_hashes->push_back(GetSliceHash(prev_key_prefix_slice));
  208. }
  209. if (file_info_.is_mmap_mode) {
  210. prev_key_prefix_slice = key_prefix_slice;
  211. } else {
  212. prev_key_prefix_buf = key_prefix_slice.ToString();
  213. prev_key_prefix_slice = prev_key_prefix_buf;
  214. }
  215. }
  216. }
  217. index_builder->AddKeyPrefix(GetPrefix(key), key_offset);
  218. if (!seekable && is_first_record) {
  219. return Status::Corruption("Key for a prefix is not seekable");
  220. }
  221. is_first_record = false;
  222. }
  223. prefix_hashes->push_back(GetSliceHash(key_prefix_slice));
  224. auto s = index_.InitFromRawData(index_builder->Finish());
  225. return s;
  226. }
  227. void PlainTableReader::AllocateBloom(int bloom_bits_per_key, int num_keys,
  228. size_t huge_page_tlb_size) {
  229. uint32_t bloom_total_bits = num_keys * bloom_bits_per_key;
  230. if (bloom_total_bits > 0) {
  231. enable_bloom_ = true;
  232. bloom_.SetTotalBits(&arena_, bloom_total_bits, ioptions_.bloom_locality,
  233. huge_page_tlb_size, ioptions_.logger);
  234. }
  235. }
  236. void PlainTableReader::FillBloom(const std::vector<uint32_t>& prefix_hashes) {
  237. assert(bloom_.IsInitialized());
  238. for (const auto prefix_hash : prefix_hashes) {
  239. bloom_.AddHash(prefix_hash);
  240. }
  241. }
  242. Status PlainTableReader::MmapDataIfNeeded() {
  243. if (file_info_.is_mmap_mode) {
  244. // Get mmapped memory.
  245. return file_info_.file->Read(IOOptions(), 0,
  246. static_cast<size_t>(file_size_),
  247. &file_info_.file_data, nullptr, nullptr);
  248. }
  249. return Status::OK();
  250. }
  251. Status PlainTableReader::PopulateIndex(TableProperties* props,
  252. int bloom_bits_per_key,
  253. double hash_table_ratio,
  254. size_t index_sparseness,
  255. size_t huge_page_tlb_size) {
  256. assert(props != nullptr);
  257. BlockContents index_block_contents;
  258. // TODO: plumb Env::IOActivity, Env::IOPriority
  259. const ReadOptions read_options;
  260. Status s =
  261. ReadMetaBlock(file_info_.file.get(), nullptr /* prefetch_buffer */,
  262. file_size_, kPlainTableMagicNumber, ioptions_, read_options,
  263. PlainTableIndexBuilder::kPlainTableIndexBlock,
  264. BlockType::kIndex, &index_block_contents);
  265. bool index_in_file = s.ok();
  266. BlockContents bloom_block_contents;
  267. bool bloom_in_file = false;
  268. // We only need to read the bloom block if index block is in file.
  269. if (index_in_file) {
  270. s = ReadMetaBlock(file_info_.file.get(), nullptr /* prefetch_buffer */,
  271. file_size_, kPlainTableMagicNumber, ioptions_,
  272. read_options, BloomBlockBuilder::kBloomBlock,
  273. BlockType::kFilter, &bloom_block_contents);
  274. bloom_in_file = s.ok() && bloom_block_contents.data.size() > 0;
  275. }
  276. Slice* bloom_block;
  277. if (bloom_in_file) {
  278. // If bloom_block_contents.allocation is not empty (which will be the case
  279. // for non-mmap mode), it holds the alloated memory for the bloom block.
  280. // It needs to be kept alive to keep `bloom_block` valid.
  281. bloom_block_alloc_ = std::move(bloom_block_contents.allocation);
  282. bloom_block = &bloom_block_contents.data;
  283. } else {
  284. bloom_block = nullptr;
  285. }
  286. Slice* index_block;
  287. if (index_in_file) {
  288. // If index_block_contents.allocation is not empty (which will be the case
  289. // for non-mmap mode), it holds the alloated memory for the index block.
  290. // It needs to be kept alive to keep `index_block` valid.
  291. index_block_alloc_ = std::move(index_block_contents.allocation);
  292. index_block = &index_block_contents.data;
  293. } else {
  294. index_block = nullptr;
  295. }
  296. if ((prefix_extractor_ == nullptr) && (hash_table_ratio != 0)) {
  297. // moptions.prefix_extractor is requried for a hash-based look-up.
  298. return Status::NotSupported(
  299. "PlainTable requires a prefix extractor enable prefix hash mode.");
  300. }
  301. // First, read the whole file, for every kIndexIntervalForSamePrefixKeys rows
  302. // for a prefix (starting from the first one), generate a record of (hash,
  303. // offset) and append it to IndexRecordList, which is a data structure created
  304. // to store them.
  305. if (!index_in_file) {
  306. // Allocate bloom filter here for total order mode.
  307. if (IsTotalOrderMode()) {
  308. AllocateBloom(bloom_bits_per_key,
  309. static_cast<uint32_t>(props->num_entries),
  310. huge_page_tlb_size);
  311. }
  312. } else if (bloom_in_file) {
  313. enable_bloom_ = true;
  314. auto num_blocks_property = props->user_collected_properties.find(
  315. PlainTablePropertyNames::kNumBloomBlocks);
  316. uint32_t num_blocks = 0;
  317. if (num_blocks_property != props->user_collected_properties.end()) {
  318. Slice temp_slice(num_blocks_property->second);
  319. if (!GetVarint32(&temp_slice, &num_blocks)) {
  320. num_blocks = 0;
  321. }
  322. }
  323. // cast away const qualifier, because bloom_ won't be changed
  324. bloom_.SetRawData(const_cast<char*>(bloom_block->data()),
  325. static_cast<uint32_t>(bloom_block->size()) * 8,
  326. num_blocks);
  327. } else {
  328. // Index in file but no bloom in file. Disable bloom filter in this case.
  329. enable_bloom_ = false;
  330. bloom_bits_per_key = 0;
  331. }
  332. PlainTableIndexBuilder index_builder(&arena_, ioptions_, prefix_extractor_,
  333. index_sparseness, hash_table_ratio,
  334. huge_page_tlb_size);
  335. std::vector<uint32_t> prefix_hashes;
  336. if (!index_in_file) {
  337. // Populates _bloom if enabled (total order mode)
  338. s = PopulateIndexRecordList(&index_builder, &prefix_hashes);
  339. if (!s.ok()) {
  340. return s;
  341. }
  342. } else {
  343. s = index_.InitFromRawData(*index_block);
  344. if (!s.ok()) {
  345. return s;
  346. }
  347. }
  348. if (!index_in_file) {
  349. if (!IsTotalOrderMode()) {
  350. // Calculated bloom filter size and allocate memory for
  351. // bloom filter based on the number of prefixes, then fill it.
  352. AllocateBloom(bloom_bits_per_key, index_.GetNumPrefixes(),
  353. huge_page_tlb_size);
  354. if (enable_bloom_) {
  355. FillBloom(prefix_hashes);
  356. }
  357. }
  358. }
  359. // Fill two table properties.
  360. if (!index_in_file) {
  361. props->user_collected_properties["plain_table_hash_table_size"] =
  362. std::to_string(index_.GetIndexSize() * PlainTableIndex::kOffsetLen);
  363. props->user_collected_properties["plain_table_sub_index_size"] =
  364. std::to_string(index_.GetSubIndexSize());
  365. } else {
  366. props->user_collected_properties["plain_table_hash_table_size"] =
  367. std::to_string(0);
  368. props->user_collected_properties["plain_table_sub_index_size"] =
  369. std::to_string(0);
  370. }
  371. return Status::OK();
  372. }
  373. Status PlainTableReader::GetOffset(PlainTableKeyDecoder* decoder,
  374. const Slice& target, const Slice& prefix,
  375. uint32_t prefix_hash, bool& prefix_matched,
  376. uint32_t* offset) const {
  377. prefix_matched = false;
  378. uint32_t prefix_index_offset;
  379. auto res = index_.GetOffset(prefix_hash, &prefix_index_offset);
  380. if (res == PlainTableIndex::kNoPrefixForBucket) {
  381. *offset = file_info_.data_end_offset;
  382. return Status::OK();
  383. } else if (res == PlainTableIndex::kDirectToFile) {
  384. *offset = prefix_index_offset;
  385. return Status::OK();
  386. }
  387. // point to sub-index, need to do a binary search
  388. uint32_t upper_bound = 0;
  389. const char* base_ptr =
  390. index_.GetSubIndexBasePtrAndUpperBound(prefix_index_offset, &upper_bound);
  391. uint32_t low = 0;
  392. uint32_t high = upper_bound;
  393. ParsedInternalKey mid_key;
  394. ParsedInternalKey parsed_target;
  395. Status s = ParseInternalKey(target, &parsed_target,
  396. false /* log_err_key */); // TODO
  397. if (!s.ok()) {
  398. return s;
  399. }
  400. // The key is between [low, high). Do a binary search between it.
  401. while (high - low > 1) {
  402. uint32_t mid = (high + low) / 2;
  403. uint32_t file_offset = GetFixed32Element(base_ptr, mid);
  404. uint32_t tmp;
  405. s = decoder->NextKeyNoValue(file_offset, &mid_key, nullptr, &tmp);
  406. if (!s.ok()) {
  407. return s;
  408. }
  409. int cmp_result = internal_comparator_.Compare(mid_key, parsed_target);
  410. if (cmp_result < 0) {
  411. low = mid;
  412. } else {
  413. if (cmp_result == 0) {
  414. // Happen to have found the exact key or target is smaller than the
  415. // first key after base_offset.
  416. prefix_matched = true;
  417. *offset = file_offset;
  418. return Status::OK();
  419. } else {
  420. high = mid;
  421. }
  422. }
  423. }
  424. // Both of the key at the position low or low+1 could share the same
  425. // prefix as target. We need to rule out one of them to avoid to go
  426. // to the wrong prefix.
  427. ParsedInternalKey low_key;
  428. uint32_t tmp;
  429. uint32_t low_key_offset = GetFixed32Element(base_ptr, low);
  430. s = decoder->NextKeyNoValue(low_key_offset, &low_key, nullptr, &tmp);
  431. if (!s.ok()) {
  432. return s;
  433. }
  434. if (GetPrefix(low_key) == prefix) {
  435. prefix_matched = true;
  436. *offset = low_key_offset;
  437. } else if (low + 1 < upper_bound) {
  438. // There is possible a next prefix, return it
  439. prefix_matched = false;
  440. *offset = GetFixed32Element(base_ptr, low + 1);
  441. } else {
  442. // target is larger than a key of the last prefix in this bucket
  443. // but with a different prefix. Key does not exist.
  444. *offset = file_info_.data_end_offset;
  445. }
  446. return Status::OK();
  447. }
  448. bool PlainTableReader::MatchBloom(uint32_t hash) const {
  449. if (!enable_bloom_) {
  450. return true;
  451. }
  452. if (bloom_.MayContainHash(hash)) {
  453. PERF_COUNTER_ADD(bloom_sst_hit_count, 1);
  454. return true;
  455. } else {
  456. PERF_COUNTER_ADD(bloom_sst_miss_count, 1);
  457. return false;
  458. }
  459. }
  460. Status PlainTableReader::Next(PlainTableKeyDecoder* decoder, uint32_t* offset,
  461. ParsedInternalKey* parsed_key,
  462. Slice* internal_key, Slice* value,
  463. bool* seekable) const {
  464. if (*offset == file_info_.data_end_offset) {
  465. *offset = file_info_.data_end_offset;
  466. return Status::OK();
  467. }
  468. if (*offset > file_info_.data_end_offset) {
  469. return Status::Corruption("Offset is out of file size");
  470. }
  471. uint32_t bytes_read;
  472. Status s = decoder->NextKey(*offset, parsed_key, internal_key, value,
  473. &bytes_read, seekable);
  474. if (!s.ok()) {
  475. return s;
  476. }
  477. *offset = *offset + bytes_read;
  478. return Status::OK();
  479. }
  480. void PlainTableReader::Prepare(const Slice& target) {
  481. if (enable_bloom_) {
  482. uint32_t prefix_hash = GetSliceHash(GetPrefix(target));
  483. bloom_.Prefetch(prefix_hash);
  484. }
  485. }
  486. Status PlainTableReader::Get(const ReadOptions& /*ro*/, const Slice& target,
  487. GetContext* get_context,
  488. const SliceTransform* /* prefix_extractor */,
  489. bool /*skip_filters*/) {
  490. // Check bloom filter first.
  491. Slice prefix_slice;
  492. uint32_t prefix_hash;
  493. if (IsTotalOrderMode()) {
  494. if (full_scan_mode_) {
  495. status_ =
  496. Status::InvalidArgument("Get() is not allowed in full scan mode.");
  497. }
  498. // Match whole user key for bloom filter check.
  499. if (!MatchBloom(GetSliceHash(ExtractUserKey(target)))) {
  500. return Status::OK();
  501. }
  502. // in total order mode, there is only one bucket 0, and we always use empty
  503. // prefix.
  504. prefix_slice = Slice();
  505. prefix_hash = 0;
  506. } else {
  507. prefix_slice = GetPrefix(target);
  508. prefix_hash = GetSliceHash(prefix_slice);
  509. if (!MatchBloom(prefix_hash)) {
  510. return Status::OK();
  511. }
  512. }
  513. uint32_t offset;
  514. bool prefix_match;
  515. PlainTableKeyDecoder decoder(&file_info_, encoding_type_, user_key_len_,
  516. prefix_extractor_);
  517. Status s = GetOffset(&decoder, target, prefix_slice, prefix_hash,
  518. prefix_match, &offset);
  519. if (!s.ok()) {
  520. return s;
  521. }
  522. ParsedInternalKey found_key;
  523. ParsedInternalKey parsed_target;
  524. s = ParseInternalKey(target, &parsed_target,
  525. false /* log_err_key */); // TODO
  526. if (!s.ok()) {
  527. return s;
  528. }
  529. Slice found_value;
  530. while (offset < file_info_.data_end_offset) {
  531. s = Next(&decoder, &offset, &found_key, nullptr, &found_value);
  532. if (!s.ok()) {
  533. return s;
  534. }
  535. if (!prefix_match) {
  536. // Need to verify prefix for the first key found if it is not yet
  537. // checked.
  538. if (GetPrefix(found_key) != prefix_slice) {
  539. return Status::OK();
  540. }
  541. prefix_match = true;
  542. }
  543. // TODO(ljin): since we know the key comparison result here,
  544. // can we enable the fast path?
  545. if (internal_comparator_.Compare(found_key, parsed_target) >= 0) {
  546. bool dont_care __attribute__((__unused__));
  547. bool ret = get_context->SaveValue(found_key, found_value, &dont_care, &s,
  548. dummy_cleanable_.get());
  549. if (!s.ok()) {
  550. return s;
  551. }
  552. if (!ret) {
  553. break;
  554. }
  555. }
  556. }
  557. return Status::OK();
  558. }
  559. uint64_t PlainTableReader::ApproximateOffsetOf(
  560. const ReadOptions& /*read_options*/, const Slice& /*key*/,
  561. TableReaderCaller /*caller*/) {
  562. return 0;
  563. }
  564. uint64_t PlainTableReader::ApproximateSize(const ReadOptions& /* read_options*/,
  565. const Slice& /*start*/,
  566. const Slice& /*end*/,
  567. TableReaderCaller /*caller*/) {
  568. return 0;
  569. }
  570. PlainTableIterator::PlainTableIterator(PlainTableReader* table,
  571. bool use_prefix_seek)
  572. : table_(table),
  573. decoder_(&table_->file_info_, table_->encoding_type_,
  574. table_->user_key_len_, table_->prefix_extractor_),
  575. use_prefix_seek_(use_prefix_seek) {
  576. next_offset_ = offset_ = table_->file_info_.data_end_offset;
  577. }
  578. PlainTableIterator::~PlainTableIterator() = default;
  579. bool PlainTableIterator::Valid() const {
  580. return offset_ < table_->file_info_.data_end_offset &&
  581. offset_ >= table_->data_start_offset_;
  582. }
  583. void PlainTableIterator::SeekToFirst() {
  584. status_ = Status::OK();
  585. next_offset_ = table_->data_start_offset_;
  586. if (next_offset_ >= table_->file_info_.data_end_offset) {
  587. next_offset_ = offset_ = table_->file_info_.data_end_offset;
  588. } else {
  589. Next();
  590. }
  591. }
  592. void PlainTableIterator::SeekToLast() {
  593. assert(false);
  594. status_ = Status::NotSupported("SeekToLast() is not supported in PlainTable");
  595. next_offset_ = offset_ = table_->file_info_.data_end_offset;
  596. }
  597. void PlainTableIterator::Seek(const Slice& target) {
  598. if (use_prefix_seek_ != !table_->IsTotalOrderMode()) {
  599. // This check is done here instead of NewIterator() to permit creating an
  600. // iterator with total_order_seek = true even if we won't be able to Seek()
  601. // it. This is needed for compaction: it creates iterator with
  602. // total_order_seek = true but usually never does Seek() on it,
  603. // only SeekToFirst().
  604. status_ = Status::InvalidArgument(
  605. "total_order_seek not implemented for PlainTable.");
  606. offset_ = next_offset_ = table_->file_info_.data_end_offset;
  607. return;
  608. }
  609. // If the user doesn't set prefix seek option and we are not able to do a
  610. // total Seek(). assert failure.
  611. if (table_->IsTotalOrderMode()) {
  612. if (table_->full_scan_mode_) {
  613. status_ =
  614. Status::InvalidArgument("Seek() is not allowed in full scan mode.");
  615. offset_ = next_offset_ = table_->file_info_.data_end_offset;
  616. return;
  617. } else if (table_->GetIndexSize() > 1) {
  618. assert(false);
  619. status_ = Status::NotSupported(
  620. "PlainTable cannot issue non-prefix seek unless in total order "
  621. "mode.");
  622. offset_ = next_offset_ = table_->file_info_.data_end_offset;
  623. return;
  624. }
  625. }
  626. Slice prefix_slice = table_->GetPrefix(target);
  627. uint32_t prefix_hash = 0;
  628. // Bloom filter is ignored in total-order mode.
  629. if (!table_->IsTotalOrderMode()) {
  630. prefix_hash = GetSliceHash(prefix_slice);
  631. if (!table_->MatchBloom(prefix_hash)) {
  632. status_ = Status::OK();
  633. offset_ = next_offset_ = table_->file_info_.data_end_offset;
  634. return;
  635. }
  636. }
  637. bool prefix_match;
  638. status_ = table_->GetOffset(&decoder_, target, prefix_slice, prefix_hash,
  639. prefix_match, &next_offset_);
  640. if (!status_.ok()) {
  641. offset_ = next_offset_ = table_->file_info_.data_end_offset;
  642. return;
  643. }
  644. if (next_offset_ < table_->file_info_.data_end_offset) {
  645. for (Next(); status_.ok() && Valid(); Next()) {
  646. if (!prefix_match) {
  647. // Need to verify the first key's prefix
  648. if (table_->GetPrefix(key()) != prefix_slice) {
  649. offset_ = next_offset_ = table_->file_info_.data_end_offset;
  650. break;
  651. }
  652. prefix_match = true;
  653. }
  654. if (table_->internal_comparator_.Compare(key(), target) >= 0) {
  655. break;
  656. }
  657. }
  658. } else {
  659. offset_ = table_->file_info_.data_end_offset;
  660. }
  661. }
  662. void PlainTableIterator::SeekForPrev(const Slice& /*target*/) {
  663. assert(false);
  664. status_ =
  665. Status::NotSupported("SeekForPrev() is not supported in PlainTable");
  666. offset_ = next_offset_ = table_->file_info_.data_end_offset;
  667. }
  668. void PlainTableIterator::Next() {
  669. offset_ = next_offset_;
  670. if (offset_ < table_->file_info_.data_end_offset) {
  671. Slice tmp_slice;
  672. ParsedInternalKey parsed_key;
  673. status_ =
  674. table_->Next(&decoder_, &next_offset_, &parsed_key, &key_, &value_);
  675. if (!status_.ok()) {
  676. offset_ = next_offset_ = table_->file_info_.data_end_offset;
  677. }
  678. }
  679. }
  680. void PlainTableIterator::Prev() { assert(false); }
  681. Slice PlainTableIterator::key() const {
  682. assert(Valid());
  683. return key_;
  684. }
  685. Slice PlainTableIterator::value() const {
  686. assert(Valid());
  687. return value_;
  688. }
  689. Status PlainTableIterator::status() const { return status_; }
  690. } // namespace ROCKSDB_NAMESPACE