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