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