index_builder.cc 15 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350
  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 "table/block_based/index_builder.h"
  10. #include <cassert>
  11. #include <cinttypes>
  12. #include <list>
  13. #include <string>
  14. #include "db/dbformat.h"
  15. #include "rocksdb/comparator.h"
  16. #include "rocksdb/flush_block_policy.h"
  17. #include "table/block_based/partitioned_filter_block.h"
  18. #include "table/format.h"
  19. namespace ROCKSDB_NAMESPACE {
  20. // Create a index builder based on its type.
  21. IndexBuilder* IndexBuilder::CreateIndexBuilder(
  22. BlockBasedTableOptions::IndexType index_type,
  23. const InternalKeyComparator* comparator,
  24. const InternalKeySliceTransform* int_key_slice_transform,
  25. const bool use_value_delta_encoding,
  26. const BlockBasedTableOptions& table_opt, size_t ts_sz,
  27. const bool persist_user_defined_timestamps) {
  28. IndexBuilder* result = nullptr;
  29. switch (index_type) {
  30. case BlockBasedTableOptions::kBinarySearch: {
  31. result = new ShortenedIndexBuilder(
  32. comparator, table_opt.index_block_restart_interval,
  33. table_opt.format_version, use_value_delta_encoding,
  34. table_opt.index_shortening, /* include_first_key */ false, ts_sz,
  35. persist_user_defined_timestamps);
  36. break;
  37. }
  38. case BlockBasedTableOptions::kHashSearch: {
  39. // Currently kHashSearch is incompatible with index_block_restart_interval
  40. // > 1
  41. assert(table_opt.index_block_restart_interval == 1);
  42. result = new HashIndexBuilder(
  43. comparator, int_key_slice_transform,
  44. table_opt.index_block_restart_interval, table_opt.format_version,
  45. use_value_delta_encoding, table_opt.index_shortening, ts_sz,
  46. persist_user_defined_timestamps);
  47. break;
  48. }
  49. case BlockBasedTableOptions::kTwoLevelIndexSearch: {
  50. result = PartitionedIndexBuilder::CreateIndexBuilder(
  51. comparator, use_value_delta_encoding, table_opt, ts_sz,
  52. persist_user_defined_timestamps);
  53. break;
  54. }
  55. case BlockBasedTableOptions::kBinarySearchWithFirstKey: {
  56. result = new ShortenedIndexBuilder(
  57. comparator, table_opt.index_block_restart_interval,
  58. table_opt.format_version, use_value_delta_encoding,
  59. table_opt.index_shortening, /* include_first_key */ true, ts_sz,
  60. persist_user_defined_timestamps);
  61. break;
  62. }
  63. default: {
  64. assert(!"Do not recognize the index type ");
  65. break;
  66. }
  67. }
  68. return result;
  69. }
  70. Slice ShortenedIndexBuilder::FindShortestInternalKeySeparator(
  71. const Comparator& comparator, const Slice& start, const Slice& limit,
  72. std::string* scratch) {
  73. // Attempt to shorten the user portion of the key
  74. Slice user_start = ExtractUserKey(start);
  75. Slice user_limit = ExtractUserKey(limit);
  76. scratch->assign(user_start.data(), user_start.size());
  77. comparator.FindShortestSeparator(scratch, user_limit);
  78. assert(comparator.Compare(user_start, *scratch) <= 0);
  79. assert(comparator.Compare(user_start, user_limit) >= 0 ||
  80. comparator.Compare(*scratch, user_limit) < 0);
  81. if (scratch->size() <= user_start.size() &&
  82. comparator.Compare(user_start, *scratch) < 0) {
  83. // User key has become shorter physically, but larger logically.
  84. // Tack on the earliest possible number to the shortened user key.
  85. PutFixed64(scratch,
  86. PackSequenceAndType(kMaxSequenceNumber, kValueTypeForSeek));
  87. assert(InternalKeyComparator(&comparator).Compare(start, *scratch) < 0);
  88. assert(InternalKeyComparator(&comparator).Compare(*scratch, limit) < 0);
  89. return *scratch;
  90. } else {
  91. return start;
  92. }
  93. }
  94. Slice ShortenedIndexBuilder::FindShortInternalKeySuccessor(
  95. const Comparator& comparator, const Slice& key, std::string* scratch) {
  96. Slice user_key = ExtractUserKey(key);
  97. scratch->assign(user_key.data(), user_key.size());
  98. comparator.FindShortSuccessor(scratch);
  99. assert(comparator.Compare(user_key, *scratch) <= 0);
  100. if (scratch->size() <= user_key.size() &&
  101. comparator.Compare(user_key, *scratch) < 0) {
  102. // User key has become shorter physically, but larger logically.
  103. // Tack on the earliest possible number to the shortened user key.
  104. PutFixed64(scratch,
  105. PackSequenceAndType(kMaxSequenceNumber, kValueTypeForSeek));
  106. assert(InternalKeyComparator(&comparator).Compare(key, *scratch) < 0);
  107. return *scratch;
  108. } else {
  109. return key;
  110. }
  111. }
  112. uint64_t ShortenedIndexBuilder::EstimateCurrentIndexSize() const {
  113. uint64_t current_size =
  114. must_use_separator_with_seq_
  115. ? index_block_builder_.CurrentSizeEstimate()
  116. : index_block_builder_without_seq_.CurrentSizeEstimate();
  117. if (num_index_entries_ == 0) {
  118. return current_size;
  119. }
  120. uint64_t avg_entry_size = current_size / num_index_entries_;
  121. // Add buffer to generously account (in most cases) for the next index entry
  122. return current_size + (2 * avg_entry_size);
  123. }
  124. PartitionedIndexBuilder* PartitionedIndexBuilder::CreateIndexBuilder(
  125. const InternalKeyComparator* comparator,
  126. const bool use_value_delta_encoding,
  127. const BlockBasedTableOptions& table_opt, size_t ts_sz,
  128. const bool persist_user_defined_timestamps) {
  129. return new PartitionedIndexBuilder(comparator, table_opt,
  130. use_value_delta_encoding, ts_sz,
  131. persist_user_defined_timestamps);
  132. }
  133. PartitionedIndexBuilder::PartitionedIndexBuilder(
  134. const InternalKeyComparator* comparator,
  135. const BlockBasedTableOptions& table_opt,
  136. const bool use_value_delta_encoding, size_t ts_sz,
  137. const bool persist_user_defined_timestamps)
  138. : IndexBuilder(comparator, ts_sz, persist_user_defined_timestamps),
  139. index_block_builder_(
  140. table_opt.index_block_restart_interval, true /*use_delta_encoding*/,
  141. use_value_delta_encoding,
  142. BlockBasedTableOptions::kDataBlockBinarySearch /* index_type */,
  143. 0.75 /* data_block_hash_table_util_ratio */, ts_sz,
  144. persist_user_defined_timestamps, false /* is_user_key */),
  145. index_block_builder_without_seq_(
  146. table_opt.index_block_restart_interval, true /*use_delta_encoding*/,
  147. use_value_delta_encoding,
  148. BlockBasedTableOptions::kDataBlockBinarySearch /* index_type */,
  149. 0.75 /* data_block_hash_table_util_ratio */, ts_sz,
  150. persist_user_defined_timestamps, true /* is_user_key */),
  151. table_opt_(table_opt),
  152. // We start by false. After each partition we revise the value based on
  153. // what the sub_index_builder has decided. If the feature is disabled
  154. // entirely, this will be set to true after switching the first
  155. // sub_index_builder. Otherwise, it could be set to true even one of the
  156. // sub_index_builders could not safely exclude seq from the keys, then it
  157. // wil be enforced on all sub_index_builders on ::Finish.
  158. must_use_separator_with_seq_(false),
  159. use_value_delta_encoding_(use_value_delta_encoding) {
  160. MakeNewSubIndexBuilder();
  161. }
  162. void PartitionedIndexBuilder::MakeNewSubIndexBuilder() {
  163. auto new_builder = std::make_unique<ShortenedIndexBuilder>(
  164. comparator_, table_opt_.index_block_restart_interval,
  165. table_opt_.format_version, use_value_delta_encoding_,
  166. table_opt_.index_shortening, /* include_first_key */ false, ts_sz_,
  167. persist_user_defined_timestamps_);
  168. sub_index_builder_ = new_builder.get();
  169. // Start next partition entry, where we will modify the key
  170. entries_.push_back({{}, std::move(new_builder)});
  171. BlockBuilder* builder_to_monitor;
  172. // Set sub_index_builder_->must_use_separator_with_seq_ to true if
  173. // must_use_separator_with_seq_ is true (internal-key mode) (set to false by
  174. // default on Creation) so that flush policy can point to
  175. // sub_index_builder_->index_block_builder_
  176. if (must_use_separator_with_seq_) {
  177. sub_index_builder_->must_use_separator_with_seq_ = true;
  178. builder_to_monitor = &sub_index_builder_->index_block_builder_;
  179. } else {
  180. builder_to_monitor = &sub_index_builder_->index_block_builder_without_seq_;
  181. }
  182. if (flush_policy_ == nullptr) {
  183. // Note: some partitions could be sub-optimal since sub_index_builder_
  184. // could later reset must_use_separator_with_seq_ but the probability and
  185. // impact of that are low.
  186. flush_policy_ = NewFlushBlockBySizePolicy(table_opt_.metadata_block_size,
  187. table_opt_.block_size_deviation,
  188. *builder_to_monitor);
  189. } else {
  190. flush_policy_->Retarget(*builder_to_monitor);
  191. }
  192. partition_cut_requested_ = false;
  193. }
  194. void PartitionedIndexBuilder::RequestPartitionCut() {
  195. partition_cut_requested_ = true;
  196. }
  197. std::unique_ptr<IndexBuilder::PreparedIndexEntry>
  198. PartitionedIndexBuilder::CreatePreparedIndexEntry() {
  199. // Fortunately, for ShortenedIndexBuilder, we can prepare an entry from one
  200. // similarly configured builder and finish it at another.
  201. return entries_.front().value->CreatePreparedIndexEntry();
  202. }
  203. void PartitionedIndexBuilder::PrepareIndexEntry(
  204. const Slice& last_key_in_current_block,
  205. const Slice* first_key_in_next_block, PreparedIndexEntry* out) {
  206. // Fortunately, for ShortenedIndexBuilder, we can prepare an entry from one
  207. // similarly configured builder and finish it at another. We just have to
  208. // keep in mind that this first sub builder keeps track of the original
  209. // must_use_separator_with_seq_ in the pipeline that is then propagated.
  210. return entries_.front().value->PrepareIndexEntry(
  211. last_key_in_current_block, first_key_in_next_block, out);
  212. }
  213. void PartitionedIndexBuilder::MaybeFlush(const Slice& index_key,
  214. const BlockHandle& index_value) {
  215. bool do_flush = !sub_index_builder_->index_block_builder_.empty() &&
  216. (partition_cut_requested_ ||
  217. flush_policy_->Update(
  218. index_key, EncodedBlockHandle(index_value).AsSlice()));
  219. if (do_flush) {
  220. assert(entries_.back().value.get() == sub_index_builder_);
  221. cut_filter_block = true;
  222. MakeNewSubIndexBuilder();
  223. }
  224. }
  225. void PartitionedIndexBuilder::FinishIndexEntry(const BlockHandle& block_handle,
  226. PreparedIndexEntry* base_entry,
  227. bool skip_delta_encoding) {
  228. using SPIE = ShortenedIndexBuilder::ShortenedPreparedIndexEntry;
  229. SPIE* entry = static_cast<SPIE*>(base_entry);
  230. MaybeFlush(entry->separator_with_seq, block_handle);
  231. sub_index_builder_->FinishIndexEntry(block_handle, base_entry,
  232. skip_delta_encoding);
  233. std::swap(entries_.back().key, entry->separator_with_seq);
  234. if (!must_use_separator_with_seq_ && entry->must_use_separator_with_seq) {
  235. // We need to apply !must_use_separator_with_seq to all sub-index builders
  236. must_use_separator_with_seq_ = true;
  237. flush_policy_->Retarget(sub_index_builder_->index_block_builder_);
  238. }
  239. // NOTE: not compatible with coupled partitioned filters so don't need to
  240. // cut_filter_block
  241. }
  242. Slice PartitionedIndexBuilder::AddIndexEntry(
  243. const Slice& last_key_in_current_block,
  244. const Slice* first_key_in_next_block, const BlockHandle& block_handle,
  245. std::string* separator_scratch, bool skip_delta_encoding) {
  246. // At least when running without parallel compression, maintain behavior of
  247. // avoiding a last index partition with just one entry
  248. if (first_key_in_next_block) {
  249. MaybeFlush(last_key_in_current_block, block_handle);
  250. }
  251. auto sep = sub_index_builder_->AddIndexEntry(
  252. last_key_in_current_block, first_key_in_next_block, block_handle,
  253. separator_scratch, skip_delta_encoding);
  254. entries_.back().key.assign(sep.data(), sep.size());
  255. if (!must_use_separator_with_seq_ &&
  256. sub_index_builder_->must_use_separator_with_seq_) {
  257. // We need to apply !must_use_separator_with_seq to all sub-index builders
  258. must_use_separator_with_seq_ = true;
  259. flush_policy_->Retarget(sub_index_builder_->index_block_builder_);
  260. }
  261. if (UNLIKELY(first_key_in_next_block == nullptr)) {
  262. // no more keys
  263. cut_filter_block = true;
  264. }
  265. return sep;
  266. }
  267. Status PartitionedIndexBuilder::Finish(
  268. IndexBlocks* index_blocks, const BlockHandle& last_partition_block_handle) {
  269. if (partition_cnt_ == 0) {
  270. sub_index_builder_ = nullptr;
  271. if (!entries_.empty()) {
  272. // Remove the last entry if it is empty
  273. if (entries_.back().value->index_block_builder_.empty()) {
  274. assert(entries_.back().key.empty());
  275. entries_.pop_back();
  276. }
  277. partition_cnt_ = entries_.size();
  278. }
  279. }
  280. if (finishing_indexes_ == true) {
  281. Entry& last_entry = entries_.front();
  282. EncodedBlockHandle handle_encoding(last_partition_block_handle);
  283. std::string handle_delta_encoding;
  284. PutVarsignedint64(
  285. &handle_delta_encoding,
  286. last_partition_block_handle.size() - last_encoded_handle_.size());
  287. last_encoded_handle_ = last_partition_block_handle;
  288. const Slice handle_delta_encoding_slice(handle_delta_encoding);
  289. index_block_builder_.Add(last_entry.key, handle_encoding.AsSlice(),
  290. &handle_delta_encoding_slice);
  291. if (!must_use_separator_with_seq_) {
  292. index_block_builder_without_seq_.Add(ExtractUserKey(last_entry.key),
  293. handle_encoding.AsSlice(),
  294. &handle_delta_encoding_slice);
  295. }
  296. entries_.pop_front();
  297. }
  298. // If there is no sub_index left, then return the 2nd level index.
  299. if (UNLIKELY(entries_.empty())) {
  300. if (must_use_separator_with_seq_) {
  301. index_blocks->index_block_contents = index_block_builder_.Finish();
  302. } else {
  303. index_blocks->index_block_contents =
  304. index_block_builder_without_seq_.Finish();
  305. }
  306. top_level_index_size_ = index_blocks->index_block_contents.size();
  307. index_size_ += top_level_index_size_;
  308. return Status::OK();
  309. } else {
  310. // Finish the next partition index in line and Incomplete() to indicate we
  311. // expect more calls to Finish
  312. Entry& entry = entries_.front();
  313. // Apply the policy to all sub-indexes
  314. entry.value->must_use_separator_with_seq_ = must_use_separator_with_seq_;
  315. auto s = entry.value->Finish(index_blocks);
  316. index_size_ += index_blocks->index_block_contents.size();
  317. finishing_indexes_ = true;
  318. return s.ok() ? Status::Incomplete() : s;
  319. }
  320. }
  321. size_t PartitionedIndexBuilder::NumPartitions() const { return partition_cnt_; }
  322. } // namespace ROCKSDB_NAMESPACE