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 //
6 // Currently we support two types of tables: plain table and block-based table.
7 //   1. Block-based table: this is the default table type that we inherited from
8 //      LevelDB, which was designed for storing data in hard disk or flash
9 //      device.
10 //   2. Plain table: it is one of RocksDB's SST file format optimized
11 //      for low query latency on pure-memory or really low-latency media.
12 //
13 // A tutorial of rocksdb table formats is available here:
14 //   https://github.com/facebook/rocksdb/wiki/A-Tutorial-of-RocksDB-SST-formats
15 //
16 // Example code is also available
17 //   https://github.com/facebook/rocksdb/wiki/A-Tutorial-of-RocksDB-SST-formats#wiki-examples
18 
19 #pragma once
20 
21 #include <memory>
22 #include <string>
23 #include <unordered_map>
24 
25 #include "rocksdb/cache.h"
26 #include "rocksdb/env.h"
27 #include "rocksdb/iterator.h"
28 #include "rocksdb/options.h"
29 #include "rocksdb/status.h"
30 
31 namespace ROCKSDB_NAMESPACE {
32 
33 // -- Block-based Table
34 class FlushBlockPolicyFactory;
35 class PersistentCache;
36 class RandomAccessFile;
37 struct TableReaderOptions;
38 struct TableBuilderOptions;
39 class TableBuilder;
40 class TableReader;
41 class WritableFileWriter;
42 struct EnvOptions;
43 struct Options;
44 
45 enum ChecksumType : char {
46   kNoChecksum = 0x0,
47   kCRC32c = 0x1,
48   kxxHash = 0x2,
49   kxxHash64 = 0x3,
50 };
51 
52 // For advanced user only
53 struct BlockBasedTableOptions {
54   // @flush_block_policy_factory creates the instances of flush block policy.
55   // which provides a configurable way to determine when to flush a block in
56   // the block based tables.  If not set, table builder will use the default
57   // block flush policy, which cut blocks by block size (please refer to
58   // `FlushBlockBySizePolicy`).
59   std::shared_ptr<FlushBlockPolicyFactory> flush_block_policy_factory;
60 
61   // TODO(kailiu) Temporarily disable this feature by making the default value
62   // to be false.
63   //
64   // TODO(ajkr) we need to update names of variables controlling meta-block
65   // caching as they should now apply to range tombstone and compression
66   // dictionary meta-blocks, in addition to index and filter meta-blocks.
67   //
68   // Indicating if we'd put index/filter blocks to the block cache.
69   // If not specified, each "table reader" object will pre-load index/filter
70   // block during table initialization.
71   bool cache_index_and_filter_blocks = false;
72 
73   // If cache_index_and_filter_blocks is enabled, cache index and filter
74   // blocks with high priority. If set to true, depending on implementation of
75   // block cache, index and filter blocks may be less likely to be evicted
76   // than data blocks.
77   bool cache_index_and_filter_blocks_with_high_priority = true;
78 
79   // if cache_index_and_filter_blocks is true and the below is true, then
80   // filter and index blocks are stored in the cache, but a reference is
81   // held in the "table reader" object so the blocks are pinned and only
82   // evicted from cache when the table reader is freed.
83   bool pin_l0_filter_and_index_blocks_in_cache = false;
84 
85   // If cache_index_and_filter_blocks is true and the below is true, then
86   // the top-level index of partitioned filter and index blocks are stored in
87   // the cache, but a reference is held in the "table reader" object so the
88   // blocks are pinned and only evicted from cache when the table reader is
89   // freed. This is not limited to l0 in LSM tree.
90   bool pin_top_level_index_and_filter = true;
91 
92   // The index type that will be used for this table.
93   enum IndexType : char {
94     // A space efficient index block that is optimized for
95     // binary-search-based index.
96     kBinarySearch = 0x00,
97 
98     // The hash index, if enabled, will do the hash lookup when
99     // `Options.prefix_extractor` is provided.
100     kHashSearch = 0x01,
101 
102     // A two-level index implementation. Both levels are binary search indexes.
103     kTwoLevelIndexSearch = 0x02,
104 
105     // Like kBinarySearch, but index also contains first key of each block.
106     // This allows iterators to defer reading the block until it's actually
107     // needed. May significantly reduce read amplification of short range scans.
108     // Without it, iterator seek usually reads one block from each level-0 file
109     // and from each level, which may be expensive.
110     // Works best in combination with:
111     //  - IndexShorteningMode::kNoShortening,
112     //  - custom FlushBlockPolicy to cut blocks at some meaningful boundaries,
113     //    e.g. when prefix changes.
114     // Makes the index significantly bigger (2x or more), especially when keys
115     // are long.
116     //
117     // IO errors are not handled correctly in this mode right now: if an error
118     // happens when lazily reading a block in value(), value() returns empty
119     // slice, and you need to call Valid()/status() afterwards.
120     // TODO(kolmike): Fix it.
121     kBinarySearchWithFirstKey = 0x03,
122   };
123 
124   IndexType index_type = kBinarySearch;
125 
126   // The index type that will be used for the data block.
127   enum DataBlockIndexType : char {
128     kDataBlockBinarySearch = 0,   // traditional block type
129     kDataBlockBinaryAndHash = 1,  // additional hash index
130   };
131 
132   DataBlockIndexType data_block_index_type = kDataBlockBinarySearch;
133 
134   // #entries/#buckets. It is valid only when data_block_hash_index_type is
135   // kDataBlockBinaryAndHash.
136   double data_block_hash_table_util_ratio = 0.75;
137 
138   // This option is now deprecated. No matter what value it is set to,
139   // it will behave as if hash_index_allow_collision=true.
140   bool hash_index_allow_collision = true;
141 
142   // Use the specified checksum type. Newly created table files will be
143   // protected with this checksum type. Old table files will still be readable,
144   // even though they have different checksum type.
145   ChecksumType checksum = kCRC32c;
146 
147   // Disable block cache. If this is set to true,
148   // then no block cache should be used, and the block_cache should
149   // point to a nullptr object.
150   bool no_block_cache = false;
151 
152   // If non-NULL use the specified cache for blocks.
153   // If NULL, rocksdb will automatically create and use an 8MB internal cache.
154   std::shared_ptr<Cache> block_cache = nullptr;
155 
156   // If non-NULL use the specified cache for pages read from device
157   // IF NULL, no page cache is used
158   std::shared_ptr<PersistentCache> persistent_cache = nullptr;
159 
160   // If non-NULL use the specified cache for compressed blocks.
161   // If NULL, rocksdb will not use a compressed block cache.
162   // Note: though it looks similar to `block_cache`, RocksDB doesn't put the
163   //       same type of object there.
164   std::shared_ptr<Cache> block_cache_compressed = nullptr;
165 
166   // Approximate size of user data packed per block.  Note that the
167   // block size specified here corresponds to uncompressed data.  The
168   // actual size of the unit read from disk may be smaller if
169   // compression is enabled.  This parameter can be changed dynamically.
170   size_t block_size = 4 * 1024;
171 
172   // This is used to close a block before it reaches the configured
173   // 'block_size'. If the percentage of free space in the current block is less
174   // than this specified number and adding a new record to the block will
175   // exceed the configured block size, then this block will be closed and the
176   // new record will be written to the next block.
177   int block_size_deviation = 10;
178 
179   // Number of keys between restart points for delta encoding of keys.
180   // This parameter can be changed dynamically.  Most clients should
181   // leave this parameter alone.  The minimum value allowed is 1.  Any smaller
182   // value will be silently overwritten with 1.
183   int block_restart_interval = 16;
184 
185   // Same as block_restart_interval but used for the index block.
186   int index_block_restart_interval = 1;
187 
188   // Block size for partitioned metadata. Currently applied to indexes when
189   // kTwoLevelIndexSearch is used and to filters when partition_filters is used.
190   // Note: Since in the current implementation the filters and index partitions
191   // are aligned, an index/filter block is created when either index or filter
192   // block size reaches the specified limit.
193   // Note: this limit is currently applied to only index blocks; a filter
194   // partition is cut right after an index block is cut
195   // TODO(myabandeh): remove the note above when filter partitions are cut
196   // separately
197   uint64_t metadata_block_size = 4096;
198 
199   // Note: currently this option requires kTwoLevelIndexSearch to be set as
200   // well.
201   // TODO(myabandeh): remove the note above once the limitation is lifted
202   // Use partitioned full filters for each SST file. This option is
203   // incompatible with block-based filters.
204   bool partition_filters = false;
205 
206   // Use delta encoding to compress keys in blocks.
207   // ReadOptions::pin_data requires this option to be disabled.
208   //
209   // Default: true
210   bool use_delta_encoding = true;
211 
212   // If non-nullptr, use the specified filter policy to reduce disk reads.
213   // Many applications will benefit from passing the result of
214   // NewBloomFilterPolicy() here.
215   std::shared_ptr<const FilterPolicy> filter_policy = nullptr;
216 
217   // If true, place whole keys in the filter (not just prefixes).
218   // This must generally be true for gets to be efficient.
219   bool whole_key_filtering = true;
220 
221   // Verify that decompressing the compressed block gives back the input. This
222   // is a verification mode that we use to detect bugs in compression
223   // algorithms.
224   bool verify_compression = false;
225 
226   // If used, For every data block we load into memory, we will create a bitmap
227   // of size ((block_size / `read_amp_bytes_per_bit`) / 8) bytes. This bitmap
228   // will be used to figure out the percentage we actually read of the blocks.
229   //
230   // When this feature is used Tickers::READ_AMP_ESTIMATE_USEFUL_BYTES and
231   // Tickers::READ_AMP_TOTAL_READ_BYTES can be used to calculate the
232   // read amplification using this formula
233   // (READ_AMP_TOTAL_READ_BYTES / READ_AMP_ESTIMATE_USEFUL_BYTES)
234   //
235   // value  =>  memory usage (percentage of loaded blocks memory)
236   // 1      =>  12.50 %
237   // 2      =>  06.25 %
238   // 4      =>  03.12 %
239   // 8      =>  01.56 %
240   // 16     =>  00.78 %
241   //
242   // Note: This number must be a power of 2, if not it will be sanitized
243   // to be the next lowest power of 2, for example a value of 7 will be
244   // treated as 4, a value of 19 will be treated as 16.
245   //
246   // Default: 0 (disabled)
247   uint32_t read_amp_bytes_per_bit = 0;
248 
249   // We currently have five versions:
250   // 0 -- This version is currently written out by all RocksDB's versions by
251   // default.  Can be read by really old RocksDB's. Doesn't support changing
252   // checksum (default is CRC32).
253   // 1 -- Can be read by RocksDB's versions since 3.0. Supports non-default
254   // checksum, like xxHash. It is written by RocksDB when
255   // BlockBasedTableOptions::checksum is something other than kCRC32c. (version
256   // 0 is silently upconverted)
257   // 2 -- Can be read by RocksDB's versions since 3.10. Changes the way we
258   // encode compressed blocks with LZ4, BZip2 and Zlib compression. If you
259   // don't plan to run RocksDB before version 3.10, you should probably use
260   // this.
261   // 3 -- Can be read by RocksDB's versions since 5.15. Changes the way we
262   // encode the keys in index blocks. If you don't plan to run RocksDB before
263   // version 5.15, you should probably use this.
264   // This option only affects newly written tables. When reading existing
265   // tables, the information about version is read from the footer.
266   // 4 -- Can be read by RocksDB's versions since 5.16. Changes the way we
267   // encode the values in index blocks. If you don't plan to run RocksDB before
268   // version 5.16 and you are using index_block_restart_interval > 1, you should
269   // probably use this as it would reduce the index size.
270   // This option only affects newly written tables. When reading existing
271   // tables, the information about version is read from the footer.
272   // 5 -- Can be read by RocksDB's versions since 6.6.0. Full and partitioned
273   // filters use a generally faster and more accurate Bloom filter
274   // implementation, with a different schema.
275   uint32_t format_version = 4;
276 
277   // Store index blocks on disk in compressed format. Changing this option to
278   // false  will avoid the overhead of decompression if index blocks are evicted
279   // and read back
280   bool enable_index_compression = true;
281 
282   // Align data blocks on lesser of page size and block size
283   bool block_align = false;
284 
285   // This enum allows trading off increased index size for improved iterator
286   // seek performance in some situations, particularly when block cache is
287   // disabled (ReadOptions::fill_cache = false) and direct IO is
288   // enabled (DBOptions::use_direct_reads = true).
289   // The default mode is the best tradeoff for most use cases.
290   // This option only affects newly written tables.
291   //
292   // The index contains a key separating each pair of consecutive blocks.
293   // Let A be the highest key in one block, B the lowest key in the next block,
294   // and I the index entry separating these two blocks:
295   // [ ... A] I [B ...]
296   // I is allowed to be anywhere in [A, B).
297   // If an iterator is seeked to a key in (A, I], we'll unnecessarily read the
298   // first block, then immediately fall through to the second block.
299   // However, if I=A, this can't happen, and we'll read only the second block.
300   // In kNoShortening mode, we use I=A. In other modes, we use the shortest
301   // key in [A, B), which usually significantly reduces index size.
302   //
303   // There's a similar story for the last index entry, which is an upper bound
304   // of the highest key in the file. If it's shortened and therefore
305   // overestimated, iterator is likely to unnecessarily read the last data block
306   // from each file on each seek.
307   enum class IndexShorteningMode : char {
308     // Use full keys.
309     kNoShortening,
310     // Shorten index keys between blocks, but use full key for the last index
311     // key, which is the upper bound of the whole file.
312     kShortenSeparators,
313     // Shorten both keys between blocks and key after last block.
314     kShortenSeparatorsAndSuccessor,
315   };
316 
317   IndexShorteningMode index_shortening =
318       IndexShorteningMode::kShortenSeparators;
319 };
320 
321 // Table Properties that are specific to block-based table properties.
322 struct BlockBasedTablePropertyNames {
323   // value of this properties is a fixed int32 number.
324   static const std::string kIndexType;
325   // value is "1" for true and "0" for false.
326   static const std::string kWholeKeyFiltering;
327   // value is "1" for true and "0" for false.
328   static const std::string kPrefixFiltering;
329 };
330 
331 // Create default block based table factory.
332 extern TableFactory* NewBlockBasedTableFactory(
333     const BlockBasedTableOptions& table_options = BlockBasedTableOptions());
334 
335 #ifndef ROCKSDB_LITE
336 
337 enum EncodingType : char {
338   // Always write full keys without any special encoding.
339   kPlain,
340   // Find opportunity to write the same prefix once for multiple rows.
341   // In some cases, when a key follows a previous key with the same prefix,
342   // instead of writing out the full key, it just writes out the size of the
343   // shared prefix, as well as other bytes, to save some bytes.
344   //
345   // When using this option, the user is required to use the same prefix
346   // extractor to make sure the same prefix will be extracted from the same key.
347   // The Name() value of the prefix extractor will be stored in the file. When
348   // reopening the file, the name of the options.prefix_extractor given will be
349   // bitwise compared to the prefix extractors stored in the file. An error
350   // will be returned if the two don't match.
351   kPrefix,
352 };
353 
354 // Table Properties that are specific to plain table properties.
355 struct PlainTablePropertyNames {
356   static const std::string kEncodingType;
357   static const std::string kBloomVersion;
358   static const std::string kNumBloomBlocks;
359 };
360 
361 const uint32_t kPlainTableVariableLength = 0;
362 
363 struct PlainTableOptions {
364   // @user_key_len: plain table has optimization for fix-sized keys, which can
365   //                be specified via user_key_len.  Alternatively, you can pass
366   //                `kPlainTableVariableLength` if your keys have variable
367   //                lengths.
368   uint32_t user_key_len = kPlainTableVariableLength;
369 
370   // @bloom_bits_per_key: the number of bits used for bloom filer per prefix.
371   //                      You may disable it by passing a zero.
372   int bloom_bits_per_key = 10;
373 
374   // @hash_table_ratio: the desired utilization of the hash table used for
375   //                    prefix hashing.
376   //                    hash_table_ratio = number of prefixes / #buckets in the
377   //                    hash table
378   double hash_table_ratio = 0.75;
379 
380   // @index_sparseness: inside each prefix, need to build one index record for
381   //                    how many keys for binary search inside each hash bucket.
382   //                    For encoding type kPrefix, the value will be used when
383   //                    writing to determine an interval to rewrite the full
384   //                    key. It will also be used as a suggestion and satisfied
385   //                    when possible.
386   size_t index_sparseness = 16;
387 
388   // @huge_page_tlb_size: if <=0, allocate hash indexes and blooms from malloc.
389   //                      Otherwise from huge page TLB. The user needs to
390   //                      reserve huge pages for it to be allocated, like:
391   //                          sysctl -w vm.nr_hugepages=20
392   //                      See linux doc Documentation/vm/hugetlbpage.txt
393   size_t huge_page_tlb_size = 0;
394 
395   // @encoding_type: how to encode the keys. See enum EncodingType above for
396   //                 the choices. The value will determine how to encode keys
397   //                 when writing to a new SST file. This value will be stored
398   //                 inside the SST file which will be used when reading from
399   //                 the file, which makes it possible for users to choose
400   //                 different encoding type when reopening a DB. Files with
401   //                 different encoding types can co-exist in the same DB and
402   //                 can be read.
403   EncodingType encoding_type = kPlain;
404 
405   // @full_scan_mode: mode for reading the whole file one record by one without
406   //                  using the index.
407   bool full_scan_mode = false;
408 
409   // @store_index_in_file: compute plain table index and bloom filter during
410   //                       file building and store it in file. When reading
411   //                       file, index will be mmaped instead of recomputation.
412   bool store_index_in_file = false;
413 };
414 
415 // -- Plain Table with prefix-only seek
416 // For this factory, you need to set Options.prefix_extractor properly to make
417 // it work. Look-up will starts with prefix hash lookup for key prefix. Inside
418 // the hash bucket found, a binary search is executed for hash conflicts.
419 // Finally, a linear search is used.
420 
421 extern TableFactory* NewPlainTableFactory(
422     const PlainTableOptions& options = PlainTableOptions());
423 
424 struct CuckooTablePropertyNames {
425   // The key that is used to fill empty buckets.
426   static const std::string kEmptyKey;
427   // Fixed length of value.
428   static const std::string kValueLength;
429   // Number of hash functions used in Cuckoo Hash.
430   static const std::string kNumHashFunc;
431   // It denotes the number of buckets in a Cuckoo Block. Given a key and a
432   // particular hash function, a Cuckoo Block is a set of consecutive buckets,
433   // where starting bucket id is given by the hash function on the key. In case
434   // of a collision during inserting the key, the builder tries to insert the
435   // key in other locations of the cuckoo block before using the next hash
436   // function. This reduces cache miss during read operation in case of
437   // collision.
438   static const std::string kCuckooBlockSize;
439   // Size of the hash table. Use this number to compute the modulo of hash
440   // function. The actual number of buckets will be kMaxHashTableSize +
441   // kCuckooBlockSize - 1. The last kCuckooBlockSize-1 buckets are used to
442   // accommodate the Cuckoo Block from end of hash table, due to cache friendly
443   // implementation.
444   static const std::string kHashTableSize;
445   // Denotes if the key sorted in the file is Internal Key (if false)
446   // or User Key only (if true).
447   static const std::string kIsLastLevel;
448   // Indicate if using identity function for the first hash function.
449   static const std::string kIdentityAsFirstHash;
450   // Indicate if using module or bit and to calculate hash value
451   static const std::string kUseModuleHash;
452   // Fixed user key length
453   static const std::string kUserKeyLength;
454 };
455 
456 struct CuckooTableOptions {
457   // Determines the utilization of hash tables. Smaller values
458   // result in larger hash tables with fewer collisions.
459   double hash_table_ratio = 0.9;
460   // A property used by builder to determine the depth to go to
461   // to search for a path to displace elements in case of
462   // collision. See Builder.MakeSpaceForKey method. Higher
463   // values result in more efficient hash tables with fewer
464   // lookups but take more time to build.
465   uint32_t max_search_depth = 100;
466   // In case of collision while inserting, the builder
467   // attempts to insert in the next cuckoo_block_size
468   // locations before skipping over to the next Cuckoo hash
469   // function. This makes lookups more cache friendly in case
470   // of collisions.
471   uint32_t cuckoo_block_size = 5;
472   // If this option is enabled, user key is treated as uint64_t and its value
473   // is used as hash value directly. This option changes builder's behavior.
474   // Reader ignore this option and behave according to what specified in table
475   // property.
476   bool identity_as_first_hash = false;
477   // If this option is set to true, module is used during hash calculation.
478   // This often yields better space efficiency at the cost of performance.
479   // If this option is set to false, # of entries in table is constrained to be
480   // power of two, and bit and is used to calculate hash, which is faster in
481   // general.
482   bool use_module_hash = true;
483 };
484 
485 // Cuckoo Table Factory for SST table format using Cache Friendly Cuckoo Hashing
486 extern TableFactory* NewCuckooTableFactory(
487     const CuckooTableOptions& table_options = CuckooTableOptions());
488 
489 #endif  // ROCKSDB_LITE
490 
491 class RandomAccessFileReader;
492 
493 // A base class for table factories.
494 class TableFactory {
495  public:
~TableFactory()496   virtual ~TableFactory() {}
497 
498   // The type of the table.
499   //
500   // The client of this package should switch to a new name whenever
501   // the table format implementation changes.
502   //
503   // Names starting with "rocksdb." are reserved and should not be used
504   // by any clients of this package.
505   virtual const char* Name() const = 0;
506 
507   // Returns a Table object table that can fetch data from file specified
508   // in parameter file. It's the caller's responsibility to make sure
509   // file is in the correct format.
510   //
511   // NewTableReader() is called in three places:
512   // (1) TableCache::FindTable() calls the function when table cache miss
513   //     and cache the table object returned.
514   // (2) SstFileDumper (for SST Dump) opens the table and dump the table
515   //     contents using the iterator of the table.
516   // (3) DBImpl::IngestExternalFile() calls this function to read the contents
517   //     of the sst file it's attempting to add
518   //
519   // table_reader_options is a TableReaderOptions which contain all the
520   //    needed parameters and configuration to open the table.
521   // file is a file handler to handle the file for the table.
522   // file_size is the physical file size of the file.
523   // table_reader is the output table reader.
524   virtual Status NewTableReader(
525       const TableReaderOptions& table_reader_options,
526       std::unique_ptr<RandomAccessFileReader>&& file, uint64_t file_size,
527       std::unique_ptr<TableReader>* table_reader,
528       bool prefetch_index_and_filter_in_cache = true) const = 0;
529 
530   // Return a table builder to write to a file for this table type.
531   //
532   // It is called in several places:
533   // (1) When flushing memtable to a level-0 output file, it creates a table
534   //     builder (In DBImpl::WriteLevel0Table(), by calling BuildTable())
535   // (2) During compaction, it gets the builder for writing compaction output
536   //     files in DBImpl::OpenCompactionOutputFile().
537   // (3) When recovering from transaction logs, it creates a table builder to
538   //     write to a level-0 output file (In DBImpl::WriteLevel0TableForRecovery,
539   //     by calling BuildTable())
540   // (4) When running Repairer, it creates a table builder to convert logs to
541   //     SST files (In Repairer::ConvertLogToTable() by calling BuildTable())
542   //
543   // Multiple configured can be accessed from there, including and not limited
544   // to compression options. file is a handle of a writable file.
545   // It is the caller's responsibility to keep the file open and close the file
546   // after closing the table builder. compression_type is the compression type
547   // to use in this table.
548   virtual TableBuilder* NewTableBuilder(
549       const TableBuilderOptions& table_builder_options,
550       uint32_t column_family_id, WritableFileWriter* file) const = 0;
551 
552   // Sanitizes the specified DB Options and ColumnFamilyOptions.
553   //
554   // If the function cannot find a way to sanitize the input DB Options,
555   // a non-ok Status will be returned.
556   virtual Status SanitizeOptions(const DBOptions& db_opts,
557                                  const ColumnFamilyOptions& cf_opts) const = 0;
558 
559   // Return a string that contains printable format of table configurations.
560   // RocksDB prints configurations at DB Open().
561   virtual std::string GetPrintableTableOptions() const = 0;
562 
GetOptionString(std::string *,const std::string &)563   virtual Status GetOptionString(std::string* /*opt_string*/,
564                                  const std::string& /*delimiter*/) const {
565     return Status::NotSupported(
566         "The table factory doesn't implement GetOptionString().");
567   }
568 
569   // Returns the raw pointer of the table options that is used by this
570   // TableFactory, or nullptr if this function is not supported.
571   // Since the return value is a raw pointer, the TableFactory owns the
572   // pointer and the caller should not delete the pointer.
573   //
574   // In certain case, it is desirable to alter the underlying options when the
575   // TableFactory is not used by any open DB by casting the returned pointer
576   // to the right class.   For instance, if BlockBasedTableFactory is used,
577   // then the pointer can be casted to BlockBasedTableOptions.
578   //
579   // Note that changing the underlying TableFactory options while the
580   // TableFactory is currently used by any open DB is undefined behavior.
581   // Developers should use DB::SetOption() instead to dynamically change
582   // options while the DB is open.
GetOptions()583   virtual void* GetOptions() { return nullptr; }
584 
585   // Return is delete range supported
IsDeleteRangeSupported()586   virtual bool IsDeleteRangeSupported() const { return false; }
587 };
588 
589 #ifndef ROCKSDB_LITE
590 // Create a special table factory that can open either of the supported
591 // table formats, based on setting inside the SST files. It should be used to
592 // convert a DB from one table format to another.
593 // @table_factory_to_write: the table factory used when writing to new files.
594 // @block_based_table_factory:  block based table factory to use. If NULL, use
595 //                              a default one.
596 // @plain_table_factory: plain table factory to use. If NULL, use a default one.
597 // @cuckoo_table_factory: cuckoo table factory to use. If NULL, use a default
598 // one.
599 extern TableFactory* NewAdaptiveTableFactory(
600     std::shared_ptr<TableFactory> table_factory_to_write = nullptr,
601     std::shared_ptr<TableFactory> block_based_table_factory = nullptr,
602     std::shared_ptr<TableFactory> plain_table_factory = nullptr,
603     std::shared_ptr<TableFactory> cuckoo_table_factory = nullptr);
604 
605 #endif  // ROCKSDB_LITE
606 
607 }  // namespace ROCKSDB_NAMESPACE
608