diff --git a/rocksdb/_rocksdb.pyx b/rocksdb/_rocksdb.pyx index 8c43dbe..6e1eccd 100644 --- a/rocksdb/_rocksdb.pyx +++ b/rocksdb/_rocksdb.pyx @@ -3,6 +3,7 @@ import cython from libcpp.string cimport string from libcpp.deque cimport deque from libcpp.vector cimport vector +from libcpp.map cimport map from cpython cimport bool as py_bool from libcpp cimport bool as cpp_bool from libc.stdint cimport uint32_t @@ -31,6 +32,7 @@ from . cimport table_factory from . cimport memtablerep from . cimport universal_compaction from . cimport transaction_db +from . cimport metadata # Enums are the only exception for direct imports # Their name als already unique enough @@ -1113,6 +1115,24 @@ cdef class ColumnFamilyOptions(object): def __set__(self, value): self.copts.max_bytes_for_level_multiplier_additional = value + property soft_rate_limit: + def __get__(self): + return self.copts.soft_rate_limit + def __set__(self, value): + self.copts.soft_rate_limit = value + + property hard_rate_limit: + def __get__(self): + return self.copts.hard_rate_limit + def __set__(self, value): + self.copts.hard_rate_limit = value + + property rate_limit_delay_max_milliseconds: + def __get__(self): + return self.copts.rate_limit_delay_max_milliseconds + def __set__(self, value): + self.copts.rate_limit_delay_max_milliseconds = value + property arena_block_size: def __get__(self): return self.copts.arena_block_size @@ -1125,6 +1145,12 @@ cdef class ColumnFamilyOptions(object): def __set__(self, value): self.copts.disable_auto_compactions = value + property purge_redundant_kvs_while_flush: + def __get__(self): + return self.copts.purge_redundant_kvs_while_flush + def __set__(self, value): + self.copts.purge_redundant_kvs_while_flush = value + # FIXME: remove to util/options_helper.h # property allow_os_buffer: # def __get__(self): @@ -1400,12 +1426,6 @@ cdef class Options(ColumnFamilyOptions): def __set__(self, value): self.opts.base_background_compactions = value - property max_background_compactions: - def __get__(self): - return self.opts.max_background_compactions - def __set__(self, value): - self.opts.max_background_compactions = value - property max_subcompactions: def __get__(self): return self.opts.max_subcompactions @@ -1472,6 +1492,18 @@ cdef class Options(ColumnFamilyOptions): def __set__(self, value): self.opts.manifest_preallocation_size = value + property enable_write_thread_adaptive_yield: + def __get__(self): + return self.opts.enable_write_thread_adaptive_yield + def __set__(self, value): + self.opts.enable_write_thread_adaptive_yield = value + + property allow_concurrent_memtable_write: + def __get__(self): + return self.opts.allow_concurrent_memtable_write + def __set__(self, value): + self.opts.allow_concurrent_memtable_write = value + property allow_mmap_reads: def __get__(self): return self.opts.allow_mmap_reads @@ -1508,6 +1540,12 @@ cdef class Options(ColumnFamilyOptions): def __set__(self, value): self.opts.is_fd_close_on_exec = value + property skip_log_error_on_recovery: + def __get__(self): + return self.opts.skip_log_error_on_recovery + def __set__(self, value): + self.opts.skip_log_error_on_recovery = value + property stats_dump_period_sec: def __get__(self): return self.opts.stats_dump_period_sec @@ -2464,8 +2502,26 @@ cdef class DB(object): else: return None + # def get_map_property(self, prop, ColumnFamilyHandle column_family=None): + # cdef map[string, string] value + # cdef Slice c_prop = bytes_to_slice(prop) + # cdef cpp_bool ret = False + # cdef db.ColumnFamilyHandle* cf_handle = NULL + # if column_family: + # cf_handle = column_family.get_handle() + # else: + # cf_handle = self.db.DefaultColumnFamily() + + # with nogil: + # ret = self.db.GetMapProperty(cf_handle, c_prop, cython.address(value)) + + # if ret: + # return value + # else: + # return None + def get_live_files_metadata(self): - cdef vector[db.LiveFileMetaData] metadata + cdef vector[metadata.LiveFileMetaData] metadata with nogil: self.wrapped_db.GetLiveFilesMetaData(cython.address(metadata)) @@ -2486,7 +2542,7 @@ cdef class DB(object): return ret def get_column_family_meta_data(self, ColumnFamilyHandle column_family=None): - cdef db.ColumnFamilyMetaData metadata + cdef metadata.ColumnFamilyMetaData metadata cdef db.ColumnFamilyHandle* cf_handle = self.wrapped_db.DefaultColumnFamily() if column_family: @@ -2644,6 +2700,18 @@ def repair_db(db_name, Options opts): st = db.RepairDB(db_path, deref(opts.opts)) check_status(st) +# TODO Figure out API to add descriptors. See constructor which deals with vector of ColumnFamilyDescriptor +# def repair_db(db_name, Options opts, descriptors): +# cdef Status st +# cdef string db_path +# cdef vector[db.ColumnFamilyDescriptor] c_descriptors + +# for d in descriptors: +# c_descriptors.push_back((d)) + +# db_path = path_to_string(db_name) +# st = db.RepairDB(db_path, deref(opts.opts), c_descriptors) +# check_status(st) def list_column_families(db_name, Options opts): cdef Status st diff --git a/rocksdb/db.pxd b/rocksdb/db.pxd index 6cc4d11..b4fb778 100644 --- a/rocksdb/db.pxd +++ b/rocksdb/db.pxd @@ -5,9 +5,18 @@ from libcpp cimport bool as cpp_bool from libcpp.string cimport string from libcpp.vector cimport vector from .types cimport SequenceNumber +from libcpp.map cimport map +from libcpp.unordered_map cimport unordered_map +from libcpp.memory cimport shared_ptr +from .types cimport SequenceNumber from .slice_ cimport Slice from .snapshot cimport Snapshot from .iterator cimport Iterator +from .env cimport Env +from .metadata cimport ColumnFamilyMetaData +from .metadata cimport LiveFileMetaData +from .metadata cimport ExportImportFilesMetaData +from .table_properties cimport TableProperties cdef extern from "rocksdb/write_batch.h" namespace "rocksdb": cdef cppclass WriteBatch: @@ -40,8 +49,15 @@ cdef extern from "cpp/write_batch_iter_helper.hpp" namespace "py_rocks": cdef extern from "rocksdb/db.h" namespace "rocksdb": + ctypedef uint64_t SequenceNumber string kDefaultColumnFamilyName + #todo TableProperties + ctypedef unordered_map[string, shared_ptr[const TableProperties]] TablePropertiesCollection + + cdef struct GetMergeOperandsOptions: + uint64_t expected_max_number_of_operands + cdef struct LiveFileMetaData: string name int level @@ -67,12 +83,6 @@ cdef extern from "rocksdb/db.h" namespace "rocksdb": # string largestkey # LiveFileMetaData files - cdef struct ColumnFamilyMetaData: - uint64_t size - uint64_t file_count - # string largestkey - # LevelMetaData levels - cdef cppclass Range: Range(const Slice&, const Slice&) @@ -88,6 +98,12 @@ cdef extern from "rocksdb/db.h" namespace "rocksdb": ColumnFamilyHandle*, const Slice&) nogil except+ + Status DeleteRange( + const options.WriteOptions&, + ColumnFamilyHandle*, + const Slice&, + const Slice&) nogil except+ + Status Merge( const options.WriteOptions&, ColumnFamilyHandle*, @@ -104,6 +120,14 @@ cdef extern from "rocksdb/db.h" namespace "rocksdb": const Slice&, string*) nogil except+ + # Status GetMergeOperands( + # const options.ReadOptions&, + # ColumnFamilyHandle*, + # const Slice&, + # PinnableSlice*, + # GetMergeOperandsOptions*, + # uint64_t*) nogil except+; + vector[Status] MultiGet( const options.ReadOptions&, const vector[ColumnFamilyHandle*]&, @@ -141,6 +165,22 @@ cdef extern from "rocksdb/db.h" namespace "rocksdb": const Slice&, string*) nogil except+ + cpp_bool GetMapProperty( + ColumnFamilyHandle*, + const Slice&, + map[string, string]*) nogil except+ + + cpp_bool GetIntProperty( + ColumnFamilyHandle*, + const Slice&, + uint64_t*) nogil except+ + + Status ResetStats() nogil except+ + + cpp_bool GetAggregatedIntProperty( + const Slice&, + uint64_t*) nogil except+ + void GetApproximateSizes( ColumnFamilyHandle*, const Range* @@ -153,35 +193,116 @@ cdef extern from "rocksdb/db.h" namespace "rocksdb": const Slice*, const Slice*) nogil except+ + Status SetOptions( + ColumnFamilyHandle*, + const unordered_map[string, string]&) nogil except+ + + Status EnableAutoCompaction( + const vector[ColumnFamilyHandle*]&) nogil except+ + + void DisableManualCompaction() nogil except+ + void EnableManualCompaction() nogil except+ + Status CreateColumnFamily( const options.ColumnFamilyOptions&, const string&, ColumnFamilyHandle**) nogil except+ + Status CreateColumnFamilies( + const options.ColumnFamilyOptions&, + const vector[string]&, + vector[ColumnFamilyHandle*]*) nogil except+ + + Status CreateColumnFamilies( + const vector[ColumnFamilyDescriptor]&, + vector[ColumnFamilyHandle*]*) nogil except+ + Status DropColumnFamily( ColumnFamilyHandle*) nogil except+ + Status DropColumnFamilies( + vector[ColumnFamilyHandle*]*) nogil except+ + + Status DestroyColumnFamilyHandle( + ColumnFamilyHandle*) nogil except+ + int NumberLevels(ColumnFamilyHandle*) nogil except+ int MaxMemCompactionLevel(ColumnFamilyHandle*) nogil except+ int Level0StopWriteTrigger(ColumnFamilyHandle*) nogil except+ const string& GetName() nogil except+ + Env* GetEnv() nogil except+ + # TODO Mandar FileSystem* GetFileSystem() nogil except+ const options.Options& GetOptions(ColumnFamilyHandle*) nogil except+ Status Flush(const options.FlushOptions&, ColumnFamilyHandle*) nogil except+ + Status Flush( + const options.FlushOptions&, + const vector[ColumnFamilyHandle*]&) nogil except+ + Status FlushWAL(bool_cpp) nogil except+ + Status SyncWAL() nogil except+ + Status LockWAL() nogil except+ + Status UnlockWAL() nogil except+ + + SequenceNumber GetLatestSequenceNumber() + cpp_bool SetPreserveDeletesSequenceNumber(SequenceNumber) nogil except+ + Status DisableFileDeletions() nogil except+ Status EnableFileDeletions() nogil except+ + Status Close() nogil except+ + Status Resume() nogil except+ + Status PauseBackgroundWork() nogil except+ + Status ContinueBackgroundWork() nogil except+ + Status GetDbIdentity(string&) nogil except+ + ColumnFamilyHandle* DefaultColumnFamily() + + # Following defined for #ifndef ROCKDDB_LITE + Status GetLiveFiles(vector[string]&, + uint64_t*, + cpp_bool) nogil except+ # TODO: Status GetSortedWalFiles(VectorLogPtr& files) - # TODO: SequenceNumber GetLatestSequenceNumber() + # TODO: Status GetCurrentWalFile(std::unique_ptr*) # TODO: Status GetUpdatesSince( - # SequenceNumber seq_number, - # unique_ptr[TransactionLogIterator]*) - + # SequenceNumber seq_number, + # unique_ptr[TransactionLogIterator]*, + # const TransactionLogIterator::ReadOptions&) + Status GetCreationTimeOfOldestFile(uint64_t*) nogil except+ Status DeleteFile(string) nogil except+ void GetLiveFilesMetaData(vector[LiveFileMetaData]*) nogil except+ void GetColumnFamilyMetaData(ColumnFamilyHandle*, ColumnFamilyMetaData*) nogil except+ - ColumnFamilyHandle* DefaultColumnFamily() + Status IngestExternalFile( + ColumnFamilyHandle*, + const vector[string]&, + const options.IngestExternalFileOptions&) nogil except+ + Status CreateColumnFamilyWithImport( + const options.ColumnFamilyOptions&, + const string&, + const options.ImportColumnFamilyOptions&, + const ExportImportFilesMetaData&, + ColumnFamilyHandle**) nogil except+ + Status VerifyChecksum(const options.ReadOptions&) nogil except+ + + DB* GetRootDB() nogil except+ + Status GetPropertiesOfAllTables( + ColumnFamilyHandle*, + TablePropertiesCollection*) nogil except+ + Status GetPropertiesOfTablesInRange( + ColumnFamilyHandle*, const Range*, size_t, + TablePropertiesCollection*) nogil except+ + + Status SuggestCompactRange(ColumnFamilyHandle*, + const Slice*, + const Slice*) nogil except+ + Status PromoteL0(ColumnFamilyHandle*, int) nogil except+ + #TODO Status StartTrace(const TraceOptions&, + # std::unique_ptr&&) nogil except+ + Status EndTrace() nogil except+ + # TODO Status StartBlockCacheTrace( + # const TraceOptions&, + # std::unique_ptr&&) nogil except+ + Status EndBlockCacheTrace() nogil except+ + Status TryCatchUpWithPrimary() nogil except+ cdef Status DB_Open "rocksdb::DB::Open"( const options.Options&, @@ -209,7 +330,13 @@ cdef extern from "rocksdb/db.h" namespace "rocksdb": DB**, cpp_bool) nogil except+ + cdef Status DestroyDB( + const string&, + const options.Options&, + const vector[ColumnFamilyDescriptor]&) nogil except+ + cdef Status RepairDB(const string& dbname, const options.Options&) + cdef Status RepairDB(const string& dbname, const options.Options&, const vector[ColumnFamilyDescriptor]&) cdef Status ListColumnFamilies "rocksdb::DB::ListColumnFamilies" ( const options.Options&, @@ -223,7 +350,7 @@ cdef extern from "rocksdb/db.h" namespace "rocksdb": cdef cppclass ColumnFamilyDescriptor: ColumnFamilyDescriptor() nogil except+ ColumnFamilyDescriptor( - const string&, + const string&, const options.ColumnFamilyOptions&) nogil except+ string name options.ColumnFamilyOptions options diff --git a/rocksdb/metadata.pxd b/rocksdb/metadata.pxd new file mode 100644 index 0000000..cea3da7 --- /dev/null +++ b/rocksdb/metadata.pxd @@ -0,0 +1,55 @@ +from libcpp cimport bool as cpp_bool +from libcpp.string cimport string +from libcpp.vector cimport vector +from libc.stdint cimport uint64_t +from libc.stdint cimport int64_t +from .logger cimport Logger +from .types cimport SequenceNumber + +cdef extern from "rocksdb/metadata.h" namespace "rocksdb": + cdef cppclass ColumnFamilyMetaData: + ColumnFamilyMetaData() except+ + ColumnFamilyMetaData(const string&, uint64_t, + const vector[LevelMetaData]&&) except+ + uint64_t size + uint64_t file_count + string name + vector[LevelMetaData] levels + + cdef cppclass LevelMetaData: + int level + uint64_t size + vector[SstFileMetaData] files + + cdef cppclass SstFileMetaData: + uint64_t size + string name + uint64_t file_number + string db_path + SequenceNumber smallest_seqno + SequenceNumber largest_seqno + string smallestkey + string largestkey + uint64_t num_reads_sampled + cpp_bool being_compacted + uint64_t num_entries + uint64_t num_deletions + uint64_t oldest_blob_file_number + uint64_t oldest_ancester_time + uint64_t file_creation_time + string file_checksum + string file_checksum_func_name + + cdef cppclass LiveFileMetaData(SstFileMetaData): + string column_family_name + int level + uint64_t size + string smallestkey + string largestkey + SequenceNumber smallest_seqno + SequenceNumber largest_seqno + + cdef cppclass ExportImportFilesMetaData: + string db_comparator_name + vector[LiveFileMetaData] files + diff --git a/rocksdb/options.pxd b/rocksdb/options.pxd index 66b68b1..ad5c349 100644 --- a/rocksdb/options.pxd +++ b/rocksdb/options.pxd @@ -15,7 +15,7 @@ from .statistics cimport Statistics from .memtablerep cimport MemTableRepFactory from .universal_compaction cimport CompactionOptionsUniversal from .cache cimport Cache -from . cimport advanced_options +from .cimport advanced_options from .advanced_options cimport CompressionOptions from .advanced_options cimport AdvancedColumnFamilyOptions from .env cimport Env @@ -26,10 +26,26 @@ from .concurrent_task_limiter cimport ConcurrentTaskLimiter cdef extern from "rocksdb/options.h" namespace "rocksdb": ctypedef enum CpuPriority: kIdle - kLow - kNormal + KLow + KNormal kHigh + cdef cppclass CompressionOptions: + int window_bits; + int level; + int strategy; + uint32_t max_dict_bytes + # FIXME: add missing fields: max_dict_bytes, zstd_max_train_bytes, + # parallel_threads, enabled + CompressionOptions() except + + CompressionOptions(int, int, int, int) except + + + ctypedef enum CompactionStyle: + kCompactionStyleLevel + kCompactionStyleUniversal + kCompactionStyleFIFO + kCompactionStyleNone + ctypedef enum CompressionType: kNoCompression kSnappyCompression @@ -67,11 +83,17 @@ cdef extern from "rocksdb/options.h" namespace "rocksdb": shared_ptr[TableFactory] table_factory vector[DbPath] cf_paths - shared_ptr[ConcurrentTaskLimiter] compaction_thread_limiter + # TODO shared_ptr[ConcurrentTaskLimiter] compaction_thread_limiter ColumnFamilyOptions() ColumnFamilyOptions(const Options& options) void Dump(Logger*) + ctypedef enum CompactionPri: + kByCompensatedSize + kOldestLargestSeqFirst + kOldestSmallestSeqFirst + kMinOverlappingRatio + # This needs to be in _rocksdb.pxd so it will export into python cpdef enum AccessHint "rocksdb::DBOptions::AccessHint": NONE, @@ -123,6 +145,7 @@ cdef extern from "rocksdb/options.h" namespace "rocksdb": size_t log_file_time_to_roll size_t keep_log_file_num size_t recycle_log_file_num + size_t stats_history_buffer_size uint64_t max_manifest_file_size int table_cache_numshardbits uint64_t WAL_ttl_seconds @@ -134,6 +157,7 @@ cdef extern from "rocksdb/options.h" namespace "rocksdb": cpp_bool use_direct_io_for_flush_and_compaction cpp_bool allow_fallocate cpp_bool is_fd_close_on_exec + cpp_bool skip_log_error_on_recovery unsigned int stats_dump_period_sec unsigned int stats_persist_period_sec cpp_bool persist_stats_to_disk @@ -232,10 +256,16 @@ cdef extern from "rocksdb/options.h" namespace "rocksdb": cpp_bool ignore_missing_column_families cpp_bool no_slowdown cpp_bool low_pri - cpp_bool memtable_insert_hint_per_batch + cpp_bool memtable_insert_hint_per_back const Slice* timestamp WriteOptions() nogil except+ + cdef cppclass ReadOptions: + cpp_bool verify_checksums + cpp_bool fill_cache + const Snapshot* snapshot + ReadTier read_tier + cdef cppclass FlushOptions: cpp_bool wait cpp_bool allow_write_stall diff --git a/rocksdb/tests/test_options.py b/rocksdb/tests/test_options.py index f42bb12..ec9aa94 100644 --- a/rocksdb/tests/test_options.py +++ b/rocksdb/tests/test_options.py @@ -120,6 +120,41 @@ def test_bottommost_compression_opts(self): self.assertIsNot(new_opts, opts.bottommost_compression_opts) for key, value in new_opts.items(): self.assertEqual(opts.bottommost_compression_opts[key], value) + opts.compression_opts = {'window_bits': 1, 'level': 2, 'strategy': 3, 'max_dict_bytes': 4, 'zstd_max_train_bytes': 15, 'parallel_threads': 4, 'enabled': True} + compression_opts = opts.compression_opts + self.assertEqual(compression_opts['window_bits'], 1) + self.assertEqual(compression_opts['level'], 2) + self.assertEqual(compression_opts['strategy'], 3) + self.assertEqual(compression_opts['max_dict_bytes'], 4) + self.assertEqual(compression_opts['zstd_max_train_bytes'], 15) + self.assertEqual(compression_opts['parallel_threads'], 4) + self.assertEqual(compression_opts['enabled'], True) + + def test_bottommost_compression_opts(self): + opts = rocksdb.Options() + bottommost_compression_opts = opts.bottommost_compression_opts + # default value + self.assertEqual(isinstance(bottommost_compression_opts, dict), True) + self.assertEqual(bottommost_compression_opts['window_bits'], -14) + self.assertEqual(bottommost_compression_opts['level'], 2**15 - 1) + self.assertEqual(bottommost_compression_opts['strategy'], 0) + self.assertEqual(bottommost_compression_opts['max_dict_bytes'], 0) + self.assertEqual(bottommost_compression_opts['zstd_max_train_bytes'], 0) + self.assertEqual(bottommost_compression_opts['parallel_threads'], 1) + self.assertEqual(bottommost_compression_opts['enabled'], False) + + with self.assertRaises(TypeError): + opts.compression_opts = list(1, 2) + + opts.bottommost_compression_opts = {'window_bits': 1, 'level': 2, 'strategy': 3, 'max_dict_bytes': 4, 'zstd_max_train_bytes': 15, 'parallel_threads': 4, 'enabled': True} + bottommost_compression_opts = opts.bottommost_compression_opts + self.assertEqual(bottommost_compression_opts['window_bits'], 1) + self.assertEqual(bottommost_compression_opts['level'], 2) + self.assertEqual(bottommost_compression_opts['strategy'], 3) + self.assertEqual(bottommost_compression_opts['max_dict_bytes'], 4) + self.assertEqual(bottommost_compression_opts['zstd_max_train_bytes'], 15) + self.assertEqual(bottommost_compression_opts['parallel_threads'], 4) + self.assertEqual(bottommost_compression_opts['enabled'], True) def test_simple(self): opts = rocksdb.Options() @@ -284,3 +319,436 @@ def test_rocksdb_options(self): else: setattr(opts, option, new_value) self.assertEqual(getattr(opts, option), new_value) + + def test_max_open_files(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.max_open_files) + opts.max_open_files = 10 + self.assertEqual(opts.max_open_files, 10) + + def test_max_file_opening_threads(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.max_file_opening_threads) + opts.max_file_opening_threads = 10 + self.assertEqual(opts.max_file_opening_threads, 10) + + def test_max_total_wal_size(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.max_total_wal_size) + opts.max_total_wal_size = 10 + self.assertEqual(opts.max_total_wal_size, 10) + + def test_max_background_jobs(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.max_background_jobs) + opts.max_background_jobs = 10 + self.assertEqual(opts.max_background_jobs, 10) + + def test_base_background_compactions(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.base_background_compactions) + opts.base_background_compactions = 10 + self.assertEqual(opts.base_background_compactions, 10) + + def test_max_background_compactions(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.max_background_compactions) + opts.max_background_compactions = 10 + self.assertEqual(opts.max_background_compactions, 10) + + def test_max_subcompactions(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.max_subcompactions) + opts.max_subcompactions = 10 + self.assertEqual(opts.max_subcompactions, 10) + + def test_max_background_flushes(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.max_background_flushes) + opts.max_background_flushes = 10 + self.assertEqual(opts.max_background_flushes, 10) + + def test_max_log_file_size(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.max_log_file_size) + opts.max_log_file_size = 10 + self.assertEqual(opts.max_log_file_size, 10) + + def test_log_file_time_to_roll(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.log_file_time_to_roll) + opts.log_file_time_to_roll = 10 + self.assertEqual(opts.log_file_time_to_roll, 10) + + def test_recycle_log_file_num(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.recycle_log_file_num) + opts.recycle_log_file_num = 10 + self.assertEqual(opts.recycle_log_file_num, 10) + + def test_stats_history_buffer_size(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.stats_history_buffer_size) + opts.stats_history_buffer_size = 10 + self.assertEqual(opts.stats_history_buffer_size, 10) + + def test_max_manifest_file_size(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.max_manifest_file_size) + opts.max_manifest_file_size = 10 + self.assertEqual(opts.max_manifest_file_size, 10) + + def test_table_cache_numshardbits(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.table_cache_numshardbits) + opts.table_cache_numshardbits = 10 + self.assertEqual(opts.table_cache_numshardbits, 10) + + def test_wal_ttl_seconds(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.wal_ttl_seconds) + opts.wal_ttl_seconds = 10 + self.assertEqual(opts.wal_ttl_seconds, 10) + + def test_wal_size_limit_mb(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.wal_size_limit_mb) + opts.wal_size_limit_mb = 10 + self.assertEqual(opts.wal_size_limit_mb, 10) + + def test_manifest_preallocation_size(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.manifest_preallocation_size) + opts.manifest_preallocation_size = 10 + self.assertEqual(opts.manifest_preallocation_size, 10) + + def test_allow_mmap_reads(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.allow_mmap_reads) + self.assertEqual(opts.allow_mmap_reads, False) + opts.allow_mmap_reads = True + self.assertEqual(opts.allow_mmap_reads, True) + + def test_allow_mmap_writes(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.allow_mmap_writes) + self.assertEqual(opts.allow_mmap_writes, False) + opts.allow_mmap_writes = True + self.assertEqual(opts.allow_mmap_writes, True) + + def test_use_direct_reads(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.use_direct_reads) + self.assertEqual(opts.use_direct_reads, False) + opts.use_direct_reads = True + self.assertEqual(opts.use_direct_reads, True) + + def test_use_direct_io_for_flush_and_compaction(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.use_direct_io_for_flush_and_compaction) + self.assertEqual(opts.use_direct_io_for_flush_and_compaction, False) + opts.use_direct_io_for_flush_and_compaction = True + self.assertEqual(opts.use_direct_io_for_flush_and_compaction, True) + + def test_allow_fallocate(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.allow_fallocate) + self.assertEqual(opts.allow_fallocate, True) + opts.allow_fallocate = False + self.assertEqual(opts.allow_fallocate, False) + + def test_is_fd_close_on_exec(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.is_fd_close_on_exec) + self.assertEqual(opts.is_fd_close_on_exec, True) + opts.is_fd_close_on_exec = False + self.assertEqual(opts.is_fd_close_on_exec, False) + + def test_skip_log_error_on_recovery(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.skip_log_error_on_recovery) + self.assertEqual(opts.skip_log_error_on_recovery, False) + opts.skip_log_error_on_recovery = True + self.assertEqual(opts.skip_log_error_on_recovery, True) + + def test_stats_dump_period_sec(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.stats_dump_period_sec) + self.assertEqual(opts.stats_dump_period_sec, 600) + opts.stats_dump_period_sec = 3600 + self.assertEqual(opts.stats_dump_period_sec, 3600) + + def test_stats_persist_period_sec(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.stats_persist_period_sec) + self.assertEqual(opts.stats_persist_period_sec, 600) + opts.stats_persist_period_sec = 3600 + self.assertEqual(opts.stats_persist_period_sec, 3600) + + def test_persist_stats_to_disk(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.persist_stats_to_disk) + self.assertEqual(opts.persist_stats_to_disk, False) + opts.persist_stats_to_disk = True + self.assertEqual(opts.persist_stats_to_disk, True) + + def test_stats_history_buffer_size(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.stats_history_buffer_size) + self.assertEqual(opts.stats_history_buffer_size, 1024*1024) + opts.stats_history_buffer_size = 3600 + self.assertEqual(opts.stats_history_buffer_size, 3600) + + def test_advise_random_on_open(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.advise_random_on_open) + self.assertEqual(opts.advise_random_on_open, True) + opts.advise_random_on_open = False + self.assertEqual(opts.advise_random_on_open, False) + + def test_db_write_buffer_size(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.db_write_buffer_size) + self.assertEqual(opts.db_write_buffer_size, 0) + opts.db_write_buffer_size = 3600 + self.assertEqual(opts.db_write_buffer_size, 3600) + + def test_new_table_reader_for_compaction_inputs(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.new_table_reader_for_compaction_inputs) + self.assertEqual(opts.new_table_reader_for_compaction_inputs, False) + opts.new_table_reader_for_compaction_inputs = True + self.assertEqual(opts.new_table_reader_for_compaction_inputs, True) + + def test_compaction_readahead_size(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.compaction_readahead_size) + self.assertEqual(opts.compaction_readahead_size, 0) + opts.compaction_readahead_size = 3600 + self.assertEqual(opts.compaction_readahead_size, 3600) + + def test_random_access_max_buffer_size(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.random_access_max_buffer_size) + self.assertEqual(opts.random_access_max_buffer_size, 1024*1024) + opts.random_access_max_buffer_size = 3600 + self.assertEqual(opts.random_access_max_buffer_size, 3600) + + def test_writable_file_max_buffer_size(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.writable_file_max_buffer_size) + self.assertEqual(opts.writable_file_max_buffer_size, 1024*1024) + opts.writable_file_max_buffer_size = 3600 + self.assertEqual(opts.writable_file_max_buffer_size, 3600) + + def test_use_adaptive_mutex(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.use_adaptive_mutex) + self.assertEqual(opts.use_adaptive_mutex, False) + opts.use_adaptive_mutex = True + self.assertEqual(opts.use_adaptive_mutex, True) + + def test_bytes_per_sync(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.bytes_per_sync) + self.assertEqual(opts.bytes_per_sync, 0) + opts.bytes_per_sync = 3600 + self.assertEqual(opts.bytes_per_sync, 3600) + + def test_wal_wal_bytes_per_sync(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.wal_bytes_per_sync) + self.assertEqual(opts.wal_bytes_per_sync, 0) + opts.wal_bytes_per_sync = 3600 + self.assertEqual(opts.wal_bytes_per_sync, 3600) + + def test_strict_bytes_per_sync(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.strict_bytes_per_sync) + self.assertEqual(opts.strict_bytes_per_sync, False) + opts.strict_bytes_per_sync = True + self.assertEqual(opts.strict_bytes_per_sync, True) + + def test_enable_thread_tracking(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.enable_thread_tracking) + self.assertEqual(opts.enable_thread_tracking, False) + opts.enable_thread_tracking = True + self.assertEqual(opts.enable_thread_tracking, True) + + def test_delayed_write_rate(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.delayed_write_rate) + self.assertEqual(opts.delayed_write_rate, 0) + opts.delayed_write_rate = 10 + self.assertEqual(opts.delayed_write_rate, 10) + + def test_enable_pipelined_write(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.enable_pipelined_write) + self.assertEqual(opts.enable_pipelined_write, False) + opts.enable_pipelined_write = True + self.assertEqual(opts.enable_pipelined_write, True) + + def test_unordered_write(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.unordered_write) + self.assertEqual(opts.unordered_write, False) + opts.unordered_write = True + self.assertEqual(opts.unordered_write, True) + + def test_allow_concurrent_memtable_write(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.allow_concurrent_memtable_write) + self.assertEqual(opts.allow_concurrent_memtable_write, True) + opts.allow_concurrent_memtable_write = False + self.assertEqual(opts.allow_concurrent_memtable_write, False) + + def test_enable_write_thread_adaptive_yield(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.enable_write_thread_adaptive_yield) + self.assertEqual(opts.enable_write_thread_adaptive_yield, True) + opts.enable_write_thread_adaptive_yield = False + self.assertEqual(opts.enable_write_thread_adaptive_yield, False) + + def test_max_write_batch_group_size_bytes(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.max_write_batch_group_size_bytes) + self.assertEqual(opts.max_write_batch_group_size_bytes, 1 << 20) + opts.max_write_batch_group_size_bytes = 200 + self.assertEqual(opts.max_write_batch_group_size_bytes, 200) + + def test_write_thread_max_yield_usec(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.write_thread_max_yield_usec) + self.assertEqual(opts.write_thread_max_yield_usec, 100) + opts.write_thread_max_yield_usec = 200 + self.assertEqual(opts.write_thread_max_yield_usec, 200) + + def test_write_thread_slow_yield_usec(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.write_thread_slow_yield_usec) + self.assertEqual(opts.write_thread_slow_yield_usec, 3) + opts.write_thread_slow_yield_usec = 200 + self.assertEqual(opts.write_thread_slow_yield_usec, 200) + + def test_skip_stats_update_on_db_open(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.skip_stats_update_on_db_open) + self.assertEqual(opts.skip_stats_update_on_db_open, False) + opts.skip_stats_update_on_db_open = True + self.assertEqual(opts.skip_stats_update_on_db_open, True) + + def test_skip_checking_sst_file_sizes_on_db_open(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.skip_checking_sst_file_sizes_on_db_open) + self.assertEqual(opts.skip_checking_sst_file_sizes_on_db_open, False) + opts.skip_checking_sst_file_sizes_on_db_open = True + self.assertEqual(opts.skip_checking_sst_file_sizes_on_db_open, True) + + def test_allow_2pc(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.allow_2pc) + self.assertEqual(opts.allow_2pc, False) + opts.allow_2pc = True + self.assertEqual(opts.allow_2pc, True) + + def test_row_cache(self): + opts = rocksdb.Options() + self.assertIsNone(opts.row_cache) + with self.assertRaises(Exception): + opts.row_cache = True + + def test_fail_if_options_file_error(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.fail_if_options_file_error) + self.assertEqual(opts.fail_if_options_file_error, False) + opts.fail_if_options_file_error = True + self.assertEqual(opts.fail_if_options_file_error, True) + + def test_dump_malloc_stats(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.dump_malloc_stats) + self.assertEqual(opts.dump_malloc_stats, False) + opts.dump_malloc_stats = True + self.assertEqual(opts.dump_malloc_stats, True) + + def test_avoid_flush_during_recovery(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.avoid_flush_during_recovery) + self.assertEqual(opts.avoid_flush_during_recovery, False) + opts.avoid_flush_during_recovery = True + self.assertEqual(opts.avoid_flush_during_recovery, True) + + def test_avoid_flush_during_shutdown(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.avoid_flush_during_shutdown) + self.assertEqual(opts.avoid_flush_during_shutdown, False) + opts.avoid_flush_during_shutdown = True + self.assertEqual(opts.avoid_flush_during_shutdown, True) + + def test_allow_ingest_behind(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.allow_ingest_behind) + self.assertEqual(opts.allow_ingest_behind, False) + opts.allow_ingest_behind = True + self.assertEqual(opts.allow_ingest_behind, True) + + def test_preserve_deletes(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.preserve_deletes) + self.assertEqual(opts.preserve_deletes, False) + opts.preserve_deletes = True + self.assertEqual(opts.preserve_deletes, True) + + def test_two_write_queues(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.two_write_queues) + self.assertEqual(opts.two_write_queues, False) + opts.two_write_queues = True + self.assertEqual(opts.two_write_queues, True) + + def test_manual_wal_flush(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.manual_wal_flush) + self.assertEqual(opts.manual_wal_flush, False) + opts.manual_wal_flush = True + self.assertEqual(opts.manual_wal_flush, True) + + def test_atomic_flush(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.atomic_flush) + self.assertEqual(opts.atomic_flush, False) + opts.atomic_flush = True + self.assertEqual(opts.atomic_flush, True) + + def test_avoid_unnecessary_blocking_io(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.avoid_unnecessary_blocking_io) + self.assertEqual(opts.avoid_unnecessary_blocking_io, False) + opts.avoid_unnecessary_blocking_io = True + self.assertEqual(opts.avoid_unnecessary_blocking_io, True) + + def test_write_dbid_to_manifest(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.write_dbid_to_manifest) + self.assertEqual(opts.write_dbid_to_manifest, False) + opts.write_dbid_to_manifest = True + self.assertEqual(opts.write_dbid_to_manifest, True) + + def test_log_readahead_size(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.log_readahead_size) + self.assertEqual(opts.log_readahead_size, 0) + opts.log_readahead_size = 10 + self.assertEqual(opts.log_readahead_size, 10) + + def test_best_efforts_recovery(self): + opts = rocksdb.Options() + self.assertIsNotNone(opts.best_efforts_recovery) + self.assertEqual(opts.best_efforts_recovery, False) + opts.best_efforts_recovery = True + self.assertEqual(opts.best_efforts_recovery, True) + + +