cuckoo_table_builder.cc 20 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528
  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. #ifndef ROCKSDB_LITE
  6. #include "table/cuckoo/cuckoo_table_builder.h"
  7. #include <assert.h>
  8. #include <algorithm>
  9. #include <limits>
  10. #include <string>
  11. #include <vector>
  12. #include "db/dbformat.h"
  13. #include "file/writable_file_writer.h"
  14. #include "rocksdb/env.h"
  15. #include "rocksdb/table.h"
  16. #include "table/block_based/block_builder.h"
  17. #include "table/cuckoo/cuckoo_table_factory.h"
  18. #include "table/format.h"
  19. #include "table/meta_blocks.h"
  20. #include "util/autovector.h"
  21. #include "util/random.h"
  22. #include "util/string_util.h"
  23. namespace ROCKSDB_NAMESPACE {
  24. const std::string CuckooTablePropertyNames::kEmptyKey =
  25. "rocksdb.cuckoo.bucket.empty.key";
  26. const std::string CuckooTablePropertyNames::kNumHashFunc =
  27. "rocksdb.cuckoo.hash.num";
  28. const std::string CuckooTablePropertyNames::kHashTableSize =
  29. "rocksdb.cuckoo.hash.size";
  30. const std::string CuckooTablePropertyNames::kValueLength =
  31. "rocksdb.cuckoo.value.length";
  32. const std::string CuckooTablePropertyNames::kIsLastLevel =
  33. "rocksdb.cuckoo.file.islastlevel";
  34. const std::string CuckooTablePropertyNames::kCuckooBlockSize =
  35. "rocksdb.cuckoo.hash.cuckooblocksize";
  36. const std::string CuckooTablePropertyNames::kIdentityAsFirstHash =
  37. "rocksdb.cuckoo.hash.identityfirst";
  38. const std::string CuckooTablePropertyNames::kUseModuleHash =
  39. "rocksdb.cuckoo.hash.usemodule";
  40. const std::string CuckooTablePropertyNames::kUserKeyLength =
  41. "rocksdb.cuckoo.hash.userkeylength";
  42. // Obtained by running echo rocksdb.table.cuckoo | sha1sum
  43. extern const uint64_t kCuckooTableMagicNumber = 0x926789d0c5f17873ull;
  44. CuckooTableBuilder::CuckooTableBuilder(
  45. WritableFileWriter* file, double max_hash_table_ratio,
  46. uint32_t max_num_hash_table, uint32_t max_search_depth,
  47. const Comparator* user_comparator, uint32_t cuckoo_block_size,
  48. bool use_module_hash, bool identity_as_first_hash,
  49. uint64_t (*get_slice_hash)(const Slice&, uint32_t, uint64_t),
  50. uint32_t column_family_id, const std::string& column_family_name)
  51. : num_hash_func_(2),
  52. file_(file),
  53. max_hash_table_ratio_(max_hash_table_ratio),
  54. max_num_hash_func_(max_num_hash_table),
  55. max_search_depth_(max_search_depth),
  56. cuckoo_block_size_(std::max(1U, cuckoo_block_size)),
  57. hash_table_size_(use_module_hash ? 0 : 2),
  58. is_last_level_file_(false),
  59. has_seen_first_key_(false),
  60. has_seen_first_value_(false),
  61. key_size_(0),
  62. value_size_(0),
  63. num_entries_(0),
  64. num_values_(0),
  65. ucomp_(user_comparator),
  66. use_module_hash_(use_module_hash),
  67. identity_as_first_hash_(identity_as_first_hash),
  68. get_slice_hash_(get_slice_hash),
  69. closed_(false) {
  70. // Data is in a huge block.
  71. properties_.num_data_blocks = 1;
  72. properties_.index_size = 0;
  73. properties_.filter_size = 0;
  74. properties_.column_family_id = column_family_id;
  75. properties_.column_family_name = column_family_name;
  76. }
  77. void CuckooTableBuilder::Add(const Slice& key, const Slice& value) {
  78. if (num_entries_ >= kMaxVectorIdx - 1) {
  79. status_ = Status::NotSupported("Number of keys in a file must be < 2^32-1");
  80. return;
  81. }
  82. ParsedInternalKey ikey;
  83. if (!ParseInternalKey(key, &ikey)) {
  84. status_ = Status::Corruption("Unable to parse key into inernal key.");
  85. return;
  86. }
  87. if (ikey.type != kTypeDeletion && ikey.type != kTypeValue) {
  88. status_ = Status::NotSupported("Unsupported key type " +
  89. ToString(ikey.type));
  90. return;
  91. }
  92. // Determine if we can ignore the sequence number and value type from
  93. // internal keys by looking at sequence number from first key. We assume
  94. // that if first key has a zero sequence number, then all the remaining
  95. // keys will have zero seq. no.
  96. if (!has_seen_first_key_) {
  97. is_last_level_file_ = ikey.sequence == 0;
  98. has_seen_first_key_ = true;
  99. smallest_user_key_.assign(ikey.user_key.data(), ikey.user_key.size());
  100. largest_user_key_.assign(ikey.user_key.data(), ikey.user_key.size());
  101. key_size_ = is_last_level_file_ ? ikey.user_key.size() : key.size();
  102. }
  103. if (key_size_ != (is_last_level_file_ ? ikey.user_key.size() : key.size())) {
  104. status_ = Status::NotSupported("all keys have to be the same size");
  105. return;
  106. }
  107. if (ikey.type == kTypeValue) {
  108. if (!has_seen_first_value_) {
  109. has_seen_first_value_ = true;
  110. value_size_ = value.size();
  111. }
  112. if (value_size_ != value.size()) {
  113. status_ = Status::NotSupported("all values have to be the same size");
  114. return;
  115. }
  116. if (is_last_level_file_) {
  117. kvs_.append(ikey.user_key.data(), ikey.user_key.size());
  118. } else {
  119. kvs_.append(key.data(), key.size());
  120. }
  121. kvs_.append(value.data(), value.size());
  122. ++num_values_;
  123. } else {
  124. if (is_last_level_file_) {
  125. deleted_keys_.append(ikey.user_key.data(), ikey.user_key.size());
  126. } else {
  127. deleted_keys_.append(key.data(), key.size());
  128. }
  129. }
  130. ++num_entries_;
  131. // In order to fill the empty buckets in the hash table, we identify a
  132. // key which is not used so far (unused_user_key). We determine this by
  133. // maintaining smallest and largest keys inserted so far in bytewise order
  134. // and use them to find a key outside this range in Finish() operation.
  135. // Note that this strategy is independent of user comparator used here.
  136. if (ikey.user_key.compare(smallest_user_key_) < 0) {
  137. smallest_user_key_.assign(ikey.user_key.data(), ikey.user_key.size());
  138. } else if (ikey.user_key.compare(largest_user_key_) > 0) {
  139. largest_user_key_.assign(ikey.user_key.data(), ikey.user_key.size());
  140. }
  141. if (!use_module_hash_) {
  142. if (hash_table_size_ < num_entries_ / max_hash_table_ratio_) {
  143. hash_table_size_ *= 2;
  144. }
  145. }
  146. }
  147. bool CuckooTableBuilder::IsDeletedKey(uint64_t idx) const {
  148. assert(closed_);
  149. return idx >= num_values_;
  150. }
  151. Slice CuckooTableBuilder::GetKey(uint64_t idx) const {
  152. assert(closed_);
  153. if (IsDeletedKey(idx)) {
  154. return Slice(&deleted_keys_[static_cast<size_t>((idx - num_values_) * key_size_)], static_cast<size_t>(key_size_));
  155. }
  156. return Slice(&kvs_[static_cast<size_t>(idx * (key_size_ + value_size_))], static_cast<size_t>(key_size_));
  157. }
  158. Slice CuckooTableBuilder::GetUserKey(uint64_t idx) const {
  159. assert(closed_);
  160. return is_last_level_file_ ? GetKey(idx) : ExtractUserKey(GetKey(idx));
  161. }
  162. Slice CuckooTableBuilder::GetValue(uint64_t idx) const {
  163. assert(closed_);
  164. if (IsDeletedKey(idx)) {
  165. static std::string empty_value(static_cast<unsigned int>(value_size_), 'a');
  166. return Slice(empty_value);
  167. }
  168. return Slice(&kvs_[static_cast<size_t>(idx * (key_size_ + value_size_) + key_size_)], static_cast<size_t>(value_size_));
  169. }
  170. Status CuckooTableBuilder::MakeHashTable(std::vector<CuckooBucket>* buckets) {
  171. buckets->resize(static_cast<size_t>(hash_table_size_ + cuckoo_block_size_ - 1));
  172. uint32_t make_space_for_key_call_id = 0;
  173. for (uint32_t vector_idx = 0; vector_idx < num_entries_; vector_idx++) {
  174. uint64_t bucket_id = 0;
  175. bool bucket_found = false;
  176. autovector<uint64_t> hash_vals;
  177. Slice user_key = GetUserKey(vector_idx);
  178. for (uint32_t hash_cnt = 0; hash_cnt < num_hash_func_ && !bucket_found;
  179. ++hash_cnt) {
  180. uint64_t hash_val = CuckooHash(user_key, hash_cnt, use_module_hash_,
  181. hash_table_size_, identity_as_first_hash_, get_slice_hash_);
  182. // If there is a collision, check next cuckoo_block_size_ locations for
  183. // empty locations. While checking, if we reach end of the hash table,
  184. // stop searching and proceed for next hash function.
  185. for (uint32_t block_idx = 0; block_idx < cuckoo_block_size_;
  186. ++block_idx, ++hash_val) {
  187. if ((*buckets)[static_cast<size_t>(hash_val)].vector_idx == kMaxVectorIdx) {
  188. bucket_id = hash_val;
  189. bucket_found = true;
  190. break;
  191. } else {
  192. if (ucomp_->Compare(user_key,
  193. GetUserKey((*buckets)[static_cast<size_t>(hash_val)].vector_idx)) == 0) {
  194. return Status::NotSupported("Same key is being inserted again.");
  195. }
  196. hash_vals.push_back(hash_val);
  197. }
  198. }
  199. }
  200. while (!bucket_found && !MakeSpaceForKey(hash_vals,
  201. ++make_space_for_key_call_id, buckets, &bucket_id)) {
  202. // Rehash by increashing number of hash tables.
  203. if (num_hash_func_ >= max_num_hash_func_) {
  204. return Status::NotSupported("Too many collisions. Unable to hash.");
  205. }
  206. // We don't really need to rehash the entire table because old hashes are
  207. // still valid and we only increased the number of hash functions.
  208. uint64_t hash_val = CuckooHash(user_key, num_hash_func_, use_module_hash_,
  209. hash_table_size_, identity_as_first_hash_, get_slice_hash_);
  210. ++num_hash_func_;
  211. for (uint32_t block_idx = 0; block_idx < cuckoo_block_size_;
  212. ++block_idx, ++hash_val) {
  213. if ((*buckets)[static_cast<size_t>(hash_val)].vector_idx == kMaxVectorIdx) {
  214. bucket_found = true;
  215. bucket_id = hash_val;
  216. break;
  217. } else {
  218. hash_vals.push_back(hash_val);
  219. }
  220. }
  221. }
  222. (*buckets)[static_cast<size_t>(bucket_id)].vector_idx = vector_idx;
  223. }
  224. return Status::OK();
  225. }
  226. Status CuckooTableBuilder::Finish() {
  227. assert(!closed_);
  228. closed_ = true;
  229. std::vector<CuckooBucket> buckets;
  230. Status s;
  231. std::string unused_bucket;
  232. if (num_entries_ > 0) {
  233. // Calculate the real hash size if module hash is enabled.
  234. if (use_module_hash_) {
  235. hash_table_size_ =
  236. static_cast<uint64_t>(num_entries_ / max_hash_table_ratio_);
  237. }
  238. s = MakeHashTable(&buckets);
  239. if (!s.ok()) {
  240. return s;
  241. }
  242. // Determine unused_user_key to fill empty buckets.
  243. std::string unused_user_key = smallest_user_key_;
  244. int curr_pos = static_cast<int>(unused_user_key.size()) - 1;
  245. while (curr_pos >= 0) {
  246. --unused_user_key[curr_pos];
  247. if (Slice(unused_user_key).compare(smallest_user_key_) < 0) {
  248. break;
  249. }
  250. --curr_pos;
  251. }
  252. if (curr_pos < 0) {
  253. // Try using the largest key to identify an unused key.
  254. unused_user_key = largest_user_key_;
  255. curr_pos = static_cast<int>(unused_user_key.size()) - 1;
  256. while (curr_pos >= 0) {
  257. ++unused_user_key[curr_pos];
  258. if (Slice(unused_user_key).compare(largest_user_key_) > 0) {
  259. break;
  260. }
  261. --curr_pos;
  262. }
  263. }
  264. if (curr_pos < 0) {
  265. return Status::Corruption("Unable to find unused key");
  266. }
  267. if (is_last_level_file_) {
  268. unused_bucket = unused_user_key;
  269. } else {
  270. ParsedInternalKey ikey(unused_user_key, 0, kTypeValue);
  271. AppendInternalKey(&unused_bucket, ikey);
  272. }
  273. }
  274. properties_.num_entries = num_entries_;
  275. properties_.num_deletions = num_entries_ - num_values_;
  276. properties_.fixed_key_len = key_size_;
  277. properties_.user_collected_properties[
  278. CuckooTablePropertyNames::kValueLength].assign(
  279. reinterpret_cast<const char*>(&value_size_), sizeof(value_size_));
  280. uint64_t bucket_size = key_size_ + value_size_;
  281. unused_bucket.resize(static_cast<size_t>(bucket_size), 'a');
  282. // Write the table.
  283. uint32_t num_added = 0;
  284. for (auto& bucket : buckets) {
  285. if (bucket.vector_idx == kMaxVectorIdx) {
  286. s = file_->Append(Slice(unused_bucket));
  287. } else {
  288. ++num_added;
  289. s = file_->Append(GetKey(bucket.vector_idx));
  290. if (s.ok()) {
  291. if (value_size_ > 0) {
  292. s = file_->Append(GetValue(bucket.vector_idx));
  293. }
  294. }
  295. }
  296. if (!s.ok()) {
  297. return s;
  298. }
  299. }
  300. assert(num_added == NumEntries());
  301. properties_.raw_key_size = num_added * properties_.fixed_key_len;
  302. properties_.raw_value_size = num_added * value_size_;
  303. uint64_t offset = buckets.size() * bucket_size;
  304. properties_.data_size = offset;
  305. unused_bucket.resize(static_cast<size_t>(properties_.fixed_key_len));
  306. properties_.user_collected_properties[
  307. CuckooTablePropertyNames::kEmptyKey] = unused_bucket;
  308. properties_.user_collected_properties[
  309. CuckooTablePropertyNames::kNumHashFunc].assign(
  310. reinterpret_cast<char*>(&num_hash_func_), sizeof(num_hash_func_));
  311. properties_.user_collected_properties[
  312. CuckooTablePropertyNames::kHashTableSize].assign(
  313. reinterpret_cast<const char*>(&hash_table_size_),
  314. sizeof(hash_table_size_));
  315. properties_.user_collected_properties[
  316. CuckooTablePropertyNames::kIsLastLevel].assign(
  317. reinterpret_cast<const char*>(&is_last_level_file_),
  318. sizeof(is_last_level_file_));
  319. properties_.user_collected_properties[
  320. CuckooTablePropertyNames::kCuckooBlockSize].assign(
  321. reinterpret_cast<const char*>(&cuckoo_block_size_),
  322. sizeof(cuckoo_block_size_));
  323. properties_.user_collected_properties[
  324. CuckooTablePropertyNames::kIdentityAsFirstHash].assign(
  325. reinterpret_cast<const char*>(&identity_as_first_hash_),
  326. sizeof(identity_as_first_hash_));
  327. properties_.user_collected_properties[
  328. CuckooTablePropertyNames::kUseModuleHash].assign(
  329. reinterpret_cast<const char*>(&use_module_hash_),
  330. sizeof(use_module_hash_));
  331. uint32_t user_key_len = static_cast<uint32_t>(smallest_user_key_.size());
  332. properties_.user_collected_properties[
  333. CuckooTablePropertyNames::kUserKeyLength].assign(
  334. reinterpret_cast<const char*>(&user_key_len),
  335. sizeof(user_key_len));
  336. // Write meta blocks.
  337. MetaIndexBuilder meta_index_builder;
  338. PropertyBlockBuilder property_block_builder;
  339. property_block_builder.AddTableProperty(properties_);
  340. property_block_builder.Add(properties_.user_collected_properties);
  341. Slice property_block = property_block_builder.Finish();
  342. BlockHandle property_block_handle;
  343. property_block_handle.set_offset(offset);
  344. property_block_handle.set_size(property_block.size());
  345. s = file_->Append(property_block);
  346. offset += property_block.size();
  347. if (!s.ok()) {
  348. return s;
  349. }
  350. meta_index_builder.Add(kPropertiesBlock, property_block_handle);
  351. Slice meta_index_block = meta_index_builder.Finish();
  352. BlockHandle meta_index_block_handle;
  353. meta_index_block_handle.set_offset(offset);
  354. meta_index_block_handle.set_size(meta_index_block.size());
  355. s = file_->Append(meta_index_block);
  356. if (!s.ok()) {
  357. return s;
  358. }
  359. Footer footer(kCuckooTableMagicNumber, 1);
  360. footer.set_metaindex_handle(meta_index_block_handle);
  361. footer.set_index_handle(BlockHandle::NullBlockHandle());
  362. std::string footer_encoding;
  363. footer.EncodeTo(&footer_encoding);
  364. s = file_->Append(footer_encoding);
  365. if (file_ != nullptr) {
  366. file_checksum_ = file_->GetFileChecksum();
  367. }
  368. return s;
  369. }
  370. void CuckooTableBuilder::Abandon() {
  371. assert(!closed_);
  372. closed_ = true;
  373. }
  374. uint64_t CuckooTableBuilder::NumEntries() const {
  375. return num_entries_;
  376. }
  377. uint64_t CuckooTableBuilder::FileSize() const {
  378. if (closed_) {
  379. return file_->GetFileSize();
  380. } else if (num_entries_ == 0) {
  381. return 0;
  382. }
  383. if (use_module_hash_) {
  384. return static_cast<uint64_t>((key_size_ + value_size_) *
  385. num_entries_ / max_hash_table_ratio_);
  386. } else {
  387. // Account for buckets being a power of two.
  388. // As elements are added, file size remains constant for a while and
  389. // doubles its size. Since compaction algorithm stops adding elements
  390. // only after it exceeds the file limit, we account for the extra element
  391. // being added here.
  392. uint64_t expected_hash_table_size = hash_table_size_;
  393. if (expected_hash_table_size < (num_entries_ + 1) / max_hash_table_ratio_) {
  394. expected_hash_table_size *= 2;
  395. }
  396. return (key_size_ + value_size_) * expected_hash_table_size - 1;
  397. }
  398. }
  399. // This method is invoked when there is no place to insert the target key.
  400. // It searches for a set of elements that can be moved to accommodate target
  401. // key. The search is a BFS graph traversal with first level (hash_vals)
  402. // being all the buckets target key could go to.
  403. // Then, from each node (curr_node), we find all the buckets that curr_node
  404. // could go to. They form the children of curr_node in the tree.
  405. // We continue the traversal until we find an empty bucket, in which case, we
  406. // move all elements along the path from first level to this empty bucket, to
  407. // make space for target key which is inserted at first level (*bucket_id).
  408. // If tree depth exceedes max depth, we return false indicating failure.
  409. bool CuckooTableBuilder::MakeSpaceForKey(
  410. const autovector<uint64_t>& hash_vals,
  411. const uint32_t make_space_for_key_call_id,
  412. std::vector<CuckooBucket>* buckets, uint64_t* bucket_id) {
  413. struct CuckooNode {
  414. uint64_t bucket_id;
  415. uint32_t depth;
  416. uint32_t parent_pos;
  417. CuckooNode(uint64_t _bucket_id, uint32_t _depth, int _parent_pos)
  418. : bucket_id(_bucket_id), depth(_depth), parent_pos(_parent_pos) {}
  419. };
  420. // This is BFS search tree that is stored simply as a vector.
  421. // Each node stores the index of parent node in the vector.
  422. std::vector<CuckooNode> tree;
  423. // We want to identify already visited buckets in the current method call so
  424. // that we don't add same buckets again for exploration in the tree.
  425. // We do this by maintaining a count of current method call in
  426. // make_space_for_key_call_id, which acts as a unique id for this invocation
  427. // of the method. We store this number into the nodes that we explore in
  428. // current method call.
  429. // It is unlikely for the increment operation to overflow because the maximum
  430. // no. of times this will be called is <= max_num_hash_func_ + num_entries_.
  431. for (uint32_t hash_cnt = 0; hash_cnt < num_hash_func_; ++hash_cnt) {
  432. uint64_t bid = hash_vals[hash_cnt];
  433. (*buckets)[static_cast<size_t>(bid)].make_space_for_key_call_id = make_space_for_key_call_id;
  434. tree.push_back(CuckooNode(bid, 0, 0));
  435. }
  436. bool null_found = false;
  437. uint32_t curr_pos = 0;
  438. while (!null_found && curr_pos < tree.size()) {
  439. CuckooNode& curr_node = tree[curr_pos];
  440. uint32_t curr_depth = curr_node.depth;
  441. if (curr_depth >= max_search_depth_) {
  442. break;
  443. }
  444. CuckooBucket& curr_bucket = (*buckets)[static_cast<size_t>(curr_node.bucket_id)];
  445. for (uint32_t hash_cnt = 0;
  446. hash_cnt < num_hash_func_ && !null_found; ++hash_cnt) {
  447. uint64_t child_bucket_id = CuckooHash(GetUserKey(curr_bucket.vector_idx),
  448. hash_cnt, use_module_hash_, hash_table_size_, identity_as_first_hash_,
  449. get_slice_hash_);
  450. // Iterate inside Cuckoo Block.
  451. for (uint32_t block_idx = 0; block_idx < cuckoo_block_size_;
  452. ++block_idx, ++child_bucket_id) {
  453. if ((*buckets)[static_cast<size_t>(child_bucket_id)].make_space_for_key_call_id ==
  454. make_space_for_key_call_id) {
  455. continue;
  456. }
  457. (*buckets)[static_cast<size_t>(child_bucket_id)].make_space_for_key_call_id =
  458. make_space_for_key_call_id;
  459. tree.push_back(CuckooNode(child_bucket_id, curr_depth + 1,
  460. curr_pos));
  461. if ((*buckets)[static_cast<size_t>(child_bucket_id)].vector_idx == kMaxVectorIdx) {
  462. null_found = true;
  463. break;
  464. }
  465. }
  466. }
  467. ++curr_pos;
  468. }
  469. if (null_found) {
  470. // There is an empty node in tree.back(). Now, traverse the path from this
  471. // empty node to top of the tree and at every node in the path, replace
  472. // child with the parent. Stop when first level is reached in the tree
  473. // (happens when 0 <= bucket_to_replace_pos < num_hash_func_) and return
  474. // this location in first level for target key to be inserted.
  475. uint32_t bucket_to_replace_pos = static_cast<uint32_t>(tree.size()) - 1;
  476. while (bucket_to_replace_pos >= num_hash_func_) {
  477. CuckooNode& curr_node = tree[bucket_to_replace_pos];
  478. (*buckets)[static_cast<size_t>(curr_node.bucket_id)] =
  479. (*buckets)[static_cast<size_t>(tree[curr_node.parent_pos].bucket_id)];
  480. bucket_to_replace_pos = curr_node.parent_pos;
  481. }
  482. *bucket_id = tree[bucket_to_replace_pos].bucket_id;
  483. }
  484. return null_found;
  485. }
  486. const char* CuckooTableBuilder::GetFileChecksumFuncName() const {
  487. if (file_ != nullptr) {
  488. return file_->GetFileChecksumFuncName();
  489. } else {
  490. return kUnknownFileChecksumFuncName.c_str();
  491. }
  492. }
  493. } // namespace ROCKSDB_NAMESPACE
  494. #endif // ROCKSDB_LITE