rust_rocksdb/
db_options.rs

1// Copyright 2020 Tyler Neely
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7// http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15use std::panic::{AssertUnwindSafe, catch_unwind};
16use std::path::Path;
17use std::ptr::{NonNull, null_mut};
18use std::slice;
19use std::sync::Arc;
20
21use libc::{self, c_char, c_double, c_int, c_uchar, c_uint, c_void, size_t};
22
23use crate::cache::Cache;
24use crate::column_family::ColumnFamilyTtl;
25use crate::event_listener::{EventListener, new_event_listener};
26use crate::ffi_util::from_cstr_and_free;
27use crate::sst_file_manager::SstFileManager;
28use crate::statistics::{Histogram, HistogramData, StatsLevel};
29use crate::write_buffer_manager::WriteBufferManager;
30use crate::{
31    ColumnFamilyDescriptor, Error, SnapshotWithThreadMode,
32    compaction_filter::{self, CompactionFilterCallback, CompactionFilterFn},
33    compaction_filter_factory::{self, CompactionFilterFactory},
34    comparator::{
35        ComparatorCallback, ComparatorWithTsCallback, CompareFn, CompareTsFn, CompareWithoutTsFn,
36    },
37    db::DBAccess,
38    env::Env,
39    ffi,
40    ffi_util::{CStrLike, to_cpath},
41    merge_operator::{
42        self, MergeFn, MergeOperatorCallback, full_merge_callback, partial_merge_callback,
43    },
44    slice_transform::SliceTransform,
45    statistics::Ticker,
46};
47
48// must be Send and Sync because it will be called by RocksDB from different threads
49type LogCallbackFn = dyn Fn(LogLevel, &str) + 'static + Send + Sync;
50
51// Holds a log callback to ensure it outlives any Options and DBs that use it.
52struct LogCallback {
53    callback: Box<LogCallbackFn>,
54}
55
56/// Options that must outlive the DB, and may be shared between DBs. This is cloned and stored
57/// with every DB that is created from the options.
58#[derive(Default)]
59pub(crate) struct OptionsMustOutliveDB {
60    env: Option<Env>,
61    row_cache: Option<Cache>,
62    blob_cache: Option<Cache>,
63    block_based: Option<BlockBasedOptionsMustOutliveDB>,
64    write_buffer_manager: Option<WriteBufferManager>,
65    sst_file_manager: Option<SstFileManager>,
66    log_callback: Option<Arc<LogCallback>>,
67    comparator: Option<Arc<OwnedComparator>>,
68    compaction_filter: Option<Arc<OwnedCompactionFilter>>,
69}
70
71impl OptionsMustOutliveDB {
72    pub(crate) fn clone(&self) -> Self {
73        Self {
74            env: self.env.clone(),
75            row_cache: self.row_cache.clone(),
76            blob_cache: self.blob_cache.clone(),
77            block_based: self
78                .block_based
79                .as_ref()
80                .map(BlockBasedOptionsMustOutliveDB::clone),
81            write_buffer_manager: self.write_buffer_manager.clone(),
82            sst_file_manager: self.sst_file_manager.clone(),
83            log_callback: self.log_callback.clone(),
84            comparator: self.comparator.clone(),
85            compaction_filter: self.compaction_filter.clone(),
86        }
87    }
88}
89
90/// Stores a `rocksdb_comparator_t` and destroys it when dropped.
91///
92/// This has an unsafe implementation of Send and Sync because it wraps a RocksDB pointer that
93/// is safe to share between threads.
94struct OwnedComparator {
95    inner: NonNull<ffi::rocksdb_comparator_t>,
96}
97
98impl OwnedComparator {
99    fn new(inner: NonNull<ffi::rocksdb_comparator_t>) -> Self {
100        Self { inner }
101    }
102}
103
104impl Drop for OwnedComparator {
105    fn drop(&mut self) {
106        unsafe {
107            ffi::rocksdb_comparator_destroy(self.inner.as_ptr());
108        }
109    }
110}
111
112/// Stores a `rocksdb_compactionfilter_t` and destroys it when dropped.
113///
114/// This has an unsafe implementation of Send and Sync because it wraps a RocksDB pointer that
115/// is safe to share between threads.
116struct OwnedCompactionFilter {
117    inner: NonNull<ffi::rocksdb_compactionfilter_t>,
118}
119
120impl OwnedCompactionFilter {
121    fn new(inner: NonNull<ffi::rocksdb_compactionfilter_t>) -> Self {
122        Self { inner }
123    }
124}
125
126impl Drop for OwnedCompactionFilter {
127    fn drop(&mut self) {
128        unsafe {
129            ffi::rocksdb_compactionfilter_destroy(self.inner.as_ptr());
130        }
131    }
132}
133
134#[derive(Default)]
135struct BlockBasedOptionsMustOutliveDB {
136    block_cache: Option<Cache>,
137}
138
139impl BlockBasedOptionsMustOutliveDB {
140    fn clone(&self) -> Self {
141        Self {
142            block_cache: self.block_cache.clone(),
143        }
144    }
145}
146
147/// Database-wide options around performance and behavior.
148///
149/// Please read the official tuning [guide](https://github.com/facebook/rocksdb/wiki/RocksDB-Tuning-Guide)
150/// and most importantly, measure performance under realistic workloads with realistic hardware.
151///
152/// # Examples
153///
154/// ```
155/// use rust_rocksdb::{Options, DB};
156/// use rust_rocksdb::DBCompactionStyle;
157///
158/// fn badly_tuned_for_somebody_elses_disk() -> DB {
159///    let path = "path/for/rocksdb/storageX";
160///    let mut opts = Options::default();
161///    opts.create_if_missing(true);
162///    opts.set_max_open_files(10000);
163///    opts.set_use_fsync(false);
164///    opts.set_bytes_per_sync(8388608);
165///    opts.optimize_for_point_lookup(1024);
166///    opts.set_table_cache_num_shard_bits(6);
167///    opts.set_max_write_buffer_number(32);
168///    opts.set_write_buffer_size(536870912);
169///    opts.set_target_file_size_base(1073741824);
170///    opts.set_min_write_buffer_number_to_merge(4);
171///    opts.set_level_zero_stop_writes_trigger(2000);
172///    opts.set_level_zero_slowdown_writes_trigger(0);
173///    opts.set_compaction_style(DBCompactionStyle::Universal);
174///    opts.set_disable_auto_compactions(true);
175///
176///    DB::open(&opts, path).unwrap()
177/// }
178/// ```
179pub struct Options {
180    pub(crate) inner: *mut ffi::rocksdb_options_t,
181    pub(crate) outlive: OptionsMustOutliveDB,
182}
183
184/// Optionally disable WAL or sync for this write.
185///
186/// # Examples
187///
188/// Making an unsafe write of a batch:
189///
190/// ```
191/// use rust_rocksdb::{DB, Options, WriteBatch, WriteOptions};
192///
193/// let tempdir = tempfile::Builder::new()
194///     .prefix("_path_for_rocksdb_storageY1")
195///     .tempdir()
196///     .expect("Failed to create temporary path for the _path_for_rocksdb_storageY1");
197/// let path = tempdir.path();
198/// {
199///     let db = DB::open_default(path).unwrap();
200///     let mut batch = WriteBatch::default();
201///     batch.put(b"my key", b"my value");
202///     batch.put(b"key2", b"value2");
203///     batch.put(b"key3", b"value3");
204///
205///     let mut write_options = WriteOptions::default();
206///     write_options.set_sync(false);
207///     write_options.disable_wal(true);
208///
209///     db.write_opt(&batch, &write_options);
210/// }
211/// let _ = DB::destroy(&Options::default(), path);
212/// ```
213pub struct WriteOptions {
214    pub(crate) inner: *mut ffi::rocksdb_writeoptions_t,
215}
216
217pub struct LruCacheOptions {
218    pub(crate) inner: *mut ffi::rocksdb_lru_cache_options_t,
219}
220
221/// Optionally wait for the memtable flush to be performed.
222///
223/// # Examples
224///
225/// Manually flushing the memtable:
226///
227/// ```
228/// use rust_rocksdb::{DB, Options, FlushOptions};
229///
230/// let tempdir = tempfile::Builder::new()
231///     .prefix("_path_for_rocksdb_storageY2")
232///     .tempdir()
233///     .expect("Failed to create temporary path for the _path_for_rocksdb_storageY2");
234/// let path = tempdir.path();
235/// {
236///     let db = DB::open_default(path).unwrap();
237///
238///     let mut flush_options = FlushOptions::default();
239///     flush_options.set_wait(true);
240///
241///     db.flush_opt(&flush_options);
242/// }
243/// let _ = DB::destroy(&Options::default(), path);
244/// ```
245pub struct FlushOptions {
246    pub(crate) inner: *mut ffi::rocksdb_flushoptions_t,
247}
248
249/// For configuring block-based file storage.
250pub struct BlockBasedOptions {
251    pub(crate) inner: *mut ffi::rocksdb_block_based_table_options_t,
252    outlive: BlockBasedOptionsMustOutliveDB,
253}
254
255pub struct ReadOptions {
256    pub(crate) inner: *mut ffi::rocksdb_readoptions_t,
257    // The `ReadOptions` owns a copy of the timestamp and iteration bounds.
258    // This is necessary to ensure the pointers we pass over the FFI live as
259    // long as the `ReadOptions`. This way, when performing the read operation,
260    // the pointers are guaranteed to be valid.
261    timestamp: Option<Vec<u8>>,
262    iter_start_ts: Option<Vec<u8>>,
263    iterate_upper_bound: Option<Vec<u8>>,
264    iterate_lower_bound: Option<Vec<u8>>,
265}
266
267/// Configuration of cuckoo-based storage.
268pub struct CuckooTableOptions {
269    pub(crate) inner: *mut ffi::rocksdb_cuckoo_table_options_t,
270}
271
272/// For configuring external files ingestion.
273///
274/// # Examples
275///
276/// Move files instead of copying them:
277///
278/// ```
279/// use rust_rocksdb::{DB, IngestExternalFileOptions, SstFileWriter, Options};
280///
281/// let writer_opts = Options::default();
282/// let mut writer = SstFileWriter::create(&writer_opts);
283/// let tempdir = tempfile::Builder::new()
284///     .tempdir()
285///     .expect("Failed to create temporary folder for the _path_for_sst_file");
286/// let path1 = tempdir.path().join("_path_for_sst_file");
287/// writer.open(path1.clone()).unwrap();
288/// writer.put(b"k1", b"v1").unwrap();
289/// writer.finish().unwrap();
290///
291/// let tempdir2 = tempfile::Builder::new()
292///     .prefix("_path_for_rocksdb_storageY3")
293///     .tempdir()
294///     .expect("Failed to create temporary path for the _path_for_rocksdb_storageY3");
295/// let path2 = tempdir2.path();
296/// {
297///   let db = DB::open_default(&path2).unwrap();
298///   let mut ingest_opts = IngestExternalFileOptions::default();
299///   ingest_opts.set_move_files(true);
300///   db.ingest_external_file_opts(&ingest_opts, vec![path1]).unwrap();
301/// }
302/// let _ = DB::destroy(&Options::default(), path2);
303/// ```
304pub struct IngestExternalFileOptions {
305    pub(crate) inner: *mut ffi::rocksdb_ingestexternalfileoptions_t,
306}
307
308// Safety note: auto-implementing Send on most db-related types is prevented by the inner FFI
309// pointer. In most cases, however, this pointer is Send-safe because it is never aliased and
310// rocksdb internally does not rely on thread-local information for its user-exposed types.
311unsafe impl Send for Options {}
312unsafe impl Send for WriteOptions {}
313unsafe impl Send for LruCacheOptions {}
314unsafe impl Send for FlushOptions {}
315unsafe impl Send for BlockBasedOptions {}
316unsafe impl Send for CuckooTableOptions {}
317unsafe impl Send for ReadOptions {}
318unsafe impl Send for IngestExternalFileOptions {}
319unsafe impl Send for CompactOptions {}
320unsafe impl Send for ImportColumnFamilyOptions {}
321unsafe impl Send for OwnedComparator {}
322unsafe impl Send for OwnedCompactionFilter {}
323
324// Sync is similarly safe for many types because they do not expose interior mutability, and their
325// use within the rocksdb library is generally behind a const reference
326unsafe impl Sync for Options {}
327unsafe impl Sync for WriteOptions {}
328unsafe impl Sync for LruCacheOptions {}
329unsafe impl Sync for FlushOptions {}
330unsafe impl Sync for BlockBasedOptions {}
331unsafe impl Sync for CuckooTableOptions {}
332unsafe impl Sync for ReadOptions {}
333unsafe impl Sync for IngestExternalFileOptions {}
334unsafe impl Sync for CompactOptions {}
335unsafe impl Sync for ImportColumnFamilyOptions {}
336unsafe impl Sync for OwnedComparator {}
337unsafe impl Sync for OwnedCompactionFilter {}
338
339impl Drop for Options {
340    fn drop(&mut self) {
341        unsafe {
342            ffi::rocksdb_options_destroy(self.inner);
343        }
344    }
345}
346
347impl Clone for Options {
348    fn clone(&self) -> Self {
349        let inner = unsafe { ffi::rocksdb_options_create_copy(self.inner) };
350        assert!(!inner.is_null(), "Could not copy RocksDB options");
351
352        Self {
353            inner,
354            outlive: self.outlive.clone(),
355        }
356    }
357}
358
359impl Drop for BlockBasedOptions {
360    fn drop(&mut self) {
361        unsafe {
362            ffi::rocksdb_block_based_options_destroy(self.inner);
363        }
364    }
365}
366
367impl Drop for CuckooTableOptions {
368    fn drop(&mut self) {
369        unsafe {
370            ffi::rocksdb_cuckoo_options_destroy(self.inner);
371        }
372    }
373}
374
375impl Drop for FlushOptions {
376    fn drop(&mut self) {
377        unsafe {
378            ffi::rocksdb_flushoptions_destroy(self.inner);
379        }
380    }
381}
382
383impl Drop for WriteOptions {
384    fn drop(&mut self) {
385        unsafe {
386            ffi::rocksdb_writeoptions_destroy(self.inner);
387        }
388    }
389}
390
391impl Drop for LruCacheOptions {
392    fn drop(&mut self) {
393        unsafe {
394            ffi::rocksdb_lru_cache_options_destroy(self.inner);
395        }
396    }
397}
398
399impl Drop for ReadOptions {
400    fn drop(&mut self) {
401        unsafe {
402            ffi::rocksdb_readoptions_destroy(self.inner);
403        }
404    }
405}
406
407impl Drop for IngestExternalFileOptions {
408    fn drop(&mut self) {
409        unsafe {
410            ffi::rocksdb_ingestexternalfileoptions_destroy(self.inner);
411        }
412    }
413}
414
415impl BlockBasedOptions {
416    /// Approximate size of user data packed per block. Note that the
417    /// block size specified here corresponds to uncompressed data. The
418    /// actual size of the unit read from disk may be smaller if
419    /// compression is enabled. This parameter can be changed dynamically.
420    pub fn set_block_size(&mut self, size: usize) {
421        unsafe {
422            ffi::rocksdb_block_based_options_set_block_size(self.inner, size);
423        }
424    }
425
426    /// Block size for partitioned metadata. Currently applied to indexes when
427    /// kTwoLevelIndexSearch is used and to filters when partition_filters is used.
428    /// Note: Since in the current implementation the filters and index partitions
429    /// are aligned, an index/filter block is created when either index or filter
430    /// block size reaches the specified limit.
431    ///
432    /// Note: this limit is currently applied to only index blocks; a filter
433    /// partition is cut right after an index block is cut.
434    pub fn set_metadata_block_size(&mut self, size: usize) {
435        unsafe {
436            ffi::rocksdb_block_based_options_set_metadata_block_size(self.inner, size as u64);
437        }
438    }
439
440    /// Note: currently this option requires kTwoLevelIndexSearch to be set as
441    /// well.
442    ///
443    /// Use partitioned full filters for each SST file. This option is
444    /// incompatible with block-based filters.
445    pub fn set_partition_filters(&mut self, size: bool) {
446        unsafe {
447            ffi::rocksdb_block_based_options_set_partition_filters(self.inner, c_uchar::from(size));
448        }
449    }
450
451    /// Sets global cache for blocks (user data is stored in a set of blocks, and
452    /// a block is the unit of reading from disk).
453    ///
454    /// If set, use the specified cache for blocks.
455    /// By default, rocksdb will automatically create and use an 8MB internal cache.
456    pub fn set_block_cache(&mut self, cache: &Cache) {
457        unsafe {
458            ffi::rocksdb_block_based_options_set_block_cache(self.inner, cache.0.inner.as_ptr());
459        }
460        self.outlive.block_cache = Some(cache.clone());
461    }
462
463    /// Disable block cache
464    pub fn disable_cache(&mut self) {
465        unsafe {
466            ffi::rocksdb_block_based_options_set_no_block_cache(self.inner, c_uchar::from(true));
467        }
468    }
469
470    /// Sets a [Bloom filter](https://github.com/facebook/rocksdb/wiki/RocksDB-Bloom-Filter)
471    /// policy to reduce disk reads.
472    ///
473    /// # Examples
474    ///
475    /// ```
476    /// use rust_rocksdb::BlockBasedOptions;
477    ///
478    /// let mut opts = BlockBasedOptions::default();
479    /// opts.set_bloom_filter(10.0, true);
480    /// ```
481    pub fn set_bloom_filter(&mut self, bits_per_key: c_double, block_based: bool) {
482        unsafe {
483            let bloom = if block_based {
484                ffi::rocksdb_filterpolicy_create_bloom(bits_per_key as _)
485            } else {
486                ffi::rocksdb_filterpolicy_create_bloom_full(bits_per_key as _)
487            };
488
489            ffi::rocksdb_block_based_options_set_filter_policy(self.inner, bloom);
490        }
491    }
492
493    /// Sets a [Ribbon filter](http://rocksdb.org/blog/2021/12/29/ribbon-filter.html)
494    /// policy to reduce disk reads.
495    ///
496    /// Ribbon filters use less memory in exchange for slightly more CPU usage
497    /// compared to an equivalent bloom filter.
498    ///
499    /// # Examples
500    ///
501    /// ```
502    /// use rust_rocksdb::BlockBasedOptions;
503    ///
504    /// let mut opts = BlockBasedOptions::default();
505    /// opts.set_ribbon_filter(10.0);
506    /// ```
507    pub fn set_ribbon_filter(&mut self, bloom_equivalent_bits_per_key: c_double) {
508        unsafe {
509            let ribbon = ffi::rocksdb_filterpolicy_create_ribbon(bloom_equivalent_bits_per_key);
510            ffi::rocksdb_block_based_options_set_filter_policy(self.inner, ribbon);
511        }
512    }
513
514    /// Sets a hybrid [Ribbon filter](http://rocksdb.org/blog/2021/12/29/ribbon-filter.html)
515    /// policy to reduce disk reads.
516    ///
517    /// Uses Bloom filters before the given level, and Ribbon filters for all
518    /// other levels. This combines the memory savings from Ribbon filters
519    /// with the lower CPU usage of Bloom filters.
520    ///
521    /// # Examples
522    ///
523    /// ```
524    /// use rust_rocksdb::BlockBasedOptions;
525    ///
526    /// let mut opts = BlockBasedOptions::default();
527    /// opts.set_hybrid_ribbon_filter(10.0, 2);
528    /// ```
529    pub fn set_hybrid_ribbon_filter(
530        &mut self,
531        bloom_equivalent_bits_per_key: c_double,
532        bloom_before_level: c_int,
533    ) {
534        unsafe {
535            let ribbon = ffi::rocksdb_filterpolicy_create_ribbon_hybrid(
536                bloom_equivalent_bits_per_key,
537                bloom_before_level,
538            );
539            ffi::rocksdb_block_based_options_set_filter_policy(self.inner, ribbon);
540        }
541    }
542
543    /// Whether to put index/filter blocks in the block cache. When false,
544    /// each "table reader" object will pre-load index/filter blocks during
545    /// table initialization. Index and filter partition blocks always use
546    /// block cache regardless of this option.
547    ///
548    /// Default: false
549    pub fn set_cache_index_and_filter_blocks(&mut self, v: bool) {
550        unsafe {
551            ffi::rocksdb_block_based_options_set_cache_index_and_filter_blocks(
552                self.inner,
553                c_uchar::from(v),
554            );
555        }
556    }
557
558    /// If `cache_index_and_filter_blocks` is enabled, cache index and filter
559    /// blocks with high priority. Depending on the block cache implementation,
560    /// index, filter, and other metadata blocks may be less likely to be
561    /// evicted than data blocks when this is set to true.
562    ///
563    /// Default: true.
564    pub fn set_cache_index_and_filter_blocks_with_high_priority(&mut self, v: bool) {
565        unsafe {
566            ffi::rocksdb_block_based_options_set_cache_index_and_filter_blocks_with_high_priority(
567                self.inner,
568                c_uchar::from(v),
569            );
570        }
571    }
572
573    /// Defines the index type to be used for SS-table lookups.
574    ///
575    /// # Examples
576    ///
577    /// ```
578    /// use rust_rocksdb::{BlockBasedOptions, BlockBasedIndexType, Options};
579    ///
580    /// let mut opts = Options::default();
581    /// let mut block_opts = BlockBasedOptions::default();
582    /// block_opts.set_index_type(BlockBasedIndexType::HashSearch);
583    /// ```
584    pub fn set_index_type(&mut self, index_type: BlockBasedIndexType) {
585        let index = index_type as i32;
586        unsafe {
587            ffi::rocksdb_block_based_options_set_index_type(self.inner, index);
588        }
589    }
590
591    /// If cache_index_and_filter_blocks is true and the below is true, then
592    /// filter and index blocks are stored in the cache, but a reference is
593    /// held in the "table reader" object so the blocks are pinned and only
594    /// evicted from cache when the table reader is freed.
595    ///
596    /// Default: false.
597    pub fn set_pin_l0_filter_and_index_blocks_in_cache(&mut self, v: bool) {
598        unsafe {
599            ffi::rocksdb_block_based_options_set_pin_l0_filter_and_index_blocks_in_cache(
600                self.inner,
601                c_uchar::from(v),
602            );
603        }
604    }
605
606    /// If cache_index_and_filter_blocks is true and the below is true, then
607    /// the top-level index of partitioned filter and index blocks are stored in
608    /// the cache, but a reference is held in the "table reader" object so the
609    /// blocks are pinned and only evicted from cache when the table reader is
610    /// freed. This is not limited to l0 in LSM tree.
611    ///
612    /// Default: true.
613    pub fn set_pin_top_level_index_and_filter(&mut self, v: bool) {
614        unsafe {
615            ffi::rocksdb_block_based_options_set_pin_top_level_index_and_filter(
616                self.inner,
617                c_uchar::from(v),
618            );
619        }
620    }
621
622    /// Format version, reserved for backward compatibility.
623    ///
624    /// See full [list](https://github.com/facebook/rocksdb/blob/v8.6.7/include/rocksdb/table.h#L493-L521)
625    /// of the supported versions.
626    ///
627    /// Default: 6.
628    pub fn set_format_version(&mut self, version: i32) {
629        unsafe {
630            ffi::rocksdb_block_based_options_set_format_version(self.inner, version);
631        }
632    }
633
634    /// Use delta encoding to compress keys in blocks.
635    /// ReadOptions::pin_data requires this option to be disabled.
636    ///
637    /// Default: true
638    pub fn set_use_delta_encoding(&mut self, enable: bool) {
639        unsafe {
640            ffi::rocksdb_block_based_options_set_use_delta_encoding(
641                self.inner,
642                c_uchar::from(enable),
643            );
644        }
645    }
646
647    /// Number of keys between restart points for delta encoding of keys.
648    /// This parameter can be changed dynamically. Most clients should
649    /// leave this parameter alone. The minimum value allowed is 1. Any smaller
650    /// value will be silently overwritten with 1.
651    ///
652    /// Default: 16.
653    pub fn set_block_restart_interval(&mut self, interval: i32) {
654        unsafe {
655            ffi::rocksdb_block_based_options_set_block_restart_interval(self.inner, interval);
656        }
657    }
658
659    /// Same as block_restart_interval but used for the index block.
660    /// If you don't plan to run RocksDB before version 5.16 and you are
661    /// using `index_block_restart_interval` > 1, you should
662    /// probably set the `format_version` to >= 4 as it would reduce the index size.
663    ///
664    /// Default: 1.
665    pub fn set_index_block_restart_interval(&mut self, interval: i32) {
666        unsafe {
667            ffi::rocksdb_block_based_options_set_index_block_restart_interval(self.inner, interval);
668        }
669    }
670
671    /// Set the data block index type for point lookups:
672    ///  `DataBlockIndexType::BinarySearch` to use binary search within the data block.
673    ///  `DataBlockIndexType::BinaryAndHash` to use the data block hash index in combination with
674    ///  the normal binary search.
675    ///
676    /// The hash table utilization ratio is adjustable using [`set_data_block_hash_ratio`](#method.set_data_block_hash_ratio), which is
677    /// valid only when using `DataBlockIndexType::BinaryAndHash`.
678    ///
679    /// Default: `BinarySearch`
680    /// # Examples
681    ///
682    /// ```
683    /// use rust_rocksdb::{BlockBasedOptions, DataBlockIndexType, Options};
684    ///
685    /// let mut opts = Options::default();
686    /// let mut block_opts = BlockBasedOptions::default();
687    /// block_opts.set_data_block_index_type(DataBlockIndexType::BinaryAndHash);
688    /// block_opts.set_data_block_hash_ratio(0.85);
689    /// ```
690    pub fn set_data_block_index_type(&mut self, index_type: DataBlockIndexType) {
691        let index_t = index_type as i32;
692        unsafe {
693            ffi::rocksdb_block_based_options_set_data_block_index_type(self.inner, index_t);
694        }
695    }
696
697    /// Set the data block hash index utilization ratio.
698    ///
699    /// The smaller the utilization ratio, the less hash collisions happen, and so reduce the risk for a
700    /// point lookup to fall back to binary search due to the collisions. A small ratio means faster
701    /// lookup at the price of more space overhead.
702    ///
703    /// Default: 0.75
704    pub fn set_data_block_hash_ratio(&mut self, ratio: f64) {
705        unsafe {
706            ffi::rocksdb_block_based_options_set_data_block_hash_ratio(self.inner, ratio);
707        }
708    }
709
710    /// If false, place only prefixes in the filter, not whole keys.
711    ///
712    /// Defaults to true.
713    pub fn set_whole_key_filtering(&mut self, v: bool) {
714        unsafe {
715            ffi::rocksdb_block_based_options_set_whole_key_filtering(self.inner, c_uchar::from(v));
716        }
717    }
718
719    /// Use the specified checksum type.
720    /// Newly created table files will be protected with this checksum type.
721    /// Old table files will still be readable, even though they have different checksum type.
722    pub fn set_checksum_type(&mut self, checksum_type: ChecksumType) {
723        unsafe {
724            ffi::rocksdb_block_based_options_set_checksum(self.inner, checksum_type as c_char);
725        }
726    }
727
728    /// If true, generate Bloom/Ribbon filters that minimize memory internal
729    /// fragmentation.
730    /// See official [wiki](
731    /// https://github.com/facebook/rocksdb/wiki/RocksDB-Bloom-Filter#reducing-internal-fragmentation)
732    /// for more information.
733    ///
734    /// Default: true.
735    /// # Examples
736    ///
737    /// ```
738    /// use rust_rocksdb::BlockBasedOptions;
739    ///
740    /// let mut opts = BlockBasedOptions::default();
741    /// opts.set_bloom_filter(10.0, true);
742    /// opts.set_optimize_filters_for_memory(true);
743    /// ```
744    pub fn set_optimize_filters_for_memory(&mut self, v: bool) {
745        unsafe {
746            ffi::rocksdb_block_based_options_set_optimize_filters_for_memory(
747                self.inner,
748                c_uchar::from(v),
749            );
750        }
751    }
752
753    /// The tier of block-based tables whose top-level index into metadata
754    /// partitions will be pinned. Currently indexes and filters may be
755    /// partitioned.
756    ///
757    /// Note `cache_index_and_filter_blocks` must be true for this option to have
758    /// any effect. Otherwise any top-level index into metadata partitions would be
759    /// held in table reader memory, outside the block cache.
760    ///
761    /// Default: `BlockBasedPinningTier:Fallback`
762    ///
763    /// # Example
764    ///
765    /// ```
766    /// use rust_rocksdb::{BlockBasedOptions, BlockBasedPinningTier, Options};
767    ///
768    /// let mut opts = Options::default();
769    /// let mut block_opts = BlockBasedOptions::default();
770    /// block_opts.set_top_level_index_pinning_tier(BlockBasedPinningTier::FlushAndSimilar);
771    /// ```
772    pub fn set_top_level_index_pinning_tier(&mut self, tier: BlockBasedPinningTier) {
773        unsafe {
774            ffi::rocksdb_block_based_options_set_top_level_index_pinning_tier(
775                self.inner,
776                tier as c_int,
777            );
778        }
779    }
780
781    /// The tier of block-based tables whose metadata partitions will be pinned.
782    /// Currently indexes and filters may be partitioned.
783    ///
784    /// Default: `BlockBasedPinningTier:Fallback`
785    ///
786    /// # Example
787    ///
788    /// ```
789    /// use rust_rocksdb::{BlockBasedOptions, BlockBasedPinningTier, Options};
790    ///
791    /// let mut opts = Options::default();
792    /// let mut block_opts = BlockBasedOptions::default();
793    /// block_opts.set_partition_pinning_tier(BlockBasedPinningTier::FlushAndSimilar);
794    /// ```
795    pub fn set_partition_pinning_tier(&mut self, tier: BlockBasedPinningTier) {
796        unsafe {
797            ffi::rocksdb_block_based_options_set_partition_pinning_tier(self.inner, tier as c_int);
798        }
799    }
800
801    /// The tier of block-based tables whose unpartitioned metadata blocks will be
802    /// pinned.
803    ///
804    /// Note `cache_index_and_filter_blocks` must be true for this option to have
805    /// any effect. Otherwise the unpartitioned meta-blocks would be held in table
806    /// reader memory, outside the block cache.
807    ///
808    /// Default: `BlockBasedPinningTier:Fallback`
809    ///
810    /// # Example
811    ///
812    /// ```
813    /// use rust_rocksdb::{BlockBasedOptions, BlockBasedPinningTier, Options};
814    ///
815    /// let mut opts = Options::default();
816    /// let mut block_opts = BlockBasedOptions::default();
817    /// block_opts.set_unpartitioned_pinning_tier(BlockBasedPinningTier::FlushAndSimilar);
818    /// ```
819    pub fn set_unpartitioned_pinning_tier(&mut self, tier: BlockBasedPinningTier) {
820        unsafe {
821            ffi::rocksdb_block_based_options_set_unpartitioned_pinning_tier(
822                self.inner,
823                tier as c_int,
824            );
825        }
826    }
827}
828
829impl Default for BlockBasedOptions {
830    fn default() -> Self {
831        let block_opts = unsafe { ffi::rocksdb_block_based_options_create() };
832        assert!(
833            !block_opts.is_null(),
834            "Could not create RocksDB block based options"
835        );
836
837        Self {
838            inner: block_opts,
839            outlive: BlockBasedOptionsMustOutliveDB::default(),
840        }
841    }
842}
843
844impl CuckooTableOptions {
845    /// Determines the utilization of hash tables. Smaller values
846    /// result in larger hash tables with fewer collisions.
847    /// Default: 0.9
848    pub fn set_hash_ratio(&mut self, ratio: f64) {
849        unsafe {
850            ffi::rocksdb_cuckoo_options_set_hash_ratio(self.inner, ratio);
851        }
852    }
853
854    /// A property used by builder to determine the depth to go to
855    /// to search for a path to displace elements in case of
856    /// collision. See Builder.MakeSpaceForKey method. Higher
857    /// values result in more efficient hash tables with fewer
858    /// lookups but take more time to build.
859    /// Default: 100
860    pub fn set_max_search_depth(&mut self, depth: u32) {
861        unsafe {
862            ffi::rocksdb_cuckoo_options_set_max_search_depth(self.inner, depth);
863        }
864    }
865
866    /// In case of collision while inserting, the builder
867    /// attempts to insert in the next cuckoo_block_size
868    /// locations before skipping over to the next Cuckoo hash
869    /// function. This makes lookups more cache friendly in case
870    /// of collisions.
871    /// Default: 5
872    pub fn set_cuckoo_block_size(&mut self, size: u32) {
873        unsafe {
874            ffi::rocksdb_cuckoo_options_set_cuckoo_block_size(self.inner, size);
875        }
876    }
877
878    /// If this option is enabled, user key is treated as uint64_t and its value
879    /// is used as hash value directly. This option changes builder's behavior.
880    /// Reader ignore this option and behave according to what specified in
881    /// table property.
882    /// Default: false
883    pub fn set_identity_as_first_hash(&mut self, flag: bool) {
884        unsafe {
885            ffi::rocksdb_cuckoo_options_set_identity_as_first_hash(self.inner, c_uchar::from(flag));
886        }
887    }
888
889    /// If this option is set to true, module is used during hash calculation.
890    /// This often yields better space efficiency at the cost of performance.
891    /// If this option is set to false, # of entries in table is constrained to
892    /// be power of two, and bit and is used to calculate hash, which is faster in general.
893    /// Default: true
894    pub fn set_use_module_hash(&mut self, flag: bool) {
895        unsafe {
896            ffi::rocksdb_cuckoo_options_set_use_module_hash(self.inner, c_uchar::from(flag));
897        }
898    }
899}
900
901impl Default for CuckooTableOptions {
902    fn default() -> Self {
903        let opts = unsafe { ffi::rocksdb_cuckoo_options_create() };
904        assert!(!opts.is_null(), "Could not create RocksDB cuckoo options");
905
906        Self { inner: opts }
907    }
908}
909
910// Verbosity of the LOG.
911#[derive(Debug, Copy, Clone, PartialEq, Eq)]
912#[repr(i32)]
913pub enum LogLevel {
914    Debug = 0,
915    Info,
916    Warn,
917    Error,
918    Fatal,
919    Header,
920}
921
922impl Options {
923    /// Constructs the DBOptions and ColumnFamilyDescriptors by loading the
924    /// latest RocksDB options file stored in the specified rocksdb database.
925    ///
926    /// *IMPORTANT*:
927    /// ROCKSDB DOES NOT STORE cf ttl in the options file. If you have set it via
928    /// [`ColumnFamilyDescriptor::new_with_ttl`] then you need to set it again after loading the options file.
929    /// Tll will be set to [`ColumnFamilyTtl::Disabled`] for all column families for your safety.
930    pub fn load_latest<P: AsRef<Path>>(
931        path: P,
932        env: Env,
933        ignore_unknown_options: bool,
934        cache: Cache,
935    ) -> Result<(Options, Vec<ColumnFamilyDescriptor>), Error> {
936        let path = to_cpath(path)?;
937        let mut db_options: *mut ffi::rocksdb_options_t = null_mut();
938        let mut num_column_families: usize = 0;
939        let mut column_family_names: *mut *mut c_char = null_mut();
940        let mut column_family_options: *mut *mut ffi::rocksdb_options_t = null_mut();
941        unsafe {
942            ffi_try!(ffi::rocksdb_load_latest_options(
943                path.as_ptr(),
944                env.0.inner,
945                ignore_unknown_options,
946                cache.0.inner.as_ptr(),
947                &raw mut db_options,
948                &raw mut num_column_families,
949                &raw mut column_family_names,
950                &raw mut column_family_options,
951            ));
952        }
953        let options = Options {
954            inner: db_options,
955            outlive: OptionsMustOutliveDB::default(),
956        };
957        // read_column_descriptors frees column_family_names and the column_family_options array.
958        // We can't call rocksdb_load_latest_options_destroy because it also frees options, and
959        // the individual `column_family_options` pointers. We want to return them.
960        let column_families = unsafe {
961            Options::read_column_descriptors(
962                num_column_families,
963                column_family_names,
964                column_family_options,
965            )
966        };
967        Ok((options, column_families))
968    }
969
970    /// Constructs a new `DBOptions` from `self` and a string `opts_str` with the syntax detailed in the blogpost
971    /// [Reading RocksDB options from a file](https://rocksdb.org/blog/2015/02/24/reading-rocksdb-options-from-a-file.html)
972    pub fn get_options_from_string<S: AsRef<str>>(
973        &mut self,
974        opts_str: S,
975    ) -> Result<Options, Error> {
976        // create the rocksdb_options_t and immediately wrap it so we don't forget to free it
977        let options = Options {
978            inner: unsafe { ffi::rocksdb_options_create() },
979            outlive: OptionsMustOutliveDB::default(),
980        };
981
982        let opts_cstr = opts_str.as_ref().into_c_string().map_err(|e| {
983            Error::new(format!(
984                "options string must not contain NUL (0x00) bytes: {e}"
985            ))
986        })?;
987        unsafe {
988            ffi_try!(ffi::rocksdb_get_options_from_string(
989                self.inner.cast_const(),
990                opts_cstr.as_ptr(),
991                options.inner,
992            ));
993        }
994        Ok(options)
995    }
996
997    /// Reads column descriptors from C pointers. This frees the `column_family_names` and
998    /// `column_family_options` arrays, and the strings contained in `column_family_names`. It does
999    /// *not* free the `rocksdb_options_t*` pointers contained in `column_family_options`.
1000    #[inline]
1001    unsafe fn read_column_descriptors(
1002        num_column_families: usize,
1003        column_family_names: *mut *mut c_char,
1004        column_family_options: *mut *mut ffi::rocksdb_options_t,
1005    ) -> Vec<ColumnFamilyDescriptor> {
1006        let column_family_names_iter = unsafe {
1007            slice::from_raw_parts(column_family_names, num_column_families)
1008                .iter()
1009                .map(|ptr| from_cstr_and_free(*ptr))
1010        };
1011        let column_family_options_iter = unsafe {
1012            slice::from_raw_parts(column_family_options, num_column_families)
1013                .iter()
1014                .map(|ptr| Options {
1015                    inner: *ptr,
1016                    outlive: OptionsMustOutliveDB::default(),
1017                })
1018        };
1019        let column_descriptors = column_family_names_iter
1020            .zip(column_family_options_iter)
1021            .map(|(name, options)| ColumnFamilyDescriptor {
1022                name,
1023                options,
1024                ttl: ColumnFamilyTtl::Disabled,
1025            })
1026            .collect::<Vec<_>>();
1027
1028        // free the arrays
1029        unsafe {
1030            // we freed each string in the column_family_names array using from_cstr_and_free
1031            ffi::rocksdb_free(column_family_names as *mut c_void);
1032            // we don't want to free the contents of this array because we return it
1033            ffi::rocksdb_free(column_family_options as *mut c_void);
1034            column_descriptors
1035        }
1036    }
1037
1038    /// By default, RocksDB uses only one background thread for flush and
1039    /// compaction. Calling this function will set it up such that total of
1040    /// `total_threads` is used. Good value for `total_threads` is the number of
1041    /// cores. You almost definitely want to call this function if your system is
1042    /// bottlenecked by RocksDB.
1043    ///
1044    /// # Examples
1045    ///
1046    /// ```
1047    /// use rust_rocksdb::Options;
1048    ///
1049    /// let mut opts = Options::default();
1050    /// opts.increase_parallelism(3);
1051    /// ```
1052    pub fn increase_parallelism(&mut self, parallelism: i32) {
1053        unsafe {
1054            ffi::rocksdb_options_increase_parallelism(self.inner, parallelism);
1055        }
1056    }
1057
1058    /// Optimize level style compaction.
1059    ///
1060    /// Default values for some parameters in `Options` are not optimized for heavy
1061    /// workloads and big datasets, which means you might observe write stalls under
1062    /// some conditions.
1063    ///
1064    /// This can be used as one of the starting points for tuning RocksDB options in
1065    /// such cases.
1066    ///
1067    /// Internally, it sets `write_buffer_size`, `min_write_buffer_number_to_merge`,
1068    /// `max_write_buffer_number`, `level0_file_num_compaction_trigger`,
1069    /// `target_file_size_base`, `max_bytes_for_level_base`, so it can override if those
1070    /// parameters were set before.
1071    ///
1072    /// It sets buffer sizes so that memory consumption would be constrained by
1073    /// `memtable_memory_budget`.
1074    pub fn optimize_level_style_compaction(&mut self, memtable_memory_budget: usize) {
1075        unsafe {
1076            ffi::rocksdb_options_optimize_level_style_compaction(
1077                self.inner,
1078                memtable_memory_budget as u64,
1079            );
1080        }
1081    }
1082
1083    /// Optimize universal style compaction.
1084    ///
1085    /// Default values for some parameters in `Options` are not optimized for heavy
1086    /// workloads and big datasets, which means you might observe write stalls under
1087    /// some conditions.
1088    ///
1089    /// This can be used as one of the starting points for tuning RocksDB options in
1090    /// such cases.
1091    ///
1092    /// Internally, it sets `write_buffer_size`, `min_write_buffer_number_to_merge`,
1093    /// `max_write_buffer_number`, `level0_file_num_compaction_trigger`,
1094    /// `target_file_size_base`, `max_bytes_for_level_base`, so it can override if those
1095    /// parameters were set before.
1096    ///
1097    /// It sets buffer sizes so that memory consumption would be constrained by
1098    /// `memtable_memory_budget`.
1099    pub fn optimize_universal_style_compaction(&mut self, memtable_memory_budget: usize) {
1100        unsafe {
1101            ffi::rocksdb_options_optimize_universal_style_compaction(
1102                self.inner,
1103                memtable_memory_budget as u64,
1104            );
1105        }
1106    }
1107
1108    /// If true, the database will be created if it is missing.
1109    ///
1110    /// Default: `false`
1111    ///
1112    /// # Examples
1113    ///
1114    /// ```
1115    /// use rust_rocksdb::Options;
1116    ///
1117    /// let mut opts = Options::default();
1118    /// opts.create_if_missing(true);
1119    /// ```
1120    pub fn create_if_missing(&mut self, create_if_missing: bool) {
1121        unsafe {
1122            ffi::rocksdb_options_set_create_if_missing(
1123                self.inner,
1124                c_uchar::from(create_if_missing),
1125            );
1126        }
1127    }
1128
1129    /// If true, any column families that didn't exist when opening the database
1130    /// will be created.
1131    ///
1132    /// Default: `false`
1133    ///
1134    /// # Examples
1135    ///
1136    /// ```
1137    /// use rust_rocksdb::Options;
1138    ///
1139    /// let mut opts = Options::default();
1140    /// opts.create_missing_column_families(true);
1141    /// ```
1142    pub fn create_missing_column_families(&mut self, create_missing_cfs: bool) {
1143        unsafe {
1144            ffi::rocksdb_options_set_create_missing_column_families(
1145                self.inner,
1146                c_uchar::from(create_missing_cfs),
1147            );
1148        }
1149    }
1150
1151    /// Specifies whether an error should be raised if the database already exists.
1152    ///
1153    /// Default: false
1154    pub fn set_error_if_exists(&mut self, enabled: bool) {
1155        unsafe {
1156            ffi::rocksdb_options_set_error_if_exists(self.inner, c_uchar::from(enabled));
1157        }
1158    }
1159
1160    /// Enable/disable paranoid checks.
1161    ///
1162    /// If true, the implementation will do aggressive checking of the
1163    /// data it is processing and will stop early if it detects any
1164    /// errors. This may have unforeseen ramifications: for example, a
1165    /// corruption of one DB entry may cause a large number of entries to
1166    /// become unreadable or for the entire DB to become unopenable.
1167    /// If any of the  writes to the database fails (Put, Delete, Merge, Write),
1168    /// the database will switch to read-only mode and fail all other
1169    /// Write operations.
1170    ///
1171    /// Default: false
1172    pub fn set_paranoid_checks(&mut self, enabled: bool) {
1173        unsafe {
1174            ffi::rocksdb_options_set_paranoid_checks(self.inner, c_uchar::from(enabled));
1175        }
1176    }
1177
1178    /// A list of paths where SST files can be put into, with its target size.
1179    /// Newer data is placed into paths specified earlier in the vector while
1180    /// older data gradually moves to paths specified later in the vector.
1181    ///
1182    /// For example, you have a flash device with 10GB allocated for the DB,
1183    /// as well as a hard drive of 2TB, you should config it to be:
1184    ///   [{"/flash_path", 10GB}, {"/hard_drive", 2TB}]
1185    ///
1186    /// The system will try to guarantee data under each path is close to but
1187    /// not larger than the target size. But current and future file sizes used
1188    /// by determining where to place a file are based on best-effort estimation,
1189    /// which means there is a chance that the actual size under the directory
1190    /// is slightly more than target size under some workloads. User should give
1191    /// some buffer room for those cases.
1192    ///
1193    /// If none of the paths has sufficient room to place a file, the file will
1194    /// be placed to the last path anyway, despite to the target size.
1195    ///
1196    /// Placing newer data to earlier paths is also best-efforts. User should
1197    /// expect user files to be placed in higher levels in some extreme cases.
1198    ///
1199    /// If left empty, only one path will be used, which is `path` passed when
1200    /// opening the DB.
1201    ///
1202    /// Default: empty
1203    pub fn set_db_paths(&mut self, paths: &[DBPath]) {
1204        let mut paths: Vec<_> = paths.iter().map(|path| path.inner.cast_const()).collect();
1205        let num_paths = paths.len();
1206        unsafe {
1207            ffi::rocksdb_options_set_db_paths(self.inner, paths.as_mut_ptr(), num_paths);
1208        }
1209    }
1210
1211    /// Use the specified object to interact with the environment,
1212    /// e.g. to read/write files, schedule background work, etc. In the near
1213    /// future, support for doing storage operations such as read/write files
1214    /// through env will be deprecated in favor of file_system.
1215    ///
1216    /// Default: Env::default()
1217    pub fn set_env(&mut self, env: &Env) {
1218        unsafe {
1219            ffi::rocksdb_options_set_env(self.inner, env.0.inner);
1220        }
1221        self.outlive.env = Some(env.clone());
1222    }
1223
1224    /// Sets the compression algorithm that will be used for compressing blocks.
1225    ///
1226    /// Default: `DBCompressionType::Snappy` (`DBCompressionType::None` if
1227    /// snappy feature is not enabled).
1228    ///
1229    /// # Examples
1230    ///
1231    /// ```
1232    /// use rust_rocksdb::{Options, DBCompressionType};
1233    ///
1234    /// let mut opts = Options::default();
1235    /// opts.set_compression_type(DBCompressionType::Snappy);
1236    /// ```
1237    pub fn set_compression_type(&mut self, t: DBCompressionType) {
1238        unsafe {
1239            ffi::rocksdb_options_set_compression(self.inner, t as c_int);
1240        }
1241    }
1242
1243    /// Number of threads for parallel compression.
1244    /// Parallel compression is enabled only if threads > 1.
1245    /// THE FEATURE IS STILL EXPERIMENTAL
1246    ///
1247    /// See [code](https://github.com/facebook/rocksdb/blob/v8.6.7/include/rocksdb/advanced_options.h#L116-L127)
1248    /// for more information.
1249    ///
1250    /// Default: 1
1251    ///
1252    /// Examples
1253    ///
1254    /// ```
1255    /// use rust_rocksdb::{Options, DBCompressionType};
1256    ///
1257    /// let mut opts = Options::default();
1258    /// opts.set_compression_type(DBCompressionType::Zstd);
1259    /// opts.set_compression_options_parallel_threads(3);
1260    /// ```
1261    pub fn set_compression_options_parallel_threads(&mut self, num: i32) {
1262        unsafe {
1263            ffi::rocksdb_options_set_compression_options_parallel_threads(self.inner, num);
1264        }
1265    }
1266
1267    /// Sets the compression algorithm that will be used for compressing WAL.
1268    ///
1269    /// At present, only ZSTD compression is supported!
1270    ///
1271    /// Default: `DBCompressionType::None`
1272    ///
1273    /// # Examples
1274    ///
1275    /// ```
1276    /// use rust_rocksdb::{Options, DBCompressionType};
1277    ///
1278    /// let mut opts = Options::default();
1279    /// opts.set_wal_compression_type(DBCompressionType::Zstd);
1280    /// // Or None to disable it
1281    /// opts.set_wal_compression_type(DBCompressionType::None);
1282    /// ```
1283    pub fn set_wal_compression_type(&mut self, t: DBCompressionType) {
1284        match t {
1285            DBCompressionType::None | DBCompressionType::Zstd => unsafe {
1286                ffi::rocksdb_options_set_wal_compression(self.inner, t as c_int);
1287            },
1288            other => unimplemented!("{:?} is not supported for WAL compression", other),
1289        }
1290    }
1291
1292    /// Sets the bottom-most compression algorithm that will be used for
1293    /// compressing blocks at the bottom-most level.
1294    ///
1295    /// Note that to actually enable bottom-most compression configuration after
1296    /// setting the compression type, it needs to be enabled by calling
1297    /// [`set_bottommost_compression_options`](#method.set_bottommost_compression_options) or
1298    /// [`set_bottommost_zstd_max_train_bytes`](#method.set_bottommost_zstd_max_train_bytes) method with `enabled` argument
1299    /// set to `true`.
1300    ///
1301    /// # Examples
1302    ///
1303    /// ```
1304    /// use rust_rocksdb::{Options, DBCompressionType};
1305    ///
1306    /// let mut opts = Options::default();
1307    /// opts.set_bottommost_compression_type(DBCompressionType::Zstd);
1308    /// opts.set_bottommost_zstd_max_train_bytes(0, true);
1309    /// ```
1310    pub fn set_bottommost_compression_type(&mut self, t: DBCompressionType) {
1311        unsafe {
1312            ffi::rocksdb_options_set_bottommost_compression(self.inner, t as c_int);
1313        }
1314    }
1315
1316    /// Different levels can have different compression policies. There
1317    /// are cases where most lower levels would like to use quick compression
1318    /// algorithms while the higher levels (which have more data) use
1319    /// compression algorithms that have better compression but could
1320    /// be slower. This array, if non-empty, should have an entry for
1321    /// each level of the database; these override the value specified in
1322    /// the previous field 'compression'.
1323    ///
1324    /// # Examples
1325    ///
1326    /// ```
1327    /// use rust_rocksdb::{Options, DBCompressionType};
1328    ///
1329    /// let mut opts = Options::default();
1330    /// opts.set_compression_per_level(&[
1331    ///     DBCompressionType::None,
1332    ///     DBCompressionType::None,
1333    ///     DBCompressionType::Snappy,
1334    ///     DBCompressionType::Snappy,
1335    ///     DBCompressionType::Snappy
1336    /// ]);
1337    /// ```
1338    pub fn set_compression_per_level(&mut self, level_types: &[DBCompressionType]) {
1339        unsafe {
1340            let mut level_types: Vec<_> = level_types.iter().map(|&t| t as c_int).collect();
1341            ffi::rocksdb_options_set_compression_per_level(
1342                self.inner,
1343                level_types.as_mut_ptr(),
1344                level_types.len() as size_t,
1345            );
1346        }
1347    }
1348
1349    /// Maximum size of dictionaries used to prime the compression library.
1350    /// Enabling dictionary can improve compression ratios when there are
1351    /// repetitions across data blocks.
1352    ///
1353    /// The dictionary is created by sampling the SST file data. If
1354    /// `zstd_max_train_bytes` is nonzero, the samples are passed through zstd's
1355    /// dictionary generator. Otherwise, the random samples are used directly as
1356    /// the dictionary.
1357    ///
1358    /// When compression dictionary is disabled, we compress and write each block
1359    /// before buffering data for the next one. When compression dictionary is
1360    /// enabled, we buffer all SST file data in-memory so we can sample it, as data
1361    /// can only be compressed and written after the dictionary has been finalized.
1362    /// So users of this feature may see increased memory usage.
1363    ///
1364    /// Default: `0`
1365    ///
1366    /// # Examples
1367    ///
1368    /// ```
1369    /// use rust_rocksdb::Options;
1370    ///
1371    /// let mut opts = Options::default();
1372    /// opts.set_compression_options(4, 5, 6, 7);
1373    /// ```
1374    pub fn set_compression_options(
1375        &mut self,
1376        w_bits: c_int,
1377        level: c_int,
1378        strategy: c_int,
1379        max_dict_bytes: c_int,
1380    ) {
1381        unsafe {
1382            ffi::rocksdb_options_set_compression_options(
1383                self.inner,
1384                w_bits,
1385                level,
1386                strategy,
1387                max_dict_bytes,
1388            );
1389        }
1390    }
1391
1392    /// Sets compression options for blocks at the bottom-most level.  Meaning
1393    /// of all settings is the same as in [`set_compression_options`](#method.set_compression_options) method but
1394    /// affect only the bottom-most compression which is set using
1395    /// [`set_bottommost_compression_type`](#method.set_bottommost_compression_type) method.
1396    ///
1397    /// # Examples
1398    ///
1399    /// ```
1400    /// use rust_rocksdb::{Options, DBCompressionType};
1401    ///
1402    /// let mut opts = Options::default();
1403    /// opts.set_bottommost_compression_type(DBCompressionType::Zstd);
1404    /// opts.set_bottommost_compression_options(4, 5, 6, 7, true);
1405    /// ```
1406    pub fn set_bottommost_compression_options(
1407        &mut self,
1408        w_bits: c_int,
1409        level: c_int,
1410        strategy: c_int,
1411        max_dict_bytes: c_int,
1412        enabled: bool,
1413    ) {
1414        unsafe {
1415            ffi::rocksdb_options_set_bottommost_compression_options(
1416                self.inner,
1417                w_bits,
1418                level,
1419                strategy,
1420                max_dict_bytes,
1421                c_uchar::from(enabled),
1422            );
1423        }
1424    }
1425
1426    /// Sets maximum size of training data passed to zstd's dictionary trainer. Using zstd's
1427    /// dictionary trainer can achieve even better compression ratio improvements than using
1428    /// `max_dict_bytes` alone.
1429    ///
1430    /// The training data will be used to generate a dictionary of max_dict_bytes.
1431    ///
1432    /// Default: 0.
1433    pub fn set_zstd_max_train_bytes(&mut self, value: c_int) {
1434        unsafe {
1435            ffi::rocksdb_options_set_compression_options_zstd_max_train_bytes(self.inner, value);
1436        }
1437    }
1438
1439    /// Sets maximum size of training data passed to zstd's dictionary trainer
1440    /// when compressing the bottom-most level. Using zstd's dictionary trainer
1441    /// can achieve even better compression ratio improvements than using
1442    /// `max_dict_bytes` alone.
1443    ///
1444    /// The training data will be used to generate a dictionary of
1445    /// `max_dict_bytes`.
1446    ///
1447    /// Default: 0.
1448    pub fn set_bottommost_zstd_max_train_bytes(&mut self, value: c_int, enabled: bool) {
1449        unsafe {
1450            ffi::rocksdb_options_set_bottommost_compression_options_zstd_max_train_bytes(
1451                self.inner,
1452                value,
1453                c_uchar::from(enabled),
1454            );
1455        }
1456    }
1457
1458    /// If non-zero, we perform bigger reads when doing compaction. If you're
1459    /// running RocksDB on spinning disks, you should set this to at least 2MB.
1460    /// That way RocksDB's compaction is doing sequential instead of random reads.
1461    ///
1462    /// Default: 2 * 1024 * 1024 (2 MB)
1463    pub fn set_compaction_readahead_size(&mut self, compaction_readahead_size: usize) {
1464        unsafe {
1465            ffi::rocksdb_options_compaction_readahead_size(self.inner, compaction_readahead_size);
1466        }
1467    }
1468
1469    /// Allow RocksDB to pick dynamic base of bytes for levels.
1470    /// With this feature turned on, RocksDB will automatically adjust max bytes for each level.
1471    /// The goal of this feature is to have lower bound on size amplification.
1472    ///
1473    /// Default: false.
1474    pub fn set_level_compaction_dynamic_level_bytes(&mut self, v: bool) {
1475        unsafe {
1476            ffi::rocksdb_options_set_level_compaction_dynamic_level_bytes(
1477                self.inner,
1478                c_uchar::from(v),
1479            );
1480        }
1481    }
1482
1483    /// This option has different meanings for different compaction styles:
1484    ///
1485    /// Leveled: files older than `periodic_compaction_seconds` will be picked up
1486    /// for compaction and will be re-written to the same level as they were
1487    /// before if level_compaction_dynamic_level_bytes is disabled. Otherwise,
1488    /// it will rewrite files to the next level except for the last level files
1489    /// to the same level.
1490    ///
1491    /// FIFO: not supported. Setting this option has no effect for FIFO compaction.
1492    ///
1493    /// Universal: when there are files older than `periodic_compaction_seconds`,
1494    /// rocksdb will try to do as large a compaction as possible including the
1495    /// last level. Such compaction is only skipped if only last level is to
1496    /// be compacted and no file in last level is older than
1497    /// `periodic_compaction_seconds`. See more in
1498    /// UniversalCompactionBuilder::PickPeriodicCompaction().
1499    /// For backward compatibility, the effective value of this option takes
1500    /// into account the value of option `ttl`. The logic is as follows:
1501    ///
1502    /// - both options are set to 30 days if they have the default value.
1503    /// - if both options are zero, zero is picked. Otherwise, we take the min
1504    ///   value among non-zero options values (i.e. takes the stricter limit).
1505    ///
1506    /// One main use of the feature is to make sure a file goes through compaction
1507    /// filters periodically. Users can also use the feature to clear up SST
1508    /// files using old format.
1509    ///
1510    /// A file's age is computed by looking at file_creation_time or creation_time
1511    /// table properties in order, if they have valid non-zero values; if not, the
1512    /// age is based on the file's last modified time (given by the underlying
1513    /// Env).
1514    ///
1515    /// This option only supports block based table format for any compaction
1516    /// style.
1517    ///
1518    /// unit: seconds. Ex: 7 days = 7 * 24 * 60 * 60
1519    ///
1520    /// Values:
1521    /// 0: Turn off Periodic compactions.
1522    /// UINT64_MAX - 1 (0xfffffffffffffffe) is special flag to allow RocksDB to
1523    /// pick default.
1524    ///
1525    /// Default: 30 days if using block based table format + compaction filter +
1526    /// leveled compaction or block based table format + universal compaction.
1527    /// 0 (disabled) otherwise.
1528    ///
1529    pub fn set_periodic_compaction_seconds(&mut self, secs: u64) {
1530        unsafe {
1531            ffi::rocksdb_options_set_periodic_compaction_seconds(self.inner, secs);
1532        }
1533    }
1534
1535    /// When an iterator scans this number of invisible entries (tombstones or
1536    /// hidden puts) from the active memtable during a single iterator operation,
1537    /// we will attempt to flush the memtable. Currently only forward scans are
1538    /// supported (SeekToFirst(), Seek() and Next()).
1539    /// This option helps to reduce the overhead of scanning through a
1540    /// large number of entries in memtable.
1541    /// Users should consider enable deletion-triggered-compaction (see
1542    /// CompactOnDeletionCollectorFactory) together with this option to compact
1543    /// away tombstones after the memtable is flushed.
1544    ///
1545    /// Default: 0 (disabled)
1546    /// Dynamically changeable through the SetOptions() API.
1547    pub fn set_memtable_op_scan_flush_trigger(&mut self, num: u32) {
1548        unsafe {
1549            ffi::rocksdb_options_set_memtable_op_scan_flush_trigger(self.inner, num);
1550        }
1551    }
1552
1553    /// Similar to `memtable_op_scan_flush_trigger`, but this option applies to
1554    /// Next() calls between Seeks or until iterator destruction. If the average
1555    /// of the number of invisible entries scanned from the active memtable, the
1556    /// memtable will be marked for flush.
1557    /// Note that to avoid the case where the window between Seeks is too small,
1558    /// the option only takes effect if the total number of hidden entries scanned
1559    /// within a window is at least `memtable_op_scan_flush_trigger`. So this
1560    /// option is only effective when `memtable_op_scan_flush_trigger` is set.
1561    ///
1562    /// This option should be set to a lower value than
1563    /// `memtable_op_scan_flush_trigger`. It covers the case where an iterator
1564    /// scans through an expensive key range with many invisible entries from the
1565    /// active memtable, but the number of invisible entries per operation does not
1566    /// exceed `memtable_op_scan_flush_trigger`.
1567    ///
1568    /// Default: 0 (disabled)
1569    /// Dynamically changeable through the SetOptions() API.
1570    pub fn set_memtable_avg_op_scan_flush_trigger(&mut self, num: u32) {
1571        unsafe {
1572            ffi::rocksdb_options_set_memtable_avg_op_scan_flush_trigger(self.inner, num);
1573        }
1574    }
1575
1576    /// This option has different meanings for different compaction styles:
1577    ///
1578    /// Leveled: Non-bottom-level files with all keys older than TTL will go
1579    ///    through the compaction process. This usually happens in a cascading
1580    ///    way so that those entries will be compacted to bottommost level/file.
1581    ///    The feature is used to remove stale entries that have been deleted or
1582    ///    updated from the file system.
1583    ///
1584    /// FIFO: Files with all keys older than TTL will be deleted. TTL is only
1585    ///    supported if option max_open_files is set to -1.
1586    ///
1587    /// Universal: users should only set the option `periodic_compaction_seconds`
1588    ///    instead. For backward compatibility, this option has the same
1589    ///    meaning as `periodic_compaction_seconds`. See more in comments for
1590    ///    `periodic_compaction_seconds` on the interaction between these two
1591    ///    options.
1592    ///
1593    /// This option only supports block based table format for any compaction
1594    /// style.
1595    ///
1596    /// unit: seconds. Ex: 1 day = 1 * 24 * 60 * 60
1597    /// 0 means disabling.
1598    /// UINT64_MAX - 1 (0xfffffffffffffffe) is special flag to allow RocksDB to
1599    /// pick default.
1600    ///
1601    /// Default: 30 days if using block based table. 0 (disable) otherwise.
1602    ///
1603    /// Dynamically changeable
1604    /// Note that dynamically changing this option only works for leveled and FIFO
1605    /// compaction. For universal compaction, dynamically changing this option has
1606    /// no effect, users should dynamically change `periodic_compaction_seconds`
1607    /// instead.
1608    pub fn set_ttl(&mut self, secs: u64) {
1609        unsafe {
1610            ffi::rocksdb_options_set_ttl(self.inner, secs);
1611        }
1612    }
1613
1614    pub fn set_merge_operator_associative<F: MergeFn + Clone>(
1615        &mut self,
1616        name: impl CStrLike,
1617        full_merge_fn: F,
1618    ) {
1619        let cb = Box::new(MergeOperatorCallback {
1620            name: name.into_c_string().unwrap(),
1621            full_merge_fn: full_merge_fn.clone(),
1622            partial_merge_fn: full_merge_fn,
1623        });
1624
1625        unsafe {
1626            let mo = ffi::rocksdb_mergeoperator_create(
1627                Box::into_raw(cb).cast::<c_void>(),
1628                Some(merge_operator::destructor_callback::<F, F>),
1629                Some(full_merge_callback::<F, F>),
1630                Some(partial_merge_callback::<F, F>),
1631                Some(merge_operator::delete_callback),
1632                Some(merge_operator::name_callback::<F, F>),
1633            );
1634            ffi::rocksdb_options_set_merge_operator(self.inner, mo);
1635        }
1636    }
1637
1638    pub fn set_merge_operator<F: MergeFn, PF: MergeFn>(
1639        &mut self,
1640        name: impl CStrLike,
1641        full_merge_fn: F,
1642        partial_merge_fn: PF,
1643    ) {
1644        let cb = Box::new(MergeOperatorCallback {
1645            name: name.into_c_string().unwrap(),
1646            full_merge_fn,
1647            partial_merge_fn,
1648        });
1649
1650        unsafe {
1651            let mo = ffi::rocksdb_mergeoperator_create(
1652                Box::into_raw(cb).cast::<c_void>(),
1653                Some(merge_operator::destructor_callback::<F, PF>),
1654                Some(full_merge_callback::<F, PF>),
1655                Some(partial_merge_callback::<F, PF>),
1656                Some(merge_operator::delete_callback),
1657                Some(merge_operator::name_callback::<F, PF>),
1658            );
1659            ffi::rocksdb_options_set_merge_operator(self.inner, mo);
1660        }
1661    }
1662
1663    #[deprecated(
1664        since = "0.5.0",
1665        note = "add_merge_operator has been renamed to set_merge_operator"
1666    )]
1667    pub fn add_merge_operator<F: MergeFn + Clone>(&mut self, name: &str, merge_fn: F) {
1668        self.set_merge_operator_associative(name, merge_fn);
1669    }
1670
1671    /// Sets a compaction filter used to determine if entries should be kept, changed,
1672    /// or removed during compaction.
1673    ///
1674    /// An example use case is to remove entries with an expired TTL.
1675    ///
1676    /// If you take a snapshot of the database, only values written since the last
1677    /// snapshot will be passed through the compaction filter.
1678    ///
1679    /// If multi-threaded compaction is used, `filter_fn` may be called multiple times
1680    /// simultaneously.
1681    pub fn set_compaction_filter<F>(&mut self, name: impl CStrLike, filter_fn: F)
1682    where
1683        F: CompactionFilterFn + Send + 'static,
1684    {
1685        let cb = Box::new(CompactionFilterCallback {
1686            name: name.into_c_string().unwrap(),
1687            filter_fn,
1688        });
1689
1690        let filter = unsafe {
1691            let cf = ffi::rocksdb_compactionfilter_create(
1692                Box::into_raw(cb).cast::<c_void>(),
1693                Some(compaction_filter::destructor_callback::<CompactionFilterCallback<F>>),
1694                Some(compaction_filter::filter_callback::<CompactionFilterCallback<F>>),
1695                Some(compaction_filter::name_callback::<CompactionFilterCallback<F>>),
1696            );
1697            ffi::rocksdb_options_set_compaction_filter(self.inner, cf);
1698
1699            OwnedCompactionFilter::new(NonNull::new(cf).unwrap())
1700        };
1701        self.outlive.compaction_filter = Some(Arc::new(filter));
1702    }
1703
1704    pub fn add_event_listener<L: EventListener>(&mut self, l: L) {
1705        let handle = new_event_listener(l);
1706        unsafe { ffi::rocksdb_options_add_eventlistener(self.inner, handle.inner) }
1707    }
1708
1709    /// This is a factory that provides compaction filter objects which allow
1710    /// an application to modify/delete a key-value during background compaction.
1711    ///
1712    /// A new filter will be created on each compaction run.  If multithreaded
1713    /// compaction is being used, each created CompactionFilter will only be used
1714    /// from a single thread and so does not need to be thread-safe.
1715    ///
1716    /// Default: nullptr
1717    pub fn set_compaction_filter_factory<F>(&mut self, factory: F)
1718    where
1719        F: CompactionFilterFactory + 'static,
1720    {
1721        let factory = Box::new(factory);
1722
1723        unsafe {
1724            let cff = ffi::rocksdb_compactionfilterfactory_create(
1725                Box::into_raw(factory).cast::<c_void>(),
1726                Some(compaction_filter_factory::destructor_callback::<F>),
1727                Some(compaction_filter_factory::create_compaction_filter_callback::<F>),
1728                Some(compaction_filter_factory::name_callback::<F>),
1729            );
1730
1731            ffi::rocksdb_options_set_compaction_filter_factory(self.inner, cff);
1732        }
1733    }
1734
1735    /// Sets the comparator used to define the order of keys in the table.
1736    /// Default: a comparator that uses lexicographic byte-wise ordering
1737    ///
1738    /// The client must ensure that the comparator supplied here has the same
1739    /// name and orders keys *exactly* the same as the comparator provided to
1740    /// previous open calls on the same DB.
1741    pub fn set_comparator(&mut self, name: impl CStrLike, compare_fn: Box<CompareFn>) {
1742        let cb = Box::new(ComparatorCallback {
1743            name: name.into_c_string().unwrap(),
1744            compare_fn,
1745        });
1746
1747        let cmp = unsafe {
1748            let cmp = ffi::rocksdb_comparator_create(
1749                Box::into_raw(cb).cast::<c_void>(),
1750                Some(ComparatorCallback::destructor_callback),
1751                Some(ComparatorCallback::compare_callback),
1752                Some(ComparatorCallback::name_callback),
1753            );
1754            ffi::rocksdb_options_set_comparator(self.inner, cmp);
1755            OwnedComparator::new(NonNull::new(cmp).unwrap())
1756        };
1757        self.outlive.comparator = Some(Arc::new(cmp));
1758    }
1759
1760    /// Sets the comparator that are timestamp-aware, used to define the order of keys in the table,
1761    /// taking timestamp into consideration.
1762    /// Find more information on timestamp-aware comparator on [here](https://github.com/facebook/rocksdb/wiki/User-defined-Timestamp)
1763    ///
1764    /// The client must ensure that the comparator supplied here has the same
1765    /// name and orders keys *exactly* the same as the comparator provided to
1766    /// previous open calls on the same DB.
1767    pub fn set_comparator_with_ts(
1768        &mut self,
1769        name: impl CStrLike,
1770        timestamp_size: usize,
1771        compare_fn: Box<CompareFn>,
1772        compare_ts_fn: Box<CompareTsFn>,
1773        compare_without_ts_fn: Box<CompareWithoutTsFn>,
1774    ) {
1775        let cb = Box::new(ComparatorWithTsCallback {
1776            name: name.into_c_string().unwrap(),
1777            compare_fn,
1778            compare_ts_fn,
1779            compare_without_ts_fn,
1780        });
1781
1782        let cmp = unsafe {
1783            let cmp = ffi::rocksdb_comparator_with_ts_create(
1784                Box::into_raw(cb).cast::<c_void>(),
1785                Some(ComparatorWithTsCallback::destructor_callback),
1786                Some(ComparatorWithTsCallback::compare_callback),
1787                Some(ComparatorWithTsCallback::compare_ts_callback),
1788                Some(ComparatorWithTsCallback::compare_without_ts_callback),
1789                Some(ComparatorWithTsCallback::name_callback),
1790                timestamp_size,
1791            );
1792            ffi::rocksdb_options_set_comparator(self.inner, cmp);
1793            OwnedComparator::new(NonNull::new(cmp).unwrap())
1794        };
1795        self.outlive.comparator = Some(Arc::new(cmp));
1796    }
1797
1798    pub fn set_prefix_extractor(&mut self, prefix_extractor: SliceTransform) {
1799        unsafe {
1800            ffi::rocksdb_options_set_prefix_extractor(self.inner, prefix_extractor.inner);
1801        }
1802    }
1803
1804    // Use this if you don't need to keep the data sorted, i.e. you'll never use
1805    // an iterator, only Put() and Get() API calls
1806    //
1807    pub fn optimize_for_point_lookup(&mut self, block_cache_size_mb: u64) {
1808        unsafe {
1809            ffi::rocksdb_options_optimize_for_point_lookup(self.inner, block_cache_size_mb);
1810        }
1811    }
1812
1813    /// Sets the optimize_filters_for_hits flag
1814    ///
1815    /// Default: `false`
1816    ///
1817    /// # Examples
1818    ///
1819    /// ```
1820    /// use rust_rocksdb::Options;
1821    ///
1822    /// let mut opts = Options::default();
1823    /// opts.set_optimize_filters_for_hits(true);
1824    /// ```
1825    pub fn set_optimize_filters_for_hits(&mut self, optimize_for_hits: bool) {
1826        unsafe {
1827            ffi::rocksdb_options_set_optimize_filters_for_hits(
1828                self.inner,
1829                c_int::from(optimize_for_hits),
1830            );
1831        }
1832    }
1833
1834    /// Sets the periodicity when obsolete files get deleted.
1835    ///
1836    /// The files that get out of scope by compaction
1837    /// process will still get automatically delete on every compaction,
1838    /// regardless of this setting.
1839    ///
1840    /// Default: 6 hours
1841    pub fn set_delete_obsolete_files_period_micros(&mut self, micros: u64) {
1842        unsafe {
1843            ffi::rocksdb_options_set_delete_obsolete_files_period_micros(self.inner, micros);
1844        }
1845    }
1846
1847    /// Prepare the DB for bulk loading.
1848    ///
1849    /// All data will be in level 0 without any automatic compaction.
1850    /// It's recommended to manually call CompactRange(NULL, NULL) before reading
1851    /// from the database, because otherwise the read can be very slow.
1852    pub fn prepare_for_bulk_load(&mut self) {
1853        unsafe {
1854            ffi::rocksdb_options_prepare_for_bulk_load(self.inner);
1855        }
1856    }
1857
1858    /// Sets the number of open files that can be used by the DB. You may need to
1859    /// increase this if your database has a large working set. Value `-1` means
1860    /// files opened are always kept open. You can estimate number of files based
1861    /// on target_file_size_base and target_file_size_multiplier for level-based
1862    /// compaction. For universal-style compaction, you can usually set it to `-1`.
1863    ///
1864    /// Default: `-1`
1865    ///
1866    /// # Examples
1867    ///
1868    /// ```
1869    /// use rust_rocksdb::Options;
1870    ///
1871    /// let mut opts = Options::default();
1872    /// opts.set_max_open_files(10);
1873    /// ```
1874    pub fn set_max_open_files(&mut self, nfiles: c_int) {
1875        unsafe {
1876            ffi::rocksdb_options_set_max_open_files(self.inner, nfiles);
1877        }
1878    }
1879
1880    /// If max_open_files is -1, DB will open all files on DB::Open(). You can
1881    /// use this option to increase the number of threads used to open the files.
1882    /// Default: 16
1883    pub fn set_max_file_opening_threads(&mut self, nthreads: c_int) {
1884        unsafe {
1885            ffi::rocksdb_options_set_max_file_opening_threads(self.inner, nthreads);
1886        }
1887    }
1888
1889    /// By default, writes to stable storage use fdatasync (on platforms
1890    /// where this function is available). If this option is true,
1891    /// fsync is used instead.
1892    ///
1893    /// fsync and fdatasync are equally safe for our purposes and fdatasync is
1894    /// faster, so it is rarely necessary to set this option. It is provided
1895    /// as a workaround for kernel/filesystem bugs, such as one that affected
1896    /// fdatasync with ext4 in kernel versions prior to 3.7.
1897    ///
1898    /// Default: `false`
1899    ///
1900    /// # Examples
1901    ///
1902    /// ```
1903    /// use rust_rocksdb::Options;
1904    ///
1905    /// let mut opts = Options::default();
1906    /// opts.set_use_fsync(true);
1907    /// ```
1908    pub fn set_use_fsync(&mut self, useit: bool) {
1909        unsafe {
1910            ffi::rocksdb_options_set_use_fsync(self.inner, c_int::from(useit));
1911        }
1912    }
1913
1914    /// Returns the value of the `use_fsync` option.
1915    pub fn get_use_fsync(&self) -> bool {
1916        let val = unsafe { ffi::rocksdb_options_get_use_fsync(self.inner) };
1917        val != 0
1918    }
1919
1920    /// Specifies the absolute info LOG dir.
1921    ///
1922    /// If it is empty, the log files will be in the same dir as data.
1923    /// If it is non empty, the log files will be in the specified dir,
1924    /// and the db data dir's absolute path will be used as the log file
1925    /// name's prefix.
1926    ///
1927    /// Default: empty
1928    pub fn set_db_log_dir<P: AsRef<Path>>(&mut self, path: P) {
1929        let p = to_cpath(path).unwrap();
1930        unsafe {
1931            ffi::rocksdb_options_set_db_log_dir(self.inner, p.as_ptr());
1932        }
1933    }
1934
1935    /// Specifies the log level.
1936    /// Consider the `LogLevel` enum for a list of possible levels.
1937    ///
1938    /// Default: Info
1939    ///
1940    /// # Examples
1941    ///
1942    /// ```
1943    /// use rust_rocksdb::{Options, LogLevel};
1944    ///
1945    /// let mut opts = Options::default();
1946    /// opts.set_log_level(LogLevel::Warn);
1947    /// ```
1948    pub fn set_log_level(&mut self, level: LogLevel) {
1949        unsafe {
1950            ffi::rocksdb_options_set_info_log_level(self.inner, level as c_int);
1951        }
1952    }
1953
1954    /// Allows OS to incrementally sync files to disk while they are being
1955    /// written, asynchronously, in the background. This operation can be used
1956    /// to smooth out write I/Os over time. Users shouldn't rely on it for
1957    /// persistency guarantee.
1958    /// Issue one request for every bytes_per_sync written. `0` turns it off.
1959    ///
1960    /// Default: `0`
1961    ///
1962    /// You may consider using rate_limiter to regulate write rate to device.
1963    /// When rate limiter is enabled, it automatically enables bytes_per_sync
1964    /// to 1MB.
1965    ///
1966    /// This option applies to table files
1967    ///
1968    /// # Examples
1969    ///
1970    /// ```
1971    /// use rust_rocksdb::Options;
1972    ///
1973    /// let mut opts = Options::default();
1974    /// opts.set_bytes_per_sync(1024 * 1024);
1975    /// ```
1976    pub fn set_bytes_per_sync(&mut self, nbytes: u64) {
1977        unsafe {
1978            ffi::rocksdb_options_set_bytes_per_sync(self.inner, nbytes);
1979        }
1980    }
1981
1982    /// Same as bytes_per_sync, but applies to WAL files.
1983    ///
1984    /// Default: 0, turned off
1985    ///
1986    /// Dynamically changeable through SetDBOptions() API.
1987    pub fn set_wal_bytes_per_sync(&mut self, nbytes: u64) {
1988        unsafe {
1989            ffi::rocksdb_options_set_wal_bytes_per_sync(self.inner, nbytes);
1990        }
1991    }
1992
1993    /// Sets the maximum buffer size that is used by WritableFileWriter.
1994    ///
1995    /// On Windows, we need to maintain an aligned buffer for writes.
1996    /// We allow the buffer to grow until it's size hits the limit in buffered
1997    /// IO and fix the buffer size when using direct IO to ensure alignment of
1998    /// write requests if the logical sector size is unusual
1999    ///
2000    /// Default: 1024 * 1024 (1 MB)
2001    ///
2002    /// Dynamically changeable through SetDBOptions() API.
2003    pub fn set_writable_file_max_buffer_size(&mut self, nbytes: u64) {
2004        unsafe {
2005            ffi::rocksdb_options_set_writable_file_max_buffer_size(self.inner, nbytes);
2006        }
2007    }
2008
2009    /// If true, allow multi-writers to update mem tables in parallel.
2010    /// Only some memtable_factory-s support concurrent writes; currently it
2011    /// is implemented only for SkipListFactory.  Concurrent memtable writes
2012    /// are not compatible with inplace_update_support or filter_deletes.
2013    /// It is strongly recommended to set enable_write_thread_adaptive_yield
2014    /// if you are going to use this feature.
2015    ///
2016    /// Default: true
2017    ///
2018    /// # Examples
2019    ///
2020    /// ```
2021    /// use rust_rocksdb::Options;
2022    ///
2023    /// let mut opts = Options::default();
2024    /// opts.set_allow_concurrent_memtable_write(false);
2025    /// ```
2026    pub fn set_allow_concurrent_memtable_write(&mut self, allow: bool) {
2027        unsafe {
2028            ffi::rocksdb_options_set_allow_concurrent_memtable_write(
2029                self.inner,
2030                c_uchar::from(allow),
2031            );
2032        }
2033    }
2034
2035    /// If true, threads synchronizing with the write batch group leader will wait for up to
2036    /// write_thread_max_yield_usec before blocking on a mutex. This can substantially improve
2037    /// throughput for concurrent workloads, regardless of whether allow_concurrent_memtable_write
2038    /// is enabled.
2039    ///
2040    /// Default: true
2041    pub fn set_enable_write_thread_adaptive_yield(&mut self, enabled: bool) {
2042        unsafe {
2043            ffi::rocksdb_options_set_enable_write_thread_adaptive_yield(
2044                self.inner,
2045                c_uchar::from(enabled),
2046            );
2047        }
2048    }
2049
2050    /// Specifies whether an iteration->Next() sequentially skips over keys with the same user-key or not.
2051    ///
2052    /// This number specifies the number of keys (with the same userkey)
2053    /// that will be sequentially skipped before a reseek is issued.
2054    ///
2055    /// Default: 8
2056    pub fn set_max_sequential_skip_in_iterations(&mut self, num: u64) {
2057        unsafe {
2058            ffi::rocksdb_options_set_max_sequential_skip_in_iterations(self.inner, num);
2059        }
2060    }
2061
2062    /// Enable direct I/O mode for reading
2063    /// they may or may not improve performance depending on the use case
2064    ///
2065    /// Files will be opened in "direct I/O" mode
2066    /// which means that data read from the disk will not be cached or
2067    /// buffered. The hardware buffer of the devices may however still
2068    /// be used. Memory mapped files are not impacted by these parameters.
2069    ///
2070    /// Default: false
2071    ///
2072    /// # Examples
2073    ///
2074    /// ```
2075    /// use rust_rocksdb::Options;
2076    ///
2077    /// let mut opts = Options::default();
2078    /// opts.set_use_direct_reads(true);
2079    /// ```
2080    pub fn set_use_direct_reads(&mut self, enabled: bool) {
2081        unsafe {
2082            ffi::rocksdb_options_set_use_direct_reads(self.inner, c_uchar::from(enabled));
2083        }
2084    }
2085
2086    /// Enable direct I/O mode for flush and compaction
2087    ///
2088    /// Files will be opened in "direct I/O" mode
2089    /// which means that data written to the disk will not be cached or
2090    /// buffered. The hardware buffer of the devices may however still
2091    /// be used. Memory mapped files are not impacted by these parameters.
2092    /// they may or may not improve performance depending on the use case
2093    ///
2094    /// Default: false
2095    ///
2096    /// # Examples
2097    ///
2098    /// ```
2099    /// use rust_rocksdb::Options;
2100    ///
2101    /// let mut opts = Options::default();
2102    /// opts.set_use_direct_io_for_flush_and_compaction(true);
2103    /// ```
2104    pub fn set_use_direct_io_for_flush_and_compaction(&mut self, enabled: bool) {
2105        unsafe {
2106            ffi::rocksdb_options_set_use_direct_io_for_flush_and_compaction(
2107                self.inner,
2108                c_uchar::from(enabled),
2109            );
2110        }
2111    }
2112
2113    /// Enable/disable child process inherit open files.
2114    ///
2115    /// Default: true
2116    pub fn set_is_fd_close_on_exec(&mut self, enabled: bool) {
2117        unsafe {
2118            ffi::rocksdb_options_set_is_fd_close_on_exec(self.inner, c_uchar::from(enabled));
2119        }
2120    }
2121
2122    /// Hints to the OS that it should not buffer disk I/O. Enabling this
2123    /// parameter may improve performance but increases pressure on the
2124    /// system cache.
2125    ///
2126    /// The exact behavior of this parameter is platform dependent.
2127    ///
2128    /// On POSIX systems, after RocksDB reads data from disk it will
2129    /// mark the pages as "unneeded". The operating system may or may not
2130    /// evict these pages from memory, reducing pressure on the system
2131    /// cache. If the disk block is requested again this can result in
2132    /// additional disk I/O.
2133    ///
2134    /// On WINDOWS systems, files will be opened in "unbuffered I/O" mode
2135    /// which means that data read from the disk will not be cached or
2136    /// bufferized. The hardware buffer of the devices may however still
2137    /// be used. Memory mapped files are not impacted by this parameter.
2138    ///
2139    /// Default: true
2140    ///
2141    /// # Examples
2142    ///
2143    /// ```
2144    /// use rust_rocksdb::Options;
2145    ///
2146    /// let mut opts = Options::default();
2147    /// #[allow(deprecated)]
2148    /// opts.set_allow_os_buffer(false);
2149    /// ```
2150    #[deprecated(
2151        since = "0.7.0",
2152        note = "replaced with set_use_direct_reads/set_use_direct_io_for_flush_and_compaction methods"
2153    )]
2154    pub fn set_allow_os_buffer(&mut self, is_allow: bool) {
2155        self.set_use_direct_reads(!is_allow);
2156        self.set_use_direct_io_for_flush_and_compaction(!is_allow);
2157    }
2158
2159    /// Sets the number of shards used for table cache.
2160    ///
2161    /// Default: `6`
2162    ///
2163    /// # Examples
2164    ///
2165    /// ```
2166    /// use rust_rocksdb::Options;
2167    ///
2168    /// let mut opts = Options::default();
2169    /// opts.set_table_cache_num_shard_bits(4);
2170    /// ```
2171    pub fn set_table_cache_num_shard_bits(&mut self, nbits: c_int) {
2172        unsafe {
2173            ffi::rocksdb_options_set_table_cache_numshardbits(self.inner, nbits);
2174        }
2175    }
2176
2177    /// By default target_file_size_multiplier is 1, which means
2178    /// by default files in different levels will have similar size.
2179    ///
2180    /// Dynamically changeable through SetOptions() API
2181    pub fn set_target_file_size_multiplier(&mut self, multiplier: i32) {
2182        unsafe {
2183            ffi::rocksdb_options_set_target_file_size_multiplier(self.inner, multiplier as c_int);
2184        }
2185    }
2186
2187    /// Sets the minimum number of write buffers that will be merged
2188    /// before writing to storage.  If set to `1`, then
2189    /// all write buffers are flushed to L0 as individual files and this increases
2190    /// read amplification because a get request has to check in all of these
2191    /// files. Also, an in-memory merge may result in writing lesser
2192    /// data to storage if there are duplicate records in each of these
2193    /// individual write buffers.
2194    ///
2195    /// Default: `1`
2196    ///
2197    /// # Examples
2198    ///
2199    /// ```
2200    /// use rust_rocksdb::Options;
2201    ///
2202    /// let mut opts = Options::default();
2203    /// opts.set_min_write_buffer_number(2);
2204    /// ```
2205    pub fn set_min_write_buffer_number(&mut self, nbuf: c_int) {
2206        unsafe {
2207            ffi::rocksdb_options_set_min_write_buffer_number_to_merge(self.inner, nbuf);
2208        }
2209    }
2210
2211    /// Sets the maximum number of write buffers that are built up in memory.
2212    /// The default and the minimum number is 2, so that when 1 write buffer
2213    /// is being flushed to storage, new writes can continue to the other
2214    /// write buffer.
2215    /// If max_write_buffer_number > 3, writing will be slowed down to
2216    /// options.delayed_write_rate if we are writing to the last write buffer
2217    /// allowed.
2218    ///
2219    /// Default: `2`
2220    ///
2221    /// # Examples
2222    ///
2223    /// ```
2224    /// use rust_rocksdb::Options;
2225    ///
2226    /// let mut opts = Options::default();
2227    /// opts.set_max_write_buffer_number(4);
2228    /// ```
2229    pub fn set_max_write_buffer_number(&mut self, nbuf: c_int) {
2230        unsafe {
2231            ffi::rocksdb_options_set_max_write_buffer_number(self.inner, nbuf);
2232        }
2233    }
2234
2235    /// Sets the amount of data to build up in memory (backed by an unsorted log
2236    /// on disk) before converting to a sorted on-disk file.
2237    ///
2238    /// Larger values increase performance, especially during bulk loads.
2239    /// Up to max_write_buffer_number write buffers may be held in memory
2240    /// at the same time,
2241    /// so you may wish to adjust this parameter to control memory usage.
2242    /// Also, a larger write buffer will result in a longer recovery time
2243    /// the next time the database is opened.
2244    ///
2245    /// Note that write_buffer_size is enforced per column family.
2246    /// See db_write_buffer_size for sharing memory across column families.
2247    ///
2248    /// Default: `0x4000000` (64MiB)
2249    ///
2250    /// Dynamically changeable through SetOptions() API
2251    ///
2252    /// # Examples
2253    ///
2254    /// ```
2255    /// use rust_rocksdb::Options;
2256    ///
2257    /// let mut opts = Options::default();
2258    /// opts.set_write_buffer_size(128 * 1024 * 1024);
2259    /// ```
2260    pub fn set_write_buffer_size(&mut self, size: usize) {
2261        unsafe {
2262            ffi::rocksdb_options_set_write_buffer_size(self.inner, size);
2263        }
2264    }
2265
2266    /// Amount of data to build up in memtables across all column
2267    /// families before writing to disk.
2268    ///
2269    /// This is distinct from write_buffer_size, which enforces a limit
2270    /// for a single memtable.
2271    ///
2272    /// This feature is disabled by default. Specify a non-zero value
2273    /// to enable it.
2274    ///
2275    /// Default: 0 (disabled)
2276    ///
2277    /// # Examples
2278    ///
2279    /// ```
2280    /// use rust_rocksdb::Options;
2281    ///
2282    /// let mut opts = Options::default();
2283    /// opts.set_db_write_buffer_size(128 * 1024 * 1024);
2284    /// ```
2285    pub fn set_db_write_buffer_size(&mut self, size: usize) {
2286        unsafe {
2287            ffi::rocksdb_options_set_db_write_buffer_size(self.inner, size);
2288        }
2289    }
2290
2291    /// Control maximum total data size for a level.
2292    /// max_bytes_for_level_base is the max total for level-1.
2293    /// Maximum number of bytes for level L can be calculated as
2294    /// (max_bytes_for_level_base) * (max_bytes_for_level_multiplier ^ (L-1))
2295    /// For example, if max_bytes_for_level_base is 200MB, and if
2296    /// max_bytes_for_level_multiplier is 10, total data size for level-1
2297    /// will be 200MB, total file size for level-2 will be 2GB,
2298    /// and total file size for level-3 will be 20GB.
2299    ///
2300    /// Default: `0x10000000` (256MiB).
2301    ///
2302    /// Dynamically changeable through SetOptions() API
2303    ///
2304    /// # Examples
2305    ///
2306    /// ```
2307    /// use rust_rocksdb::Options;
2308    ///
2309    /// let mut opts = Options::default();
2310    /// opts.set_max_bytes_for_level_base(512 * 1024 * 1024);
2311    /// ```
2312    pub fn set_max_bytes_for_level_base(&mut self, size: u64) {
2313        unsafe {
2314            ffi::rocksdb_options_set_max_bytes_for_level_base(self.inner, size);
2315        }
2316    }
2317
2318    /// Default: `10`
2319    ///
2320    /// # Examples
2321    ///
2322    /// ```
2323    /// use rust_rocksdb::Options;
2324    ///
2325    /// let mut opts = Options::default();
2326    /// opts.set_max_bytes_for_level_multiplier(4.0);
2327    /// ```
2328    pub fn set_max_bytes_for_level_multiplier(&mut self, mul: f64) {
2329        unsafe {
2330            ffi::rocksdb_options_set_max_bytes_for_level_multiplier(self.inner, mul);
2331        }
2332    }
2333
2334    /// The manifest file is rolled over on reaching this limit.
2335    /// The older manifest file be deleted.
2336    /// The default value is MAX_INT so that roll-over does not take place.
2337    ///
2338    /// # Examples
2339    ///
2340    /// ```
2341    /// use rust_rocksdb::Options;
2342    ///
2343    /// let mut opts = Options::default();
2344    /// opts.set_max_manifest_file_size(20 * 1024 * 1024);
2345    /// ```
2346    pub fn set_max_manifest_file_size(&mut self, size: usize) {
2347        unsafe {
2348            ffi::rocksdb_options_set_max_manifest_file_size(self.inner, size);
2349        }
2350    }
2351
2352    /// Sets the target file size for compaction.
2353    /// target_file_size_base is per-file size for level-1.
2354    /// Target file size for level L can be calculated by
2355    /// target_file_size_base * (target_file_size_multiplier ^ (L-1))
2356    /// For example, if target_file_size_base is 2MB and
2357    /// target_file_size_multiplier is 10, then each file on level-1 will
2358    /// be 2MB, and each file on level 2 will be 20MB,
2359    /// and each file on level-3 will be 200MB.
2360    ///
2361    /// Default: `0x4000000` (64MiB)
2362    ///
2363    /// Dynamically changeable through SetOptions() API
2364    ///
2365    /// # Examples
2366    ///
2367    /// ```
2368    /// use rust_rocksdb::Options;
2369    ///
2370    /// let mut opts = Options::default();
2371    /// opts.set_target_file_size_base(128 * 1024 * 1024);
2372    /// ```
2373    pub fn set_target_file_size_base(&mut self, size: u64) {
2374        unsafe {
2375            ffi::rocksdb_options_set_target_file_size_base(self.inner, size);
2376        }
2377    }
2378
2379    /// Sets the minimum number of write buffers that will be merged together
2380    /// before writing to storage.  If set to `1`, then
2381    /// all write buffers are flushed to L0 as individual files and this increases
2382    /// read amplification because a get request has to check in all of these
2383    /// files. Also, an in-memory merge may result in writing lesser
2384    /// data to storage if there are duplicate records in each of these
2385    /// individual write buffers.
2386    ///
2387    /// Default: `1`
2388    ///
2389    /// # Examples
2390    ///
2391    /// ```
2392    /// use rust_rocksdb::Options;
2393    ///
2394    /// let mut opts = Options::default();
2395    /// opts.set_min_write_buffer_number_to_merge(2);
2396    /// ```
2397    pub fn set_min_write_buffer_number_to_merge(&mut self, to_merge: c_int) {
2398        unsafe {
2399            ffi::rocksdb_options_set_min_write_buffer_number_to_merge(self.inner, to_merge);
2400        }
2401    }
2402
2403    /// Sets the number of files to trigger level-0 compaction. A value < `0` means that
2404    /// level-0 compaction will not be triggered by number of files at all.
2405    ///
2406    /// Default: `4`
2407    ///
2408    /// Dynamically changeable through SetOptions() API
2409    ///
2410    /// # Examples
2411    ///
2412    /// ```
2413    /// use rust_rocksdb::Options;
2414    ///
2415    /// let mut opts = Options::default();
2416    /// opts.set_level_zero_file_num_compaction_trigger(8);
2417    /// ```
2418    pub fn set_level_zero_file_num_compaction_trigger(&mut self, n: c_int) {
2419        unsafe {
2420            ffi::rocksdb_options_set_level0_file_num_compaction_trigger(self.inner, n);
2421        }
2422    }
2423
2424    /// Sets the soft limit on number of level-0 files. We start slowing down writes at this
2425    /// point. A value < `0` means that no writing slowdown will be triggered by
2426    /// number of files in level-0.
2427    ///
2428    /// Default: `20`
2429    ///
2430    /// Dynamically changeable through SetOptions() API
2431    ///
2432    /// # Examples
2433    ///
2434    /// ```
2435    /// use rust_rocksdb::Options;
2436    ///
2437    /// let mut opts = Options::default();
2438    /// opts.set_level_zero_slowdown_writes_trigger(10);
2439    /// ```
2440    pub fn set_level_zero_slowdown_writes_trigger(&mut self, n: c_int) {
2441        unsafe {
2442            ffi::rocksdb_options_set_level0_slowdown_writes_trigger(self.inner, n);
2443        }
2444    }
2445
2446    /// Sets the maximum number of level-0 files.  We stop writes at this point.
2447    ///
2448    /// Default: `24`
2449    ///
2450    /// Dynamically changeable through SetOptions() API
2451    ///
2452    /// # Examples
2453    ///
2454    /// ```
2455    /// use rust_rocksdb::Options;
2456    ///
2457    /// let mut opts = Options::default();
2458    /// opts.set_level_zero_stop_writes_trigger(48);
2459    /// ```
2460    pub fn set_level_zero_stop_writes_trigger(&mut self, n: c_int) {
2461        unsafe {
2462            ffi::rocksdb_options_set_level0_stop_writes_trigger(self.inner, n);
2463        }
2464    }
2465
2466    /// Sets the compaction style.
2467    ///
2468    /// Default: DBCompactionStyle::Level
2469    ///
2470    /// # Examples
2471    ///
2472    /// ```
2473    /// use rust_rocksdb::{Options, DBCompactionStyle};
2474    ///
2475    /// let mut opts = Options::default();
2476    /// opts.set_compaction_style(DBCompactionStyle::Universal);
2477    /// ```
2478    pub fn set_compaction_style(&mut self, style: DBCompactionStyle) {
2479        unsafe {
2480            ffi::rocksdb_options_set_compaction_style(self.inner, style as c_int);
2481        }
2482    }
2483
2484    /// Sets the options needed to support Universal Style compactions.
2485    pub fn set_universal_compaction_options(&mut self, uco: &UniversalCompactOptions) {
2486        unsafe {
2487            ffi::rocksdb_options_set_universal_compaction_options(self.inner, uco.inner);
2488        }
2489    }
2490
2491    /// Sets the options for FIFO compaction style.
2492    pub fn set_fifo_compaction_options(&mut self, fco: &FifoCompactOptions) {
2493        unsafe {
2494            ffi::rocksdb_options_set_fifo_compaction_options(self.inner, fco.inner);
2495        }
2496    }
2497
2498    /// Sets unordered_write to true trades higher write throughput with
2499    /// relaxing the immutability guarantee of snapshots. This violates the
2500    /// repeatability one expects from ::Get from a snapshot, as well as
2501    /// ::MultiGet and Iterator's consistent-point-in-time view property.
2502    /// If the application cannot tolerate the relaxed guarantees, it can implement
2503    /// its own mechanisms to work around that and yet benefit from the higher
2504    /// throughput. Using TransactionDB with WRITE_PREPARED write policy and
2505    /// two_write_queues=true is one way to achieve immutable snapshots despite
2506    /// unordered_write.
2507    ///
2508    /// By default, i.e., when it is false, rocksdb does not advance the sequence
2509    /// number for new snapshots unless all the writes with lower sequence numbers
2510    /// are already finished. This provides the immutability that we expect from
2511    /// snapshots. Moreover, since Iterator and MultiGet internally depend on
2512    /// snapshots, the snapshot immutability results into Iterator and MultiGet
2513    /// offering consistent-point-in-time view. If set to true, although
2514    /// Read-Your-Own-Write property is still provided, the snapshot immutability
2515    /// property is relaxed: the writes issued after the snapshot is obtained (with
2516    /// larger sequence numbers) will be still not visible to the reads from that
2517    /// snapshot, however, there still might be pending writes (with lower sequence
2518    /// number) that will change the state visible to the snapshot after they are
2519    /// landed to the memtable.
2520    ///
2521    /// Default: false
2522    pub fn set_unordered_write(&mut self, unordered: bool) {
2523        unsafe {
2524            ffi::rocksdb_options_set_unordered_write(self.inner, c_uchar::from(unordered));
2525        }
2526    }
2527
2528    /// Sets maximum number of threads that will
2529    /// concurrently perform a compaction job by breaking it into multiple,
2530    /// smaller ones that are run simultaneously.
2531    ///
2532    /// Default: 1 (i.e. no subcompactions)
2533    pub fn set_max_subcompactions(&mut self, num: u32) {
2534        unsafe {
2535            ffi::rocksdb_options_set_max_subcompactions(self.inner, num);
2536        }
2537    }
2538
2539    /// Sets maximum number of concurrent background jobs
2540    /// (compactions and flushes).
2541    ///
2542    /// Default: 2
2543    ///
2544    /// Dynamically changeable through SetDBOptions() API.
2545    pub fn set_max_background_jobs(&mut self, jobs: c_int) {
2546        unsafe {
2547            ffi::rocksdb_options_set_max_background_jobs(self.inner, jobs);
2548        }
2549    }
2550
2551    /// Sets the maximum number of concurrent background compaction jobs, submitted to
2552    /// the default LOW priority thread pool.
2553    /// We first try to schedule compactions based on
2554    /// `base_background_compactions`. If the compaction cannot catch up , we
2555    /// will increase number of compaction threads up to
2556    /// `max_background_compactions`.
2557    ///
2558    /// If you're increasing this, also consider increasing number of threads in
2559    /// LOW priority thread pool. For more information, see
2560    /// Env::SetBackgroundThreads
2561    ///
2562    /// Default: `1`
2563    ///
2564    /// # Examples
2565    ///
2566    /// ```
2567    /// use rust_rocksdb::Options;
2568    ///
2569    /// let mut opts = Options::default();
2570    /// #[allow(deprecated)]
2571    /// opts.set_max_background_compactions(2);
2572    /// ```
2573    #[deprecated(
2574        since = "0.15.0",
2575        note = "RocksDB automatically decides this based on the value of max_background_jobs"
2576    )]
2577    pub fn set_max_background_compactions(&mut self, n: c_int) {
2578        unsafe {
2579            ffi::rocksdb_options_set_max_background_compactions(self.inner, n);
2580        }
2581    }
2582
2583    /// Sets the maximum number of concurrent background memtable flush jobs, submitted to
2584    /// the HIGH priority thread pool.
2585    ///
2586    /// By default, all background jobs (major compaction and memtable flush) go
2587    /// to the LOW priority pool. If this option is set to a positive number,
2588    /// memtable flush jobs will be submitted to the HIGH priority pool.
2589    /// It is important when the same Env is shared by multiple db instances.
2590    /// Without a separate pool, long running major compaction jobs could
2591    /// potentially block memtable flush jobs of other db instances, leading to
2592    /// unnecessary Put stalls.
2593    ///
2594    /// If you're increasing this, also consider increasing number of threads in
2595    /// HIGH priority thread pool. For more information, see
2596    /// Env::SetBackgroundThreads
2597    ///
2598    /// Default: `1`
2599    ///
2600    /// # Examples
2601    ///
2602    /// ```
2603    /// use rust_rocksdb::Options;
2604    ///
2605    /// let mut opts = Options::default();
2606    /// #[allow(deprecated)]
2607    /// opts.set_max_background_flushes(2);
2608    /// ```
2609    #[deprecated(
2610        since = "0.15.0",
2611        note = "RocksDB automatically decides this based on the value of max_background_jobs"
2612    )]
2613    pub fn set_max_background_flushes(&mut self, n: c_int) {
2614        unsafe {
2615            ffi::rocksdb_options_set_max_background_flushes(self.inner, n);
2616        }
2617    }
2618
2619    /// Disables automatic compactions. Manual compactions can still
2620    /// be issued on this column family
2621    ///
2622    /// Default: `false`
2623    ///
2624    /// Dynamically changeable through SetOptions() API
2625    ///
2626    /// # Examples
2627    ///
2628    /// ```
2629    /// use rust_rocksdb::Options;
2630    ///
2631    /// let mut opts = Options::default();
2632    /// opts.set_disable_auto_compactions(true);
2633    /// ```
2634    pub fn set_disable_auto_compactions(&mut self, disable: bool) {
2635        unsafe {
2636            ffi::rocksdb_options_set_disable_auto_compactions(self.inner, c_int::from(disable));
2637        }
2638    }
2639
2640    /// SetMemtableHugePageSize sets the page size for huge page for
2641    /// arena used by the memtable.
2642    /// If <=0, it won't allocate from huge page but from malloc.
2643    /// Users are responsible to reserve huge pages for it to be allocated. For
2644    /// example:
2645    ///      sysctl -w vm.nr_hugepages=20
2646    /// See linux doc Documentation/vm/hugetlbpage.txt
2647    /// If there isn't enough free huge page available, it will fall back to
2648    /// malloc.
2649    ///
2650    /// Dynamically changeable through SetOptions() API
2651    pub fn set_memtable_huge_page_size(&mut self, size: size_t) {
2652        unsafe {
2653            ffi::rocksdb_options_set_memtable_huge_page_size(self.inner, size);
2654        }
2655    }
2656
2657    /// Sets the maximum number of successive merge operations on a key in the memtable.
2658    ///
2659    /// When a merge operation is added to the memtable and the maximum number of
2660    /// successive merges is reached, the value of the key will be calculated and
2661    /// inserted into the memtable instead of the merge operation. This will
2662    /// ensure that there are never more than max_successive_merges merge
2663    /// operations in the memtable.
2664    ///
2665    /// Default: 0 (disabled)
2666    pub fn set_max_successive_merges(&mut self, num: usize) {
2667        unsafe {
2668            ffi::rocksdb_options_set_max_successive_merges(self.inner, num);
2669        }
2670    }
2671
2672    /// Control locality of bloom filter probes to improve cache miss rate.
2673    /// This option only applies to memtable prefix bloom and plaintable
2674    /// prefix bloom. It essentially limits the max number of cache lines each
2675    /// bloom filter check can touch.
2676    ///
2677    /// This optimization is turned off when set to 0. The number should never
2678    /// be greater than number of probes. This option can boost performance
2679    /// for in-memory workload but should use with care since it can cause
2680    /// higher false positive rate.
2681    ///
2682    /// Default: 0
2683    pub fn set_bloom_locality(&mut self, v: u32) {
2684        unsafe {
2685            ffi::rocksdb_options_set_bloom_locality(self.inner, v);
2686        }
2687    }
2688
2689    /// Enable/disable thread-safe inplace updates.
2690    ///
2691    /// Requires updates if
2692    /// * key exists in current memtable
2693    /// * new sizeof(new_value) <= sizeof(old_value)
2694    /// * old_value for that key is a put i.e. kTypeValue
2695    ///
2696    /// Default: false.
2697    pub fn set_inplace_update_support(&mut self, enabled: bool) {
2698        unsafe {
2699            ffi::rocksdb_options_set_inplace_update_support(self.inner, c_uchar::from(enabled));
2700        }
2701    }
2702
2703    /// Sets the number of locks used for inplace update.
2704    ///
2705    /// Default: 10000 when inplace_update_support = true, otherwise 0.
2706    pub fn set_inplace_update_locks(&mut self, num: usize) {
2707        unsafe {
2708            ffi::rocksdb_options_set_inplace_update_num_locks(self.inner, num);
2709        }
2710    }
2711
2712    /// Different max-size multipliers for different levels.
2713    /// These are multiplied by max_bytes_for_level_multiplier to arrive
2714    /// at the max-size of each level.
2715    ///
2716    /// Default: 1
2717    ///
2718    /// Dynamically changeable through SetOptions() API
2719    pub fn set_max_bytes_for_level_multiplier_additional(&mut self, level_values: &[i32]) {
2720        let count = level_values.len();
2721        unsafe {
2722            ffi::rocksdb_options_set_max_bytes_for_level_multiplier_additional(
2723                self.inner,
2724                level_values.as_ptr().cast_mut(),
2725                count,
2726            );
2727        }
2728    }
2729
2730    /// If true, then DB::Open() will not fetch and check sizes of all sst files.
2731    /// This may significantly speed up startup if there are many sst files,
2732    /// especially when using non-default Env with expensive GetFileSize().
2733    /// We'll still check that all required sst files exist.
2734    /// If paranoid_checks is false, this option is ignored, and sst files are
2735    /// not checked at all.
2736    ///
2737    /// Default: false
2738    #[deprecated(note = "RocksDB >= 10.5: option is ignored: checking done with a thread pool")]
2739    pub fn set_skip_checking_sst_file_sizes_on_db_open(&mut self, value: bool) {
2740        unsafe {
2741            ffi::rocksdb_options_set_skip_checking_sst_file_sizes_on_db_open(
2742                self.inner,
2743                c_uchar::from(value),
2744            );
2745        }
2746    }
2747
2748    /// The total maximum size(bytes) of write buffers to maintain in memory
2749    /// including copies of buffers that have already been flushed. This parameter
2750    /// only affects trimming of flushed buffers and does not affect flushing.
2751    /// This controls the maximum amount of write history that will be available
2752    /// in memory for conflict checking when Transactions are used. The actual
2753    /// size of write history (flushed Memtables) might be higher than this limit
2754    /// if further trimming will reduce write history total size below this
2755    /// limit. For example, if max_write_buffer_size_to_maintain is set to 64MB,
2756    /// and there are three flushed Memtables, with sizes of 32MB, 20MB, 20MB.
2757    /// Because trimming the next Memtable of size 20MB will reduce total memory
2758    /// usage to 52MB which is below the limit, RocksDB will stop trimming.
2759    ///
2760    /// When using an OptimisticTransactionDB:
2761    /// If this value is too low, some transactions may fail at commit time due
2762    /// to not being able to determine whether there were any write conflicts.
2763    ///
2764    /// When using a TransactionDB:
2765    /// If Transaction::SetSnapshot is used, TransactionDB will read either
2766    /// in-memory write buffers or SST files to do write-conflict checking.
2767    /// Increasing this value can reduce the number of reads to SST files
2768    /// done for conflict detection.
2769    ///
2770    /// Setting this value to 0 will cause write buffers to be freed immediately
2771    /// after they are flushed. If this value is set to -1,
2772    /// 'max_write_buffer_number * write_buffer_size' will be used.
2773    ///
2774    /// Default:
2775    /// If using a TransactionDB/OptimisticTransactionDB, the default value will
2776    /// be set to the value of 'max_write_buffer_number * write_buffer_size'
2777    /// if it is not explicitly set by the user.  Otherwise, the default is 0.
2778    pub fn set_max_write_buffer_size_to_maintain(&mut self, size: i64) {
2779        unsafe {
2780            ffi::rocksdb_options_set_max_write_buffer_size_to_maintain(self.inner, size);
2781        }
2782    }
2783
2784    /// By default, a single write thread queue is maintained. The thread gets
2785    /// to the head of the queue becomes write batch group leader and responsible
2786    /// for writing to WAL and memtable for the batch group.
2787    ///
2788    /// If enable_pipelined_write is true, separate write thread queue is
2789    /// maintained for WAL write and memtable write. A write thread first enter WAL
2790    /// writer queue and then memtable writer queue. Pending thread on the WAL
2791    /// writer queue thus only have to wait for previous writers to finish their
2792    /// WAL writing but not the memtable writing. Enabling the feature may improve
2793    /// write throughput and reduce latency of the prepare phase of two-phase
2794    /// commit.
2795    ///
2796    /// Default: false
2797    pub fn set_enable_pipelined_write(&mut self, value: bool) {
2798        unsafe {
2799            ffi::rocksdb_options_set_enable_pipelined_write(self.inner, c_uchar::from(value));
2800        }
2801    }
2802
2803    /// Defines the underlying memtable implementation.
2804    /// See official [wiki](https://github.com/facebook/rocksdb/wiki/MemTable) for more information.
2805    /// Defaults to using a skiplist.
2806    ///
2807    /// # Examples
2808    ///
2809    /// ```
2810    /// use rust_rocksdb::{Options, MemtableFactory};
2811    /// let mut opts = Options::default();
2812    /// let factory = MemtableFactory::HashSkipList {
2813    ///     bucket_count: 1_000_000,
2814    ///     height: 4,
2815    ///     branching_factor: 4,
2816    /// };
2817    ///
2818    /// opts.set_allow_concurrent_memtable_write(false);
2819    /// opts.set_memtable_factory(factory);
2820    /// ```
2821    pub fn set_memtable_factory(&mut self, factory: MemtableFactory) {
2822        match factory {
2823            MemtableFactory::Vector => unsafe {
2824                ffi::rocksdb_options_set_memtable_vector_rep(self.inner);
2825            },
2826            MemtableFactory::HashSkipList {
2827                bucket_count,
2828                height,
2829                branching_factor,
2830            } => unsafe {
2831                ffi::rocksdb_options_set_hash_skip_list_rep(
2832                    self.inner,
2833                    bucket_count,
2834                    height,
2835                    branching_factor,
2836                );
2837            },
2838            MemtableFactory::HashLinkList { bucket_count } => unsafe {
2839                ffi::rocksdb_options_set_hash_link_list_rep(self.inner, bucket_count);
2840            },
2841        }
2842    }
2843
2844    pub fn set_block_based_table_factory(&mut self, factory: &BlockBasedOptions) {
2845        unsafe {
2846            ffi::rocksdb_options_set_block_based_table_factory(self.inner, factory.inner);
2847        }
2848        self.outlive.block_based = Some(factory.outlive.clone());
2849    }
2850
2851    /// Sets the table factory to a CuckooTableFactory (the default table
2852    /// factory is a block-based table factory that provides a default
2853    /// implementation of TableBuilder and TableReader with default
2854    /// BlockBasedTableOptions).
2855    /// See official [wiki](https://github.com/facebook/rocksdb/wiki/CuckooTable-Format) for more information on this table format.
2856    /// # Examples
2857    ///
2858    /// ```
2859    /// use rust_rocksdb::{Options, CuckooTableOptions};
2860    ///
2861    /// let mut opts = Options::default();
2862    /// let mut factory_opts = CuckooTableOptions::default();
2863    /// factory_opts.set_hash_ratio(0.8);
2864    /// factory_opts.set_max_search_depth(20);
2865    /// factory_opts.set_cuckoo_block_size(10);
2866    /// factory_opts.set_identity_as_first_hash(true);
2867    /// factory_opts.set_use_module_hash(false);
2868    ///
2869    /// opts.set_cuckoo_table_factory(&factory_opts);
2870    /// ```
2871    pub fn set_cuckoo_table_factory(&mut self, factory: &CuckooTableOptions) {
2872        unsafe {
2873            ffi::rocksdb_options_set_cuckoo_table_factory(self.inner, factory.inner);
2874        }
2875    }
2876
2877    // This is a factory that provides TableFactory objects.
2878    // Default: a block-based table factory that provides a default
2879    // implementation of TableBuilder and TableReader with default
2880    // BlockBasedTableOptions.
2881    /// Sets the factory as plain table.
2882    /// See official [wiki](https://github.com/facebook/rocksdb/wiki/PlainTable-Format) for more
2883    /// information.
2884    ///
2885    /// # Examples
2886    ///
2887    /// ```
2888    /// use rust_rocksdb::{KeyEncodingType, Options, PlainTableFactoryOptions};
2889    ///
2890    /// let mut opts = Options::default();
2891    /// let factory_opts = PlainTableFactoryOptions {
2892    ///   user_key_length: 0,
2893    ///   bloom_bits_per_key: 20,
2894    ///   hash_table_ratio: 0.75,
2895    ///   index_sparseness: 16,
2896    ///   huge_page_tlb_size: 0,
2897    ///   encoding_type: KeyEncodingType::Plain,
2898    ///   full_scan_mode: false,
2899    ///   store_index_in_file: false,
2900    /// };
2901    ///
2902    /// opts.set_plain_table_factory(&factory_opts);
2903    /// ```
2904    pub fn set_plain_table_factory(&mut self, options: &PlainTableFactoryOptions) {
2905        unsafe {
2906            ffi::rocksdb_options_set_plain_table_factory(
2907                self.inner,
2908                options.user_key_length,
2909                options.bloom_bits_per_key,
2910                options.hash_table_ratio,
2911                options.index_sparseness,
2912                options.huge_page_tlb_size,
2913                options.encoding_type as c_char,
2914                c_uchar::from(options.full_scan_mode),
2915                c_uchar::from(options.store_index_in_file),
2916            );
2917        }
2918    }
2919
2920    /// Sets the start level to use compression.
2921    pub fn set_min_level_to_compress(&mut self, lvl: c_int) {
2922        unsafe {
2923            ffi::rocksdb_options_set_min_level_to_compress(self.inner, lvl);
2924        }
2925    }
2926
2927    /// Measure IO stats in compactions and flushes, if `true`.
2928    ///
2929    /// Default: `false`
2930    ///
2931    /// # Examples
2932    ///
2933    /// ```
2934    /// use rust_rocksdb::Options;
2935    ///
2936    /// let mut opts = Options::default();
2937    /// opts.set_report_bg_io_stats(true);
2938    /// ```
2939    pub fn set_report_bg_io_stats(&mut self, enable: bool) {
2940        unsafe {
2941            ffi::rocksdb_options_set_report_bg_io_stats(self.inner, c_int::from(enable));
2942        }
2943    }
2944
2945    /// Once write-ahead logs exceed this size, we will start forcing the flush of
2946    /// column families whose memtables are backed by the oldest live WAL file
2947    /// (i.e. the ones that are causing all the space amplification).
2948    ///
2949    /// Default: `0`
2950    ///
2951    /// # Examples
2952    ///
2953    /// ```
2954    /// use rust_rocksdb::Options;
2955    ///
2956    /// let mut opts = Options::default();
2957    /// // Set max total wal size to 1G.
2958    /// opts.set_max_total_wal_size(1 << 30);
2959    /// ```
2960    pub fn set_max_total_wal_size(&mut self, size: u64) {
2961        unsafe {
2962            ffi::rocksdb_options_set_max_total_wal_size(self.inner, size);
2963        }
2964    }
2965
2966    /// Recovery mode to control the consistency while replaying WAL.
2967    ///
2968    /// Default: DBRecoveryMode::PointInTime
2969    ///
2970    /// # Examples
2971    ///
2972    /// ```
2973    /// use rust_rocksdb::{Options, DBRecoveryMode};
2974    ///
2975    /// let mut opts = Options::default();
2976    /// opts.set_wal_recovery_mode(DBRecoveryMode::AbsoluteConsistency);
2977    /// ```
2978    pub fn set_wal_recovery_mode(&mut self, mode: DBRecoveryMode) {
2979        unsafe {
2980            ffi::rocksdb_options_set_wal_recovery_mode(self.inner, mode as c_int);
2981        }
2982    }
2983
2984    pub fn enable_statistics(&mut self) {
2985        unsafe {
2986            ffi::rocksdb_options_enable_statistics(self.inner);
2987        }
2988    }
2989
2990    pub fn get_statistics(&self) -> Option<String> {
2991        unsafe {
2992            let value = ffi::rocksdb_options_statistics_get_string(self.inner);
2993            if value.is_null() {
2994                return None;
2995            }
2996
2997            // Must have valid UTF-8 format.
2998            Some(from_cstr_and_free(value))
2999        }
3000    }
3001
3002    /// StatsLevel can be used to reduce statistics overhead by skipping certain
3003    /// types of stats in the stats collection process.
3004    pub fn set_statistics_level(&self, level: StatsLevel) {
3005        unsafe { ffi::rocksdb_options_set_statistics_level(self.inner, level as c_int) }
3006    }
3007
3008    /// Returns the value of cumulative db counters if stat collection is enabled.
3009    pub fn get_ticker_count(&self, ticker: Ticker) -> u64 {
3010        unsafe { ffi::rocksdb_options_statistics_get_ticker_count(self.inner, ticker as u32) }
3011    }
3012
3013    /// Gets Histogram data from collected db stats. Requires stats to be enabled.
3014    pub fn get_histogram_data(&self, histogram: Histogram) -> HistogramData {
3015        unsafe {
3016            let data = HistogramData::default();
3017            ffi::rocksdb_options_statistics_get_histogram_data(
3018                self.inner,
3019                histogram as u32,
3020                data.inner,
3021            );
3022            data
3023        }
3024    }
3025
3026    /// If not zero, dump `rocksdb.stats` to LOG every `stats_dump_period_sec`.
3027    ///
3028    /// Default: `600` (10 mins)
3029    ///
3030    /// # Examples
3031    ///
3032    /// ```
3033    /// use rust_rocksdb::Options;
3034    ///
3035    /// let mut opts = Options::default();
3036    /// opts.set_stats_dump_period_sec(300);
3037    /// ```
3038    pub fn set_stats_dump_period_sec(&mut self, period: c_uint) {
3039        unsafe {
3040            ffi::rocksdb_options_set_stats_dump_period_sec(self.inner, period);
3041        }
3042    }
3043
3044    /// If not zero, dump rocksdb.stats to RocksDB to LOG every `stats_persist_period_sec`.
3045    ///
3046    /// Default: `600` (10 mins)
3047    ///
3048    /// # Examples
3049    ///
3050    /// ```
3051    /// use rust_rocksdb::Options;
3052    ///
3053    /// let mut opts = Options::default();
3054    /// opts.set_stats_persist_period_sec(5);
3055    /// ```
3056    pub fn set_stats_persist_period_sec(&mut self, period: c_uint) {
3057        unsafe {
3058            ffi::rocksdb_options_set_stats_persist_period_sec(self.inner, period);
3059        }
3060    }
3061
3062    /// When set to true, reading SST files will opt out of the filesystem's
3063    /// readahead. Setting this to false may improve sequential iteration
3064    /// performance.
3065    ///
3066    /// Default: `true`
3067    pub fn set_advise_random_on_open(&mut self, advise: bool) {
3068        unsafe {
3069            ffi::rocksdb_options_set_advise_random_on_open(self.inner, c_uchar::from(advise));
3070        }
3071    }
3072
3073    /// Enable/disable adaptive mutex, which spins in the user space before resorting to kernel.
3074    ///
3075    /// This could reduce context switch when the mutex is not
3076    /// heavily contended. However, if the mutex is hot, we could end up
3077    /// wasting spin time.
3078    ///
3079    /// Default: false
3080    pub fn set_use_adaptive_mutex(&mut self, enabled: bool) {
3081        unsafe {
3082            ffi::rocksdb_options_set_use_adaptive_mutex(self.inner, c_uchar::from(enabled));
3083        }
3084    }
3085
3086    /// Sets the number of levels for this database.
3087    pub fn set_num_levels(&mut self, n: c_int) {
3088        unsafe {
3089            ffi::rocksdb_options_set_num_levels(self.inner, n);
3090        }
3091    }
3092
3093    /// When a `prefix_extractor` is defined through `opts.set_prefix_extractor` this
3094    /// creates a prefix bloom filter for each memtable with the size of
3095    /// `write_buffer_size * memtable_prefix_bloom_ratio` (capped at 0.25).
3096    ///
3097    /// Default: `0`
3098    ///
3099    /// # Examples
3100    ///
3101    /// ```
3102    /// use rust_rocksdb::{Options, SliceTransform};
3103    ///
3104    /// let mut opts = Options::default();
3105    /// let transform = SliceTransform::create_fixed_prefix(10);
3106    /// opts.set_prefix_extractor(transform);
3107    /// opts.set_memtable_prefix_bloom_ratio(0.2);
3108    /// ```
3109    pub fn set_memtable_prefix_bloom_ratio(&mut self, ratio: f64) {
3110        unsafe {
3111            ffi::rocksdb_options_set_memtable_prefix_bloom_size_ratio(self.inner, ratio);
3112        }
3113    }
3114
3115    /// Sets the maximum number of bytes in all compacted files.
3116    /// We try to limit number of bytes in one compaction to be lower than this
3117    /// threshold. But it's not guaranteed.
3118    ///
3119    /// Value 0 will be sanitized.
3120    ///
3121    /// Default: target_file_size_base * 25
3122    pub fn set_max_compaction_bytes(&mut self, nbytes: u64) {
3123        unsafe {
3124            ffi::rocksdb_options_set_max_compaction_bytes(self.inner, nbytes);
3125        }
3126    }
3127
3128    /// Specifies the absolute path of the directory the
3129    /// write-ahead log (WAL) should be written to.
3130    ///
3131    /// Default: same directory as the database
3132    ///
3133    /// # Examples
3134    ///
3135    /// ```
3136    /// use rust_rocksdb::Options;
3137    ///
3138    /// let mut opts = Options::default();
3139    /// opts.set_wal_dir("/path/to/dir");
3140    /// ```
3141    pub fn set_wal_dir<P: AsRef<Path>>(&mut self, path: P) {
3142        let p = to_cpath(path).unwrap();
3143        unsafe {
3144            ffi::rocksdb_options_set_wal_dir(self.inner, p.as_ptr());
3145        }
3146    }
3147
3148    /// Sets the WAL ttl in seconds.
3149    ///
3150    /// The following two options affect how archived logs will be deleted.
3151    /// 1. If both set to 0, logs will be deleted asap and will not get into
3152    ///    the archive.
3153    /// 2. If wal_ttl_seconds is 0 and wal_size_limit_mb is not 0,
3154    ///    WAL files will be checked every 10 min and if total size is greater
3155    ///    then wal_size_limit_mb, they will be deleted starting with the
3156    ///    earliest until size_limit is met. All empty files will be deleted.
3157    /// 3. If wal_ttl_seconds is not 0 and wall_size_limit_mb is 0, then
3158    ///    WAL files will be checked every wal_ttl_seconds / 2 and those that
3159    ///    are older than wal_ttl_seconds will be deleted.
3160    /// 4. If both are not 0, WAL files will be checked every 10 min and both
3161    ///    checks will be performed with ttl being first.
3162    ///
3163    /// Default: 0
3164    pub fn set_wal_ttl_seconds(&mut self, secs: u64) {
3165        unsafe {
3166            ffi::rocksdb_options_set_WAL_ttl_seconds(self.inner, secs);
3167        }
3168    }
3169
3170    /// Sets the WAL size limit in MB.
3171    ///
3172    /// If total size of WAL files is greater then wal_size_limit_mb,
3173    /// they will be deleted starting with the earliest until size_limit is met.
3174    ///
3175    /// Default: 0
3176    pub fn set_wal_size_limit_mb(&mut self, size: u64) {
3177        unsafe {
3178            ffi::rocksdb_options_set_WAL_size_limit_MB(self.inner, size);
3179        }
3180    }
3181
3182    /// Sets the number of bytes to preallocate (via fallocate) the manifest files.
3183    ///
3184    /// Default is 4MB, which is reasonable to reduce random IO
3185    /// as well as prevent overallocation for mounts that preallocate
3186    /// large amounts of data (such as xfs's allocsize option).
3187    pub fn set_manifest_preallocation_size(&mut self, size: usize) {
3188        unsafe {
3189            ffi::rocksdb_options_set_manifest_preallocation_size(self.inner, size);
3190        }
3191    }
3192
3193    /// If true, then DB::Open() will not update the statistics used to optimize
3194    /// compaction decision by loading table properties from many files.
3195    /// Turning off this feature will improve DBOpen time especially in disk environment.
3196    ///
3197    /// Default: false
3198    pub fn set_skip_stats_update_on_db_open(&mut self, skip: bool) {
3199        unsafe {
3200            ffi::rocksdb_options_set_skip_stats_update_on_db_open(self.inner, c_uchar::from(skip));
3201        }
3202    }
3203
3204    /// Specify the maximal number of info log files to be kept.
3205    ///
3206    /// Default: 1000
3207    ///
3208    /// # Examples
3209    ///
3210    /// ```
3211    /// use rust_rocksdb::Options;
3212    ///
3213    /// let mut options = Options::default();
3214    /// options.set_keep_log_file_num(100);
3215    /// ```
3216    pub fn set_keep_log_file_num(&mut self, nfiles: usize) {
3217        unsafe {
3218            ffi::rocksdb_options_set_keep_log_file_num(self.inner, nfiles);
3219        }
3220    }
3221
3222    /// Allow the OS to mmap file for writing.
3223    ///
3224    /// Default: false
3225    ///
3226    /// # Examples
3227    ///
3228    /// ```
3229    /// use rust_rocksdb::Options;
3230    ///
3231    /// let mut options = Options::default();
3232    /// options.set_allow_mmap_writes(true);
3233    /// ```
3234    pub fn set_allow_mmap_writes(&mut self, is_enabled: bool) {
3235        unsafe {
3236            ffi::rocksdb_options_set_allow_mmap_writes(self.inner, c_uchar::from(is_enabled));
3237        }
3238    }
3239
3240    /// Allow the OS to mmap file for reading sst tables.
3241    ///
3242    /// Default: false
3243    ///
3244    /// # Examples
3245    ///
3246    /// ```
3247    /// use rust_rocksdb::Options;
3248    ///
3249    /// let mut options = Options::default();
3250    /// options.set_allow_mmap_reads(true);
3251    /// ```
3252    pub fn set_allow_mmap_reads(&mut self, is_enabled: bool) {
3253        unsafe {
3254            ffi::rocksdb_options_set_allow_mmap_reads(self.inner, c_uchar::from(is_enabled));
3255        }
3256    }
3257
3258    /// If enabled, WAL is not flushed automatically after each write. Instead it
3259    /// relies on manual invocation of `DB::flush_wal()` to write the WAL buffer
3260    /// to its file.
3261    ///
3262    /// Default: false
3263    ///
3264    /// # Examples
3265    ///
3266    /// ```
3267    /// use rust_rocksdb::Options;
3268    ///
3269    /// let mut options = Options::default();
3270    /// options.set_manual_wal_flush(true);
3271    /// ```
3272    pub fn set_manual_wal_flush(&mut self, is_enabled: bool) {
3273        unsafe {
3274            ffi::rocksdb_options_set_manual_wal_flush(self.inner, c_uchar::from(is_enabled));
3275        }
3276    }
3277
3278    /// Guarantee that all column families are flushed together atomically.
3279    /// This option applies to both manual flushes (`db.flush()`) and automatic
3280    /// background flushes caused when memtables are filled.
3281    ///
3282    /// Note that this is only useful when the WAL is disabled. When using the
3283    /// WAL, writes are always consistent across column families.
3284    ///
3285    /// Default: false
3286    ///
3287    /// # Examples
3288    ///
3289    /// ```
3290    /// use rust_rocksdb::Options;
3291    ///
3292    /// let mut options = Options::default();
3293    /// options.set_atomic_flush(true);
3294    /// ```
3295    pub fn set_atomic_flush(&mut self, atomic_flush: bool) {
3296        unsafe {
3297            ffi::rocksdb_options_set_atomic_flush(self.inner, c_uchar::from(atomic_flush));
3298        }
3299    }
3300
3301    /// Sets global cache for table-level rows.
3302    ///
3303    /// Default: null (disabled)
3304    /// Not supported in ROCKSDB_LITE mode!
3305    pub fn set_row_cache(&mut self, cache: &Cache) {
3306        unsafe {
3307            ffi::rocksdb_options_set_row_cache(self.inner, cache.0.inner.as_ptr());
3308        }
3309        self.outlive.row_cache = Some(cache.clone());
3310    }
3311
3312    /// Use to control write rate of flush and compaction. Flush has higher
3313    /// priority than compaction.
3314    /// If rate limiter is enabled, bytes_per_sync is set to 1MB by default.
3315    ///
3316    /// Default: disable
3317    ///
3318    /// # Examples
3319    ///
3320    /// ```
3321    /// use rust_rocksdb::Options;
3322    ///
3323    /// let mut options = Options::default();
3324    /// options.set_ratelimiter(1024 * 1024, 100 * 1000, 10);
3325    /// ```
3326    pub fn set_ratelimiter(
3327        &mut self,
3328        rate_bytes_per_sec: i64,
3329        refill_period_us: i64,
3330        fairness: i32,
3331    ) {
3332        unsafe {
3333            let ratelimiter =
3334                ffi::rocksdb_ratelimiter_create(rate_bytes_per_sec, refill_period_us, fairness);
3335            ffi::rocksdb_options_set_ratelimiter(self.inner, ratelimiter);
3336            ffi::rocksdb_ratelimiter_destroy(ratelimiter);
3337        }
3338    }
3339
3340    /// Use to control write rate of flush and compaction. Flush has higher
3341    /// priority than compaction.
3342    /// If rate limiter is enabled, bytes_per_sync is set to 1MB by default.
3343    ///
3344    /// Default: disable
3345    pub fn set_auto_tuned_ratelimiter(
3346        &mut self,
3347        rate_bytes_per_sec: i64,
3348        refill_period_us: i64,
3349        fairness: i32,
3350    ) {
3351        unsafe {
3352            let ratelimiter = ffi::rocksdb_ratelimiter_create_auto_tuned(
3353                rate_bytes_per_sec,
3354                refill_period_us,
3355                fairness,
3356            );
3357            ffi::rocksdb_options_set_ratelimiter(self.inner, ratelimiter);
3358            ffi::rocksdb_ratelimiter_destroy(ratelimiter);
3359        }
3360    }
3361
3362    /// Create a RateLimiter object, which can be shared among RocksDB instances to
3363    /// control write rate of flush and compaction.
3364    ///
3365    /// rate_bytes_per_sec: this is the only parameter you want to set most of the
3366    /// time. It controls the total write rate of compaction and flush in bytes per
3367    /// second. Currently, RocksDB does not enforce rate limit for anything other
3368    /// than flush and compaction, e.g. write to WAL.
3369    ///
3370    /// refill_period_us: this controls how often tokens are refilled. For example,
3371    /// when rate_bytes_per_sec is set to 10MB/s and refill_period_us is set to
3372    /// 100ms, then 1MB is refilled every 100ms internally. Larger value can lead to
3373    /// burstier writes while smaller value introduces more CPU overhead.
3374    /// The default should work for most cases.
3375    ///
3376    /// fairness: RateLimiter accepts high-pri requests and low-pri requests.
3377    /// A low-pri request is usually blocked in favor of hi-pri request. Currently,
3378    /// RocksDB assigns low-pri to request from compaction and high-pri to request
3379    /// from flush. Low-pri requests can get blocked if flush requests come in
3380    /// continuously. This fairness parameter grants low-pri requests permission by
3381    /// 1/fairness chance even though high-pri requests exist to avoid starvation.
3382    /// You should be good by leaving it at default 10.
3383    ///
3384    /// mode: Mode indicates which types of operations count against the limit.
3385    ///
3386    /// auto_tuned: Enables dynamic adjustment of rate limit within the range
3387    ///              `[rate_bytes_per_sec / 20, rate_bytes_per_sec]`, according to
3388    ///              the recent demand for background I/O.
3389    pub fn set_ratelimiter_with_mode(
3390        &mut self,
3391        rate_bytes_per_sec: i64,
3392        refill_period_us: i64,
3393        fairness: i32,
3394        mode: RateLimiterMode,
3395        auto_tuned: bool,
3396    ) {
3397        unsafe {
3398            let ratelimiter = ffi::rocksdb_ratelimiter_create_with_mode(
3399                rate_bytes_per_sec,
3400                refill_period_us,
3401                fairness,
3402                mode as c_int,
3403                auto_tuned,
3404            );
3405            ffi::rocksdb_options_set_ratelimiter(self.inner, ratelimiter);
3406            ffi::rocksdb_ratelimiter_destroy(ratelimiter);
3407        }
3408    }
3409
3410    /// Sets the maximal size of the info log file.
3411    ///
3412    /// If the log file is larger than `max_log_file_size`, a new info log file
3413    /// will be created. If `max_log_file_size` is equal to zero, all logs will
3414    /// be written to one log file.
3415    ///
3416    /// Default: 0
3417    ///
3418    /// # Examples
3419    ///
3420    /// ```
3421    /// use rust_rocksdb::Options;
3422    ///
3423    /// let mut options = Options::default();
3424    /// options.set_max_log_file_size(0);
3425    /// ```
3426    pub fn set_max_log_file_size(&mut self, size: usize) {
3427        unsafe {
3428            ffi::rocksdb_options_set_max_log_file_size(self.inner, size);
3429        }
3430    }
3431
3432    /// Sets the time for the info log file to roll (in seconds).
3433    ///
3434    /// If specified with non-zero value, log file will be rolled
3435    /// if it has been active longer than `log_file_time_to_roll`.
3436    /// Default: 0 (disabled)
3437    pub fn set_log_file_time_to_roll(&mut self, secs: usize) {
3438        unsafe {
3439            ffi::rocksdb_options_set_log_file_time_to_roll(self.inner, secs);
3440        }
3441    }
3442
3443    /// Controls the recycling of log files.
3444    ///
3445    /// If non-zero, previously written log files will be reused for new logs,
3446    /// overwriting the old data. The value indicates how many such files we will
3447    /// keep around at any point in time for later use. This is more efficient
3448    /// because the blocks are already allocated and fdatasync does not need to
3449    /// update the inode after each write.
3450    ///
3451    /// Default: 0
3452    ///
3453    /// # Examples
3454    ///
3455    /// ```
3456    /// use rust_rocksdb::Options;
3457    ///
3458    /// let mut options = Options::default();
3459    /// options.set_recycle_log_file_num(5);
3460    /// ```
3461    pub fn set_recycle_log_file_num(&mut self, num: usize) {
3462        unsafe {
3463            ffi::rocksdb_options_set_recycle_log_file_num(self.inner, num);
3464        }
3465    }
3466
3467    /// Prints logs to stderr for faster debugging
3468    /// See official [wiki](https://github.com/facebook/rocksdb/wiki/Logger) for more information.
3469    pub fn set_stderr_logger(&mut self, log_level: LogLevel, prefix: impl CStrLike) {
3470        let p = prefix.into_c_string().unwrap();
3471
3472        unsafe {
3473            let logger = ffi::rocksdb_logger_create_stderr_logger(log_level as c_int, p.as_ptr());
3474            ffi::rocksdb_options_set_info_log(self.inner, logger);
3475            ffi::rocksdb_logger_destroy(logger);
3476        }
3477    }
3478
3479    /// Invokes `callback` with RocksDB log messages with level >= `log_level`.
3480    ///
3481    /// The callback can be called concurrently by multiple RocksDB threads.
3482    ///
3483    /// # Examples
3484    /// ```
3485    /// use rust_rocksdb::{LogLevel, Options};
3486    ///
3487    /// let mut options = Options::default();
3488    /// options.set_callback_logger(LogLevel::Debug, move |level, msg| println!("{level:?} {msg}"));
3489    /// ```
3490    pub fn set_callback_logger(
3491        &mut self,
3492        log_level: LogLevel,
3493        callback: impl Fn(LogLevel, &str) + 'static + Send + Sync,
3494    ) {
3495        // store the closure in an Arc so it can be shared across multiple Option/DBs
3496        let holder = Arc::new(LogCallback {
3497            callback: Box::new(callback),
3498        });
3499        let holder_ptr = holder.as_ref() as *const LogCallback;
3500        let holder_cvoid = holder_ptr.cast::<c_void>().cast_mut();
3501
3502        unsafe {
3503            let logger = ffi::rocksdb_logger_create_callback_logger(
3504                log_level as c_int,
3505                Some(Self::logger_callback),
3506                holder_cvoid,
3507            );
3508            ffi::rocksdb_options_set_info_log(self.inner, logger);
3509            ffi::rocksdb_logger_destroy(logger);
3510        }
3511
3512        self.outlive.log_callback = Some(holder);
3513    }
3514
3515    extern "C" fn logger_callback(func: *mut c_void, level: u32, msg: *mut c_char, len: usize) {
3516        use std::{mem, process, str};
3517
3518        let level = unsafe { mem::transmute::<u32, LogLevel>(level) };
3519        let slice = unsafe { slice::from_raw_parts_mut(msg.cast::<u8>(), len) };
3520        let msg = unsafe { str::from_utf8_unchecked(slice) };
3521
3522        let holder = unsafe { &mut *func.cast::<LogCallback>() };
3523        let mut callback_in_catch_unwind = AssertUnwindSafe(&mut holder.callback);
3524        if catch_unwind(move || callback_in_catch_unwind(level, msg)).is_err() {
3525            process::abort();
3526        }
3527    }
3528
3529    /// Sets the threshold at which all writes will be slowed down to at least delayed_write_rate if estimated
3530    /// bytes needed to be compaction exceed this threshold.
3531    ///
3532    /// Default: 64GB
3533    pub fn set_soft_pending_compaction_bytes_limit(&mut self, limit: usize) {
3534        unsafe {
3535            ffi::rocksdb_options_set_soft_pending_compaction_bytes_limit(self.inner, limit);
3536        }
3537    }
3538
3539    /// Sets the bytes threshold at which all writes are stopped if estimated bytes needed to be compaction exceed
3540    /// this threshold.
3541    ///
3542    /// Default: 256GB
3543    pub fn set_hard_pending_compaction_bytes_limit(&mut self, limit: usize) {
3544        unsafe {
3545            ffi::rocksdb_options_set_hard_pending_compaction_bytes_limit(self.inner, limit);
3546        }
3547    }
3548
3549    /// Sets the size of one block in arena memory allocation.
3550    ///
3551    /// If <= 0, a proper value is automatically calculated (usually 1/10 of
3552    /// writer_buffer_size).
3553    ///
3554    /// Default: 0
3555    pub fn set_arena_block_size(&mut self, size: usize) {
3556        unsafe {
3557            ffi::rocksdb_options_set_arena_block_size(self.inner, size);
3558        }
3559    }
3560
3561    /// If true, then print malloc stats together with rocksdb.stats when printing to LOG.
3562    ///
3563    /// Default: false
3564    pub fn set_dump_malloc_stats(&mut self, enabled: bool) {
3565        unsafe {
3566            ffi::rocksdb_options_set_dump_malloc_stats(self.inner, c_uchar::from(enabled));
3567        }
3568    }
3569
3570    /// Enable whole key bloom filter in memtable. Note this will only take effect
3571    /// if memtable_prefix_bloom_size_ratio is not 0. Enabling whole key filtering
3572    /// can potentially reduce CPU usage for point-look-ups.
3573    ///
3574    /// Default: false (disable)
3575    ///
3576    /// Dynamically changeable through SetOptions() API
3577    pub fn set_memtable_whole_key_filtering(&mut self, whole_key_filter: bool) {
3578        unsafe {
3579            ffi::rocksdb_options_set_memtable_whole_key_filtering(
3580                self.inner,
3581                c_uchar::from(whole_key_filter),
3582            );
3583        }
3584    }
3585
3586    /// Enable the use of key-value separation.
3587    ///
3588    /// More details can be found here: [Integrated BlobDB](http://rocksdb.org/blog/2021/05/26/integrated-blob-db.html).
3589    ///
3590    /// Default: false (disable)
3591    ///
3592    /// Dynamically changeable through SetOptions() API
3593    pub fn set_enable_blob_files(&mut self, val: bool) {
3594        unsafe {
3595            ffi::rocksdb_options_set_enable_blob_files(self.inner, u8::from(val));
3596        }
3597    }
3598
3599    /// Sets the minimum threshold value at or above which will be written
3600    /// to blob files during flush or compaction.
3601    ///
3602    /// Dynamically changeable through SetOptions() API
3603    pub fn set_min_blob_size(&mut self, val: u64) {
3604        unsafe {
3605            ffi::rocksdb_options_set_min_blob_size(self.inner, val);
3606        }
3607    }
3608
3609    /// Sets the size limit for blob files.
3610    ///
3611    /// Dynamically changeable through SetOptions() API
3612    pub fn set_blob_file_size(&mut self, val: u64) {
3613        unsafe {
3614            ffi::rocksdb_options_set_blob_file_size(self.inner, val);
3615        }
3616    }
3617
3618    /// Sets the blob compression type. All blob files use the same
3619    /// compression type.
3620    ///
3621    /// Dynamically changeable through SetOptions() API
3622    pub fn set_blob_compression_type(&mut self, val: DBCompressionType) {
3623        unsafe {
3624            ffi::rocksdb_options_set_blob_compression_type(self.inner, val as _);
3625        }
3626    }
3627
3628    /// If this is set to true RocksDB will actively relocate valid blobs from the oldest blob files
3629    /// as they are encountered during compaction.
3630    ///
3631    /// Dynamically changeable through SetOptions() API
3632    pub fn set_enable_blob_gc(&mut self, val: bool) {
3633        unsafe {
3634            ffi::rocksdb_options_set_enable_blob_gc(self.inner, u8::from(val));
3635        }
3636    }
3637
3638    /// Sets the threshold that the GC logic uses to determine which blob files should be considered “old.”
3639    ///
3640    /// For example, the default value of 0.25 signals to RocksDB that blobs residing in the
3641    /// oldest 25% of blob files should be relocated by GC. This parameter can be tuned to adjust
3642    /// the trade-off between write amplification and space amplification.
3643    ///
3644    /// Dynamically changeable through SetOptions() API
3645    pub fn set_blob_gc_age_cutoff(&mut self, val: c_double) {
3646        unsafe {
3647            ffi::rocksdb_options_set_blob_gc_age_cutoff(self.inner, val);
3648        }
3649    }
3650
3651    /// Sets the blob GC force threshold.
3652    ///
3653    /// Dynamically changeable through SetOptions() API
3654    pub fn set_blob_gc_force_threshold(&mut self, val: c_double) {
3655        unsafe {
3656            ffi::rocksdb_options_set_blob_gc_force_threshold(self.inner, val);
3657        }
3658    }
3659
3660    /// Sets the blob compaction read ahead size.
3661    ///
3662    /// Dynamically changeable through SetOptions() API
3663    pub fn set_blob_compaction_readahead_size(&mut self, val: u64) {
3664        unsafe {
3665            ffi::rocksdb_options_set_blob_compaction_readahead_size(self.inner, val);
3666        }
3667    }
3668
3669    /// Sets the blob cache.
3670    ///
3671    /// Using a dedicated object for blobs and using the same object for the block and blob caches
3672    /// are both supported. In the latter case, note that blobs are less valuable from a caching
3673    /// perspective than SST blocks, and some cache implementations have configuration options that
3674    /// can be used to prioritize items accordingly (see Cache::Priority and
3675    /// LRUCacheOptions::{high,low}_pri_pool_ratio).
3676    ///
3677    /// Default: disabled
3678    pub fn set_blob_cache(&mut self, cache: &Cache) {
3679        unsafe {
3680            ffi::rocksdb_options_set_blob_cache(self.inner, cache.0.inner.as_ptr());
3681        }
3682        self.outlive.blob_cache = Some(cache.clone());
3683    }
3684
3685    /// Set this option to true during creation of database if you want
3686    /// to be able to ingest behind (call IngestExternalFile() skipping keys
3687    /// that already exist, rather than overwriting matching keys).
3688    /// Setting this option to true has the following effects:
3689    ///
3690    /// 1. Disable some internal optimizations around SST file compression.
3691    /// 2. Reserve the last level for ingested files only.
3692    /// 3. Compaction will not include any file from the last level.
3693    ///
3694    /// Note that only Universal Compaction supports allow_ingest_behind.
3695    /// `num_levels` should be >= 3 if this option is turned on.
3696    ///
3697    /// DEFAULT: false
3698    /// Immutable.
3699    pub fn set_allow_ingest_behind(&mut self, val: bool) {
3700        unsafe {
3701            ffi::rocksdb_options_set_allow_ingest_behind(self.inner, c_uchar::from(val));
3702        }
3703    }
3704
3705    // A factory of a table property collector that marks an SST
3706    // file as need-compaction when it observe at least "D" deletion
3707    // entries in any "N" consecutive entries, or the ratio of tombstone
3708    // entries >= deletion_ratio.
3709    //
3710    // `window_size`: is the sliding window size "N"
3711    // `num_dels_trigger`: is the deletion trigger "D"
3712    // `deletion_ratio`: if <= 0 or > 1, disable triggering compaction based on
3713    // deletion ratio.
3714    pub fn add_compact_on_deletion_collector_factory(
3715        &mut self,
3716        window_size: size_t,
3717        num_dels_trigger: size_t,
3718        deletion_ratio: f64,
3719    ) {
3720        unsafe {
3721            ffi::rocksdb_options_add_compact_on_deletion_collector_factory_del_ratio(
3722                self.inner,
3723                window_size,
3724                num_dels_trigger,
3725                deletion_ratio,
3726            );
3727        }
3728    }
3729
3730    /// Like [`Self::add_compact_on_deletion_collector_factory`], but only triggers
3731    /// compaction if the SST file size is at least `min_file_size` bytes.
3732    pub fn add_compact_on_deletion_collector_factory_min_file_size(
3733        &mut self,
3734        window_size: size_t,
3735        num_dels_trigger: size_t,
3736        deletion_ratio: f64,
3737        min_file_size: u64,
3738    ) {
3739        unsafe {
3740            ffi::rocksdb_options_add_compact_on_deletion_collector_factory_min_file_size(
3741                self.inner,
3742                window_size,
3743                num_dels_trigger,
3744                deletion_ratio,
3745                min_file_size,
3746            );
3747        }
3748    }
3749
3750    /// <https://github.com/facebook/rocksdb/wiki/Write-Buffer-Manager>
3751    /// Write buffer manager helps users control the total memory used by memtables across multiple column families and/or DB instances.
3752    /// Users can enable this control by 2 ways:
3753    ///
3754    /// 1- Limit the total memtable usage across multiple column families and DBs under a threshold.
3755    /// 2- Cost the memtable memory usage to block cache so that memory of RocksDB can be capped by the single limit.
3756    /// The usage of a write buffer manager is similar to rate_limiter and sst_file_manager.
3757    /// Users can create one write buffer manager object and pass it to all the options of column families or DBs whose memtable size they want to be controlled by this object.
3758    pub fn set_write_buffer_manager(&mut self, write_buffer_manager: &WriteBufferManager) {
3759        unsafe {
3760            ffi::rocksdb_options_set_write_buffer_manager(
3761                self.inner,
3762                write_buffer_manager.0.inner.as_ptr(),
3763            );
3764        }
3765        self.outlive.write_buffer_manager = Some(write_buffer_manager.clone());
3766    }
3767
3768    /// Sets an `SstFileManager` for this `Options`.
3769    ///
3770    /// SstFileManager tracks and controls total SST file space usage, enabling
3771    /// applications to cap disk utilization and throttle deletions.
3772    pub fn set_sst_file_manager(&mut self, sst_file_manager: &SstFileManager) {
3773        unsafe {
3774            ffi::rocksdb_options_set_sst_file_manager(
3775                self.inner,
3776                sst_file_manager.0.inner.as_ptr(),
3777            );
3778        }
3779        self.outlive.sst_file_manager = Some(sst_file_manager.clone());
3780    }
3781
3782    /// If true, working thread may avoid doing unnecessary and long-latency
3783    /// operation (such as deleting obsolete files directly or deleting memtable)
3784    /// and will instead schedule a background job to do it.
3785    ///
3786    /// Use it if you're latency-sensitive.
3787    ///
3788    /// Default: false (disabled)
3789    pub fn set_avoid_unnecessary_blocking_io(&mut self, val: bool) {
3790        unsafe {
3791            ffi::rocksdb_options_set_avoid_unnecessary_blocking_io(self.inner, u8::from(val));
3792        }
3793    }
3794
3795    /// Sets the compaction priority.
3796    ///
3797    /// If level compaction_style =
3798    /// kCompactionStyleLevel, for each level, which files are prioritized to be
3799    /// picked to compact.
3800    ///
3801    /// Default: `DBCompactionPri::MinOverlappingRatio`
3802    ///
3803    /// # Examples
3804    ///
3805    /// ```
3806    /// use rust_rocksdb::{Options, DBCompactionPri};
3807    ///
3808    /// let mut opts = Options::default();
3809    /// opts.set_compaction_pri(DBCompactionPri::RoundRobin);
3810    /// ```
3811    pub fn set_compaction_pri(&mut self, pri: DBCompactionPri) {
3812        unsafe {
3813            ffi::rocksdb_options_set_compaction_pri(self.inner, pri as c_int);
3814        }
3815    }
3816
3817    /// If true, the log numbers and sizes of the synced WALs are tracked
3818    /// in MANIFEST. During DB recovery, if a synced WAL is missing
3819    /// from disk, or the WAL's size does not match the recorded size in
3820    /// MANIFEST, an error will be reported and the recovery will be aborted.
3821    ///
3822    /// This is one additional protection against WAL corruption besides the
3823    /// per-WAL-entry checksum.
3824    ///
3825    /// Note that this option does not work with secondary instance.
3826    /// Currently, only syncing closed WALs are tracked. Calling `DB::SyncWAL()`,
3827    /// etc. or writing with `WriteOptions::sync=true` to sync the live WAL is not
3828    /// tracked for performance/efficiency reasons.
3829    ///
3830    /// See: <https://github.com/facebook/rocksdb/wiki/Track-WAL-in-MANIFEST>
3831    ///
3832    /// Default: false (disabled)
3833    pub fn set_track_and_verify_wals_in_manifest(&mut self, val: bool) {
3834        unsafe {
3835            ffi::rocksdb_options_set_track_and_verify_wals_in_manifest(self.inner, u8::from(val));
3836        }
3837    }
3838
3839    /// Returns the value of the `track_and_verify_wals_in_manifest` option.
3840    pub fn get_track_and_verify_wals_in_manifest(&self) -> bool {
3841        let val_u8 =
3842            unsafe { ffi::rocksdb_options_get_track_and_verify_wals_in_manifest(self.inner) };
3843        val_u8 != 0
3844    }
3845
3846    /// The DB unique ID can be saved in the DB manifest (preferred, this option)
3847    /// or an IDENTITY file (historical, deprecated), or both. If this option is
3848    /// set to false (old behavior), then `write_identity_file` must be set to true.
3849    /// The manifest is preferred because
3850    ///
3851    /// 1. The IDENTITY file is not checksummed, so it is not as safe against
3852    ///    corruption.
3853    /// 2. The IDENTITY file may or may not be copied with the DB (e.g. not
3854    ///    copied by BackupEngine), so is not reliable for the provenance of a DB.
3855    ///
3856    /// This option might eventually be obsolete and removed as Identity files
3857    /// are phased out.
3858    ///
3859    /// Default: true (enabled)
3860    pub fn set_write_dbid_to_manifest(&mut self, val: bool) {
3861        unsafe {
3862            ffi::rocksdb_options_set_write_dbid_to_manifest(self.inner, u8::from(val));
3863        }
3864    }
3865
3866    /// Returns the value of the `write_dbid_to_manifest` option.
3867    pub fn get_write_dbid_to_manifest(&self) -> bool {
3868        let val_u8 = unsafe { ffi::rocksdb_options_get_write_dbid_to_manifest(self.inner) };
3869        val_u8 != 0
3870    }
3871}
3872
3873impl Default for Options {
3874    fn default() -> Self {
3875        unsafe {
3876            let opts = ffi::rocksdb_options_create();
3877            assert!(!opts.is_null(), "Could not create RocksDB options");
3878
3879            Self {
3880                inner: opts,
3881                outlive: OptionsMustOutliveDB::default(),
3882            }
3883        }
3884    }
3885}
3886
3887impl FlushOptions {
3888    pub fn new() -> FlushOptions {
3889        FlushOptions::default()
3890    }
3891
3892    /// Waits until the flush is done.
3893    ///
3894    /// Default: true
3895    ///
3896    /// # Examples
3897    ///
3898    /// ```
3899    /// use rust_rocksdb::FlushOptions;
3900    ///
3901    /// let mut options = FlushOptions::default();
3902    /// options.set_wait(false);
3903    /// ```
3904    pub fn set_wait(&mut self, wait: bool) {
3905        unsafe {
3906            ffi::rocksdb_flushoptions_set_wait(self.inner, c_uchar::from(wait));
3907        }
3908    }
3909}
3910
3911impl Default for FlushOptions {
3912    fn default() -> Self {
3913        let flush_opts = unsafe { ffi::rocksdb_flushoptions_create() };
3914        assert!(
3915            !flush_opts.is_null(),
3916            "Could not create RocksDB flush options"
3917        );
3918
3919        Self { inner: flush_opts }
3920    }
3921}
3922
3923impl WriteOptions {
3924    pub fn new() -> WriteOptions {
3925        WriteOptions::default()
3926    }
3927
3928    /// Sets the sync mode. If true, the write will be flushed
3929    /// from the operating system buffer cache before the write is considered complete.
3930    /// If this flag is true, writes will be slower.
3931    ///
3932    /// Default: false
3933    pub fn set_sync(&mut self, sync: bool) {
3934        unsafe {
3935            ffi::rocksdb_writeoptions_set_sync(self.inner, c_uchar::from(sync));
3936        }
3937    }
3938
3939    /// Sets whether WAL should be active or not.
3940    /// If true, writes will not first go to the write ahead log,
3941    /// and the write may got lost after a crash.
3942    ///
3943    /// Default: false
3944    pub fn disable_wal(&mut self, disable: bool) {
3945        unsafe {
3946            ffi::rocksdb_writeoptions_disable_WAL(self.inner, c_int::from(disable));
3947        }
3948    }
3949
3950    /// If true and if user is trying to write to column families that don't exist (they were dropped),
3951    /// ignore the write (don't return an error). If there are multiple writes in a WriteBatch,
3952    /// other writes will succeed.
3953    ///
3954    /// Default: false
3955    pub fn set_ignore_missing_column_families(&mut self, ignore: bool) {
3956        unsafe {
3957            ffi::rocksdb_writeoptions_set_ignore_missing_column_families(
3958                self.inner,
3959                c_uchar::from(ignore),
3960            );
3961        }
3962    }
3963
3964    /// If true and we need to wait or sleep for the write request, fails
3965    /// immediately with Status::Incomplete().
3966    ///
3967    /// Default: false
3968    pub fn set_no_slowdown(&mut self, no_slowdown: bool) {
3969        unsafe {
3970            ffi::rocksdb_writeoptions_set_no_slowdown(self.inner, c_uchar::from(no_slowdown));
3971        }
3972    }
3973
3974    /// If true, this write request is of lower priority if compaction is
3975    /// behind. In this case, no_slowdown = true, the request will be cancelled
3976    /// immediately with Status::Incomplete() returned. Otherwise, it will be
3977    /// slowed down. The slowdown value is determined by RocksDB to guarantee
3978    /// it introduces minimum impacts to high priority writes.
3979    ///
3980    /// Default: false
3981    pub fn set_low_pri(&mut self, v: bool) {
3982        unsafe {
3983            ffi::rocksdb_writeoptions_set_low_pri(self.inner, c_uchar::from(v));
3984        }
3985    }
3986
3987    /// If true, writebatch will maintain the last insert positions of each
3988    /// memtable as hints in concurrent write. It can improve write performance
3989    /// in concurrent writes if keys in one writebatch are sequential. In
3990    /// non-concurrent writes (when concurrent_memtable_writes is false) this
3991    /// option will be ignored.
3992    ///
3993    /// Default: false
3994    pub fn set_memtable_insert_hint_per_batch(&mut self, v: bool) {
3995        unsafe {
3996            ffi::rocksdb_writeoptions_set_memtable_insert_hint_per_batch(
3997                self.inner,
3998                c_uchar::from(v),
3999            );
4000        }
4001    }
4002}
4003
4004impl Default for WriteOptions {
4005    fn default() -> Self {
4006        let write_opts = unsafe { ffi::rocksdb_writeoptions_create() };
4007        assert!(
4008            !write_opts.is_null(),
4009            "Could not create RocksDB write options"
4010        );
4011
4012        Self { inner: write_opts }
4013    }
4014}
4015
4016impl LruCacheOptions {
4017    /// Capacity of the cache, in the same units as the `charge` of each entry.
4018    /// This is typically measured in bytes, but can be a different unit if using
4019    /// kDontChargeCacheMetadata.
4020    pub fn set_capacity(&mut self, cap: usize) {
4021        unsafe {
4022            ffi::rocksdb_lru_cache_options_set_capacity(self.inner, cap);
4023        }
4024    }
4025
4026    /// Cache is sharded into 2^num_shard_bits shards, by hash of key.
4027    /// If < 0, a good default is chosen based on the capacity and the
4028    /// implementation. (Mutex-based implementations are much more reliant
4029    /// on many shards for parallel scalability.)
4030    pub fn set_num_shard_bits(&mut self, val: c_int) {
4031        unsafe {
4032            ffi::rocksdb_lru_cache_options_set_num_shard_bits(self.inner, val);
4033        }
4034    }
4035}
4036
4037impl Default for LruCacheOptions {
4038    fn default() -> Self {
4039        let inner = unsafe { ffi::rocksdb_lru_cache_options_create() };
4040        assert!(
4041            !inner.is_null(),
4042            "Could not create RocksDB LRU cache options"
4043        );
4044
4045        Self { inner }
4046    }
4047}
4048
4049#[derive(Debug, Copy, Clone, PartialEq, Eq)]
4050#[cfg_attr(feature = "serde1", derive(serde::Serialize, serde::Deserialize))]
4051#[repr(i32)]
4052pub enum ReadTier {
4053    /// Reads data in memtable, block cache, OS cache or storage.
4054    All = 0,
4055    /// Reads data in memtable or block cache.
4056    BlockCache,
4057    /// Reads persisted data. When WAL is disabled, this option will skip data in memtable.
4058    Persisted,
4059    /// Reads data in memtable. Used for memtable only iterators.
4060    Memtable,
4061}
4062
4063impl ReadOptions {
4064    // TODO add snapshot setting here
4065    // TODO add snapshot wrapper structs with proper destructors;
4066    // that struct needs an "iterator" impl too.
4067
4068    /// Specify whether the "data block"/"index block"/"filter block"
4069    /// read for this iteration should be cached in memory?
4070    /// Callers may wish to set this field to false for bulk scans.
4071    ///
4072    /// Default: true
4073    pub fn fill_cache(&mut self, v: bool) {
4074        unsafe {
4075            ffi::rocksdb_readoptions_set_fill_cache(self.inner, c_uchar::from(v));
4076        }
4077    }
4078
4079    /// Sets the snapshot which should be used for the read.
4080    /// The snapshot must belong to the DB that is being read and must
4081    /// not have been released.
4082    pub fn set_snapshot<D: DBAccess>(&mut self, snapshot: &SnapshotWithThreadMode<D>) {
4083        unsafe {
4084            ffi::rocksdb_readoptions_set_snapshot(self.inner, snapshot.inner);
4085        }
4086    }
4087
4088    /// Sets the lower bound for an iterator.
4089    pub fn set_iterate_lower_bound<K: Into<Vec<u8>>>(&mut self, key: K) {
4090        self.set_lower_bound_impl(Some(key.into()));
4091    }
4092
4093    /// Sets the upper bound for an iterator.
4094    /// The upper bound itself is not included on the iteration result.
4095    pub fn set_iterate_upper_bound<K: Into<Vec<u8>>>(&mut self, key: K) {
4096        self.set_upper_bound_impl(Some(key.into()));
4097    }
4098
4099    /// Sets lower and upper bounds based on the provided range.  This is
4100    /// similar to setting lower and upper bounds separately except that it also
4101    /// allows either bound to be reset.
4102    ///
4103    /// The argument can be a regular Rust range, e.g. `lower..upper`.  However,
4104    /// since RocksDB upper bound is always excluded (i.e. range can never be
4105    /// fully closed) inclusive ranges (`lower..=upper` and `..=upper`) are not
4106    /// supported.  For example:
4107    ///
4108    /// ```
4109    /// let mut options = rust_rocksdb::ReadOptions::default();
4110    /// options.set_iterate_range("xy".as_bytes().."xz".as_bytes());
4111    /// ```
4112    ///
4113    /// In addition, [`crate::PrefixRange`] can be used to specify a range of
4114    /// keys with a given prefix.  In particular, the above example is
4115    /// equivalent to:
4116    ///
4117    /// ```
4118    /// let mut options = rust_rocksdb::ReadOptions::default();
4119    /// options.set_iterate_range(rust_rocksdb::PrefixRange("xy".as_bytes()));
4120    /// ```
4121    ///
4122    /// Note that setting range using this method is separate to using prefix
4123    /// iterators.  Prefix iterators use prefix extractor configured for
4124    /// a column family.  Setting bounds via [`crate::PrefixRange`] is more akin
4125    /// to using manual prefix.
4126    ///
4127    /// Using this method clears any previously set bounds.  In other words, the
4128    /// bounds can be reset by setting the range to `..` as in:
4129    ///
4130    /// ```
4131    /// let mut options = rust_rocksdb::ReadOptions::default();
4132    /// options.set_iterate_range(..);
4133    /// ```
4134    pub fn set_iterate_range(&mut self, range: impl crate::IterateBounds) {
4135        let (lower, upper) = range.into_bounds();
4136        self.set_lower_bound_impl(lower);
4137        self.set_upper_bound_impl(upper);
4138    }
4139
4140    fn set_lower_bound_impl(&mut self, bound: Option<Vec<u8>>) {
4141        let (ptr, len) = if let Some(ref bound) = bound {
4142            (bound.as_ptr() as *const c_char, bound.len())
4143        } else if self.iterate_lower_bound.is_some() {
4144            (std::ptr::null(), 0)
4145        } else {
4146            return;
4147        };
4148        self.iterate_lower_bound = bound;
4149        unsafe {
4150            ffi::rocksdb_readoptions_set_iterate_lower_bound(self.inner, ptr, len);
4151        }
4152    }
4153
4154    fn set_upper_bound_impl(&mut self, bound: Option<Vec<u8>>) {
4155        let (ptr, len) = if let Some(ref bound) = bound {
4156            (bound.as_ptr() as *const c_char, bound.len())
4157        } else if self.iterate_upper_bound.is_some() {
4158            (std::ptr::null(), 0)
4159        } else {
4160            return;
4161        };
4162        self.iterate_upper_bound = bound;
4163        unsafe {
4164            ffi::rocksdb_readoptions_set_iterate_upper_bound(self.inner, ptr, len);
4165        }
4166    }
4167
4168    /// Specify if this read request should process data that ALREADY
4169    /// resides on a particular cache. If the required data is not
4170    /// found at the specified cache, then Status::Incomplete is returned.
4171    ///
4172    /// Default: ::All
4173    pub fn set_read_tier(&mut self, tier: ReadTier) {
4174        unsafe {
4175            ffi::rocksdb_readoptions_set_read_tier(self.inner, tier as c_int);
4176        }
4177    }
4178
4179    /// Enforce that the iterator only iterates over the same
4180    /// prefix as the seek.
4181    /// This option is effective only for prefix seeks, i.e. prefix_extractor is
4182    /// non-null for the column family and total_order_seek is false.  Unlike
4183    /// iterate_upper_bound, prefix_same_as_start only works within a prefix
4184    /// but in both directions.
4185    ///
4186    /// Default: false
4187    pub fn set_prefix_same_as_start(&mut self, v: bool) {
4188        unsafe {
4189            ffi::rocksdb_readoptions_set_prefix_same_as_start(self.inner, c_uchar::from(v));
4190        }
4191    }
4192
4193    /// Enable a total order seek regardless of index format (e.g. hash index)
4194    /// used in the table. Some table format (e.g. plain table) may not support
4195    /// this option.
4196    ///
4197    /// If true when calling Get(), we also skip prefix bloom when reading from
4198    /// block based table. It provides a way to read existing data after
4199    /// changing implementation of prefix extractor.
4200    pub fn set_total_order_seek(&mut self, v: bool) {
4201        unsafe {
4202            ffi::rocksdb_readoptions_set_total_order_seek(self.inner, c_uchar::from(v));
4203        }
4204    }
4205
4206    /// Sets a threshold for the number of keys that can be skipped
4207    /// before failing an iterator seek as incomplete. The default value of 0 should be used to
4208    /// never fail a request as incomplete, even on skipping too many keys.
4209    ///
4210    /// Default: 0
4211    pub fn set_max_skippable_internal_keys(&mut self, num: u64) {
4212        unsafe {
4213            ffi::rocksdb_readoptions_set_max_skippable_internal_keys(self.inner, num);
4214        }
4215    }
4216
4217    /// If true, when PurgeObsoleteFile is called in CleanupIteratorState, we schedule a background job
4218    /// in the flush job queue and delete obsolete files in background.
4219    ///
4220    /// Default: false
4221    pub fn set_background_purge_on_iterator_cleanup(&mut self, v: bool) {
4222        unsafe {
4223            ffi::rocksdb_readoptions_set_background_purge_on_iterator_cleanup(
4224                self.inner,
4225                c_uchar::from(v),
4226            );
4227        }
4228    }
4229
4230    /// If true, keys deleted using the DeleteRange() API will be visible to
4231    /// readers until they are naturally deleted during compaction.
4232    ///
4233    /// Default: false
4234    #[deprecated(
4235        note = "deprecated in RocksDB 10.2.1: no performance impact if DeleteRange is not used"
4236    )]
4237    pub fn set_ignore_range_deletions(&mut self, v: bool) {
4238        unsafe {
4239            ffi::rocksdb_readoptions_set_ignore_range_deletions(self.inner, c_uchar::from(v));
4240        }
4241    }
4242
4243    /// If true, all data read from underlying storage will be
4244    /// verified against corresponding checksums.
4245    ///
4246    /// Default: true
4247    pub fn set_verify_checksums(&mut self, v: bool) {
4248        unsafe {
4249            ffi::rocksdb_readoptions_set_verify_checksums(self.inner, c_uchar::from(v));
4250        }
4251    }
4252
4253    /// If non-zero, an iterator will create a new table reader which
4254    /// performs reads of the given size. Using a large size (> 2MB) can
4255    /// improve the performance of forward iteration on spinning disks.
4256    /// Default: 0
4257    ///
4258    /// ```
4259    /// use rust_rocksdb::{ReadOptions};
4260    ///
4261    /// let mut opts = ReadOptions::default();
4262    /// opts.set_readahead_size(4_194_304); // 4mb
4263    /// ```
4264    pub fn set_readahead_size(&mut self, v: usize) {
4265        unsafe {
4266            ffi::rocksdb_readoptions_set_readahead_size(self.inner, v as size_t);
4267        }
4268    }
4269
4270    /// If auto_readahead_size is set to true, it will auto tune the readahead_size
4271    /// during scans internally.
4272    /// For this feature to be enabled, iterate_upper_bound must also be specified.
4273    ///
4274    /// NOTE: - Recommended for forward Scans only.
4275    ///       - If there is a backward scans, this option will be
4276    ///         disabled internally and won't be enabled again if the forward scan
4277    ///         is issued again.
4278    ///
4279    /// Default: true
4280    pub fn set_auto_readahead_size(&mut self, v: bool) {
4281        unsafe {
4282            ffi::rocksdb_readoptions_set_auto_readahead_size(self.inner, c_uchar::from(v));
4283        }
4284    }
4285
4286    /// If true, create a tailing iterator. Note that tailing iterators
4287    /// only support moving in the forward direction. Iterating in reverse
4288    /// or seek_to_last are not supported.
4289    pub fn set_tailing(&mut self, v: bool) {
4290        unsafe {
4291            ffi::rocksdb_readoptions_set_tailing(self.inner, c_uchar::from(v));
4292        }
4293    }
4294
4295    /// Specifies the value of "pin_data". If true, it keeps the blocks
4296    /// loaded by the iterator pinned in memory as long as the iterator is not deleted,
4297    /// If used when reading from tables created with
4298    /// BlockBasedTableOptions::use_delta_encoding = false,
4299    /// Iterator's property "rocksdb.iterator.is-key-pinned" is guaranteed to
4300    /// return 1.
4301    ///
4302    /// Default: false
4303    pub fn set_pin_data(&mut self, v: bool) {
4304        unsafe {
4305            ffi::rocksdb_readoptions_set_pin_data(self.inner, c_uchar::from(v));
4306        }
4307    }
4308
4309    /// Asynchronously prefetch some data.
4310    ///
4311    /// Used for sequential reads and internal automatic prefetching.
4312    ///
4313    /// Default: `false`
4314    pub fn set_async_io(&mut self, v: bool) {
4315        unsafe {
4316            ffi::rocksdb_readoptions_set_async_io(self.inner, c_uchar::from(v));
4317        }
4318    }
4319
4320    /// Deadline for completing an API call (Get/MultiGet/Seek/Next for now)
4321    /// in microseconds.
4322    /// It should be set to microseconds since epoch, i.e, gettimeofday or
4323    /// equivalent plus allowed duration in microseconds.
4324    /// This is best effort. The call may exceed the deadline if there is IO
4325    /// involved and the file system doesn't support deadlines, or due to
4326    /// checking for deadline periodically rather than for every key if
4327    /// processing a batch
4328    pub fn set_deadline(&mut self, microseconds: u64) {
4329        unsafe {
4330            ffi::rocksdb_readoptions_set_deadline(self.inner, microseconds);
4331        }
4332    }
4333
4334    /// A timeout in microseconds to be passed to the underlying FileSystem for
4335    /// reads. As opposed to deadline, this determines the timeout for each
4336    /// individual file read request. If a MultiGet/Get/Seek/Next etc call
4337    /// results in multiple reads, each read can last up to io_timeout us.
4338    pub fn set_io_timeout(&mut self, microseconds: u64) {
4339        unsafe {
4340            ffi::rocksdb_readoptions_set_io_timeout(self.inner, microseconds);
4341        }
4342    }
4343
4344    /// Timestamp of operation. Read should return the latest data visible to the
4345    /// specified timestamp. All timestamps of the same database must be of the
4346    /// same length and format. The user is responsible for providing a customized
4347    /// compare function via Comparator to order <key, timestamp> tuples.
4348    /// For iterator, iter_start_ts is the lower bound (older) and timestamp
4349    /// serves as the upper bound. Versions of the same record that fall in
4350    /// the timestamp range will be returned. If iter_start_ts is nullptr,
4351    /// only the most recent version visible to timestamp is returned.
4352    /// The user-specified timestamp feature is still under active development,
4353    /// and the API is subject to change.
4354    pub fn set_timestamp<S: Into<Vec<u8>>>(&mut self, ts: S) {
4355        self.set_timestamp_impl(Some(ts.into()));
4356    }
4357
4358    fn set_timestamp_impl(&mut self, ts: Option<Vec<u8>>) {
4359        let (ptr, len) = if let Some(ref ts) = ts {
4360            (ts.as_ptr() as *const c_char, ts.len())
4361        } else if self.timestamp.is_some() {
4362            // The stored timestamp is a `Some` but we're updating it to a `None`.
4363            // This means to cancel a previously set timestamp.
4364            // To do this, use a null pointer and zero length.
4365            (std::ptr::null(), 0)
4366        } else {
4367            return;
4368        };
4369        self.timestamp = ts;
4370        unsafe {
4371            ffi::rocksdb_readoptions_set_timestamp(self.inner, ptr, len);
4372        }
4373    }
4374
4375    /// See `set_timestamp`
4376    pub fn set_iter_start_ts<S: Into<Vec<u8>>>(&mut self, ts: S) {
4377        self.set_iter_start_ts_impl(Some(ts.into()));
4378    }
4379
4380    fn set_iter_start_ts_impl(&mut self, ts: Option<Vec<u8>>) {
4381        let (ptr, len) = if let Some(ref ts) = ts {
4382            (ts.as_ptr() as *const c_char, ts.len())
4383        } else if self.timestamp.is_some() {
4384            (std::ptr::null(), 0)
4385        } else {
4386            return;
4387        };
4388        self.iter_start_ts = ts;
4389        unsafe {
4390            ffi::rocksdb_readoptions_set_iter_start_ts(self.inner, ptr, len);
4391        }
4392    }
4393}
4394
4395impl Default for ReadOptions {
4396    fn default() -> Self {
4397        unsafe {
4398            Self {
4399                inner: ffi::rocksdb_readoptions_create(),
4400                timestamp: None,
4401                iter_start_ts: None,
4402                iterate_upper_bound: None,
4403                iterate_lower_bound: None,
4404            }
4405        }
4406    }
4407}
4408
4409impl IngestExternalFileOptions {
4410    /// Can be set to true to move the files instead of copying them.
4411    pub fn set_move_files(&mut self, v: bool) {
4412        unsafe {
4413            ffi::rocksdb_ingestexternalfileoptions_set_move_files(self.inner, c_uchar::from(v));
4414        }
4415    }
4416
4417    /// If set to false, an ingested file keys could appear in existing snapshots
4418    /// that where created before the file was ingested.
4419    pub fn set_snapshot_consistency(&mut self, v: bool) {
4420        unsafe {
4421            ffi::rocksdb_ingestexternalfileoptions_set_snapshot_consistency(
4422                self.inner,
4423                c_uchar::from(v),
4424            );
4425        }
4426    }
4427
4428    /// If set to false, IngestExternalFile() will fail if the file key range
4429    /// overlaps with existing keys or tombstones in the DB.
4430    pub fn set_allow_global_seqno(&mut self, v: bool) {
4431        unsafe {
4432            ffi::rocksdb_ingestexternalfileoptions_set_allow_global_seqno(
4433                self.inner,
4434                c_uchar::from(v),
4435            );
4436        }
4437    }
4438
4439    /// If set to false and the file key range overlaps with the memtable key range
4440    /// (memtable flush required), IngestExternalFile will fail.
4441    pub fn set_allow_blocking_flush(&mut self, v: bool) {
4442        unsafe {
4443            ffi::rocksdb_ingestexternalfileoptions_set_allow_blocking_flush(
4444                self.inner,
4445                c_uchar::from(v),
4446            );
4447        }
4448    }
4449
4450    /// Set to true if you would like duplicate keys in the file being ingested
4451    /// to be skipped rather than overwriting existing data under that key.
4452    /// Usecase: back-fill of some historical data in the database without
4453    /// over-writing existing newer version of data.
4454    /// This option could only be used if the DB has been running
4455    /// with allow_ingest_behind=true since the dawn of time.
4456    /// All files will be ingested at the bottommost level with seqno=0.
4457    pub fn set_ingest_behind(&mut self, v: bool) {
4458        unsafe {
4459            ffi::rocksdb_ingestexternalfileoptions_set_ingest_behind(self.inner, c_uchar::from(v));
4460        }
4461    }
4462}
4463
4464impl Default for IngestExternalFileOptions {
4465    fn default() -> Self {
4466        unsafe {
4467            Self {
4468                inner: ffi::rocksdb_ingestexternalfileoptions_create(),
4469            }
4470        }
4471    }
4472}
4473
4474/// Used by BlockBasedOptions::set_index_type.
4475pub enum BlockBasedIndexType {
4476    /// A space efficient index block that is optimized for
4477    /// binary-search-based index.
4478    BinarySearch,
4479
4480    /// The hash index, if enabled, will perform a hash lookup if
4481    /// a prefix extractor has been provided through Options::set_prefix_extractor.
4482    HashSearch,
4483
4484    /// A two-level index implementation. Both levels are binary search indexes.
4485    TwoLevelIndexSearch,
4486}
4487
4488/// Used by BlockBasedOptions::set_data_block_index_type.
4489#[repr(C)]
4490pub enum DataBlockIndexType {
4491    /// Use binary search when performing point lookup for keys in data blocks.
4492    /// This is the default.
4493    BinarySearch = 0,
4494
4495    /// Appends a compact hash table to the end of the data block for efficient indexing. Backwards
4496    /// compatible with databases created without this feature. Once turned on, existing data will
4497    /// be gradually converted to the hash index format.
4498    BinaryAndHash = 1,
4499}
4500
4501/// Defines the underlying memtable implementation.
4502/// See official [wiki](https://github.com/facebook/rocksdb/wiki/MemTable) for more information.
4503pub enum MemtableFactory {
4504    Vector,
4505    HashSkipList {
4506        bucket_count: usize,
4507        height: i32,
4508        branching_factor: i32,
4509    },
4510    HashLinkList {
4511        bucket_count: usize,
4512    },
4513}
4514
4515/// Used by BlockBasedOptions::set_checksum_type.
4516pub enum ChecksumType {
4517    NoChecksum = 0,
4518    CRC32c = 1,
4519    XXHash = 2,
4520    XXHash64 = 3,
4521    XXH3 = 4, // Supported since RocksDB 6.27
4522}
4523
4524/// Used in [`PlainTableFactoryOptions`].
4525#[derive(Debug, Copy, Clone, PartialEq, Eq, Default)]
4526pub enum KeyEncodingType {
4527    /// Always write full keys.
4528    #[default]
4529    Plain = 0,
4530    /// Find opportunities to write the same prefix for multiple rows.
4531    Prefix = 1,
4532}
4533
4534/// Used with DBOptions::set_plain_table_factory.
4535/// See official [wiki](https://github.com/facebook/rocksdb/wiki/PlainTable-Format) for more
4536/// information.
4537///
4538/// Defaults:
4539///  user_key_length: 0 (variable length)
4540///  bloom_bits_per_key: 10
4541///  hash_table_ratio: 0.75
4542///  index_sparseness: 16
4543///  huge_page_tlb_size: 0
4544///  encoding_type: KeyEncodingType::Plain
4545///  full_scan_mode: false
4546///  store_index_in_file: false
4547pub struct PlainTableFactoryOptions {
4548    pub user_key_length: u32,
4549    pub bloom_bits_per_key: i32,
4550    pub hash_table_ratio: f64,
4551    pub index_sparseness: usize,
4552    pub huge_page_tlb_size: usize,
4553    pub encoding_type: KeyEncodingType,
4554    pub full_scan_mode: bool,
4555    pub store_index_in_file: bool,
4556}
4557
4558#[derive(Debug, Copy, Clone, PartialEq, Eq)]
4559#[cfg_attr(feature = "serde1", derive(serde::Serialize, serde::Deserialize))]
4560pub enum DBCompressionType {
4561    None = ffi::rocksdb_no_compression as isize,
4562    Snappy = ffi::rocksdb_snappy_compression as isize,
4563    Zlib = ffi::rocksdb_zlib_compression as isize,
4564    Bz2 = ffi::rocksdb_bz2_compression as isize,
4565    Lz4 = ffi::rocksdb_lz4_compression as isize,
4566    Lz4hc = ffi::rocksdb_lz4hc_compression as isize,
4567    Zstd = ffi::rocksdb_zstd_compression as isize,
4568}
4569
4570#[derive(Debug, Copy, Clone, PartialEq, Eq)]
4571#[cfg_attr(feature = "serde1", derive(serde::Serialize, serde::Deserialize))]
4572pub enum DBCompactionStyle {
4573    Level = ffi::rocksdb_level_compaction as isize,
4574    Universal = ffi::rocksdb_universal_compaction as isize,
4575    Fifo = ffi::rocksdb_fifo_compaction as isize,
4576}
4577
4578#[derive(Debug, Copy, Clone, PartialEq, Eq)]
4579#[cfg_attr(feature = "serde1", derive(serde::Serialize, serde::Deserialize))]
4580pub enum DBRecoveryMode {
4581    TolerateCorruptedTailRecords = ffi::rocksdb_tolerate_corrupted_tail_records_recovery as isize,
4582    AbsoluteConsistency = ffi::rocksdb_absolute_consistency_recovery as isize,
4583    PointInTime = ffi::rocksdb_point_in_time_recovery as isize,
4584    SkipAnyCorruptedRecord = ffi::rocksdb_skip_any_corrupted_records_recovery as isize,
4585}
4586
4587#[derive(Debug, Copy, Clone, PartialEq, Eq)]
4588#[repr(i32)]
4589pub enum RateLimiterMode {
4590    KReadsOnly = 0,
4591    KWritesOnly = 1,
4592    KAllIo = 2,
4593}
4594
4595#[derive(Debug, Copy, Clone, PartialEq, Eq)]
4596#[cfg_attr(feature = "serde1", derive(serde::Serialize, serde::Deserialize))]
4597pub enum DBCompactionPri {
4598    ByCompensatedSize = ffi::rocksdb_k_by_compensated_size_compaction_pri as isize,
4599    OldestLargestSeqFirst = ffi::rocksdb_k_oldest_largest_seq_first_compaction_pri as isize,
4600    OldestSmallestSeqFirst = ffi::rocksdb_k_oldest_smallest_seq_first_compaction_pri as isize,
4601    MinOverlappingRatio = ffi::rocksdb_k_min_overlapping_ratio_compaction_pri as isize,
4602    RoundRobin = ffi::rocksdb_k_round_robin_compaction_pri as isize,
4603}
4604
4605#[derive(Debug, Copy, Clone, PartialEq, Eq)]
4606#[cfg_attr(feature = "serde1", derive(serde::Serialize, serde::Deserialize))]
4607pub enum BlockBasedPinningTier {
4608    Fallback = ffi::rocksdb_block_based_k_fallback_pinning_tier as isize,
4609    None = ffi::rocksdb_block_based_k_none_pinning_tier as isize,
4610    FlushAndSimilar = ffi::rocksdb_block_based_k_flush_and_similar_pinning_tier as isize,
4611    All = ffi::rocksdb_block_based_k_all_pinning_tier as isize,
4612}
4613
4614pub struct FifoCompactOptions {
4615    pub(crate) inner: *mut ffi::rocksdb_fifo_compaction_options_t,
4616}
4617
4618impl Default for FifoCompactOptions {
4619    fn default() -> Self {
4620        let opts = unsafe { ffi::rocksdb_fifo_compaction_options_create() };
4621        assert!(
4622            !opts.is_null(),
4623            "Could not create RocksDB Fifo Compaction Options"
4624        );
4625
4626        Self { inner: opts }
4627    }
4628}
4629
4630impl Drop for FifoCompactOptions {
4631    fn drop(&mut self) {
4632        unsafe {
4633            ffi::rocksdb_fifo_compaction_options_destroy(self.inner);
4634        }
4635    }
4636}
4637
4638impl FifoCompactOptions {
4639    /// Sets the max table file size.
4640    ///
4641    /// Once the total sum of table files reaches this, we will delete the oldest
4642    /// table file
4643    ///
4644    /// Default: 1GB
4645    pub fn set_max_table_files_size(&mut self, nbytes: u64) {
4646        unsafe {
4647            ffi::rocksdb_fifo_compaction_options_set_max_table_files_size(self.inner, nbytes);
4648        }
4649    }
4650}
4651
4652#[derive(Debug, Copy, Clone, PartialEq, Eq)]
4653#[cfg_attr(feature = "serde1", derive(serde::Serialize, serde::Deserialize))]
4654pub enum UniversalCompactionStopStyle {
4655    Similar = ffi::rocksdb_similar_size_compaction_stop_style as isize,
4656    Total = ffi::rocksdb_total_size_compaction_stop_style as isize,
4657}
4658
4659pub struct UniversalCompactOptions {
4660    pub(crate) inner: *mut ffi::rocksdb_universal_compaction_options_t,
4661}
4662
4663impl Default for UniversalCompactOptions {
4664    fn default() -> Self {
4665        let opts = unsafe { ffi::rocksdb_universal_compaction_options_create() };
4666        assert!(
4667            !opts.is_null(),
4668            "Could not create RocksDB Universal Compaction Options"
4669        );
4670
4671        Self { inner: opts }
4672    }
4673}
4674
4675impl Drop for UniversalCompactOptions {
4676    fn drop(&mut self) {
4677        unsafe {
4678            ffi::rocksdb_universal_compaction_options_destroy(self.inner);
4679        }
4680    }
4681}
4682
4683impl UniversalCompactOptions {
4684    /// Sets the percentage flexibility while comparing file size.
4685    /// If the candidate file(s) size is 1% smaller than the next file's size,
4686    /// then include next file into this candidate set.
4687    ///
4688    /// Default: 1
4689    pub fn set_size_ratio(&mut self, ratio: c_int) {
4690        unsafe {
4691            ffi::rocksdb_universal_compaction_options_set_size_ratio(self.inner, ratio);
4692        }
4693    }
4694
4695    /// Sets the minimum number of files in a single compaction run.
4696    ///
4697    /// Default: 2
4698    pub fn set_min_merge_width(&mut self, num: c_int) {
4699        unsafe {
4700            ffi::rocksdb_universal_compaction_options_set_min_merge_width(self.inner, num);
4701        }
4702    }
4703
4704    /// Sets the maximum number of files in a single compaction run.
4705    ///
4706    /// Default: UINT_MAX
4707    pub fn set_max_merge_width(&mut self, num: c_int) {
4708        unsafe {
4709            ffi::rocksdb_universal_compaction_options_set_max_merge_width(self.inner, num);
4710        }
4711    }
4712
4713    /// sets the size amplification.
4714    ///
4715    /// It is defined as the amount (in percentage) of
4716    /// additional storage needed to store a single byte of data in the database.
4717    /// For example, a size amplification of 2% means that a database that
4718    /// contains 100 bytes of user-data may occupy upto 102 bytes of
4719    /// physical storage. By this definition, a fully compacted database has
4720    /// a size amplification of 0%. Rocksdb uses the following heuristic
4721    /// to calculate size amplification: it assumes that all files excluding
4722    /// the earliest file contribute to the size amplification.
4723    ///
4724    /// Default: 200, which means that a 100 byte database could require upto 300 bytes of storage.
4725    pub fn set_max_size_amplification_percent(&mut self, v: c_int) {
4726        unsafe {
4727            ffi::rocksdb_universal_compaction_options_set_max_size_amplification_percent(
4728                self.inner, v,
4729            );
4730        }
4731    }
4732
4733    /// Sets the percentage of compression size.
4734    ///
4735    /// If this option is set to be -1, all the output files
4736    /// will follow compression type specified.
4737    ///
4738    /// If this option is not negative, we will try to make sure compressed
4739    /// size is just above this value. In normal cases, at least this percentage
4740    /// of data will be compressed.
4741    /// When we are compacting to a new file, here is the criteria whether
4742    /// it needs to be compressed: assuming here are the list of files sorted
4743    /// by generation time:
4744    ///    A1...An B1...Bm C1...Ct
4745    /// where A1 is the newest and Ct is the oldest, and we are going to compact
4746    /// B1...Bm, we calculate the total size of all the files as total_size, as
4747    /// well as  the total size of C1...Ct as total_C, the compaction output file
4748    /// will be compressed iff
4749    ///   total_C / total_size < this percentage
4750    ///
4751    /// Default: -1
4752    pub fn set_compression_size_percent(&mut self, v: c_int) {
4753        unsafe {
4754            ffi::rocksdb_universal_compaction_options_set_compression_size_percent(self.inner, v);
4755        }
4756    }
4757
4758    /// Sets the algorithm used to stop picking files into a single compaction run.
4759    ///
4760    /// Default: ::Total
4761    pub fn set_stop_style(&mut self, style: UniversalCompactionStopStyle) {
4762        unsafe {
4763            ffi::rocksdb_universal_compaction_options_set_stop_style(self.inner, style as c_int);
4764        }
4765    }
4766}
4767
4768#[derive(Debug, Copy, Clone, PartialEq, Eq)]
4769#[cfg_attr(feature = "serde1", derive(serde::Serialize, serde::Deserialize))]
4770#[repr(u8)]
4771pub enum BottommostLevelCompaction {
4772    /// Skip bottommost level compaction
4773    Skip = 0,
4774    /// Only compact bottommost level if there is a compaction filter
4775    /// This is the default option
4776    IfHaveCompactionFilter,
4777    /// Always compact bottommost level
4778    Force,
4779    /// Always compact bottommost level but in bottommost level avoid
4780    /// double-compacting files created in the same compaction
4781    ForceOptimized,
4782}
4783
4784pub struct CompactOptions {
4785    pub(crate) inner: *mut ffi::rocksdb_compactoptions_t,
4786    full_history_ts_low: Option<Vec<u8>>,
4787}
4788
4789impl Default for CompactOptions {
4790    fn default() -> Self {
4791        let opts = unsafe { ffi::rocksdb_compactoptions_create() };
4792        assert!(!opts.is_null(), "Could not create RocksDB Compact Options");
4793
4794        Self {
4795            inner: opts,
4796            full_history_ts_low: None,
4797        }
4798    }
4799}
4800
4801impl Drop for CompactOptions {
4802    fn drop(&mut self) {
4803        unsafe {
4804            ffi::rocksdb_compactoptions_destroy(self.inner);
4805        }
4806    }
4807}
4808
4809impl CompactOptions {
4810    /// If more than one thread calls manual compaction,
4811    /// only one will actually schedule it while the other threads will simply wait
4812    /// for the scheduled manual compaction to complete. If exclusive_manual_compaction
4813    /// is set to true, the call will disable scheduling of automatic compaction jobs
4814    /// and wait for existing automatic compaction jobs to finish.
4815    pub fn set_exclusive_manual_compaction(&mut self, v: bool) {
4816        unsafe {
4817            ffi::rocksdb_compactoptions_set_exclusive_manual_compaction(
4818                self.inner,
4819                c_uchar::from(v),
4820            );
4821        }
4822    }
4823
4824    /// Sets bottommost level compaction.
4825    pub fn set_bottommost_level_compaction(&mut self, lvl: BottommostLevelCompaction) {
4826        unsafe {
4827            ffi::rocksdb_compactoptions_set_bottommost_level_compaction(self.inner, lvl as c_uchar);
4828        }
4829    }
4830
4831    /// If true, compacted files will be moved to the minimum level capable
4832    /// of holding the data or given level (specified non-negative target_level).
4833    pub fn set_change_level(&mut self, v: bool) {
4834        unsafe {
4835            ffi::rocksdb_compactoptions_set_change_level(self.inner, c_uchar::from(v));
4836        }
4837    }
4838
4839    /// If change_level is true and target_level have non-negative value, compacted
4840    /// files will be moved to target_level.
4841    pub fn set_target_level(&mut self, lvl: c_int) {
4842        unsafe {
4843            ffi::rocksdb_compactoptions_set_target_level(self.inner, lvl);
4844        }
4845    }
4846
4847    /// Set user-defined timestamp low bound, the data with older timestamp than
4848    /// low bound maybe GCed by compaction. Default: nullptr
4849    pub fn set_full_history_ts_low<S: Into<Vec<u8>>>(&mut self, ts: S) {
4850        self.set_full_history_ts_low_impl(Some(ts.into()));
4851    }
4852
4853    fn set_full_history_ts_low_impl(&mut self, ts: Option<Vec<u8>>) {
4854        let (ptr, len) = if let Some(ref ts) = ts {
4855            (ts.as_ptr() as *mut c_char, ts.len())
4856        } else if self.full_history_ts_low.is_some() {
4857            (std::ptr::null::<Vec<u8>>() as *mut c_char, 0)
4858        } else {
4859            return;
4860        };
4861        self.full_history_ts_low = ts;
4862        unsafe {
4863            ffi::rocksdb_compactoptions_set_full_history_ts_low(self.inner, ptr, len);
4864        }
4865    }
4866}
4867
4868pub struct WaitForCompactOptions {
4869    pub(crate) inner: *mut ffi::rocksdb_wait_for_compact_options_t,
4870}
4871
4872impl Default for WaitForCompactOptions {
4873    fn default() -> Self {
4874        let opts = unsafe { ffi::rocksdb_wait_for_compact_options_create() };
4875        assert!(
4876            !opts.is_null(),
4877            "Could not create RocksDB Wait For Compact Options"
4878        );
4879
4880        Self { inner: opts }
4881    }
4882}
4883
4884impl Drop for WaitForCompactOptions {
4885    fn drop(&mut self) {
4886        unsafe {
4887            ffi::rocksdb_wait_for_compact_options_destroy(self.inner);
4888        }
4889    }
4890}
4891
4892impl WaitForCompactOptions {
4893    /// If true, abort waiting if background jobs are paused. If false,
4894    /// ContinueBackgroundWork() must be called to resume the background jobs.
4895    /// Otherwise, jobs that were queued, but not scheduled yet may never finish
4896    /// and WaitForCompact() may wait indefinitely (if timeout is set, it will
4897    /// abort after the timeout).
4898    ///
4899    /// Default: false
4900    pub fn set_abort_on_pause(&mut self, v: bool) {
4901        unsafe {
4902            ffi::rocksdb_wait_for_compact_options_set_abort_on_pause(self.inner, c_uchar::from(v));
4903        }
4904    }
4905
4906    /// If true, flush all column families before starting to wait.
4907    ///
4908    /// Default: false
4909    pub fn set_flush(&mut self, v: bool) {
4910        unsafe {
4911            ffi::rocksdb_wait_for_compact_options_set_flush(self.inner, c_uchar::from(v));
4912        }
4913    }
4914
4915    /// Timeout in microseconds for waiting for compaction to complete.
4916    /// when timeout == 0, WaitForCompact() will wait as long as there's background
4917    /// work to finish.
4918    ///
4919    /// Default: 0
4920    pub fn set_timeout(&mut self, microseconds: u64) {
4921        unsafe {
4922            ffi::rocksdb_wait_for_compact_options_set_timeout(self.inner, microseconds);
4923        }
4924    }
4925}
4926
4927/// Represents a path where sst files can be put into
4928pub struct DBPath {
4929    pub(crate) inner: *mut ffi::rocksdb_dbpath_t,
4930}
4931
4932impl DBPath {
4933    /// Create a new path
4934    pub fn new<P: AsRef<Path>>(path: P, target_size: u64) -> Result<Self, Error> {
4935        let p = to_cpath(path.as_ref()).unwrap();
4936        let dbpath = unsafe { ffi::rocksdb_dbpath_create(p.as_ptr(), target_size) };
4937        if dbpath.is_null() {
4938            Err(Error::new(format!(
4939                "Could not create path for storing sst files at location: {}",
4940                path.as_ref().display()
4941            )))
4942        } else {
4943            Ok(DBPath { inner: dbpath })
4944        }
4945    }
4946}
4947
4948impl Drop for DBPath {
4949    fn drop(&mut self) {
4950        unsafe {
4951            ffi::rocksdb_dbpath_destroy(self.inner);
4952        }
4953    }
4954}
4955
4956/// Options for importing column families. See
4957/// [DB::create_column_family_with_import](crate::DB::create_column_family_with_import).
4958pub struct ImportColumnFamilyOptions {
4959    pub(crate) inner: *mut ffi::rocksdb_import_column_family_options_t,
4960}
4961
4962impl ImportColumnFamilyOptions {
4963    pub fn new() -> Self {
4964        let inner = unsafe { ffi::rocksdb_import_column_family_options_create() };
4965        ImportColumnFamilyOptions { inner }
4966    }
4967
4968    /// Determines whether to move the provided set of files on import. The default
4969    /// behavior is to copy the external files on import. Setting `move_files` to `true`
4970    /// will move the files instead of copying them. See
4971    /// [DB::create_column_family_with_import](crate::DB::create_column_family_with_import)
4972    /// for more information.
4973    pub fn set_move_files(&mut self, move_files: bool) {
4974        unsafe {
4975            ffi::rocksdb_import_column_family_options_set_move_files(
4976                self.inner,
4977                c_uchar::from(move_files),
4978            );
4979        }
4980    }
4981}
4982
4983impl Default for ImportColumnFamilyOptions {
4984    fn default() -> Self {
4985        Self::new()
4986    }
4987}
4988
4989impl Drop for ImportColumnFamilyOptions {
4990    fn drop(&mut self) {
4991        unsafe { ffi::rocksdb_import_column_family_options_destroy(self.inner) }
4992    }
4993}
4994
4995#[cfg(test)]
4996mod tests {
4997    use crate::cache::Cache;
4998    use crate::db_options::WriteBufferManager;
4999    use crate::{MemtableFactory, Options};
5000
5001    #[test]
5002    fn test_enable_statistics() {
5003        let mut opts = Options::default();
5004        opts.enable_statistics();
5005        opts.set_stats_dump_period_sec(60);
5006        assert!(opts.get_statistics().is_some());
5007
5008        let opts = Options::default();
5009        assert!(opts.get_statistics().is_none());
5010    }
5011
5012    #[test]
5013    fn test_set_memtable_factory() {
5014        let mut opts = Options::default();
5015        opts.set_memtable_factory(MemtableFactory::Vector);
5016        opts.set_memtable_factory(MemtableFactory::HashLinkList { bucket_count: 100 });
5017        opts.set_memtable_factory(MemtableFactory::HashSkipList {
5018            bucket_count: 100,
5019            height: 4,
5020            branching_factor: 4,
5021        });
5022    }
5023
5024    #[test]
5025    fn test_use_fsync() {
5026        let mut opts = Options::default();
5027        assert!(!opts.get_use_fsync());
5028        opts.set_use_fsync(true);
5029        assert!(opts.get_use_fsync());
5030    }
5031
5032    #[test]
5033    fn test_set_stats_persist_period_sec() {
5034        let mut opts = Options::default();
5035        opts.enable_statistics();
5036        opts.set_stats_persist_period_sec(5);
5037        assert!(opts.get_statistics().is_some());
5038
5039        let opts = Options::default();
5040        assert!(opts.get_statistics().is_none());
5041    }
5042
5043    #[test]
5044    fn test_set_write_buffer_manager() {
5045        let mut opts = Options::default();
5046        let lrucache = Cache::new_lru_cache(100);
5047        let write_buffer_manager =
5048            WriteBufferManager::new_write_buffer_manager_with_cache(100, false, lrucache);
5049        assert_eq!(write_buffer_manager.get_buffer_size(), 100);
5050        assert_eq!(write_buffer_manager.get_usage(), 0);
5051        assert!(write_buffer_manager.enabled());
5052
5053        opts.set_write_buffer_manager(&write_buffer_manager);
5054        drop(opts);
5055
5056        // WriteBufferManager outlives options
5057        assert!(write_buffer_manager.enabled());
5058    }
5059}