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 
10 #pragma once
11 
12 #include <assert.h>
13 #include <cinttypes>
14 
15 #include <list>
16 #include <string>
17 #include <unordered_map>
18 
19 #include "rocksdb/comparator.h"
20 #include "table/block_based/block_based_table_factory.h"
21 #include "table/block_based/block_builder.h"
22 #include "table/format.h"
23 
24 namespace ROCKSDB_NAMESPACE {
25 // The interface for building index.
26 // Instruction for adding a new concrete IndexBuilder:
27 //  1. Create a subclass instantiated from IndexBuilder.
28 //  2. Add a new entry associated with that subclass in TableOptions::IndexType.
29 //  3. Add a create function for the new subclass in CreateIndexBuilder.
30 // Note: we can devise more advanced design to simplify the process for adding
31 // new subclass, which will, on the other hand, increase the code complexity and
32 // catch unwanted attention from readers. Given that we won't add/change
33 // indexes frequently, it makes sense to just embrace a more straightforward
34 // design that just works.
35 class IndexBuilder {
36  public:
37   static IndexBuilder* CreateIndexBuilder(
38       BlockBasedTableOptions::IndexType index_type,
39       const ROCKSDB_NAMESPACE::InternalKeyComparator* comparator,
40       const InternalKeySliceTransform* int_key_slice_transform,
41       const bool use_value_delta_encoding,
42       const BlockBasedTableOptions& table_opt);
43 
44   // Index builder will construct a set of blocks which contain:
45   //  1. One primary index block.
46   //  2. (Optional) a set of metablocks that contains the metadata of the
47   //     primary index.
48   struct IndexBlocks {
49     Slice index_block_contents;
50     std::unordered_map<std::string, Slice> meta_blocks;
51   };
IndexBuilder(const InternalKeyComparator * comparator)52   explicit IndexBuilder(const InternalKeyComparator* comparator)
53       : comparator_(comparator) {}
54 
~IndexBuilder()55   virtual ~IndexBuilder() {}
56 
57   // Add a new index entry to index block.
58   // To allow further optimization, we provide `last_key_in_current_block` and
59   // `first_key_in_next_block`, based on which the specific implementation can
60   // determine the best index key to be used for the index block.
61   // Called before the OnKeyAdded() call for first_key_in_next_block.
62   // @last_key_in_current_block: this parameter maybe overridden with the value
63   //                             "substitute key".
64   // @first_key_in_next_block: it will be nullptr if the entry being added is
65   //                           the last one in the table
66   //
67   // REQUIRES: Finish() has not yet been called.
68   virtual void AddIndexEntry(std::string* last_key_in_current_block,
69                              const Slice* first_key_in_next_block,
70                              const BlockHandle& block_handle) = 0;
71 
72   // This method will be called whenever a key is added. The subclasses may
73   // override OnKeyAdded() if they need to collect additional information.
OnKeyAdded(const Slice &)74   virtual void OnKeyAdded(const Slice& /*key*/) {}
75 
76   // Inform the index builder that all entries has been written. Block builder
77   // may therefore perform any operation required for block finalization.
78   //
79   // REQUIRES: Finish() has not yet been called.
Finish(IndexBlocks * index_blocks)80   inline Status Finish(IndexBlocks* index_blocks) {
81     // Throw away the changes to last_partition_block_handle. It has no effect
82     // on the first call to Finish anyway.
83     BlockHandle last_partition_block_handle;
84     return Finish(index_blocks, last_partition_block_handle);
85   }
86 
87   // This override of Finish can be utilized to build the 2nd level index in
88   // PartitionIndexBuilder.
89   //
90   // index_blocks will be filled with the resulting index data. If the return
91   // value is Status::InComplete() then it means that the index is partitioned
92   // and the callee should keep calling Finish until Status::OK() is returned.
93   // In that case, last_partition_block_handle is pointer to the block written
94   // with the result of the last call to Finish. This can be utilized to build
95   // the second level index pointing to each block of partitioned indexes. The
96   // last call to Finish() that returns Status::OK() populates index_blocks with
97   // the 2nd level index content.
98   virtual Status Finish(IndexBlocks* index_blocks,
99                         const BlockHandle& last_partition_block_handle) = 0;
100 
101   // Get the size for index block. Must be called after ::Finish.
102   virtual size_t IndexSize() const = 0;
103 
seperator_is_key_plus_seq()104   virtual bool seperator_is_key_plus_seq() { return true; }
105 
106  protected:
107   const InternalKeyComparator* comparator_;
108   // Set after ::Finish is called
109   size_t index_size_ = 0;
110 };
111 
112 // This index builder builds space-efficient index block.
113 //
114 // Optimizations:
115 //  1. Made block's `block_restart_interval` to be 1, which will avoid linear
116 //     search when doing index lookup (can be disabled by setting
117 //     index_block_restart_interval).
118 //  2. Shorten the key length for index block. Other than honestly using the
119 //     last key in the data block as the index key, we instead find a shortest
120 //     substitute key that serves the same function.
121 class ShortenedIndexBuilder : public IndexBuilder {
122  public:
ShortenedIndexBuilder(const InternalKeyComparator * comparator,const int index_block_restart_interval,const uint32_t format_version,const bool use_value_delta_encoding,BlockBasedTableOptions::IndexShorteningMode shortening_mode,bool include_first_key)123   explicit ShortenedIndexBuilder(
124       const InternalKeyComparator* comparator,
125       const int index_block_restart_interval, const uint32_t format_version,
126       const bool use_value_delta_encoding,
127       BlockBasedTableOptions::IndexShorteningMode shortening_mode,
128       bool include_first_key)
129       : IndexBuilder(comparator),
130         index_block_builder_(index_block_restart_interval,
131                              true /*use_delta_encoding*/,
132                              use_value_delta_encoding),
133         index_block_builder_without_seq_(index_block_restart_interval,
134                                          true /*use_delta_encoding*/,
135                                          use_value_delta_encoding),
136         use_value_delta_encoding_(use_value_delta_encoding),
137         include_first_key_(include_first_key),
138         shortening_mode_(shortening_mode) {
139     // Making the default true will disable the feature for old versions
140     seperator_is_key_plus_seq_ = (format_version <= 2);
141   }
142 
OnKeyAdded(const Slice & key)143   virtual void OnKeyAdded(const Slice& key) override {
144     if (include_first_key_ && current_block_first_internal_key_.empty()) {
145       current_block_first_internal_key_.assign(key.data(), key.size());
146     }
147   }
148 
AddIndexEntry(std::string * last_key_in_current_block,const Slice * first_key_in_next_block,const BlockHandle & block_handle)149   virtual void AddIndexEntry(std::string* last_key_in_current_block,
150                              const Slice* first_key_in_next_block,
151                              const BlockHandle& block_handle) override {
152     if (first_key_in_next_block != nullptr) {
153       if (shortening_mode_ !=
154           BlockBasedTableOptions::IndexShorteningMode::kNoShortening) {
155         comparator_->FindShortestSeparator(last_key_in_current_block,
156                                            *first_key_in_next_block);
157       }
158       if (!seperator_is_key_plus_seq_ &&
159           comparator_->user_comparator()->Compare(
160               ExtractUserKey(*last_key_in_current_block),
161               ExtractUserKey(*first_key_in_next_block)) == 0) {
162         seperator_is_key_plus_seq_ = true;
163       }
164     } else {
165       if (shortening_mode_ == BlockBasedTableOptions::IndexShorteningMode::
166                                   kShortenSeparatorsAndSuccessor) {
167         comparator_->FindShortSuccessor(last_key_in_current_block);
168       }
169     }
170     auto sep = Slice(*last_key_in_current_block);
171 
172     assert(!include_first_key_ || !current_block_first_internal_key_.empty());
173     IndexValue entry(block_handle, current_block_first_internal_key_);
174     std::string encoded_entry;
175     std::string delta_encoded_entry;
176     entry.EncodeTo(&encoded_entry, include_first_key_, nullptr);
177     if (use_value_delta_encoding_ && !last_encoded_handle_.IsNull()) {
178       entry.EncodeTo(&delta_encoded_entry, include_first_key_,
179                      &last_encoded_handle_);
180     } else {
181       // If it's the first block, or delta encoding is disabled,
182       // BlockBuilder::Add() below won't use delta-encoded slice.
183     }
184     last_encoded_handle_ = block_handle;
185     const Slice delta_encoded_entry_slice(delta_encoded_entry);
186     index_block_builder_.Add(sep, encoded_entry, &delta_encoded_entry_slice);
187     if (!seperator_is_key_plus_seq_) {
188       index_block_builder_without_seq_.Add(ExtractUserKey(sep), encoded_entry,
189                                            &delta_encoded_entry_slice);
190     }
191 
192     current_block_first_internal_key_.clear();
193   }
194 
195   using IndexBuilder::Finish;
Finish(IndexBlocks * index_blocks,const BlockHandle &)196   virtual Status Finish(
197       IndexBlocks* index_blocks,
198       const BlockHandle& /*last_partition_block_handle*/) override {
199     if (seperator_is_key_plus_seq_) {
200       index_blocks->index_block_contents = index_block_builder_.Finish();
201     } else {
202       index_blocks->index_block_contents =
203           index_block_builder_without_seq_.Finish();
204     }
205     index_size_ = index_blocks->index_block_contents.size();
206     return Status::OK();
207   }
208 
IndexSize()209   virtual size_t IndexSize() const override { return index_size_; }
210 
seperator_is_key_plus_seq()211   virtual bool seperator_is_key_plus_seq() override {
212     return seperator_is_key_plus_seq_;
213   }
214 
215   friend class PartitionedIndexBuilder;
216 
217  private:
218   BlockBuilder index_block_builder_;
219   BlockBuilder index_block_builder_without_seq_;
220   const bool use_value_delta_encoding_;
221   bool seperator_is_key_plus_seq_;
222   const bool include_first_key_;
223   BlockBasedTableOptions::IndexShorteningMode shortening_mode_;
224   BlockHandle last_encoded_handle_ = BlockHandle::NullBlockHandle();
225   std::string current_block_first_internal_key_;
226 };
227 
228 // HashIndexBuilder contains a binary-searchable primary index and the
229 // metadata for secondary hash index construction.
230 // The metadata for hash index consists two parts:
231 //  - a metablock that compactly contains a sequence of prefixes. All prefixes
232 //    are stored consectively without any metadata (like, prefix sizes) being
233 //    stored, which is kept in the other metablock.
234 //  - a metablock contains the metadata of the prefixes, including prefix size,
235 //    restart index and number of block it spans. The format looks like:
236 //
237 // +-----------------+---------------------------+---------------------+
238 // <=prefix 1
239 // | length: 4 bytes | restart interval: 4 bytes | num-blocks: 4 bytes |
240 // +-----------------+---------------------------+---------------------+
241 // <=prefix 2
242 // | length: 4 bytes | restart interval: 4 bytes | num-blocks: 4 bytes |
243 // +-----------------+---------------------------+---------------------+
244 // |                                                                   |
245 // | ....                                                              |
246 // |                                                                   |
247 // +-----------------+---------------------------+---------------------+
248 // <=prefix n
249 // | length: 4 bytes | restart interval: 4 bytes | num-blocks: 4 bytes |
250 // +-----------------+---------------------------+---------------------+
251 //
252 // The reason of separating these two metablocks is to enable the efficiently
253 // reuse the first metablock during hash index construction without unnecessary
254 // data copy or small heap allocations for prefixes.
255 class HashIndexBuilder : public IndexBuilder {
256  public:
HashIndexBuilder(const InternalKeyComparator * comparator,const SliceTransform * hash_key_extractor,int index_block_restart_interval,int format_version,bool use_value_delta_encoding,BlockBasedTableOptions::IndexShorteningMode shortening_mode)257   explicit HashIndexBuilder(
258       const InternalKeyComparator* comparator,
259       const SliceTransform* hash_key_extractor,
260       int index_block_restart_interval, int format_version,
261       bool use_value_delta_encoding,
262       BlockBasedTableOptions::IndexShorteningMode shortening_mode)
263       : IndexBuilder(comparator),
264         primary_index_builder_(comparator, index_block_restart_interval,
265                                format_version, use_value_delta_encoding,
266                                shortening_mode, /* include_first_key */ false),
267         hash_key_extractor_(hash_key_extractor) {}
268 
AddIndexEntry(std::string * last_key_in_current_block,const Slice * first_key_in_next_block,const BlockHandle & block_handle)269   virtual void AddIndexEntry(std::string* last_key_in_current_block,
270                              const Slice* first_key_in_next_block,
271                              const BlockHandle& block_handle) override {
272     ++current_restart_index_;
273     primary_index_builder_.AddIndexEntry(last_key_in_current_block,
274                                          first_key_in_next_block, block_handle);
275   }
276 
OnKeyAdded(const Slice & key)277   virtual void OnKeyAdded(const Slice& key) override {
278     auto key_prefix = hash_key_extractor_->Transform(key);
279     bool is_first_entry = pending_block_num_ == 0;
280 
281     // Keys may share the prefix
282     if (is_first_entry || pending_entry_prefix_ != key_prefix) {
283       if (!is_first_entry) {
284         FlushPendingPrefix();
285       }
286 
287       // need a hard copy otherwise the underlying data changes all the time.
288       // TODO(kailiu) ToString() is expensive. We may speed up can avoid data
289       // copy.
290       pending_entry_prefix_ = key_prefix.ToString();
291       pending_block_num_ = 1;
292       pending_entry_index_ = static_cast<uint32_t>(current_restart_index_);
293     } else {
294       // entry number increments when keys share the prefix reside in
295       // different data blocks.
296       auto last_restart_index = pending_entry_index_ + pending_block_num_ - 1;
297       assert(last_restart_index <= current_restart_index_);
298       if (last_restart_index != current_restart_index_) {
299         ++pending_block_num_;
300       }
301     }
302   }
303 
Finish(IndexBlocks * index_blocks,const BlockHandle & last_partition_block_handle)304   virtual Status Finish(
305       IndexBlocks* index_blocks,
306       const BlockHandle& last_partition_block_handle) override {
307     if (pending_block_num_ != 0) {
308       FlushPendingPrefix();
309     }
310     primary_index_builder_.Finish(index_blocks, last_partition_block_handle);
311     index_blocks->meta_blocks.insert(
312         {kHashIndexPrefixesBlock.c_str(), prefix_block_});
313     index_blocks->meta_blocks.insert(
314         {kHashIndexPrefixesMetadataBlock.c_str(), prefix_meta_block_});
315     return Status::OK();
316   }
317 
IndexSize()318   virtual size_t IndexSize() const override {
319     return primary_index_builder_.IndexSize() + prefix_block_.size() +
320            prefix_meta_block_.size();
321   }
322 
seperator_is_key_plus_seq()323   virtual bool seperator_is_key_plus_seq() override {
324     return primary_index_builder_.seperator_is_key_plus_seq();
325   }
326 
327  private:
FlushPendingPrefix()328   void FlushPendingPrefix() {
329     prefix_block_.append(pending_entry_prefix_.data(),
330                          pending_entry_prefix_.size());
331     PutVarint32Varint32Varint32(
332         &prefix_meta_block_,
333         static_cast<uint32_t>(pending_entry_prefix_.size()),
334         pending_entry_index_, pending_block_num_);
335   }
336 
337   ShortenedIndexBuilder primary_index_builder_;
338   const SliceTransform* hash_key_extractor_;
339 
340   // stores a sequence of prefixes
341   std::string prefix_block_;
342   // stores the metadata of prefixes
343   std::string prefix_meta_block_;
344 
345   // The following 3 variables keeps unflushed prefix and its metadata.
346   // The details of block_num and entry_index can be found in
347   // "block_hash_index.{h,cc}"
348   uint32_t pending_block_num_ = 0;
349   uint32_t pending_entry_index_ = 0;
350   std::string pending_entry_prefix_;
351 
352   uint64_t current_restart_index_ = 0;
353 };
354 
355 /**
356  * IndexBuilder for two-level indexing. Internally it creates a new index for
357  * each partition and Finish then in order when Finish is called on it
358  * continiously until Status::OK() is returned.
359  *
360  * The format on the disk would be I I I I I I IP where I is block containing a
361  * partition of indexes built using ShortenedIndexBuilder and IP is a block
362  * containing a secondary index on the partitions, built using
363  * ShortenedIndexBuilder.
364  */
365 class PartitionedIndexBuilder : public IndexBuilder {
366  public:
367   static PartitionedIndexBuilder* CreateIndexBuilder(
368       const ROCKSDB_NAMESPACE::InternalKeyComparator* comparator,
369       const bool use_value_delta_encoding,
370       const BlockBasedTableOptions& table_opt);
371 
372   explicit PartitionedIndexBuilder(const InternalKeyComparator* comparator,
373                                    const BlockBasedTableOptions& table_opt,
374                                    const bool use_value_delta_encoding);
375 
376   virtual ~PartitionedIndexBuilder();
377 
378   virtual void AddIndexEntry(std::string* last_key_in_current_block,
379                              const Slice* first_key_in_next_block,
380                              const BlockHandle& block_handle) override;
381 
382   virtual Status Finish(
383       IndexBlocks* index_blocks,
384       const BlockHandle& last_partition_block_handle) override;
385 
IndexSize()386   virtual size_t IndexSize() const override { return index_size_; }
TopLevelIndexSize(uint64_t)387   size_t TopLevelIndexSize(uint64_t) const { return top_level_index_size_; }
388   size_t NumPartitions() const;
389 
ShouldCutFilterBlock()390   inline bool ShouldCutFilterBlock() {
391     // Current policy is to align the partitions of index and filters
392     if (cut_filter_block) {
393       cut_filter_block = false;
394       return true;
395     }
396     return false;
397   }
398 
GetPartitionKey()399   std::string& GetPartitionKey() { return sub_index_last_key_; }
400 
401   // Called when an external entity (such as filter partition builder) request
402   // cutting the next partition
403   void RequestPartitionCut();
404 
seperator_is_key_plus_seq()405   virtual bool seperator_is_key_plus_seq() override {
406     return seperator_is_key_plus_seq_;
407   }
408 
get_use_value_delta_encoding()409   bool get_use_value_delta_encoding() { return use_value_delta_encoding_; }
410 
411  private:
412   // Set after ::Finish is called
413   size_t top_level_index_size_ = 0;
414   // Set after ::Finish is called
415   size_t partition_cnt_ = 0;
416 
417   void MakeNewSubIndexBuilder();
418 
419   struct Entry {
420     std::string key;
421     std::unique_ptr<ShortenedIndexBuilder> value;
422   };
423   std::list<Entry> entries_;  // list of partitioned indexes and their keys
424   BlockBuilder index_block_builder_;              // top-level index builder
425   BlockBuilder index_block_builder_without_seq_;  // same for user keys
426   // the active partition index builder
427   ShortenedIndexBuilder* sub_index_builder_;
428   // the last key in the active partition index builder
429   std::string sub_index_last_key_;
430   std::unique_ptr<FlushBlockPolicy> flush_policy_;
431   // true if Finish is called once but not complete yet.
432   bool finishing_indexes = false;
433   const BlockBasedTableOptions& table_opt_;
434   bool seperator_is_key_plus_seq_;
435   bool use_value_delta_encoding_;
436   // true if an external entity (such as filter partition builder) request
437   // cutting the next partition
438   bool partition_cut_requested_ = true;
439   // true if it should cut the next filter partition block
440   bool cut_filter_block = false;
441   BlockHandle last_encoded_handle_;
442 };
443 }  // namespace ROCKSDB_NAMESPACE
444