diff --git a/HISTORY.md b/HISTORY.md index a93b137baf5..d6ec32e290d 100644 --- a/HISTORY.md +++ b/HISTORY.md @@ -1,6 +1,9 @@ # Rocksdb Change Log + ## Unreleased + ### Bug Fixes + * Fix a race condition in WAL size tracking which is caused by an unsafe iterator access after container is changed. * Fix unprotected concurrent accesses to `WritableFileWriter::filesize_` by `DB::SyncWAL()` and `DB::Put()` in two write queue mode. * Fix a bug in WAL tracking. Before this PR (#10087), calling `SyncWAL()` on the only WAL file of the db will not log the event in MANIFEST, thus allowing a subsequent `DB::Open` even if the WAL file is missing or corrupted. @@ -8,18 +11,22 @@ * Fix a bug in which backup/checkpoint can include a WAL deleted by RocksDB. ### Public API changes + * Remove ReadOptions::iter_start_seqnum which has been deprecated. * Remove DBOptions::preserved_deletes and DB::SetPreserveDeletesSequenceNumber(). * Removed timestamp from WriteOptions. Accordingly, added to DB APIs Put, Delete, SingleDelete, etc. accepting an additional argument 'timestamp'. Added Put, Delete, SingleDelete, etc to WriteBatch accepting an additional argument 'timestamp'. Removed WriteBatch::AssignTimestamps(vector) API. Renamed WriteBatch::AssignTimestamp() to WriteBatch::UpdateTimestamps() with clarified comments. ### Performance Improvements + * Reduce DB mutex holding time when finding obsolete files to delete. When a file is trivial moved to another level, the internal files will be referenced twice internally and sometimes opened twice too. If a deletion candidate file is not the last reference, we need to destroy the reference and close the file but not deleting the file. Right now we determine it by building a set of all live files. With the improvement, we check the file against all live LSM-tree versions instead. ## New Features + * Improved the SstDumpTool to read the comparator from table properties and use it to read the SST File. * Add an extra sanity check in `GetSortedWalFiles()` (also used by `GetLiveFilesStorageInfo()`, `BackupEngine`, and `Checkpoint`) to reduce risk of successfully created backup or checkpoint failing to open because of missing WAL file. ## Behavior Changes + * For track_and_verify_wals_in_manifest, revert to the original behavior before #10087: syncing of live WAL file is not tracked, and we track only the synced sizes of **closed** WALs. (PR #10330). * DB::Write does not hold global `mutex_` if this db instance does not need to switch wal and mem-table (#7516). * If `CompactRange()` is called with `CompactRangeOptions::bottommost_level_compaction=kForce*` to compact from L0 to L1, RocksDB now will try to do trivial move from L0 to L1 and then do an intra L1 compaction, instead of a L0 to L1 compaction with trivial move disabled (#11375). @@ -27,7 +34,9 @@ For Leveled Compaction users, `CompactRange()` with `bottommost_level_compaction = BottommostLevelCompaction::kIfHaveCompactionFilter` will behave similar to `kForceOptimized` in that it will skip files created during this manual compaction when compacting files in the bottommost level. (#11468) ## 6.29.5 (03/29/2022) + ### Bug Fixes + * Fixed a race condition for `alive_log_files_` in non-two-write-queues mode. The race is between the write_thread_ in WriteToWAL() and another thread executing `FindObsoleteFiles()`. The race condition will be caught if `__glibcxx_requires_nonempty` is enabled. * Fixed a race condition when mmaping a WritableFile on POSIX. * Fixed a race condition when 2PC is disabled and WAL tracking in the MANIFEST is enabled. The race condition is between two background flush threads trying to install flush results, causing a WAL deletion not tracked in the MANIFEST. A future DB open may fail. @@ -35,7 +44,9 @@ For Leveled Compaction users, `CompactRange()` with `bottommost_level_compaction * Fixed a bug that `rocksdb.read.block.compaction.micros` cannot track compaction stats (#9722). ## 6.29.4 (03/22/2022) + ### Bug Fixes + * Fixed a bug caused by race among flush, incoming writes and taking snapshots. Queries to snapshots created with these race condition can return incorrect result, e.g. resurfacing deleted data. * Fixed a bug that DisableManualCompaction may assert when disable an unscheduled manual compaction. * Fixed a bug that `Iterator::Refresh()` reads stale keys after DeleteRange() performed. @@ -46,20 +57,29 @@ For Leveled Compaction users, `CompactRange()` with `bottommost_level_compaction * Fixed NUM_INDEX_AND_FILTER_BLOCKS_READ_PER_LEVEL, NUM_DATA_BLOCKS_READ_PER_LEVEL, and NUM_SST_READ_PER_LEVEL stats to be reported once per MultiGet batch per level. ## 6.29.3 (02/17/2022) + ### Bug Fixes + * Fix a data loss bug for 2PC write-committed transaction caused by concurrent transaction commit and memtable switch (#9571). ## 6.29.2 (02/15/2022) + ### Performance Improvements + * DisableManualCompaction() doesn't have to wait scheduled manual compaction to be executed in thread-pool to cancel the job. ## 6.29.1 (01/31/2022) + ### Bug Fixes + * Fixed a major bug in which batched MultiGet could return old values for keys deleted by DeleteRange when memtable Bloom filter is enabled (memtable_prefix_bloom_size_ratio > 0). (The fix includes a substantial MultiGet performance improvement in the unusual case of both memtable_whole_key_filtering and prefix_extractor.) ## 6.29.0 (01/21/2022) -Note: The next release will be major release 7.0. See https://github.com/facebook/rocksdb/issues/9390 for more info. + +Note: The next release will be major release 7.0. See for more info. + ### Public API change + * Added values to `TraceFilterType`: `kTraceFilterIteratorSeek`, `kTraceFilterIteratorSeekForPrev`, and `kTraceFilterMultiGet`. They can be set in `TraceOptions` to filter out the operation types after which they are named. * Added `TraceOptions::preserve_write_order`. When enabled it guarantees write records are traced in the same order they are logged to WAL and applied to the DB. By default it is disabled (false) to match the legacy behavior and prevent regression. * Made the Env class extend the Customizable class. Implementations need to be registered with the ObjectRegistry and to implement a Name() method in order to be created via this method. @@ -69,29 +89,37 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Added API warning against using `Iterator::Refresh()` together with `DB::DeleteRange()`, which are incompatible and have always risked causing the refreshed iterator to return incorrect results. ### Behavior Changes + * `DB::DestroyColumnFamilyHandle()` will return Status::InvalidArgument() if called with `DB::DefaultColumnFamily()`. * On 32-bit platforms, mmap reads are no longer quietly disabled, just discouraged. ### New Features + * Added `Options::DisableExtraChecks()` that can be used to improve peak write performance by disabling checks that should not be necessary in the absence of software logic errors or CPU+memory hardware errors. (Default options are slowly moving toward some performance overheads for extra correctness checking.) ### Performance Improvements + * Improved read performance when a prefix extractor is used (Seek, Get, MultiGet), even compared to version 6.25 baseline (see bug fix below), by optimizing the common case of prefix extractor compatible with table file and unchanging. ### Bug Fixes + * Fix a bug that FlushMemTable may return ok even flush not succeed. * Fixed a bug of Sync() and Fsync() not using `fcntl(F_FULLFSYNC)` on OS X and iOS. * Fixed a significant performance regression in version 6.26 when a prefix extractor is used on the read path (Seek, Get, MultiGet). (Excessive time was spent in SliceTransform::AsString().) ### New Features + * Added RocksJava support for MacOS universal binary (ARM+x86) ## 6.28.0 (2021-12-17) + ### New Features + * Introduced 'CommitWithTimestamp' as a new tag. Currently, there is no API for user to trigger a write with this tag to the WAL. This is part of the efforts to support write-commited transactions with user-defined timestamps. * Introduce SimulatedHybridFileSystem which can help simulating HDD latency in db_bench. Tiered Storage latency simulation can be enabled using -simulate_hybrid_fs_file (note that it doesn't work if db_bench is interrupted in the middle). -simulate_hdd can also be used to simulate all files on HDD. ### Bug Fixes + * Fixed a bug in rocksdb automatic implicit prefetching which got broken because of new feature adaptive_readahead and internal prefetching got disabled when iterator moves from one file to next. * Fixed a bug in TableOptions.prepopulate_block_cache which causes segmentation fault when used with TableOptions.partition_filters = true and TableOptions.cache_index_and_filter_blocks = true. * Fixed a bug affecting custom memtable factories which are not registered with the `ObjectRegistry`. The bug could result in failure to save the OPTIONS file. @@ -101,23 +129,29 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Fixed a bug in C bindings causing iterator to return incorrect result (#9343). ### Behavior Changes + * MemTableList::TrimHistory now use allocated bytes when max_write_buffer_size_to_maintain > 0(default in TrasactionDB, introduced in PR#5022) Fix #8371. ### Public API change + * Extend WriteBatch::AssignTimestamp and AssignTimestamps API so that both functions can accept an optional `checker` argument that performs additional checking on timestamp sizes. * Introduce a new EventListener callback that will be called upon the end of automatic error recovery. * Add IncreaseFullHistoryTsLow API so users can advance each column family's full_history_ts_low seperately. * Add GetFullHistoryTsLow API so users can query current full_history_low value of specified column family. ### Performance Improvements + * Replaced map property `TableProperties::properties_offsets` with uint64_t property `external_sst_file_global_seqno_offset` to save table properties's memory. * Block cache accesses are faster by RocksDB using cache keys of fixed size (16 bytes). ### Java API Changes + * Removed Java API `TableProperties.getPropertiesOffsets()` as it exposed internal details to external users. ## 6.27.0 (2021-11-19) + ### New Features + * Added new ChecksumType kXXH3 which is faster than kCRC32c on almost all x86\_64 hardware. * Added a new online consistency check for BlobDB which validates that the number/total size of garbage blobs does not exceed the number/total size of all blobs in any given blob file. * Provided support for tracking per-sst user-defined timestamp information in MANIFEST. @@ -128,6 +162,7 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Added compaction readahead support for blob files to the integrated BlobDB implementation, which can improve compaction performance when the database resides on higher-latency storage like HDDs or remote filesystems. Readahead can be configured using the column family option `blob_compaction_readahead_size`. ### Bug Fixes + * Prevent a `CompactRange()` with `CompactRangeOptions::change_level == true` from possibly causing corruption to the LSM state (overlapping files within a level) when run in parallel with another manual compaction. Note that setting `force_consistency_checks == true` (the default) would cause the DB to enter read-only mode in this scenario and return `Status::Corruption`, rather than committing any corruption. * Fixed a bug in CompactionIterator when write-prepared transaction is used. A released earliest write conflict snapshot may cause assertion failure in dbg mode and unexpected key in opt mode. * Fix ticker WRITE_WITH_WAL("rocksdb.write.wal"), this bug is caused by a bad extra `RecordTick(stats_, WRITE_WITH_WAL)` (at 2 place), this fix remove the extra `RecordTick`s and fix the corresponding test case. @@ -144,11 +179,13 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Fixed a bug that could, with WAL enabled, cause backups, checkpoints, and `GetSortedWalFiles()` to fail randomly with an error like `IO error: 001234.log: No such file or directory` ### Behavior Changes + * `NUM_FILES_IN_SINGLE_COMPACTION` was only counting the first input level files, now it's including all input files. * `TransactionUtil::CheckKeyForConflicts` can also perform conflict-checking based on user-defined timestamps in addition to sequence numbers. * Removed `GenericRateLimiter`'s minimum refill bytes per period previously enforced. ### Public API change + * When options.ttl is used with leveled compaction with compactinon priority kMinOverlappingRatio, files exceeding half of TTL value will be prioritized more, so that by the time TTL is reached, fewer extra compactions will be scheduled to clear them up. At the same time, when compacting files with data older than half of TTL, output files may be cut off based on those files' boundaries, in order for the early TTL compaction to work properly. * Made FileSystem and RateLimiter extend the Customizable class and added a CreateFromString method. Implementations need to be registered with the ObjectRegistry and to implement a Name() method in order to be created via this method. * Clarified in API comments that RocksDB is not exception safe for callbacks and custom extensions. An exception propagating into RocksDB can lead to undefined behavior, including data loss, unreported corruption, deadlocks, and more. @@ -161,13 +198,17 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Deprecating `ReadOptions::iter_start_seqnum` and `DBOptions::preserve_deletes`, please try using user defined timestamp feature instead. The options will be removed in a future release, currently it logs a warning message when using. ### Performance Improvements + * Released some memory related to filter construction earlier in `BlockBasedTableBuilder` for `FullFilter` and `PartitionedFilter` case (#9070) ### Behavior Changes + * `NUM_FILES_IN_SINGLE_COMPACTION` was only counting the first input level files, now it's including all input files. ## 6.26.0 (2021-10-20) + ### Bug Fixes + * Fixes a bug in directed IO mode when calling MultiGet() for blobs in the same blob file. The bug is caused by not sorting the blob read requests by file offsets. * Fix the incorrect disabling of SST rate limited deletion when the WAL and DB are in different directories. Only WAL rate limited deletion should be disabled if its in a different directory. * Fix `DisableManualCompaction()` to cancel compactions even when they are waiting on automatic compactions to drain due to `CompactRangeOptions::exclusive_manual_compactions == true`. @@ -180,6 +221,7 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Fix a bug in `BackupEngine` where some internal callers of `GenericRateLimiter::Request()` do not honor `bytes <= GetSingleBurstBytes()`. ### New Features + * Print information about blob files when using "ldb list_live_files_metadata" * Provided support for SingleDelete with user defined timestamp. * Experimental new function DB::GetLiveFilesStorageInfo offers essentially a unified version of other functions like GetLiveFiles, GetLiveFilesChecksumInfo, and GetSortedWalFiles. Checkpoints and backups could show small behavioral changes and/or improved performance as they now use this new API. @@ -190,6 +232,7 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Added `GetMapProperty()` support for "rocksdb.dbstats" (`DB::Properties::kDBStats`). As a map property, it includes DB-level internal stats accumulated over the DB's lifetime, such as user write related stats and uptime. ### Public API change + * Made SystemClock extend the Customizable class and added a CreateFromString method. Implementations need to be registered with the ObjectRegistry and to implement a Name() method in order to be created via this method. * Made SliceTransform extend the Customizable class and added a CreateFromString method. Implementations need to be registered with the ObjectRegistry and to implement a Name() method in order to be created via this method. The Capped and Prefixed transform classes return a short name (no length); use GetId for the fully qualified name. * Made FileChecksumGenFactory, SstPartitionerFactory, TablePropertiesCollectorFactory, and WalFilter extend the Customizable class and added a CreateFromString method. @@ -200,15 +243,19 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Even when options.max_compaction_bytes is hit, compaction output files are only cut when it aligns with grandparent files' boundaries. options.max_compaction_bytes could be slightly violated with the change, but the violation is no more than one target SST file size, which is usually much smaller. ### Performance Improvements + * Improved CPU efficiency of building block-based table (SST) files (#9039 and #9040). ### Java API Changes + * Add Java API bindings for new integrated BlobDB options * `keyMayExist()` supports ByteBuffer. -* Fix multiget throwing Null Pointer Exception for num of keys > 70k (https://github.com/facebook/rocksdb/issues/8039). +* Fix multiget throwing Null Pointer Exception for num of keys > 70k (). ## 6.25.0 (2021-09-20) + ### Bug Fixes + * Allow secondary instance to refresh iterator. Assign read seq after referencing SuperVersion. * Fixed a bug of secondary instance's last_sequence going backward, and reads on the secondary fail to see recent updates from the primary. * Fixed a bug that could lead to duplicate DB ID or DB session ID in POSIX environments without /proc/sys/kernel/random/uuid. @@ -224,6 +271,7 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Fix the bug that when direct I/O is used and MultiRead() returns a short result, RandomAccessFileReader::MultiRead() still returns full size buffer, with returned short value together with some data in original buffer. This bug is unlikely cause incorrect results, because (1) since FileSystem layer is expected to retry on short result, returning short results is only possible when asking more bytes in the end of the file, which RocksDB doesn't do when using MultiRead(); (2) checksum is unlikely to match. ### New Features + * RemoteCompaction's interface now includes `db_name`, `db_id`, `session_id`, which could help the user uniquely identify compaction job between db instances and sessions. * Added a ticker statistic, "rocksdb.verify_checksum.read.bytes", reporting how many bytes were read from file to serve `VerifyChecksum()` and `VerifyFileChecksums()` queries. * Added ticker statistics, "rocksdb.backup.read.bytes" and "rocksdb.backup.write.bytes", reporting how many bytes were read and written during backup. @@ -239,6 +287,7 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Charge memory usage during data buffering, from which training samples are gathered for dictionary compression, to block cache. Unbuffering data can now be triggered if the block cache becomes full and `strict_capacity_limit=true` for the block cache, in addition to existing conditions that can trigger unbuffering. ### Public API change + * Remove obsolete implementation details FullKey and ParseFullKey from public API * Change `SstFileMetaData::size` from `size_t` to `uint64_t`. * Made Statistics extend the Customizable class and added a CreateFromString method. Implementations of Statistics need to be registered with the ObjectRegistry and to implement a Name() method in order to be created via this method. @@ -248,10 +297,13 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Add a new field `level_at_creation` in `TablePropertiesCollectorFactory::Context` to capture the level at creating the SST file (i.e, table), of which the properties are being collected. ### Miscellaneous + * Add a paranoid check where in case FileSystem layer doesn't fill the buffer but returns succeed, checksum is unlikely to match even if buffer contains a previous block. The byte modified is not useful anyway, so it isn't expected to change any behavior when FileSystem is satisfying its contract. ## 6.24.0 (2021-08-20) + ### Bug Fixes + * If the primary's CURRENT file is missing or inaccessible, the secondary instance should not hang repeatedly trying to switch to a new MANIFEST. It should instead return the error code encountered while accessing the file. * Restoring backups with BackupEngine is now a logically atomic operation, so that if a restore operation is interrupted, DB::Open on it will fail. Using BackupEngineOptions::sync (default) ensures atomicity even in case of power loss or OS crash. * Fixed a race related to the destruction of `ColumnFamilyData` objects. The earlier logic unlocked the DB mutex before destroying the thread-local `SuperVersion` pointers, which could result in a process crash if another thread managed to get a reference to the `ColumnFamilyData` object. @@ -263,6 +315,7 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Fixed MultiGet not updating the block_read_count and block_read_byte PerfContext counters. ### New Features + * Made the EventListener extend the Customizable class. * EventListeners that have a non-empty Name() and that are registered with the ObjectRegistry can now be serialized to/from the OPTIONS file. * Insert warm blocks (data blocks, uncompressed dict blocks, index and filter blocks) in Block cache during flush under option BlockBasedTableOptions.prepopulate_block_cache. Previously it was enabled for only data blocks. @@ -277,19 +330,24 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Added hybrid configuration of Ribbon filter and Bloom filter where some LSM levels use Ribbon for memory space efficiency and some use Bloom for speed. See NewRibbonFilterPolicy. This also changes the default behavior of NewRibbonFilterPolicy to use Bloom for flushes under Leveled and Universal compaction and Ribbon otherwise. The C API function `rocksdb_filterpolicy_create_ribbon` is unchanged but adds new `rocksdb_filterpolicy_create_ribbon_hybrid`. ### Public API change + * Added APIs to decode and replay trace file via Replayer class. Added `DB::NewDefaultReplayer()` to create a default Replayer instance. Added `TraceReader::Reset()` to restart reading a trace file. Created trace_record.h, trace_record_result.h and utilities/replayer.h files to access the decoded Trace records, replay them, and query the actual operation results. * Added Configurable::GetOptionsMap to the public API for use in creating new Customizable classes. * Generalized bits_per_key parameters in C API from int to double for greater configurability. Although this is a compatible change for existing C source code, anything depending on C API signatures, such as foreign function interfaces, will need to be updated. ### Performance Improvements + * Try to avoid updating DBOptions if `SetDBOptions()` does not change any option value. ### Behavior Changes + * `StringAppendOperator` additionally accepts a string as the delimiter. * BackupEngineOptions::sync (default true) now applies to restoring backups in addition to creating backups. This could slow down restores, but ensures they are fully persisted before returning OK. (Consider increasing max_background_operations to improve performance.) ## 6.23.0 (2021-07-16) + ### Bug Fixes + * Blob file checksums are now printed in hexadecimal format when using the `manifest_dump` `ldb` command. * `GetLiveFilesMetaData()` now populates the `temperature`, `oldest_ancester_time`, and `file_creation_time` fields of its `LiveFileMetaData` results when the information is available. Previously these fields always contained zero indicating unknown. * Fix mismatches of OnCompaction{Begin,Completed} in case of DisableManualCompaction(). @@ -299,6 +357,7 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Fixed handling of DBOptions::wal_dir with LoadLatestOptions() or ldb --try_load_options on a copied or moved DB. Previously, when the WAL directory is same as DB directory (default), a copied or moved DB would reference the old path of the DB as the WAL directory, potentially corrupting both copies. Under this change, the wal_dir from DB::GetOptions() or LoadLatestOptions() may now be empty, indicating that the current DB directory is used for WALs. This is also a subtle API change. ### New Features + * ldb has a new feature, `list_live_files_metadata`, that shows the live SST files, as well as their LSM storage level and the column family they belong to. * The new BlobDB implementation now tracks the amount of garbage in each blob file in the MANIFEST. * Integrated BlobDB now supports Merge with base values (Put/Delete etc.). @@ -306,14 +365,19 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Expose statistics option in RemoteCompaction worker. ### Public API change + * Added APIs to the Customizable class to allow developers to create their own Customizable classes. Created the utilities/customizable_util.h file to contain helper methods for developing new Customizable classes. * Change signature of SecondaryCache::Name(). Make SecondaryCache customizable and add SecondaryCache::CreateFromString method. ## 6.22.0 (2021-06-18) + ### Behavior Changes + * Added two additional tickers, MEMTABLE_PAYLOAD_BYTES_AT_FLUSH and MEMTABLE_GARBAGE_BYTES_AT_FLUSH. These stats can be used to estimate the ratio of "garbage" (outdated) bytes in the memtable that are discarded at flush time. * Added API comments clarifying safe usage of Disable/EnableManualCompaction and EventListener callbacks for compaction. + ### Bug Fixes + * fs_posix.cc GetFreeSpace() always report disk space available to root even when running as non-root. Linux defaults often have disk mounts with 5 to 10 percent of total space reserved only for root. Out of space could result for non-root users. * Subcompactions are now disabled when user-defined timestamps are used, since the subcompaction boundary picking logic is currently not timestamp-aware, which could lead to incorrect results when different subcompactions process keys that only differ by timestamp. * Fix an issue that `DeleteFilesInRange()` may cause ongoing compaction reports corruption exception, or ASSERT for debug build. There's no actual data loss or corruption that we find. @@ -321,19 +385,24 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Fixed performance bugs in background gathering of block cache entry statistics, that could consume a lot of CPU when there are many column families with a shared block cache. ### New Features + * Marked the Ribbon filter and optimize_filters_for_memory features as production-ready, each enabling memory savings for Bloom-like filters. Use `NewRibbonFilterPolicy` in place of `NewBloomFilterPolicy` to use Ribbon filters instead of Bloom, or `ribbonfilter` in place of `bloomfilter` in configuration string. * Allow `DBWithTTL` to use `DeleteRange` api just like other DBs. `DeleteRangeCF()` which executes `WriteBatchInternal::DeleteRange()` has been added to the handler in `DBWithTTLImpl::Write()` to implement it. * Add BlockBasedTableOptions.prepopulate_block_cache. If enabled, it prepopulate warm/hot data blocks which are already in memory into block cache at the time of flush. On a flush, the data block that is in memory (in memtables) get flushed to the device. If using Direct IO, additional IO is incurred to read this data back into memory again, which is avoided by enabling this option and it also helps with Distributed FileSystem. More details in include/rocksdb/table.h. * Added a `cancel` field to `CompactRangeOptions`, allowing individual in-process manual range compactions to be cancelled. ### New Features + * Added BlobMetaData to the ColumnFamilyMetaData to return information about blob files ### Public API change + * Added GetAllColumnFamilyMetaData API to retrieve the ColumnFamilyMetaData about all column families. ## 6.21.0 (2021-05-21) + ### Bug Fixes + * Fixed a bug in handling file rename error in distributed/network file systems when the server succeeds but client returns error. The bug can cause CURRENT file to point to non-existing MANIFEST file, thus DB cannot be opened. * Fixed a bug where ingested files were written with incorrect boundary key metadata. In rare cases this could have led to a level's files being wrongly ordered and queries for the boundary keys returning wrong results. * Fixed a data race between insertion into memtables and the retrieval of the DB properties `rocksdb.cur-size-active-mem-table`, `rocksdb.cur-size-all-mem-tables`, and `rocksdb.size-all-mem-tables`. @@ -344,9 +413,11 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Fixed a bug that `AdvancedColumnFamilyOptions.max_compaction_bytes` is under-calculated for manual compaction (`CompactRange()`). Manual compaction is split to multiple compactions if the compaction size exceed the `max_compaction_bytes`. The bug creates much larger compaction which size exceed the user setting. On the other hand, larger manual compaction size can increase the subcompaction parallelism, you can tune that by setting `max_compaction_bytes`. ### Behavior Changes + * Due to the fix of false-postive alert of "SST file is ahead of WAL", all the CFs with no SST file (CF empty) will bypass the consistency check. We fixed a false-positive, but introduced a very rare true-negative which will be triggered in the following conditions: A CF with some delete operations in the last a few queries which will result in an empty CF (those are flushed to SST file and a compaction triggered which combines this file and all other SST files and generates an empty CF, or there is another reason to write a manifest entry for this CF after a flush that generates no SST file from an empty CF). The deletion entries are logged in a WAL and this WAL was corrupted, while the CF's log number points to the next WAL (due to the flush). Therefore, the DB can only recover to the point without these trailing deletions and cause the inconsistent DB status. ### New Features + * Add new option allow_stall passed during instance creation of WriteBufferManager. When allow_stall is set, WriteBufferManager will stall all writers shared across multiple DBs and columns if memory usage goes beyond specified WriteBufferManager::buffer_size (soft limit). Stall will be cleared when memory is freed after flush and memory usage goes down below buffer_size. * Allow `CompactionFilter`s to apply in more table file creation scenarios such as flush and recovery. For compatibility, `CompactionFilter`s by default apply during compaction. Users can customize this behavior by overriding `CompactionFilterFactory::ShouldFilterTableFileCreation()`. * Added more fields to FilterBuildingContext with LSM details, for custom filter policies that vary behavior based on where they are in the LSM-tree. @@ -356,9 +427,11 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Add `TableProperties::num_filter_entries`, which can be used with `TableProperties::filter_size` to calculate the effective bits per filter entry (unique user key or prefix) for a table file. ### Performance Improvements + * BlockPrefetcher is used by iterators to prefetch data if they anticipate more data to be used in future. It is enabled implicitly by rocksdb. Added change to take in account read pattern if reads are sequential. This would disable prefetching for random reads in MultiGet and iterators as readahead_size is increased exponential doing large prefetches. ### Public API change + * Removed a parameter from TableFactory::NewTableBuilder, which should not be called by user code because TableBuilder is not a public API. * Removed unused structure `CompactionFilterContext`. * The `skip_filters` parameter to SstFileWriter is now considered deprecated. Use `BlockBasedTableOptions::filter_policy` to control generation of filters. @@ -369,13 +442,17 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Deprecated backupable_db.h and BackupableDBOptions in favor of new versions with appropriate names: backup_engine.h and BackupEngineOptions. Old API compatibility is preserved. ### Default Option Change + * When options.arena_block_size <= 0 (default value 0), still use writer_buffer_size / 8 but cap to 1MB. Too large alloation size might not be friendly to allocator and might cause performance issues in extreme cases. ### Build + * By default, try to build with liburing. For make, if ROCKSDB_USE_IO_URING is not set, treat as enable, which means RocksDB will try to build with liburing. Users can disable it with ROCKSDB_USE_IO_URING=0. For cmake, add WITH_LIBURING to control it, with default on. ## 6.20.0 (2021-04-16) + ### Behavior Changes + * `ColumnFamilyOptions::sample_for_compression` now takes effect for creation of all block-based tables. Previously it only took effect for block-based tables created by flush. * `CompactFiles()` can no longer compact files from lower level to up level, which has the risk to corrupt DB (details: #8063). The validation is also added to all compactions. * Fixed some cases in which DB::OpenForReadOnly() could write to the filesystem. If you want a Logger with a read-only DB, you must now set DBOptions::info_log yourself, such as using CreateLoggerFromOptions(). @@ -384,6 +461,7 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Improve the behavior of WriteBatchWithIndex for Merge operations. Now more operations may be stored in order to return the correct merged result. ### Bug Fixes + * Use thread-safe `strerror_r()` to get error messages. * Fixed a potential hang in shutdown for a DB whose `Env` has high-pri thread pool disabled (`Env::GetBackgroundThreads(Env::Priority::HIGH) == 0`) * Made BackupEngine thread-safe and added documentation comments to clarify what is safe for multiple BackupEngine objects accessing the same backup directory. @@ -392,15 +470,18 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Fixed a bug that allowed `DBOptions::max_open_files` to be set with a non-negative integer with `ColumnFamilyOptions::compaction_style = kCompactionStyleFIFO`. ### Performance Improvements + * On ARM platform, use `yield` instead of `wfe` to relax cpu to gain better performance. ### Public API change + * Added `TableProperties::slow_compression_estimated_data_size` and `TableProperties::fast_compression_estimated_data_size`. When `ColumnFamilyOptions::sample_for_compression > 0`, they estimate what `TableProperties::data_size` would have been if the "fast" or "slow" (see `ColumnFamilyOptions::sample_for_compression` API doc for definitions) compression had been used instead. * Update DB::StartIOTrace and remove Env object from the arguments as its redundant and DB already has Env object that is passed down to IOTracer::StartIOTrace * Added `FlushReason::kWalFull`, which is reported when a memtable is flushed due to the WAL reaching its size limit; those flushes were previously reported as `FlushReason::kWriteBufferManager`. Also, changed the reason for flushes triggered by the write buffer manager to `FlushReason::kWriteBufferManager`; they were previously reported as `FlushReason::kWriteBufferFull`. * Extend file_checksum_dump ldb command and DB::GetLiveFilesChecksumInfo API for IntegratedBlobDB and get checksum of blob files along with SST files. ### New Features + * Added the ability to open BackupEngine backups as read-only DBs, using BackupInfo::name_for_open and env_for_open provided by BackupEngine::GetBackupInfo() with include_file_details=true. * Added BackupEngine support for integrated BlobDB, with blob files shared between backups when table files are shared. Because of current limitations, blob files always use the kLegacyCrc32cAndFileSize naming scheme, and incremental backups must read and checksum all blob files in a DB, even for files that are already backed up. * Added an optional output parameter to BackupEngine::CreateNewBackup(WithMetadata) to return the BackupID of the new backup. @@ -408,11 +489,14 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Made the Ribbon filter a long-term supported feature in terms of the SST schema(compatible with version >= 6.15.0) though the API for enabling it is expected to change. ## 6.19.0 (2021-03-21) + ### Bug Fixes + * Fixed the truncation error found in APIs/tools when dumping block-based SST files in a human-readable format. After fix, the block-based table can be fully dumped as a readable file. * When hitting a write slowdown condition, no write delay (previously 1 millisecond) is imposed until `delayed_write_rate` is actually exceeded, with an initial burst allowance of 1 millisecond worth of bytes. Also, beyond the initial burst allowance, `delayed_write_rate` is now more strictly enforced, especially with multiple column families. ### Public API change + * Changed default `BackupableDBOptions::share_files_with_checksum` to `true` and deprecated `false` because of potential for data loss. Note that accepting this change in behavior can temporarily increase backup data usage because files are not shared between backups using the two different settings. Also removed obsolete option kFlagMatchInterimNaming. * Add a new option BlockBasedTableOptions::max_auto_readahead_size. RocksDB does auto-readahead for iterators on noticing more than two reads for a table file if user doesn't provide readahead_size. The readahead starts at 8KB and doubles on every additional read upto max_auto_readahead_size and now max_auto_readahead_size can be configured dynamically as well. Found that 256 KB readahead size provides the best performance, based on experiments, for auto readahead. Experiment data is in PR #3282. If value is set 0 then no automatic prefetching will be done by rocksdb. Also changing the value will only affect files opened after the change. * Add suppport to extend DB::VerifyFileChecksums API to also verify blob files checksum. @@ -426,6 +510,7 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Add new Append and PositionedAppend API with checksum handoff to legacy Env. ### New Features + * Support compaction filters for the new implementation of BlobDB. Add `FilterBlobByKey()` to `CompactionFilter`. Subclasses can override this method so that compaction filters can determine whether the actual blob value has to be read during compaction. Use a new `kUndetermined` in `CompactionFilter::Decision` to indicated that further action is necessary for compaction filter to make a decision. * Add support to extend retrieval of checksums for blob files from the MANIFEST when checkpointing. During backup, rocksdb can detect corruption in blob files during file copies. * Add new options for db_bench --benchmarks: flush, waitforcompaction, compact0, compact1. @@ -434,14 +519,18 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Add statistics and info log for error handler: counters for bg error, bg io error, bg retryable io error, auto resume count, auto resume total retry number, and auto resume sucess; Histogram for auto resume retry count in each recovery call. Note that, each auto resume attempt will have one or multiple retries. ### Behavior Changes + * During flush, only WAL sync retryable IO error is mapped to hard error, which will stall the writes. When WAL is used but only SST file write has retryable IO error, it will be mapped to soft error and write will not be affected. ## 6.18.0 (2021-02-19) + ### Behavior Changes + * When retryable IO error occurs during compaction, it is mapped to soft error and set the BG error. However, auto resume is not called to clean the soft error since compaction will reschedule by itself. In this change, When retryable IO error occurs during compaction, BG error is not set. User will be informed the error via EventHelper. * Introduce a new trace file format for query tracing and replay and trace file version is bump up to 0.2. A payload map is added as the first portion of the payload. We will not have backward compatible issues when adding new entries to trace records. Added the iterator_upper_bound and iterator_lower_bound in Seek and SeekForPrev tracing function. Added them as the new payload member for iterator tracing. ### New Features + * Add support for key-value integrity protection in live updates from the user buffers provided to `WriteBatch` through the write to RocksDB's in-memory update buffer (memtable). This is intended to detect some cases of in-memory data corruption, due to either software or hardware errors. Users can enable protection by constructing their `WriteBatch` with `protection_bytes_per_key == 8`. * Add support for updating `full_history_ts_low` option in manual compaction, which is for old timestamp data GC. * Add a mechanism for using Makefile to build external plugin code into the RocksDB libraries/binaries. This intends to simplify compatibility and distribution for plugins (e.g., special-purpose `FileSystem`s) whose source code resides outside the RocksDB repo. See "plugin/README.md" for developer details, and "PLUGINS.md" for a listing of available plugins. @@ -449,36 +538,44 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * A new, experimental version of BlobDB (key-value separation) is now available. The new implementation is integrated into the RocksDB core, i.e. it is accessible via the usual `rocksdb::DB` API, as opposed to the separate `rocksdb::blob_db::BlobDB` interface used by the earlier version, and can be configured on a per-column family basis using the configuration options `enable_blob_files`, `min_blob_size`, `blob_file_size`, `blob_compression_type`, `enable_blob_garbage_collection`, and `blob_garbage_collection_age_cutoff`. It extends RocksDB's consistency guarantees to blobs, and offers more features and better performance. Note that some features, most notably `Merge`, compaction filters, and backup/restore are not yet supported, and there is no support for migrating a database created by the old implementation. ### Bug Fixes + * Since 6.15.0, `TransactionDB` returns error `Status`es from calls to `DeleteRange()` and calls to `Write()` where the `WriteBatch` contains a range deletion. Previously such operations may have succeeded while not providing the expected transactional guarantees. There are certain cases where range deletion can still be used on such DBs; see the API doc on `TransactionDB::DeleteRange()` for details. * `OptimisticTransactionDB` now returns error `Status`es from calls to `DeleteRange()` and calls to `Write()` where the `WriteBatch` contains a range deletion. Previously such operations may have succeeded while not providing the expected transactional guarantees. * Fix `WRITE_PREPARED`, `WRITE_UNPREPARED` TransactionDB `MultiGet()` may return uncommitted data with snapshot. * In DB::OpenForReadOnly, if any error happens while checking Manifest file path, it was overridden by Status::NotFound. It has been fixed and now actual error is returned. ### Public API Change + * Added a "only_mutable_options" flag to the ConfigOptions. When this flag is "true", the Configurable functions and convenience methods (such as GetDBOptionsFromString) will only deal with options that are marked as mutable. When this flag is true, only options marked as mutable can be configured (a Status::InvalidArgument will be returned) and options not marked as mutable will not be returned or compared. The default is "false", meaning to compare all options. * Add new Append and PositionedAppend APIs to FileSystem to bring the data verification information (data checksum information) from upper layer (e.g., WritableFileWriter) to the storage layer. In this way, the customized FileSystem is able to verify the correctness of data being written to the storage on time. Add checksum_handoff_file_types to DBOptions. User can use this option to control which file types (Currently supported file tyes: kWALFile, kTableFile, kDescriptorFile.) should use the new Append and PositionedAppend APIs to handoff the verification information. Currently, RocksDB only use crc32c to calculate the checksum for write handoff. * Add an option, `CompressionOptions::max_dict_buffer_bytes`, to limit the in-memory buffering for selecting samples for generating/training a dictionary. The limit is currently loosely adhered to. - ## 6.17.0 (2021-01-15) + ### Behavior Changes + * When verifying full file checksum with `DB::VerifyFileChecksums()`, we now fail with `Status::InvalidArgument` if the name of the checksum generator used for verification does not match the name of the checksum generator used for protecting the file when it was created. * Since RocksDB does not continue write the same file if a file write fails for any reason, the file scope write IO error is treated the same as retryable IO error. More information about error handling of file scope IO error is included in `ErrorHandler::SetBGError`. ### Bug Fixes + * Version older than 6.15 cannot decode VersionEdits `WalAddition` and `WalDeletion`, fixed this by changing the encoded format of them to be ignorable by older versions. * Fix a race condition between DB startups and shutdowns in managing the periodic background worker threads. One effect of this race condition could be the process being terminated. ### Public API Change + * Add a public API WriteBufferManager::dummy_entries_in_cache_usage() which reports the size of dummy entries stored in cache (passed to WriteBufferManager). Dummy entries are used to account for DataBlocks. * Add a SystemClock class that contains the time-related methods from Env. The original methods in Env may be deprecated in a future release. This class will allow easier testing, development, and expansion of time-related features. * Add a public API GetRocksBuildProperties and GetRocksBuildInfoAsString to get properties about the current build. These properties may include settings related to the GIT settings (branch, timestamp). This change also sets the "build date" based on the GIT properties, rather than the actual build time, thereby enabling more reproducible builds. ## 6.16.0 (2020-12-18) + ### Behavior Changes + * Attempting to write a merge operand without explicitly configuring `merge_operator` now fails immediately, causing the DB to enter read-only mode. Previously, failure was deferred until the `merge_operator` was needed by a user read or a background operation. ### Bug Fixes + * Truncated WALs ending in incomplete records can no longer produce gaps in the recovered data when `WALRecoveryMode::kPointInTimeRecovery` is used. Gaps are still possible when WALs are truncated exactly on record boundaries; for complete protection, users should enable `track_and_verify_wals_in_manifest`. * Fix a bug where compressed blocks read by MultiGet are not inserted into the compressed block cache when use_direct_reads = true. * Fixed the issue of full scanning on obsolete files when there are too many outstanding compactions with ConcurrentTaskLimiter enabled. @@ -489,11 +586,13 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * When ldb --try_load_options is used with the --column_family option, the ColumnFamilyOptions for the specified column family was not loaded from the OPTIONS file. Fix it so its loaded from OPTIONS and then overridden with command line overrides. ### New Features + * User defined timestamp feature supports `CompactRange` and `GetApproximateSizes`. * Support getting aggregated table properties (kAggregatedTableProperties and kAggregatedTablePropertiesAtLevel) with DB::GetMapProperty, for easier access to the data in a structured format. * Experimental option BlockBasedTableOptions::optimize_filters_for_memory now works with experimental Ribbon filter (as well as Bloom filter). ### Public API Change + * Deprecated public but rarely-used FilterBitsBuilder::CalculateNumEntry, which is replaced with ApproximateNumEntries taking a size_t parameter and returning size_t. * To improve portability the functions `Env::GetChildren` and `Env::GetChildrenFileAttributes` will no longer return entries for the special directories `.` or `..`. * Added a new option `track_and_verify_wals_in_manifest`. If `true`, the log numbers and sizes of the synced WALs are tracked in MANIFEST, then during DB recovery, if a synced WAL is missing from disk, or the WAL's size does not match the recorded size in MANIFEST, an error will be reported and the recovery will be aborted. Note that this option does not work with secondary instance. @@ -501,7 +600,9 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * All overloads of DB::GetApproximateSizes now return Status, so that any failure to obtain the sizes is indicated to the caller. ## 6.15.0 (2020-11-13) + ### Bug Fixes + * Fixed a bug in the following combination of features: indexes with user keys (`format_version >= 3`), indexes are partitioned (`index_type == kTwoLevelIndexSearch`), and some index partitions are pinned in memory (`BlockBasedTableOptions::pin_l0_filter_and_index_blocks_in_cache`). The bug could cause keys to be truncated when read from the index leading to wrong read results or other unexpected behavior. * Fixed a bug when indexes are partitioned (`index_type == kTwoLevelIndexSearch`), some index partitions are pinned in memory (`BlockBasedTableOptions::pin_l0_filter_and_index_blocks_in_cache`), and partitions reads could be mixed between block cache and directly from the file (e.g., with `enable_index_compression == 1` and `mmap_read == 1`, partitions that were stored uncompressed due to poor compression ratio would be read directly from the file via mmap, while partitions that were stored compressed would be read from block cache). The bug could cause index partitions to be mistakenly considered empty during reads leading to wrong read results. * Since 6.12, memtable lookup should report unrecognized value_type as corruption (#7121). @@ -518,22 +619,28 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Fixed a bug of a recovery corner case, details in PR7621. ### Public API Change + * Deprecate `BlockBasedTableOptions::pin_l0_filter_and_index_blocks_in_cache` and `BlockBasedTableOptions::pin_top_level_index_and_filter`. These options still take effect until users migrate to the replacement APIs in `BlockBasedTableOptions::metadata_cache_options`. Migration guidance can be found in the API comments on the deprecated options. * Add new API `DB::VerifyFileChecksums` to verify SST file checksum with corresponding entries in the MANIFEST if present. Current implementation requires scanning and recomputing file checksums. ### Behavior Changes + * The dictionary compression settings specified in `ColumnFamilyOptions::compression_opts` now additionally affect files generated by flush and compaction to non-bottommost level. Previously those settings at most affected files generated by compaction to bottommost level, depending on whether `ColumnFamilyOptions::bottommost_compression_opts` overrode them. Users who relied on dictionary compression settings in `ColumnFamilyOptions::compression_opts` affecting only the bottommost level can keep the behavior by moving their dictionary settings to `ColumnFamilyOptions::bottommost_compression_opts` and setting its `enabled` flag. * When the `enabled` flag is set in `ColumnFamilyOptions::bottommost_compression_opts`, those compression options now take effect regardless of the value in `ColumnFamilyOptions::bottommost_compression`. Previously, those compression options only took effect when `ColumnFamilyOptions::bottommost_compression != kDisableCompressionOption`. Now, they additionally take effect when `ColumnFamilyOptions::bottommost_compression == kDisableCompressionOption` (such a setting causes bottommost compression type to fall back to `ColumnFamilyOptions::compression_per_level` if configured, and otherwise fall back to `ColumnFamilyOptions::compression`). ### New Features + * An EXPERIMENTAL new Bloom alternative that saves about 30% space compared to Bloom filters, with about 3-4x construction time and similar query times is available using NewExperimentalRibbonFilterPolicy. ## 6.14 (2020-10-09) + ### Bug fixes + * Fixed a bug after a `CompactRange()` with `CompactRangeOptions::change_level` set fails due to a conflict in the level change step, which caused all subsequent calls to `CompactRange()` with `CompactRangeOptions::change_level` set to incorrectly fail with a `Status::NotSupported("another thread is refitting")` error. * Fixed a bug that the bottom most level compaction could still be a trivial move even if `BottommostLevelCompaction.kForce` or `kForceOptimized` is set. ### Public API Change + * The methods to create and manage EncrypedEnv have been changed. The EncryptionProvider is now passed to NewEncryptedEnv as a shared pointer, rather than a raw pointer. Comparably, the CTREncryptedProvider now takes a shared pointer, rather than a reference, to a BlockCipher. CreateFromString methods have been added to BlockCipher and EncryptionProvider to provide a single API by which different ciphers and providers can be created, respectively. * The internal classes (CTREncryptionProvider, ROT13BlockCipher, CTRCipherStream) associated with the EncryptedEnv have been moved out of the public API. To create a CTREncryptionProvider, one can either use EncryptionProvider::NewCTRProvider, or EncryptionProvider::CreateFromString("CTR"). To create a new ROT13BlockCipher, one can either use BlockCipher::NewROT13Cipher or BlockCipher::CreateFromString("ROT13"). * The EncryptionProvider::AddCipher method has been added to allow keys to be added to an EncryptionProvider. This API will allow future providers to support multiple cipher keys. @@ -541,6 +648,7 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * AdvancedColumnFamilyOptions::force_consistency_checks is now true by default, for more proactive DB corruption detection at virtually no cost (estimated two extra CPU cycles per million on a major production workload). Corruptions reported by these checks now mention "force_consistency_checks" in case a false positive corruption report is suspected and the option needs to be disabled (unlikely). Since existing column families have a saved setting for force_consistency_checks, only new column families will pick up the new default. ### General Improvements + * The settings of the DBOptions and ColumnFamilyOptions are now managed by Configurable objects (see New Features). The same convenience methods to configure these options still exist but the backend implementation has been unified under a common implementation. ### New Features @@ -553,7 +661,9 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Methods to create customizable extensions -- such as TableFactory -- are exposed directly through the Customizable base class (from which these objects inherit). This change will allow these Customizable classes to be loaded and configured in a standard way (via CreateFromString). More information on how to write and use Customizable classes is in the customizable.h header file. ## 6.13 (2020-09-12) + ### Bug fixes + * Fix a performance regression introduced in 6.4 that makes a upper bound check for every Next() even if keys are within a data block that is within the upper bound. * Fix a possible corruption to the LSM state (overlapping files within a level) when a `CompactRange()` for refitting levels (`CompactRangeOptions::change_level == true`) and another manual compaction are executed in parallel. * Sanitize `recycle_log_file_num` to zero when the user attempts to enable it in combination with `WALRecoveryMode::kTolerateCorruptedTailRecords`. Previously the two features were allowed together, which compromised the user's configured crash-recovery guarantees. @@ -567,10 +677,12 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Fix a bug in which bottommost compaction continues to advance the underlying InternalIterator to skip tombstones even after shutdown. ### New Features + * A new field `std::string requested_checksum_func_name` is added to `FileChecksumGenContext`, which enables the checksum factory to create generators for a suite of different functions. * Added a new subcommand, `ldb unsafe_remove_sst_file`, which removes a lost or corrupt SST file from a DB's metadata. This command involves data loss and must not be used on a live DB. ### Performance Improvements + * Reduce thread number for multiple DB instances by re-using one global thread for statistics dumping and persisting. * Reduce write-amp in heavy write bursts in `kCompactionStyleLevel` compaction style with `level_compaction_dynamic_level_bytes` set. * BackupEngine incremental backups no longer read DB table files that are already saved to a shared part of the backup directory, unless `share_files_with_checksum` is used with `kLegacyCrc32cAndFileSize` naming (discouraged). @@ -579,6 +691,7 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * `DB::VerifyChecksum` and `BackupEngine::VerifyBackup` with checksum checking are still able to catch corruptions that `CreateNewBackup` does not. ### Public API Change + * Expose kTypeDeleteWithTimestamp in EntryType and update GetEntryType() accordingly. * Added file_checksum and file_checksum_func_name to TableFileCreationInfo, which can pass the table file checksum information through the OnTableFileCreated callback during flush and compaction. * A warning is added to `DB::DeleteFile()` API describing its known problems and deprecation plan. @@ -586,15 +699,19 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Added a new status code IOStatus::IOFenced() for the Env/FileSystem to indicate that writes from this instance are fenced off. Like any other background error, this error is returned to the user in Put/Merge/Delete/Flush calls and can be checked using Status::IsIOFenced(). ### Behavior Changes + * File abstraction `FSRandomAccessFile.Prefetch()` default return status is changed from `OK` to `NotSupported`. If the user inherited file doesn't implement prefetch, RocksDB will create internal prefetch buffer to improve read performance. * When retryabel IO error happens during Flush (manifest write error is excluded) and WAL is disabled, originally it is mapped to kHardError. Now,it is mapped to soft error. So DB will not stall the writes unless the memtable is full. At the same time, when auto resume is triggered to recover the retryable IO error during Flush, SwitchMemtable is not called to avoid generating to many small immutable memtables. If WAL is enabled, no behavior changes. * When considering whether a table file is already backed up in a shared part of backup directory, BackupEngine would already query the sizes of source (DB) and pre-existing destination (backup) files. BackupEngine now uses these file sizes to detect corruption, as at least one of (a) old backup, (b) backup in progress, or (c) current DB is corrupt if there's a size mismatch. ### Others + * Error in prefetching partitioned index blocks will not be swallowed. It will fail the query and return the IOError users. ## 6.12 (2020-07-28) + ### Public API Change + * Encryption file classes now exposed for inheritance in env_encryption.h * File I/O listener is extended to cover more I/O operations. Now class `EventListener` in listener.h contains new callback functions: `OnFileFlushFinish()`, `OnFileSyncFinish()`, `OnFileRangeSyncFinish()`, `OnFileTruncateFinish()`, and ``OnFileCloseFinish()``. * `FileOperationInfo` now reports `duration` measured by `std::chrono::steady_clock` and `start_ts` measured by `std::chrono::system_clock` instead of start and finish timestamps measured by `system_clock`. Note that `system_clock` is called before `steady_clock` in program order at operation starts. @@ -603,6 +720,7 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * A parameter `verify_with_checksum` is added to `BackupEngine::VerifyBackup`, which is false by default. If it is ture, `BackupEngine::VerifyBackup` verifies checksums and file sizes of backup files. Pass `false` for `verify_with_checksum` to maintain the previous behavior and performance of `BackupEngine::VerifyBackup`, by only verifying sizes of backup files. ### Behavior Changes + * Best-efforts recovery ignores CURRENT file completely. If CURRENT file is missing during recovery, best-efforts recovery still proceeds with MANIFEST file(s). * In best-efforts recovery, an error that is not Corruption or IOError::kNotFound or IOError::kPathNotFound will be overwritten silently. Fix this by checking all non-ok cases and return early. * When `file_checksum_gen_factory` is set to `GetFileChecksumGenCrc32cFactory()`, BackupEngine will compare the crc32c checksums of table files computed when creating a backup to the expected checksums stored in the DB manifest, and will fail `CreateNewBackup()` on mismatch (corruption). If the `file_checksum_gen_factory` is not set or set to any other customized factory, there is no checksum verification to detect if SST files in a DB are corrupt when read, copied, and independently checksummed by BackupEngine. @@ -610,6 +728,7 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * When the paranoid_file_checks option is true, a hash is generated of all keys and values are generated when the SST file is written, and then the values are read back in to validate the file. A corruption is signaled if the two hashes do not match. ### Bug fixes + * Compressed block cache was automatically disabled with read-only DBs by mistake. Now it is fixed: compressed block cache will be in effective with read-only DB too. * Fix a bug of wrong iterator result if another thread finishes an update and a DB flush between two statement. * Disable file deletion after MANIFEST write/sync failure until db re-open or Resume() so that subsequent re-open will not see MANIFEST referencing deleted SSTs. @@ -620,6 +739,7 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Fixed a bug in size-amp-triggered and periodic-triggered universal compaction, where the compression settings for the first input level were used rather than the compression settings for the output (bottom) level. ### New Features + * DB identity (`db_id`) and DB session identity (`db_session_id`) are added to table properties and stored in SST files. SST files generated from SstFileWriter and Repairer have DB identity “SST Writer” and “DB Repairer”, respectively. Their DB session IDs are generated in the same way as `DB::GetDbSessionId`. The session ID for SstFileWriter (resp., Repairer) resets every time `SstFileWriter::Open` (resp., `Repairer::Run`) is called. * Added experimental option BlockBasedTableOptions::optimize_filters_for_memory for reducing allocated memory size of Bloom filters (~10% savings with Jemalloc) while preserving the same general accuracy. To have an effect, the option requires format_version=5 and malloc_usable_size. Enabling this option is forward and backward compatible with existing format_version=5. * `BackupableDBOptions::share_files_with_checksum_naming` is added with new default behavior for naming backup files with `share_files_with_checksum`, to address performance and backup integrity issues. See API comments for details. @@ -628,12 +748,15 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Added experimental ColumnFamilyOptions::sst_partitioner_factory to define determine the partitioning of sst files. This helps compaction to split the files on interesting boundaries (key prefixes) to make propagation of sst files less write amplifying (covering the whole key space). ### Performance Improvements + * Eliminate key copies for internal comparisons while accessing ingested block-based tables. * Reduce key comparisons during random access in all block-based tables. * BackupEngine avoids unnecessary repeated checksum computation for backing up a table file to the `shared_checksum` directory when using `share_files_with_checksum_naming = kUseDbSessionId` (new default), except on SST files generated before this version of RocksDB, which fall back on using `kLegacyCrc32cAndFileSize`. ## 6.11 (2020-06-12) + ### Bug Fixes + * Fix consistency checking error swallowing in some cases when options.force_consistency_checks = true. * Fix possible false NotFound status from batched MultiGet using index type kHashSearch. * Fix corruption caused by enabling delete triggered compaction (NewCompactOnDeletionCollectorFactory) in universal compaction mode, along with parallel compactions. The bug can result in two parallel compactions picking the same input files, resulting in the DB resurrecting older and deleted versions of some keys. @@ -647,6 +770,7 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Fix incorrect results from batched MultiGet for duplicate keys, when the duplicate key matches the largest key of an SST file and the value type for the key in the file is a merge value. ### Public API Change + * Flush(..., column_family) may return Status::ColumnFamilyDropped() instead of Status::InvalidArgument() if column_family is dropped while processing the flush request. * BlobDB now explicitly disallows using the default column family's storage directories as blob directory. * DeleteRange now returns `Status::InvalidArgument` if the range's end key comes before its start key according to the user comparator. Previously the behavior was undefined. @@ -659,16 +783,20 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * A new method `Env::LowerThreadPoolCPUPriority(Priority, CpuPriority)` is added to `Env` to be able to lower to a specific priority such as `CpuPriority::kIdle`. ### New Features + * sst_dump to add a new --readahead_size argument. Users can specify read size when scanning the data. Sst_dump also tries to prefetch tail part of the SST files so usually some number of I/Os are saved there too. * Generate file checksum in SstFileWriter if Options.file_checksum_gen_factory is set. The checksum and checksum function name are stored in ExternalSstFileInfo after the sst file write is finished. * Add a value_size_soft_limit in read options which limits the cumulative value size of keys read in batches in MultiGet. Once the cumulative value size of found keys exceeds read_options.value_size_soft_limit, all the remaining keys are returned with status Abort without further finding their values. By default the value_size_soft_limit is std::numeric_limits::max(). * Enable SST file ingestion with file checksum information when calling IngestExternalFiles(const std::vector& args). Added files_checksums and files_checksum_func_names to IngestExternalFileArg such that user can ingest the sst files with their file checksum information. Added verify_file_checksum to IngestExternalFileOptions (default is True). To be backward compatible, if DB does not enable file checksum or user does not provide checksum information (vectors of files_checksums and files_checksum_func_names are both empty), verification of file checksum is always sucessful. If DB enables file checksum, DB will always generate the checksum for each ingested SST file during Prepare stage of ingestion and store the checksum in Manifest, unless verify_file_checksum is False and checksum information is provided by the application. In this case, we only verify the checksum function name and directly store the ingested checksum in Manifest. If verify_file_checksum is set to True, DB will verify the ingested checksum and function name with the genrated ones. Any mismatch will fail the ingestion. Note that, if IngestExternalFileOptions::write_global_seqno is True, the seqno will be changed in the ingested file. Therefore, the checksum of the file will be changed. In this case, a new checksum will be generated after the seqno is updated and be stored in the Manifest. ### Performance Improvements + * Eliminate redundant key comparisons during random access in block-based tables. ## 6.10 (2020-05-02) + ### Bug Fixes + * Fix wrong result being read from ingested file. May happen when a key in the file happen to be prefix of another key also in the file. The issue can further cause more data corruption. The issue exists with rocksdb >= 5.0.0 since DB::IngestExternalFile() was introduced. * Finish implementation of BlockBasedTableOptions::IndexType::kBinarySearchWithFirstKey. It's now ready for use. Significantly reduces read amplification in some setups, especially for iterator seeks. * Fix a bug by updating CURRENT file so that it points to the correct MANIFEST file after best-efforts recovery. @@ -679,11 +807,13 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Upgraded version of bzip library (1.0.6 -> 1.0.8) used with RocksJava to address potential vulnerabilities if an attacker can manipulate compressed data saved and loaded by RocksDB (not normal). See issue #6703. ### Public API Change + * Add a ConfigOptions argument to the APIs dealing with converting options to and from strings and files. The ConfigOptions is meant to replace some of the options (such as input_strings_escaped and ignore_unknown_options) and allow for more parameters to be passed in the future without changing the function signature. * Add NewFileChecksumGenCrc32cFactory to the file checksum public API, such that the builtin Crc32c based file checksum generator factory can be used by applications. * Add IsDirectory to Env and FS to indicate if a path is a directory. ### New Features + * Added support for pipelined & parallel compression optimization for `BlockBasedTableBuilder`. This optimization makes block building, block compression and block appending a pipeline, and uses multiple threads to accelerate block compression. Users can set `CompressionOptions::parallel_threads` greater than 1 to enable compression parallelism. This feature is experimental for now. * Provide an allocator for memkind to be used with block cache. This is to work with memory technologies (Intel DCPMM is one such technology currently available) that require different libraries for allocation and management (such as PMDK and memkind). The high capacities available make it possible to provision large caches (up to several TBs in size) beyond what is achievable with DRAM. * Option `max_background_flushes` can be set dynamically using DB::SetDBOptions(). @@ -691,47 +821,58 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Added statistics for redundant insertions into block cache: rocksdb.block.cache.*add.redundant. (There is currently no coordination to ensure that only one thread loads a table block when many threads are trying to access that same table block.) ### Bug Fixes + * Fix a bug when making options.bottommost_compression, options.compression_opts and options.bottommost_compression_opts dynamically changeable: the modified values are not written to option files or returned back to users when being queried. * Fix a bug where index key comparisons were unaccounted in `PerfContext::user_key_comparison_count` for lookups in files written with `format_version >= 3`. * Fix many bloom.filter statistics not being updated in batch MultiGet. ### Performance Improvements + * Improve performance of batch MultiGet with partitioned filters, by sharing block cache lookups to applicable filter blocks. * Reduced memory copies when fetching and uncompressing compressed blocks from sst files. ## 6.9.0 (2020-03-29) + ### Behavior changes + * Since RocksDB 6.8, ttl-based FIFO compaction can drop a file whose oldest key becomes older than options.ttl while others have not. This fix reverts this and makes ttl-based FIFO compaction use the file's flush time as the criterion. This fix also requires that max_open_files = -1 and compaction_options_fifo.allow_compaction = false to function properly. ### Public API Change + * Fix spelling so that API now has correctly spelled transaction state name `COMMITTED`, while the old misspelled `COMMITED` is still available as an alias. * Updated default format_version in BlockBasedTableOptions from 2 to 4. SST files generated with the new default can be read by RocksDB versions 5.16 and newer, and use more efficient encoding of keys in index blocks. * A new parameter `CreateBackupOptions` is added to both `BackupEngine::CreateNewBackup` and `BackupEngine::CreateNewBackupWithMetadata`, you can decrease CPU priority of `BackupEngine`'s background threads by setting `decrease_background_thread_cpu_priority` and `background_thread_cpu_priority` in `CreateBackupOptions`. * Updated the public API of SST file checksum. Introduce the FileChecksumGenFactory to create the FileChecksumGenerator for each SST file, such that the FileChecksumGenerator is not shared and it can be more general for checksum implementations. Changed the FileChecksumGenerator interface from Value, Extend, and GetChecksum to Update, Finalize, and GetChecksum. Finalize should be only called once after all data is processed to generate the final checksum. Temproal data should be maintained by the FileChecksumGenerator object itself and finally it can return the checksum string. ### Bug Fixes + * Fix a bug where range tombstone blocks in ingested files were cached incorrectly during ingestion. If range tombstones were read from those incorrectly cached blocks, the keys they covered would be exposed. * Fix a data race that might cause crash when calling DB::GetCreationTimeOfOldestFile() by a small chance. The bug was introduced in 6.6 Release. * Fix a bug where a boolean value optimize_filters_for_hits was for max threads when calling load table handles after a flush or compaction. The value is correct to 1. The bug should not cause user visible problems. * Fix a bug which might crash the service when write buffer manager fails to insert the dummy handle to the block cache. ### Performance Improvements + * In CompactRange, for levels starting from 0, if the level does not have any file with any key falling in the specified range, the level is skipped. So instead of always compacting from level 0, the compaction starts from the first level with keys in the specified range until the last such level. * Reduced memory copy when reading sst footer and blobdb in direct IO mode. * When restarting a database with large numbers of sst files, large amount of CPU time is spent on getting logical block size of the sst files, which slows down the starting progress, this inefficiency is optimized away with an internal cache for the logical block sizes. ### New Features + * Basic support for user timestamp in iterator. Seek/SeekToFirst/Next and lower/upper bounds are supported. Reverse iteration is not supported. Merge is not considered. * When file lock failure when the lock is held by the current process, return acquiring time and thread ID in the error message. * Added a new option, best_efforts_recovery (default: false), to allow database to open in a db dir with missing table files. During best efforts recovery, missing table files are ignored, and database recovers to the most recent state without missing table file. Cross-column-family consistency is not guaranteed even if WAL is enabled. * options.bottommost_compression, options.compression_opts and options.bottommost_compression_opts are now dynamically changeable. ## 6.8.0 (2020-02-24) + ### Java API Changes + * Major breaking changes to Java comparators, toward standardizing on ByteBuffer for performant, locale-neutral operations on keys (#6252). * Added overloads of common API methods using direct ByteBuffers for keys and values (#2283). ### Bug Fixes + * Fix incorrect results while block-based table uses kHashSearch, together with Prev()/SeekForPrev(). * Fix a bug that prevents opening a DB after two consecutive crash with TransactionDB, where the first crash recovers from a corrupted WAL with kPointInTimeRecovery but the second cannot. * Fixed issue #6316 that can cause a corruption of the MANIFEST file in the middle when writing to it fails due to no disk space. @@ -742,25 +883,31 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Assign new MANIFEST file number when caller tries to create a new MANIFEST by calling LogAndApply(..., new_descriptor_log=true). This bug can cause MANIFEST being overwritten during recovery if options.write_dbid_to_manifest = true and there are WAL file(s). ### Performance Improvements + * Perfom readahead when reading from option files. Inside DB, options.log_readahead_size will be used as the readahead size. In other cases, a default 512KB is used. ### Public API Change + * The BlobDB garbage collector now emits the statistics `BLOB_DB_GC_NUM_FILES` (number of blob files obsoleted during GC), `BLOB_DB_GC_NUM_NEW_FILES` (number of new blob files generated during GC), `BLOB_DB_GC_FAILURES` (number of failed GC passes), `BLOB_DB_GC_NUM_KEYS_RELOCATED` (number of blobs relocated during GC), and `BLOB_DB_GC_BYTES_RELOCATED` (total size of blobs relocated during GC). On the other hand, the following statistics, which are not relevant for the new GC implementation, are now deprecated: `BLOB_DB_GC_NUM_KEYS_OVERWRITTEN`, `BLOB_DB_GC_NUM_KEYS_EXPIRED`, `BLOB_DB_GC_BYTES_OVERWRITTEN`, `BLOB_DB_GC_BYTES_EXPIRED`, and `BLOB_DB_GC_MICROS`. * Disable recycle_log_file_num when an inconsistent recovery modes are requested: kPointInTimeRecovery and kAbsoluteConsistency ### New Features + * Added the checksum for each SST file generated by Flush or Compaction. Added sst_file_checksum_func to Options such that user can plugin their own SST file checksum function via override the FileChecksumFunc class. If user does not set the sst_file_checksum_func, SST file checksum calculation will not be enabled. The checksum information inlcuding uint32_t checksum value and a checksum function name (string). The checksum information is stored in FileMetadata in version store and also logged to MANIFEST. A new tool is added to LDB such that user can dump out a list of file checksum information from MANIFEST (stored in an unordered_map). * `db_bench` now supports `value_size_distribution_type`, `value_size_min`, `value_size_max` options for generating random variable sized value. Added `blob_db_compression_type` option for BlobDB to enable blob compression. * Replace RocksDB namespace "rocksdb" with flag "ROCKSDB_NAMESPACE" which if is not defined, defined as "rocksdb" in header file rocksdb_namespace.h. ## 6.7.0 (2020-01-21) + ### Public API Change + * Added a rocksdb::FileSystem class in include/rocksdb/file_system.h to encapsulate file creation/read/write operations, and an option DBOptions::file_system to allow a user to pass in an instance of rocksdb::FileSystem. If its a non-null value, this will take precendence over DBOptions::env for file operations. A new API rocksdb::FileSystem::Default() returns a platform default object. The DBOptions::env option and Env::Default() API will continue to be used for threading and other OS related functions, and where DBOptions::file_system is not specified, for file operations. For storage developers who are accustomed to rocksdb::Env, the interface in rocksdb::FileSystem is new and will probably undergo some changes as more storage systems are ported to it from rocksdb::Env. As of now, no env other than Posix has been ported to the new interface. * A new rocksdb::NewSstFileManager() API that allows the caller to pass in separate Env and FileSystem objects. * Changed Java API for RocksDB.keyMayExist functions to use Holder instead of StringBuilder, so that retrieved values need not decode to Strings. * A new `OptimisticTransactionDBOptions` Option that allows users to configure occ validation policy. The default policy changes from kValidateSerial to kValidateParallel to reduce mutex contention. ### Bug Fixes + * Fix a bug that can cause unnecessary bg thread to be scheduled(#6104). * Fix crash caused by concurrent CF iterations and drops(#6147). * Fix a race condition for cfd->log_number_ between manifest switch and memtable switch (PR 6249) when number of column families is greater than 1. @@ -772,6 +919,7 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Fix a regression bug that causes segfault when hash is used, max_open_files != -1 and total order seek is used and switched back. ### New Features + * It is now possible to enable periodic compactions for the base DB when using BlobDB. * BlobDB now garbage collects non-TTL blobs when `enable_garbage_collection` is set to `true` in `BlobDBOptions`. Garbage collection is performed during compaction: any valid blobs located in the oldest N files (where N is the number of non-TTL blob files multiplied by the value of `BlobDBOptions::garbage_collection_cutoff`) encountered during compaction get relocated to new blob files, and old blob files are dropped once they are no longer needed. Note: we recommend enabling periodic compactions for the base DB when using this feature to deal with the case when some old blob files are kept alive by SSTs that otherwise do not get picked for compaction. * `db_bench` now supports the `garbage_collection_cutoff` option for BlobDB. @@ -779,11 +927,15 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * MultiGet() can use IO Uring to parallelize read from the same SST file. This featuer is by default disabled. It can be enabled with environment variable ROCKSDB_USE_IO_URING. ## 6.6.2 (2020-01-13) + ### Bug Fixes + * Fixed a bug where non-L0 compaction input files were not considered to compute the `creation_time` of new compaction outputs. ## 6.6.1 (2020-01-02) + ### Bug Fixes + * Fix a bug in WriteBatchWithIndex::MultiGetFromBatchAndDB, which is called by Transaction::MultiGet, that causes due to stale pointer access when the number of keys is > 32 * Fixed two performance issues related to memtable history trimming. First, a new SuperVersion is now created only if some memtables were actually trimmed. Second, trimming is only scheduled if there is at least one flushed memtable that is kept in memory for the purposes of transaction conflict checking. * BlobDB no longer updates the SST to blob file mapping upon failed compactions. @@ -793,7 +945,9 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Fix use-after-free and double-deleting files in BackgroundCallPurge(). ## 6.6.0 (2019-11-25) + ### Bug Fixes + * Fix data corruption caused by output of intra-L0 compaction on ingested file not being placed in correct order in L0. * Fix a data race between Version::GetColumnFamilyMetaData() and Compaction::MarkFilesBeingCompacted() for access to being_compacted (#6056). The current fix acquires the db mutex during Version::GetColumnFamilyMetaData(), which may cause regression. * Fix a bug in DBIter that is_blob_ state isn't updated when iterating backward using seek. @@ -807,6 +961,7 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Fix a bug when a crash happens while calling WriteLevel0TableForRecovery for multiple column families, leading to a column family's log number greater than the first corrutped log number when the DB is being opened in PointInTime recovery mode during next recovery attempt (#5856). ### New Features + * Universal compaction to support options.periodic_compaction_seconds. A full compaction will be triggered if any file is over the threshold. * `GetLiveFilesMetaData` and `GetColumnFamilyMetaData` now expose the file number of SST files as well as the oldest blob file referenced by each SST. * A batched MultiGet API (DB::MultiGet()) that supports retrieving keys from multiple column families. @@ -823,6 +978,7 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * A number of new information elements are now exposed through the EventListener interface. For flushes, the file numbers of the new SST file and the oldest blob file referenced by the SST are propagated. For compactions, the level, file number, and the oldest blob file referenced are passed to the client for each compaction input and output file. ### Public API Change + * RocksDB release 4.1 or older will not be able to open DB generated by the new release. 4.2 was released on Feb 23, 2016. * TTL Compactions in Level compaction style now initiate successive cascading compactions on a key range so that it reaches the bottom level quickly on TTL expiry. `creation_time` table property for compaction output files is now set to the minimum of the creation times of all compaction inputs. * With FIFO compaction style, options.periodic_compaction_seconds will have the same meaning as options.ttl. Whichever stricter will be used. With the default options.periodic_compaction_seconds value with options.ttl's default of 0, RocksDB will give a default of 30 days. @@ -838,65 +994,80 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Flush sets file name to "(nil)" for OnTableFileCreationCompleted() if the flush does not produce any L0. This can happen if the file is empty thus delete by RocksDB. ### Default Option Changes + * Changed the default value of periodic_compaction_seconds to `UINT64_MAX - 1` which allows RocksDB to auto-tune periodic compaction scheduling. When using the default value, periodic compactions are now auto-enabled if a compaction filter is used. A value of `0` will turn off the feature completely. * Changed the default value of ttl to `UINT64_MAX - 1` which allows RocksDB to auto-tune ttl value. When using the default value, TTL will be auto-enabled to 30 days, when the feature is supported. To revert the old behavior, you can explicitly set it to 0. ### Performance Improvements + * For 64-bit hashing, RocksDB is standardizing on a slightly modified preview version of XXH3. This function is now used for many non-persisted hashes, along with fastrange64() in place of the modulus operator, and some benchmarks show a slight improvement. * Level iterator to invlidate the iterator more often in prefix seek and the level is filtered out by prefix bloom. ## 6.5.2 (2019-11-15) + ### Bug Fixes + * Fix a assertion failure in MultiGet() when BlockBasedTableOptions::no_block_cache is true and there is no compressed block cache * Fix a buffer overrun problem in BlockBasedTable::MultiGet() when compression is enabled and no compressed block cache is configured. * If a call to BackupEngine::PurgeOldBackups or BackupEngine::DeleteBackup suffered a crash, power failure, or I/O error, files could be left over from old backups that could only be purged with a call to GarbageCollect. Any call to PurgeOldBackups, DeleteBackup, or GarbageCollect should now suffice to purge such files. ## 6.5.1 (2019-10-16) + ### Bug Fixes + * Revert the feature "Merging iterator to avoid child iterator reseek for some cases (#5286)" since it might cause strange results when reseek happens with a different iterator upper bound. * Fix a bug in BlockBasedTableIterator that might return incorrect results when reseek happens with a different iterator upper bound. * Fix a bug when partitioned filters and prefix search are used in conjunction, ::SeekForPrev could return invalid for an existing prefix. ::SeekForPrev might be called by the user, or internally on ::Prev, or within ::Seek if the return value involves Delete or a Merge operand. ## 6.5.0 (2019-09-13) + ### Bug Fixes + * Fixed a number of data races in BlobDB. * Fix a bug where the compaction snapshot refresh feature is not disabled as advertised when `snap_refresh_nanos` is set to 0.. * Fix bloom filter lookups by the MultiGet batching API when BlockBasedTableOptions::whole_key_filtering is false, by checking that a key is in the perfix_extractor domain and extracting the prefix before looking up. * Fix a bug in file ingestion caused by incorrect file number allocation when the number of column families involved in the ingestion exceeds 2. ### New Features + * Introduced DBOptions::max_write_batch_group_size_bytes to configure maximum limit on number of bytes that are written in a single batch of WAL or memtable write. It is followed when the leader write size is larger than 1/8 of this limit. * VerifyChecksum() by default will issue readahead. Allow ReadOptions to be passed in to those functions to override the readhead size. For checksum verifying before external SST file ingestion, a new option IngestExternalFileOptions.verify_checksums_readahead_size, is added for this readahead setting. * When user uses options.force_consistency_check in RocksDb, instead of crashing the process, we now pass the error back to the users without killing the process. * Add an option `memtable_insert_hint_per_batch` to WriteOptions. If it is true, each WriteBatch will maintain its own insert hints for each memtable in concurrent write. See include/rocksdb/options.h for more details. ### Public API Change + * Added max_write_buffer_size_to_maintain option to better control memory usage of immutable memtables. * Added a lightweight API GetCurrentWalFile() to get last live WAL filename and size. Meant to be used as a helper for backup/restore tooling in a larger ecosystem such as MySQL with a MyRocks storage engine. * The MemTable Bloom filter, when enabled, now always uses cache locality. Options::bloom_locality now only affects the PlainTable SST format. ### Performance Improvements + * Improve the speed of the MemTable Bloom filter, reducing the write overhead of enabling it by 1/3 to 1/2, with similar benefit to read performance. ## 6.4.0 (2019-07-30) + ### Default Option Change + * LRUCacheOptions.high_pri_pool_ratio is set to 0.5 (previously 0.0) by default, which means that by default midpoint insertion is enabled. The same change is made for the default value of high_pri_pool_ratio argument in NewLRUCache(). When block cache is not explicitly created, the small block cache created by BlockBasedTable will still has this option to be 0.0. * Change BlockBasedTableOptions.cache_index_and_filter_blocks_with_high_priority's default value from false to true. ### Public API Change + * Filter and compression dictionary blocks are now handled similarly to data blocks with regards to the block cache: instead of storing objects in the cache, only the blocks themselves are cached. In addition, filter and compression dictionary blocks (as well as filter partitions) no longer get evicted from the cache when a table is closed. * Due to the above refactoring, block cache eviction statistics for filter and compression dictionary blocks are temporarily broken. We plan to reintroduce them in a later phase. * The semantics of the per-block-type block read counts in the performance context now match those of the generic block_read_count. * Errors related to the retrieval of the compression dictionary are now propagated to the user. * db_bench adds a "benchmark" stats_history, which prints out the whole stats history. * Overload GetAllKeyVersions() to support non-default column family. -* Added new APIs ExportColumnFamily() and CreateColumnFamilyWithImport() to support export and import of a Column Family. https://github.com/facebook/rocksdb/issues/3469 +* Added new APIs ExportColumnFamily() and CreateColumnFamilyWithImport() to support export and import of a Column Family. * ldb sometimes uses a string-append merge operator if no merge operator is passed in. This is to allow users to print keys from a DB with a merge operator. * Replaces old Registra with ObjectRegistry to allow user to create custom object from string, also add LoadEnv() to Env. * Added new overload of GetApproximateSizes which gets SizeApproximationOptions object and returns a Status. The older overloads are redirecting their calls to this new method and no longer assert if the include_flags doesn't have either of INCLUDE_MEMTABLES or INCLUDE_FILES bits set. It's recommended to use the new method only, as it is more type safe and returns a meaningful status in case of errors. * LDBCommandRunner::RunCommand() to return the status code as an integer, rather than call exit() using the code. ### New Features + * Add argument `--secondary_path` to ldb to open the database as the secondary instance. This would keep the original DB intact. * Compression dictionary blocks are now prefetched and pinned in the cache (based on the customer's settings) the same way as index and filter blocks. * Added DBOptions::log_readahead_size which specifies the number of bytes to prefetch when reading the log. This is mostly useful for reading a remotely located log, as it can save the number of round-trips. If 0 (default), then the prefetching is disabled. @@ -904,30 +1075,39 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Support loading custom objects in unit tests. In the affected unit tests, RocksDB will create custom Env objects based on environment variable TEST_ENV_URI. Users need to make sure custom object types are properly registered. For example, a static library should expose a `RegisterCustomObjects` function. By linking the unit test binary with the static library, the unit test can execute this function. ### Performance Improvements + * Reduce iterator key comparison for upper/lower bound check. * Improve performance of row_cache: make reads with newer snapshots than data in an SST file share the same cache key, except in some transaction cases. * The compression dictionary is no longer copied to a new object upon retrieval. ### Bug Fixes + * Fix ingested file and directory not being fsync. * Return TryAgain status in place of Corruption when new tail is not visible to TransactionLogIterator. * Fixed a regression where the fill_cache read option also affected index blocks. * Fixed an issue where using cache_index_and_filter_blocks==false affected partitions of partitioned indexes/filters as well. ## 6.3.2 (2019-08-15) + ### Public API Change + * The semantics of the per-block-type block read counts in the performance context now match those of the generic block_read_count. ### Bug Fixes + * Fixed a regression where the fill_cache read option also affected index blocks. * Fixed an issue where using cache_index_and_filter_blocks==false affected partitions of partitioned indexes as well. ## 6.3.1 (2019-07-24) + ### Bug Fixes + * Fix auto rolling bug introduced in 6.3.0, which causes segfault if log file creation fails. ## 6.3.0 (2019-06-18) + ### Public API Change + * Now DB::Close() will return Aborted() error when there is unreleased snapshot. Users can retry after all snapshots are released. * Index blocks are now handled similarly to data blocks with regards to the block cache: instead of storing objects in the cache, only the blocks themselves are cached. In addition, index blocks no longer get evicted from the cache when a table is closed, can now use the compressed block cache (if any), and can be shared among multiple table readers. * Partitions of partitioned indexes no longer affect the read amplification statistics. @@ -939,6 +1119,7 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Rate limited deletion of WALs is only enabled if DBOptions::wal_dir is not set, or explicitly set to db_name passed to DB::Open and DBOptions::db_paths is empty, or same as db_paths[0].path ### New Features + * Add an option `snap_refresh_nanos` (default to 0) to periodically refresh the snapshot list in compaction jobs. Assign to 0 to disable the feature. * Add an option `unordered_write` which trades snapshot guarantees with higher write throughput. When used with WRITE_PREPARED transactions with two_write_queues=true, it offers higher throughput with however no compromise on guarantees. * Allow DBImplSecondary to remove memtables with obsolete data after replaying MANIFEST and WAL. @@ -946,6 +1127,7 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Add command `list_file_range_deletes` in ldb, which prints out tombstones in SST files. ### Performance Improvements + * Reduce binary search when iterator reseek into the same data block. * DBIter::Next() can skip user key checking if previous entry's seqnum is 0. * Merging iterator to avoid child iterator reseek for some cases @@ -953,10 +1135,12 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Lower MultiGet batching API latency by reading data blocks from disk in parallel ### General Improvements + * Added new status code kColumnFamilyDropped to distinguish between Column Family Dropped and DB Shutdown in progress. * Improve ColumnFamilyOptions validation when creating a new column family. ### Bug Fixes + * Fix a bug in WAL replay of secondary instance by skipping write batches with older sequence numbers than the current last sequence number. * Fix flush's/compaction's merge processing logic which allowed `Put`s covered by range tombstones to reappear. Note `Put`s may exist even if the user only ever called `Merge()` due to an internal conversion during compaction to the bottommost level. * Fix/improve memtable earliest sequence assignment and WAL replay so that WAL entries of unflushed column families will not be skipped after replaying the MANIFEST and increasing db sequence due to another flushed/compacted column family. @@ -964,7 +1148,9 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * On DB open, delete WAL trash files left behind in wal_dir ## 6.2.0 (2019-04-30) + ### New Features + * Add an option `strict_bytes_per_sync` that causes a file-writing thread to block rather than exceed the limit on bytes pending writeback specified by `bytes_per_sync` or `wal_bytes_per_sync`. * Improve range scan performance by avoiding per-key upper bound check in BlockBasedTableIterator. * Introduce Periodic Compaction for Level style compaction. Files are re-compacted periodically and put in the same level. @@ -974,11 +1160,13 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Add index type BlockBasedTableOptions::IndexType::kBinarySearchWithFirstKey. It significantly reduces read amplification in some setups, especially for iterator seeks. It's not fully implemented yet: IO errors are not handled right. ### Public API Change + * Change the behavior of OptimizeForPointLookup(): move away from hash-based block-based-table index, and use whole key memtable filtering. * Change the behavior of OptimizeForSmallDb(): use a 16MB block cache, put index and filter blocks into it, and cost the memtable size to it. DBOptions.OptimizeForSmallDb() and ColumnFamilyOptions.OptimizeForSmallDb() start to take an optional cache object. * Added BottommostLevelCompaction::kForceOptimized to avoid double compacting newly compacted files in the bottommost level compaction of manual compaction. Note this option may prohibit the manual compaction to produce a single file in the bottommost level. ### Bug Fixes + * Adjust WriteBufferManager's dummy entry size to block cache from 1MB to 256KB. * Fix a race condition between WritePrepared::Get and ::Put with duplicate keys. * Fix crash when memtable prefix bloom is enabled and read/write a key out of domain of prefix extractor. @@ -986,23 +1174,29 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Fix an assertion failure `IsFlushPending() == true` caused by one bg thread releasing the db mutex in ~ColumnFamilyData and another thread clearing `flush_requested_` flag. ## 6.1.1 (2019-04-09) + ### New Features + * When reading from option file/string/map, customized comparators and/or merge operators can be filled according to object registry. ### Public API Change ### Bug Fixes + * Fix a bug in 2PC where a sequence of txn prepare, memtable flush, and crash could result in losing the prepared transaction. * Fix a bug in Encryption Env which could cause encrypted files to be read beyond file boundaries. ## 6.1.0 (2019-03-27) + ### New Features + * Introduce two more stats levels, kExceptHistogramOrTimers and kExceptTimers. * Added a feature to perform data-block sampling for compressibility, and report stats to user. * Add support for trace filtering. * Add DBOptions.avoid_unnecessary_blocking_io. If true, we avoid file deletion when destroying ColumnFamilyHandle and Iterator. Instead, a job is scheduled to delete the files in background. ### Public API Change + * Remove bundled fbson library. * statistics.stats_level_ becomes atomic. It is preferred to use statistics.set_stats_level() and statistics.get_stats_level() to access it. * Introduce a new IOError subcode, PathNotFound, to indicate trying to open a nonexistent file or directory for read. @@ -1010,11 +1204,14 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Removed some "using std::xxx" from public headers. ### Bug Fixes + * Fix JEMALLOC_CXX_THROW macro missing from older Jemalloc versions, causing build failures on some platforms. * Fix SstFileReader not able to open file ingested with write_glbal_seqno=true. ## 6.0.0 (2019-02-19) + ### New Features + * Enabled checkpoint on readonly db (DBImplReadOnly). * Make DB ignore dropped column families while committing results of atomic flush. * RocksDB may choose to preopen some files even if options.max_open_files != -1. This may make DB open slightly longer. @@ -1030,6 +1227,7 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Files written by `SstFileWriter` will now use dictionary compression if it is configured in the file writer's `CompressionOptions`. ### Public API Change + * Disallow CompactionFilter::IgnoreSnapshots() = false, because it is not very useful and the behavior is confusing. The filter will filter everything if there is no snapshot declared by the time the compaction starts. However, users can define a snapshot after the compaction starts and before it finishes and this new snapshot won't be repeatable, because after the compaction finishes, some keys may be dropped. * CompactionPri = kMinOverlappingRatio also uses compensated file size, which boosts file with lots of tombstones to be compacted first. * Transaction::GetForUpdate is extended with a do_validate parameter with default value of true. If false it skips validating the snapshot before doing the read. Similarly ::Merge, ::Put, ::Delete, and ::SingleDelete are extended with assume_tracked with default value of false. If true it indicates that call is assumed to be after a ::GetForUpdate. @@ -1045,6 +1243,7 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Remove Lua compaction filter. ### Bug Fixes + * Fix a deadlock caused by compaction and file ingestion waiting for each other in the event of write stalls. * Fix a memory leak when files with range tombstones are read in mmap mode and block cache is enabled * Fix handling of corrupt range tombstone blocks such that corruptions cannot cause deleted keys to reappear @@ -1053,10 +1252,13 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Fix with pipelined write, write leaders's callback failure lead to the whole write group fail. ### Change Default Options + * Change options.compaction_pri's default to kMinOverlappingRatio ## 5.18.0 (2018-11-30) + ### New Features + * Introduced `JemallocNodumpAllocator` memory allocator. When being use, block cache will be excluded from core dump. * Introduced `PerfContextByLevel` as part of `PerfContext` which allows storing perf context at each level. Also replaced `__thread` with `thread_local` keyword for perf_context. Added per-level perf context for bloom filter and `Get` query. * With level_compaction_dynamic_level_bytes = true, level multiplier may be adjusted automatically when Level 0 to 1 compaction is lagged behind. @@ -1068,10 +1270,12 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Improved `DeleteRange` to prevent read performance degradation. The feature is no longer marked as experimental. ### Public API Change + * `DBOptions::use_direct_reads` now affects reads issued by `BackupEngine` on the database's SSTs. * `NO_ITERATORS` is divided into two counters `NO_ITERATOR_CREATED` and `NO_ITERATOR_DELETE`. Both of them are only increasing now, just as other counters. ### Bug Fixes + * Fix corner case where a write group leader blocked due to write stall blocks other writers in queue with WriteOptions::no_slowdown set. * Fix in-memory range tombstone truncation to avoid erroneously covering newer keys at a lower level, and include range tombstones in compacted files whose largest key is the range tombstone's start key. * Properly set the stop key for a truncated manual CompactRange @@ -1084,49 +1288,65 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * The default value of NewBloomFilterPolicy()'s argument use_block_based_builder is changed to false. Note that this new default may cause large temp memory usage when building very large SST files. ## 5.17.0 (2018-10-05) + ### Public API Change + * `OnTableFileCreated` will now be called for empty files generated during compaction. In that case, `TableFileCreationInfo::file_path` will be "(nil)" and `TableFileCreationInfo::file_size` will be zero. * Add `FlushOptions::allow_write_stall`, which controls whether Flush calls start working immediately, even if it causes user writes to stall, or will wait until flush can be performed without causing write stall (similar to `CompactRangeOptions::allow_write_stall`). Note that the default value is false, meaning we add delay to Flush calls until stalling can be avoided when possible. This is behavior change compared to previous RocksDB versions, where Flush calls didn't check if they might cause stall or not. * Application using PessimisticTransactionDB is expected to rollback/commit recovered transactions before starting new ones. This assumption is used to skip concurrency control during recovery. * Expose column family id to `OnCompactionCompleted`. ### New Features + * TransactionOptions::skip_concurrency_control allows pessimistic transactions to skip the overhead of concurrency control. Could be used for optimizing certain transactions or during recovery. ### Bug Fixes + * Avoid creating empty SSTs and subsequently deleting them in certain cases during compaction. * Sync CURRENT file contents during checkpoint. ## 5.16.3 (2018-10-01) + ### Bug Fixes + * Fix crash caused when `CompactFiles` run with `CompactionOptions::compression == CompressionType::kDisableCompressionOption`. Now that setting causes the compression type to be chosen according to the column family-wide compression options. ## 5.16.2 (2018-09-21) + ### Bug Fixes + * Fix bug in partition filters with format_version=4. ## 5.16.1 (2018-09-17) + ### Bug Fixes + * Remove trace_analyzer_tool from rocksdb_lib target in TARGETS file. * Fix RocksDB Java build and tests. * Remove sync point in Block destructor. ## 5.16.0 (2018-08-21) + ### Public API Change + * The merge operands are passed to `MergeOperator::ShouldMerge` in the reversed order relative to how they were merged (passed to FullMerge or FullMergeV2) for performance reasons * GetAllKeyVersions() to take an extra argument of `max_num_ikeys`. * Using ZSTD dictionary trainer (i.e., setting `CompressionOptions::zstd_max_train_bytes` to a nonzero value) now requires ZSTD version 1.1.3 or later. ### New Features + * Changes the format of index blocks by delta encoding the index values, which are the block handles. This saves the encoding of BlockHandle::offset of the non-head index entries in each restart interval. The feature is backward compatible but not forward compatible. It is disabled by default unless format_version 4 or above is used. * Add a new tool: trace_analyzer. Trace_analyzer analyzes the trace file generated by using trace_replay API. It can convert the binary format trace file to a human readable txt file, output the statistics of the analyzed query types such as access statistics and size statistics, combining the dumped whole key space file to analyze, support query correlation analyzing, and etc. Current supported query types are: Get, Put, Delete, SingleDelete, DeleteRange, Merge, Iterator (Seek, SeekForPrev only). * Add hash index support to data blocks, which helps reducing the cpu utilization of point-lookup operations. This feature is backward compatible with the data block created without the hash index. It is disabled by default unless BlockBasedTableOptions::data_block_index_type is set to data_block_index_type = kDataBlockBinaryAndHash. ### Bug Fixes + * Fix a bug in misreporting the estimated partition index size in properties block. ## 5.15.0 (2018-07-17) + ### Public API Change + * Remove managed iterator. ReadOptions.managed is not effective anymore. * For bottommost_compression, a compatible CompressionOptions is added via `bottommost_compression_opts`. To keep backward compatible, a new boolean `enabled` is added to CompressionOptions. For compression_opts, it will be always used no matter what value of `enabled` is. For bottommost_compression_opts, it will only be used when user set `enabled=true`, otherwise, compression_opts will be used for bottommost_compression as default. * With LRUCache, when high_pri_pool_ratio > 0, midpoint insertion strategy will be enabled to put low-pri items to the tail of low-pri list (the midpoint) when they first inserted into the cache. This is to make cache entries never get hit age out faster, improving cache efficiency when large background scan presents. @@ -1134,6 +1354,7 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * The "rocksdb.num.entries" table property no longer counts range deletion tombstones as entries. ### New Features + * Changes the format of index blocks by storing the key in their raw form rather than converting them to InternalKey. This saves 8 bytes per index key. The feature is backward compatible but not forward compatible. It is disabled by default unless format_version 3 or above is used. * Avoid memcpy when reading mmap files with OpenReadOnly and max_open_files==-1. * Support dynamically changing `ColumnFamilyOptions::ttl` via `SetOptions()`. @@ -1143,6 +1364,7 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Write properties meta-block at the end of block-based table to save read-ahead IO. ### Bug Fixes + * Fix deadlock with enable_pipelined_write=true and max_successive_merges > 0 * Check conflict at output level in CompactFiles. * Fix corruption in non-iterator reads when mmap is used for file reads @@ -1152,7 +1374,9 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Fix write can stuck indefinitely if enable_pipelined_write=true. The issue exists since pipelined write was introduced in 5.5.0. ## 5.14.0 (2018-05-16) + ### Public API Change + * Add a BlockBasedTableOption to align uncompressed data blocks on the smaller of block size or page size boundary, to reduce flash reads by avoiding reads spanning 4K pages. * The background thread naming convention changed (on supporting platforms) to "rocksdb:", e.g., "rocksdb:low0". * Add a new ticker stat rocksdb.number.multiget.keys.found to count number of keys successfully read in MultiGet calls @@ -1164,6 +1388,7 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Introduced `CompressionOptions::kDefaultCompressionLevel`, which is a generic way to tell RocksDB to use the compression library's default level. It is now the default value for `CompressionOptions::level`. Previously the level defaulted to -1, which gave poor compression ratios in ZSTD. ### New Features + * Introduce TTL for level compaction so that all files older than ttl go through the compaction process to get rid of old data. * TransactionDBOptions::write_policy can be configured to enable WritePrepared 2PC transactions. Read more about them in the wiki. * Add DB properties "rocksdb.block-cache-capacity", "rocksdb.block-cache-usage", "rocksdb.block-cache-pinned-usage" to show block cache usage. @@ -1173,6 +1398,7 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * `prefix_extractor` has been moved from ImmutableCFOptions to MutableCFOptions, meaning it can be dynamically changed without a DB restart. ### Bug Fixes + * Fsync after writing global seq number to the ingestion file in ExternalSstFileIngestionJob. * Fix WAL corruption caused by race condition between user write thread and FlushWAL when two_write_queue is not set. * Fix `BackupableDBOptions::max_valid_backups_to_open` to not delete backup files when refcount cannot be accurately determined. @@ -1181,15 +1407,19 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Fix wrong results by ReverseBytewiseComparator::FindShortSuccessor() ### Java API Changes + * Add `BlockBasedTableConfig.setBlockCache` to allow sharing a block cache across DB instances. * Added SstFileManager to the Java API to allow managing SST files across DB instances. ## 5.13.0 (2018-03-20) + ### Public API Change + * RocksDBOptionsParser::Parse()'s `ignore_unknown_options` argument will only be effective if the option file shows it is generated using a higher version of RocksDB than the current version. * Remove CompactionEventListener. ### New Features + * SstFileManager now can cancel compactions if they will result in max space errors. SstFileManager users can also use SetCompactionBufferSize to specify how much space must be leftover during a compaction for auxiliary file functions such as logging and flushing. * Avoid unnecessarily flushing in `CompactRange()` when the range specified by the user does not overlap unflushed memtables. * If `ColumnFamilyOptions::max_subcompactions` is set greater than one, we now parallelize large manual level-based compactions. @@ -1197,11 +1427,14 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * NewSstFileManager to add an argument bytes_max_delete_chunk with default 64MB. With this argument, a file larger than 64MB will be ftruncated multiple times based on this size. ### Bug Fixes + * Fix a leak in prepared_section_completed_ where the zeroed entries would not removed from the map. * Fix WAL corruption caused by race condition between user write thread and backup/checkpoint thread. ## 5.12.0 (2018-02-14) + ### Public API Change + * Iterator::SeekForPrev is now a pure virtual method. This is to prevent user who implement the Iterator interface fail to implement SeekForPrev by mistake. * Add `include_end` option to make the range end exclusive when `include_end == false` in `DeleteFilesInRange()`. * Add `CompactRangeOptions::allow_write_stall`, which makes `CompactRange` start working immediately, even if it causes user writes to stall. The default value is false, meaning we add delay to `CompactRange` calls until stalling can be avoided when possible. Note this delay is not present in previous RocksDB versions. @@ -1210,6 +1443,7 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Close() method now returns a status when closing a db. ### New Features + * Improve the performance of iterators doing long range scans by using readahead. * Add new function `DeleteFilesInRanges()` to delete files in multiple ranges at once for better performance. * FreeBSD build support for RocksDB and RocksJava. @@ -1217,22 +1451,27 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Updated to and now continuously tested in Visual Studio 2017. ### Bug Fixes + * Fix `DisableFileDeletions()` followed by `GetSortedWalFiles()` to not return obsolete WAL files that `PurgeObsoleteFiles()` is going to delete. * Fix Handle error return from WriteBuffer() during WAL file close and DB close. * Fix advance reservation of arena block addresses. * Fix handling of empty string as checkpoint directory. ## 5.11.0 (2018-01-08) + ### Public API Change + * Add `autoTune` and `getBytesPerSecond()` to RocksJava RateLimiter ### New Features + * Add a new histogram stat called rocksdb.db.flush.micros for memtable flush. * Add "--use_txn" option to use transactional API in db_stress. * Disable onboard cache for compaction output in Windows platform. * Improve the performance of iterators doing long range scans by using readahead. ### Bug Fixes + * Fix a stack-use-after-scope bug in ForwardIterator. * Fix builds on platforms including Linux, Windows, and PowerPC. * Fix buffer overrun in backup engine for DBs with huge number of files. @@ -1240,22 +1479,28 @@ Note: The next release will be major release 7.0. See https://github.com/faceboo * Fix DB::Flush() keep waiting after flush finish under certain condition. ## 5.10.0 (2017-12-11) + ### Public API Change + * When running `make` with environment variable `USE_SSE` set and `PORTABLE` unset, will use all machine features available locally. Previously this combination only compiled SSE-related features. ### New Features + * Provide lifetime hints when writing files on Linux. This reduces hardware write-amp on storage devices supporting multiple streams. * Add a DB stat, `NUMBER_ITER_SKIP`, which returns how many internal keys were skipped during iterations (e.g., due to being tombstones or duplicate versions of a key). * Add PerfContext counters, `key_lock_wait_count` and `key_lock_wait_time`, which measure the number of times transactions wait on key locks and total amount of time waiting. ### Bug Fixes + * Fix IOError on WAL write doesn't propagate to write group follower * Make iterator invalid on merge error. * Fix performance issue in `IngestExternalFile()` affecting databases with large number of SST files. * Fix possible corruption to LSM structure when `DeleteFilesInRange()` deletes a subset of files spanned by a `DeleteRange()` marker. ## 5.9.0 (2017-11-01) + ### Public API Change + * `BackupableDBOptions::max_valid_backups_to_open == 0` now means no backups will be opened during BackupEngine initialization. Previously this condition disabled limiting backups opened. * `DBOptions::preserve_deletes` is a new option that allows one to specify that DB should not drop tombstones for regular deletes if they have sequence number larger than what was set by the new API call `DB::SetPreserveDeletesSequenceNumber(SequenceNumber seqnum)`. Disabled by default. * API call `DB::SetPreserveDeletesSequenceNumber(SequenceNumber seqnum)` was added, users who wish to preserve deletes are expected to periodically call this function to advance the cutoff seqnum (all deletes made before this seqnum can be dropped by DB). It's user responsibility to figure out how to advance the seqnum in the way so the tombstones are kept for the desired period of time, yet are eventually processed in time and don't eat up too much space. @@ -1266,6 +1511,7 @@ if set to something > 0 user will see 2 changes in iterators behavior 1) only ke * Return an error on write if write_options.sync = true and write_options.disableWAL = true to warn user of inconsistent options. Previously we will not write to WAL and not respecting the sync options in this case. ### New Features + * CRC32C is now using the 3-way pipelined SSE algorithm `crc32c_3way` on supported platforms to improve performance. The system will choose to use this algorithm on supported platforms automatically whenever possible. If PCLMULQDQ is not supported it will fall back to the old Fast_CRC32 algorithm. * `DBOptions::writable_file_max_buffer_size` can now be changed dynamically. * `DBOptions::bytes_per_sync`, `DBOptions::compaction_readahead_size`, and `DBOptions::wal_bytes_per_sync` can now be changed dynamically, `DBOptions::wal_bytes_per_sync` will flush all memtables and switch to a new WAL file. @@ -1278,16 +1524,20 @@ if set to something > 0 user will see 2 changes in iterators behavior 1) only ke * Support for differential snapshots (via iterator emitting the sequence of key-values representing the difference between DB state at two different sequence numbers). Supports preserving and emitting puts and regular deletes, doesn't support SingleDeletes, MergeOperator, Blobs and Range Deletes. ### Bug Fixes + * Fix a potential data inconsistency issue during point-in-time recovery. `DB:Open()` will abort if column family inconsistency is found during PIT recovery. * Fix possible metadata corruption in databases using `DeleteRange()`. ## 5.8.0 (2017-08-30) + ### Public API Change + * Users of `Statistics::getHistogramString()` will see fewer histogram buckets and different bucket endpoints. * `Slice::compare` and BytewiseComparator `Compare` no longer accept `Slice`s containing nullptr. * `Transaction::Get` and `Transaction::GetForUpdate` variants with `PinnableSlice` added. ### New Features + * Add Iterator::Refresh(), which allows users to update the iterator state so that they can avoid some initialization costs of recreating iterators. * Replace dynamic_cast<> (except unit test) so people can choose to build with RTTI off. With make, release mode is by default built with -fno-rtti and debug mode is built without it. Users can override it by setting USE_RTTI=0 or 1. * Universal compactions including the bottom level can be executed in a dedicated thread pool. This alleviates head-of-line blocking in the compaction queue, which cause write stalling, particularly in multi-instance use cases. Users can enable this feature via `Env::SetBackgroundThreads(N, Env::Priority::BOTTOM)`, where `N > 0`. @@ -1296,34 +1546,41 @@ if set to something > 0 user will see 2 changes in iterators behavior 1) only ke * Block-based table support for disabling checksums by setting `BlockBasedTableOptions::checksum = kNoChecksum`. ### Bug Fixes + * Fix wrong latencies in `rocksdb.db.get.micros`, `rocksdb.db.write.micros`, and `rocksdb.sst.read.micros`. * Fix incorrect dropping of deletions during intra-L0 compaction. * Fix transient reappearance of keys covered by range deletions when memtable prefix bloom filter is enabled. * Fix potentially wrong file smallest key when range deletions separated by snapshot are written together. ## 5.7.0 (2017-07-13) + ### Public API Change + * DB property "rocksdb.sstables" now prints keys in hex form. ### New Features + * Measure estimated number of reads per file. The information can be accessed through DB::GetColumnFamilyMetaData or "rocksdb.sstables" DB property. * RateLimiter support for throttling background reads, or throttling the sum of background reads and writes. This can give more predictable I/O usage when compaction reads more data than it writes, e.g., due to lots of deletions. * [Experimental] FIFO compaction with TTL support. It can be enabled by setting CompactionOptionsFIFO.ttl > 0. * Introduce `EventListener::OnBackgroundError()` callback. Users can implement it to be notified of errors causing the DB to enter read-only mode, and optionally override them. * Partitioned Index/Filters exiting the experimental mode. To enable partitioned indexes set index_type to kTwoLevelIndexSearch and to further enable partitioned filters set partition_filters to true. To configure the partition size set metadata_block_size. - ### Bug Fixes + * Fix discarding empty compaction output files when `DeleteRange()` is used together with subcompactions. ## 5.6.0 (2017-06-06) + ### Public API Change + * Scheduling flushes and compactions in the same thread pool is no longer supported by setting `max_background_flushes=0`. Instead, users can achieve this by configuring their high-pri thread pool to have zero threads. * Replace `Options::max_background_flushes`, `Options::max_background_compactions`, and `Options::base_background_compactions` all with `Options::max_background_jobs`, which automatically decides how many threads to allocate towards flush/compaction. * options.delayed_write_rate by default take the value of options.rate_limiter rate. * Replace global variable `IOStatsContext iostats_context` with `IOStatsContext* get_iostats_context()`; replace global variable `PerfContext perf_context` with `PerfContext* get_perf_context()`. ### New Features + * Change ticker/histogram statistics implementations to use core-local storage. This improves aggregation speed compared to our previous thread-local approach, particularly for applications with many threads. * Users can pass a cache object to write buffer manager, so that they can cap memory usage for memtable and block cache using one single limit. * Flush will be triggered when 7/8 of the limit introduced by write_buffer_manager or db_write_buffer_size is triggered, so that the hard threshold is hard to hit. @@ -1331,10 +1588,13 @@ if set to something > 0 user will see 2 changes in iterators behavior 1) only ke * `DB::IngestExternalFile()` now supports ingesting files into a database containing range deletions. ### Bug Fixes + * Shouldn't ignore return value of fsync() in flush. ## 5.5.0 (2017-05-17) + ### New Features + * FIFO compaction to support Intra L0 compaction too with CompactionOptionsFIFO.allow_compaction=true. * DB::ResetStats() to reset internal stats. * Statistics::Reset() to reset user stats. @@ -1348,64 +1608,79 @@ if set to something > 0 user will see 2 changes in iterators behavior 1) only ke * New option enable_pipelined_write which may improve write throughput in case writing from multiple threads and WAL enabled. ### Bug Fixes + * Fix the bug that Direct I/O uses direct reads for non-SST file ## 5.4.0 (2017-04-11) + ### Public API Change + * random_access_max_buffer_size no longer has any effect * Removed Env::EnableReadAhead(), Env::ShouldForwardRawRequest() * Support dynamically change `stats_dump_period_sec` option via SetDBOptions(). * Added ReadOptions::max_skippable_internal_keys to set a threshold to fail a request as incomplete when too many keys are being skipped when using iterators. * DB::Get in place of std::string accepts PinnableSlice, which avoids the extra memcpy of value to std::string in most of cases. - * PinnableSlice releases the pinned resources that contain the value when it is destructed or when ::Reset() is called on it. - * The old API that accepts std::string, although discouraged, is still supported. + * PinnableSlice releases the pinned resources that contain the value when it is destructed or when ::Reset() is called on it. + * The old API that accepts std::string, although discouraged, is still supported. * Replace Options::use_direct_writes with Options::use_direct_io_for_flush_and_compaction. Read Direct IO wiki for details. * Added CompactionEventListener and EventListener::OnFlushBegin interfaces. ### New Features + * Memtable flush can be avoided during checkpoint creation if total log file size is smaller than a threshold specified by the user. * Introduce level-based L0->L0 compactions to reduce file count, so write delays are incurred less often. * (Experimental) Partitioning filters which creates an index on the partitions. The feature can be enabled by setting partition_filters when using kFullFilter. Currently the feature also requires two-level indexing to be enabled. Number of partitions is the same as the number of partitions for indexes, which is controlled by metadata_block_size. ## 5.3.0 (2017-03-08) + ### Public API Change + * Remove disableDataSync option. * Remove timeout_hint_us option from WriteOptions. The option has been deprecated and has no effect since 3.13.0. * Remove option min_partial_merge_operands. Partial merge operands will always be merged in flush or compaction if there are more than one. * Remove option verify_checksums_in_compaction. Compaction will always verify checksum. ### Bug Fixes + * Fix the bug that iterator may skip keys ## 5.2.0 (2017-02-08) + ### Public API Change + * NewLRUCache() will determine number of shard bits automatically based on capacity, if the user doesn't pass one. This also impacts the default block cache when the user doesn't explicit provide one. * Change the default of delayed slowdown value to 16MB/s and further increase the L0 stop condition to 36 files. * Options::use_direct_writes and Options::use_direct_reads are now ready to use. * (Experimental) Two-level indexing that partition the index and creates a 2nd level index on the partitions. The feature can be enabled by setting kTwoLevelIndexSearch as IndexType and configuring index_per_partition. ### New Features + * Added new overloaded function GetApproximateSizes that allows to specify if memtable stats should be computed only without computing SST files' stats approximations. * Added new function GetApproximateMemTableStats that approximates both number of records and size of memtables. * Add Direct I/O mode for SST file I/O ### Bug Fixes + * RangeSync() should work if ROCKSDB_FALLOCATE_PRESENT is not set * Fix wrong results in a data race case in Get() * Some fixes related to 2PC. * Fix bugs of data corruption in direct I/O ## 5.1.0 (2017-01-13) + * Support dynamically change `delete_obsolete_files_period_micros` option via SetDBOptions(). * Added EventListener::OnExternalFileIngested which will be called when IngestExternalFile() add a file successfully. * BackupEngine::Open and BackupEngineReadOnly::Open now always return error statuses matching those of the backup Env. ### Bug Fixes + * Fix the bug that if 2PC is enabled, checkpoints may loss some recent transactions. * When file copying is needed when creating checkpoints or bulk loading files, fsync the file after the file copying. ## 5.0.0 (2016-11-17) + ### Public API Change + * Options::max_bytes_for_level_multiplier is now a double along with all getters and setters. * Support dynamically change `delayed_write_rate` and `max_total_wal_size` options via SetDBOptions(). * Introduce DB::DeleteRange for optimized deletion of large ranges of contiguous keys. @@ -1418,101 +1693,130 @@ if set to something > 0 user will see 2 changes in iterators behavior 1) only ke * DB::AddFile() is deprecated and is replaced with DB::IngestExternalFile(). DB::IngestExternalFile() remove all the restrictions that existed for DB::AddFile. ### New Features + * Add avoid_flush_during_shutdown option, which speeds up DB shutdown by not flushing unpersisted data (i.e. with disableWAL = true). Unpersisted data will be lost. The options is dynamically changeable via SetDBOptions(). * Add memtable_insert_with_hint_prefix_extractor option. The option is mean to reduce CPU usage for inserting keys into memtable, if keys can be group by prefix and insert for each prefix are sequential or almost sequential. See include/rocksdb/options.h for more details. * Add LuaCompactionFilter in utilities. This allows developers to write compaction filters in Lua. To use this feature, LUA_PATH needs to be set to the root directory of Lua. * No longer populate "LATEST_BACKUP" file in backup directory, which formerly contained the number of the latest backup. The latest backup can be determined by finding the highest numbered file in the "meta/" subdirectory. ## 4.13.0 (2016-10-18) + ### Public API Change + * DB::GetOptions() reflect dynamic changed options (i.e. through DB::SetOptions()) and return copy of options instead of reference. * Added Statistics::getAndResetTickerCount(). ### New Features + * Add DB::SetDBOptions() to dynamic change base_background_compactions and max_background_compactions. * Added Iterator::SeekForPrev(). This new API will seek to the last key that less than or equal to the target key. ## 4.12.0 (2016-09-12) + ### Public API Change + * CancelAllBackgroundWork() flushes all memtables for databases containing writes that have bypassed the WAL (writes issued with WriteOptions::disableWAL=true) before shutting down background threads. * Merge options source_compaction_factor, max_grandparent_overlap_bytes and expanded_compaction_factor into max_compaction_bytes. * Remove ImmutableCFOptions. * Add a compression type ZSTD, which can work with ZSTD 0.8.0 or up. Still keep ZSTDNotFinal for compatibility reasons. ### New Features + * Introduce NewClockCache, which is based on CLOCK algorithm with better concurrent performance in some cases. It can be used to replace the default LRU-based block cache and table cache. To use it, RocksDB need to be linked with TBB lib. * Change ticker/histogram statistics implementations to accumulate data in thread-local storage, which improves CPU performance by reducing cache coherency costs. Callers of CreateDBStatistics do not need to change anything to use this feature. * Block cache mid-point insertion, where index and filter block are inserted into LRU block cache with higher priority. The feature can be enabled by setting BlockBasedTableOptions::cache_index_and_filter_blocks_with_high_priority to true and high_pri_pool_ratio > 0 when creating NewLRUCache. ## 4.11.0 (2016-08-01) + ### Public API Change + * options.memtable_prefix_bloom_huge_page_tlb_size => memtable_huge_page_size. When it is set, RocksDB will try to allocate memory from huge page for memtable too, rather than just memtable bloom filter. ### New Features + * A tool to migrate DB after options change. See include/rocksdb/utilities/option_change_migration.h. * Add ReadOptions.background_purge_on_iterator_cleanup. If true, we avoid file deletion when destroying iterators. ## 4.10.0 (2016-07-05) + ### Public API Change + * options.memtable_prefix_bloom_bits changes to options.memtable_prefix_bloom_bits_ratio and deprecate options.memtable_prefix_bloom_probes * enum type CompressionType and PerfLevel changes from char to unsigned char. Value of all PerfLevel shift by one. * Deprecate options.filter_deletes. ### New Features + * Add avoid_flush_during_recovery option. * Add a read option background_purge_on_iterator_cleanup to avoid deleting files in foreground when destroying iterators. Instead, a job is scheduled in high priority queue and would be executed in a separate background thread. * RepairDB support for column families. RepairDB now associates data with non-default column families using information embedded in the SST/WAL files (4.7 or later). For data written by 4.6 or earlier, RepairDB associates it with the default column family. * Add options.write_buffer_manager which allows users to control total memtable sizes across multiple DB instances. ## 4.9.0 (2016-06-09) + ### Public API changes + * Add bottommost_compression option, This option can be used to set a specific compression algorithm for the bottommost level (Last level containing files in the DB). * Introduce CompactionJobInfo::compression, This field state the compression algorithm used to generate the output files of the compaction. * Deprecate BlockBaseTableOptions.hash_index_allow_collision=false * Deprecate options builder (GetOptions()). ### New Features + * Introduce NewSimCache() in rocksdb/utilities/sim_cache.h. This function creates a block cache that is able to give simulation results (mainly hit rate) of simulating block behavior with a configurable cache size. ## 4.8.0 (2016-05-02) + ### Public API Change + * Allow preset compression dictionary for improved compression of block-based tables. This is supported for zlib, zstd, and lz4. The compression dictionary's size is configurable via CompressionOptions::max_dict_bytes. -* Delete deprecated classes for creating backups (BackupableDB) and restoring from backups (RestoreBackupableDB). Now, BackupEngine should be used for creating backups, and BackupEngineReadOnly should be used for restorations. For more details, see https://github.com/facebook/rocksdb/wiki/How-to-backup-RocksDB%3F +* Delete deprecated classes for creating backups (BackupableDB) and restoring from backups (RestoreBackupableDB). Now, BackupEngine should be used for creating backups, and BackupEngineReadOnly should be used for restorations. For more details, see * Expose estimate of per-level compression ratio via DB property: "rocksdb.compression-ratio-at-levelN". * Added EventListener::OnTableFileCreationStarted. EventListener::OnTableFileCreated will be called on failure case. User can check creation status via TableFileCreationInfo::status. ### New Features + * Add ReadOptions::readahead_size. If non-zero, NewIterator will create a new table reader which performs reads of the given size. ## 4.7.0 (2016-04-08) + ### Public API Change + * rename options compaction_measure_io_stats to report_bg_io_stats and include flush too. -* Change some default options. Now default options will optimize for server-workloads. Also enable slowdown and full stop triggers for pending compaction bytes. These changes may cause sub-optimal performance or significant increase of resource usage. To avoid these risks, users can open existing RocksDB with options extracted from RocksDB option files. See https://github.com/facebook/rocksdb/wiki/RocksDB-Options-File for how to use RocksDB option files. Or you can call Options.OldDefaults() to recover old defaults. DEFAULT_OPTIONS_HISTORY.md will track change history of default options. +* Change some default options. Now default options will optimize for server-workloads. Also enable slowdown and full stop triggers for pending compaction bytes. These changes may cause sub-optimal performance or significant increase of resource usage. To avoid these risks, users can open existing RocksDB with options extracted from RocksDB option files. See for how to use RocksDB option files. Or you can call Options.OldDefaults() to recover old defaults. DEFAULT_OPTIONS_HISTORY.md will track change history of default options. ## 4.6.0 (2016-03-10) + ### Public API Changes + * Change default of BlockBasedTableOptions.format_version to 2. It means default DB created by 4.6 or up cannot be opened by RocksDB version 3.9 or earlier. * Added strict_capacity_limit option to NewLRUCache. If the flag is set to true, insert to cache will fail if no enough capacity can be free. Signature of Cache::Insert() is updated accordingly. * Tickers [NUMBER_DB_NEXT, NUMBER_DB_PREV, NUMBER_DB_NEXT_FOUND, NUMBER_DB_PREV_FOUND, ITER_BYTES_READ] are not updated immediately. The are updated when the Iterator is deleted. * Add monotonically increasing counter (DB property "rocksdb.current-super-version-number") that increments upon any change to the LSM tree. ### New Features + * Add CompactionPri::kMinOverlappingRatio, a compaction picking mode friendly to write amplification. * Deprecate Iterator::IsKeyPinned() and replace it with Iterator::GetProperty() with prop_name="rocksdb.iterator.is.key.pinned" ## 4.5.0 (2016-02-05) + ### Public API Changes + * Add a new perf context level between kEnableCount and kEnableTime. Level 2 now does not include timers for mutexes. * Statistics of mutex operation durations will not be measured by default. If you want to have them enabled, you need to set Statistics::stats_level_ to kAll. * DBOptions::delete_scheduler and NewDeleteScheduler() are removed, please use DBOptions::sst_file_manager and NewSstFileManager() instead ### New Features + * ldb tool now supports operations to non-default column families. * Add kPersistedTier to ReadTier. This option allows Get and MultiGet to read only the persited data and skip mem-tables if writes were done with disableWAL = true. * Add DBOptions::sst_file_manager. Use NewSstFileManager() in include/rocksdb/sst_file_manager.h to create a SstFileManager that can be used to track the total size of SST files and control the SST files deletion rate. ## 4.4.0 (2016-01-14) + ### Public API Changes + * Change names in CompactionPri and add a new one. * Deprecate options.soft_rate_limit and add options.soft_pending_compaction_bytes_limit. * If options.max_write_buffer_number > 3, writes will be slowed down when writing to the last write buffer to delay a full stop. @@ -1522,36 +1826,44 @@ if set to something > 0 user will see 2 changes in iterators behavior 1) only ke * Added a new parameter --path to ldb tool. --path accepts the name of either MANIFEST, SST or a WAL file. Either --db or --path can be used when calling ldb. ## 4.3.0 (2015-12-08) + ### New Features + * CompactionFilter has new member function called IgnoreSnapshots which allows CompactionFilter to be called even if there are snapshots later than the key. * RocksDB will now persist options under the same directory as the RocksDB database on successful DB::Open, CreateColumnFamily, DropColumnFamily, and SetOptions. * Introduce LoadLatestOptions() in rocksdb/utilities/options_util.h. This function can construct the latest DBOptions / ColumnFamilyOptions used by the specified RocksDB intance. * Introduce CheckOptionsCompatibility() in rocksdb/utilities/options_util.h. This function checks whether the input set of options is able to open the specified DB successfully. ### Public API Changes + * When options.db_write_buffer_size triggers, only the column family with the largest column family size will be flushed, not all the column families. ## 4.2.0 (2015-11-09) + ### New Features + * Introduce CreateLoggerFromOptions(), this function create a Logger for provided DBOptions. * Add GetAggregatedIntProperty(), which returns the sum of the GetIntProperty of all the column families. * Add MemoryUtil in rocksdb/utilities/memory.h. It currently offers a way to get the memory usage by type from a list rocksdb instances. ### Public API Changes + * CompactionFilter::Context includes information of Column Family ID * The need-compaction hint given by TablePropertiesCollector::NeedCompact() will be persistent and recoverable after DB recovery. This introduces a breaking format change. If you use this experimental feature, including NewCompactOnDeletionCollectorFactory() in the new version, you may not be able to directly downgrade the DB back to version 4.0 or lower. * TablePropertiesCollectorFactory::CreateTablePropertiesCollector() now takes an option Context, containing the information of column family ID for the file being written. * Remove DefaultCompactionFilterFactory. - ## 4.1.0 (2015-10-08) + ### New Features + * Added single delete operation as a more efficient way to delete keys that have not been overwritten. * Added experimental AddFile() to DB interface that allow users to add files created by SstFileWriter into an empty Database, see include/rocksdb/sst_file_writer.h and DB::AddFile() for more info. * Added support for opening SST files with .ldb suffix which enables opening LevelDB databases. * CompactionFilter now supports filtering of merge operands and merge results. ### Public API Changes + * Added SingleDelete() to the DB interface. * Added AddFile() to DB interface. * Added SstFileWriter class. @@ -1559,25 +1871,31 @@ if set to something > 0 user will see 2 changes in iterators behavior 1) only ke * We removed CompactionFilterV2 interfaces from include/rocksdb/compaction_filter.h. The functionality was deprecated already in version 3.13. ## 4.0.0 (2015-09-09) + ### New Features + * Added support for transactions. See include/rocksdb/utilities/transaction.h for more info. * DB::GetProperty() now accepts "rocksdb.aggregated-table-properties" and "rocksdb.aggregated-table-properties-at-levelN", in which case it returns aggregated table properties of the target column family, or the aggregated table properties of the specified level N if the "at-level" version is used. * Add compression option kZSTDNotFinalCompression for people to experiment ZSTD although its format is not finalized. * We removed the need for LATEST_BACKUP file in BackupEngine. We still keep writing it when we create new backups (because of backward compatibility), but we don't read it anymore. ### Public API Changes + * Removed class Env::RandomRWFile and Env::NewRandomRWFile(). * Renamed DBOptions.num_subcompactions to DBOptions.max_subcompactions to make the name better match the actual functionality of the option. * Added Equal() method to the Comparator interface that can optionally be overwritten in cases where equality comparisons can be done more efficiently than three-way comparisons. * Previous 'experimental' OptimisticTransaction class has been replaced by Transaction class. ## 3.13.0 (2015-08-06) + ### New Features + * RollbackToSavePoint() in WriteBatch/WriteBatchWithIndex * Add NewCompactOnDeletionCollectorFactory() in utilities/table_properties_collectors, which allows rocksdb to mark a SST file as need-compaction when it observes at least D deletion entries in any N consecutive entries in that SST file. Note that this feature depends on an experimental NeedCompact() API --- the result of this API will not persist after DB restart. * Add DBOptions::delete_scheduler. Use NewDeleteScheduler() in include/rocksdb/delete_scheduler.h to create a DeleteScheduler that can be shared among multiple RocksDB instances to control the file deletion rate of SST files that exist in the first db_path. ### Public API Changes + * Deprecated WriteOptions::timeout_hint_us. We no longer support write timeout. If you really need this option, talk to us and we might consider returning it. * Deprecated purge_redundant_kvs_while_flush option. * Removed BackupEngine::NewBackupEngine() and NewReadOnlyBackupEngine() that were deprecated in RocksDB 3.8. Please use BackupEngine::Open() instead. @@ -1587,16 +1905,21 @@ if set to something > 0 user will see 2 changes in iterators behavior 1) only ke * Add DBOptions::skip_stats_update_on_db_open. When it is on, DB::Open() will run faster as it skips the random reads required for loading necessary stats from SST files to optimize compaction. ## 3.12.0 (2015-07-02) + ### New Features + * Added experimental support for optimistic transactions. See include/rocksdb/utilities/optimistic_transaction.h for more info. * Added a new way to report QPS from db_bench (check out --report_file and --report_interval_seconds) * Added a cache for individual rows. See DBOptions::row_cache for more info. * Several new features on EventListener (see include/rocksdb/listener.h): - - OnCompationCompleted() now returns per-compaction job statistics, defined in include/rocksdb/compaction_job_stats.h. - - Added OnTableFileCreated() and OnTableFileDeleted(). + +* OnCompationCompleted() now returns per-compaction job statistics, defined in include/rocksdb/compaction_job_stats.h. +* Added OnTableFileCreated() and OnTableFileDeleted(). + * Add compaction_options_universal.enable_trivial_move to true, to allow trivial move while performing universal compaction. Trivial move will happen only when all the input files are non overlapping. ### Public API changes + * EventListener::OnFlushCompleted() now passes FlushJobInfo instead of a list of parameters. * DB::GetDbIdentity() is now a const function. If this function is overridden in your application, be sure to also make GetDbIdentity() const to avoid compile error. * Move listeners from ColumnFamilyOptions to DBOptions. @@ -1617,23 +1940,30 @@ if set to something > 0 user will see 2 changes in iterators behavior 1) only ke * Add DB::SyncWAL() that does a WAL sync without blocking writers. ## 3.11.0 (2015-05-19) + ### New Features + * Added a new API Cache::SetCapacity(size_t capacity) to dynamically change the maximum configured capacity of the cache. If the new capacity is less than the existing cache usage, the implementation will try to lower the usage by evicting the necessary number of elements following a strict LRU policy. * Added an experimental API for handling flashcache devices (blacklists background threads from caching their reads) -- NewFlashcacheAwareEnv * If universal compaction is used and options.num_levels > 1, compact files are tried to be stored in none-L0 with smaller files based on options.target_file_size_base. The limitation of DB size when using universal compaction is greatly mitigated by using more levels. You can set num_levels = 1 to make universal compaction behave as before. If you set num_levels > 1 and want to roll back to a previous version, you need to compact all files to a big file in level 0 (by setting target_file_size_base to be large and CompactRange(, nullptr, nullptr, true, 0) and reopen the DB with the same version to rewrite the manifest, and then you can open it using previous releases. * More information about rocksdb background threads are available in Env::GetThreadList(), including the number of bytes read / written by a compaction job, mem-table size and current number of bytes written by a flush job and many more. Check include/rocksdb/thread_status.h for more detail. ### Public API changes + * TablePropertiesCollector::AddUserKey() is added to replace TablePropertiesCollector::Add(). AddUserKey() exposes key type, sequence number and file size up to now to users. * DBOptions::bytes_per_sync used to apply to both WAL and table files. As of 3.11 it applies only to table files. If you want to use this option to sync WAL in the background, please use wal_bytes_per_sync ## 3.10.0 (2015-03-24) + ### New Features + * GetThreadStatus() is now able to report detailed thread status, including: - - Thread Operation including flush and compaction. - - The stage of the current thread operation. - - The elapsed time in micros since the current thread operation started. + +* Thread Operation including flush and compaction. +* The stage of the current thread operation. +* The elapsed time in micros since the current thread operation started. More information can be found in include/rocksdb/thread_status.h. In addition, when running db_bench with --thread_status_per_interval, db_bench will also report thread status periodically. + * Changed the LRU caching algorithm so that referenced blocks (by iterators) are never evicted. This change made parameter removeScanCountLimit obsolete. Because of that NewLRUCache doesn't take three arguments anymore. table_cache_remove_scan_limit option is also removed * By default we now optimize the compilation for the compilation platform (using -march=native). If you want to build portable binary, use 'PORTABLE=1' before the make command. * We now allow level-compaction to place files in different paths by @@ -1653,10 +1983,12 @@ if set to something > 0 user will see 2 changes in iterators behavior 1) only ke * Fixed a bug where we start deleting files of a dropped column families even if there are still live references to it ### Public API changes + * Deprecated skip_log_error_on_recovery and table_cache_remove_scan_count_limit options. * Logger method logv with log level parameter is now virtual ### RocksJava + * Added compression per level API. * MemEnv is now available in RocksJava via RocksMemEnv class. * lz4 compression is now included in rocksjava static library when running `make rocksdbjavastatic`. @@ -1665,6 +1997,7 @@ if set to something > 0 user will see 2 changes in iterators behavior 1) only ke ## 3.9.0 (2014-12-08) ### New Features + * Add rocksdb::GetThreadList(), which in the future will return the current status of all rocksdb-related threads. We will have more code instruments in the following RocksDB releases. @@ -1672,6 +2005,7 @@ if set to something > 0 user will see 2 changes in iterators behavior 1) only ke Also add support for nested options in convert function ### Public API changes + * New API to create a checkpoint added. Given a directory name, creates a new database which is an image of the existing database. * New API LinkFile added to Env. If you implement your own Env class, an @@ -1679,44 +2013,56 @@ if set to something > 0 user will see 2 changes in iterators behavior 1) only ke * MemTableRep takes MemTableAllocator instead of Arena ### Improvements + * RocksDBLite library now becomes smaller and will be compiled with -fno-exceptions flag. ## 3.8.0 (2014-11-14) ### Public API changes + * BackupEngine::NewBackupEngine() was deprecated; please use BackupEngine::Open() from now on. * BackupableDB/RestoreBackupableDB have new GarbageCollect() methods, which will clean up files from corrupt and obsolete backups. * BackupableDB/RestoreBackupableDB have new GetCorruptedBackups() methods which list corrupt backups. ### Cleanup + * Bunch of code cleanup, some extra warnings turned on (-Wshadow, -Wshorten-64-to-32, -Wnon-virtual-dtor) ### New features + * CompactFiles and EventListener, although they are still in experimental state * Full ColumnFamily support in RocksJava. ## 3.7.0 (2014-11-06) + ### Public API changes + * Introduce SetOptions() API to allow adjusting a subset of options dynamically online * Introduce 4 new convenient functions for converting Options from string: GetColumnFamilyOptionsFromMap(), GetColumnFamilyOptionsFromString(), GetDBOptionsFromMap(), GetDBOptionsFromString() * Remove WriteBatchWithIndex.Delete() overloads using SliceParts * When opening a DB, if options.max_background_compactions is larger than the existing low pri pool of options.env, it will enlarge it. Similarly, options.max_background_flushes is larger than the existing high pri pool of options.env, it will enlarge it. ## 3.6.0 (2014-10-07) + ### Disk format changes + * If you're using RocksDB on ARM platforms and you're using default bloom filter, there is a disk format change you need to be aware of. There are three steps you need to do when you convert to new release: 1. turn off filter policy, 2. compact the whole database, 3. turn on filter policy ### Behavior changes + * We have refactored our system of stalling writes. Any stall-related statistics' meanings are changed. Instead of per-write stall counts, we now count stalls per-epoch, where epochs are periods between flushes and compactions. You'll find more information in our Tuning Perf Guide once we release RocksDB 3.6. * When disableDataSync=true, we no longer sync the MANIFEST file. * Add identity_as_first_hash property to CuckooTable. SST file needs to be rebuilt to be opened by reader properly. ### Public API changes + * Change target_file_size_base type to uint64_t from int. * Remove allow_thread_local. This feature was proved to be stable, so we are turning it always-on. ## 3.5.0 (2014-09-03) + ### New Features + * Add include/utilities/write_batch_with_index.h, providing a utility class to query data out of WriteBatch when building it. * Move BlockBasedTable related options to BlockBasedTableOptions from Options. Change corresponding JNI interface. Options affected include: no_block_cache, block_cache, block_cache_compressed, block_size, block_size_deviation, block_restart_interval, filter_policy, whole_key_filtering. filter_policy is changed to shared_ptr from a raw pointer. @@ -1724,16 +2070,20 @@ if set to something > 0 user will see 2 changes in iterators behavior 1) only ke * OptimizeForPointLookup() takes one parameter for block cache size. It now builds hash index, bloom filter, and block cache. ### Public API changes + * The Prefix Extractor used with V2 compaction filters is now passed user key to SliceTransform::Transform instead of unparsed RocksDB key. ## 3.4.0 (2014-08-18) + ### New Features + * Support Multiple DB paths in universal style compactions * Add feature of storing plain table index and bloom filter in SST file. * CompactRange() will never output compacted files to level 0. This used to be the case when all the compaction input files were at level 0. * Added iterate_upper_bound to define the extent upto which the forward iterator will return entries. This will prevent iterating over delete markers and overwritten entries for edge cases where you want to break out the iterator anyways. This may improve performance in case there are a large number of delete markers or overwritten entries. ### Public API changes + * DBOptions.db_paths now is a vector of a DBPath structure which indicates both of path and target size * NewPlainTableFactory instead of bunch of parameters now accepts PlainTableOptions, which is defined in include/rocksdb/table.h * Moved include/utilities/*.h to include/rocksdb/utilities/*.h @@ -1743,7 +2093,9 @@ if set to something > 0 user will see 2 changes in iterators behavior 1) only ke * The Prefix Extractor used with V2 compaction filters is now passed user key to SliceTransform::Transform instead of unparsed RocksDB key. ## 3.3.0 (2014-07-10) + ### New Features + * Added JSON API prototype. * HashLinklist reduces performance outlier caused by skewed bucket by switching data in the bucket from linked list to skip list. Add parameter threshold_use_skiplist in NewHashLinkListRepFactory(). * RocksDB is now able to reclaim storage space more effectively during the compaction process. This is done by compensating the size of each deletion entry by the 2X average value size, which makes compaction to be triggered by deletion entries more easily. @@ -1751,43 +2103,53 @@ if set to something > 0 user will see 2 changes in iterators behavior 1) only ke * Add a rate_limiter option, which controls total throughput of flush and compaction. The throughput is specified in bytes/sec. Flush always has precedence over compaction when available bandwidth is constrained. ### Public API changes + * Removed NewTotalOrderPlainTableFactory because it is not used and implemented semantically incorrect. ## 3.2.0 (2014-06-20) ### Public API changes + * We removed seek compaction as a concept from RocksDB because: + 1) It makes more sense for spinning disk workloads, while RocksDB is primarily designed for flash and memory, 2) It added some complexity to the important code-paths, 3) None of our internal customers were really using it. Because of that, Options::disable_seek_compaction is now obsolete. It is still a parameter in Options, so it does not break the build, but it does not have any effect. We plan to completely remove it at some point, so we ask users to please remove this option from your code base. + * Add two parameters to NewHashLinkListRepFactory() for logging on too many entries in a hash bucket when flushing. * Added new option BlockBasedTableOptions::hash_index_allow_collision. When enabled, prefix hash index for block-based table will not store prefix and allow hash collision, reducing memory consumption. ### New Features + * PlainTable now supports a new key encoding: for keys of the same prefix, the prefix is only written once. It can be enabled through encoding_type parameter of NewPlainTableFactory() * Add AdaptiveTableFactory, which is used to convert from a DB of PlainTable to BlockBasedTabe, or vise versa. It can be created using NewAdaptiveTableFactory() ### Performance Improvements + * Tailing Iterator re-implemeted with ForwardIterator + Cascading Search Hint , see ~20% throughput improvement. ## 3.1.0 (2014-05-21) ### Public API changes + * Replaced ColumnFamilyOptions::table_properties_collectors with ColumnFamilyOptions::table_properties_collector_factories ### New Features + * Hash index for block-based table will be materialized and reconstructed more efficiently. Previously hash index is constructed by scanning the whole table during every table open. * FIFO compaction style ## 3.0.0 (2014-05-05) ### Public API changes + * Added _LEVEL to all InfoLogLevel enums -* Deprecated ReadOptions.prefix and ReadOptions.prefix_seek. Seek() defaults to prefix-based seek when Options.prefix_extractor is supplied. More detail is documented in https://github.com/facebook/rocksdb/wiki/Prefix-Seek-API-Changes +* Deprecated ReadOptions.prefix and ReadOptions.prefix_seek. Seek() defaults to prefix-based seek when Options.prefix_extractor is supplied. More detail is documented in * MemTableRepFactory::CreateMemTableRep() takes info logger as an extra parameter. ### New Features + * Column family support * Added an option to use different checksum functions in BlockBasedTableOptions * Added ApplyToAllCacheEntries() function to Cache @@ -1812,6 +2174,7 @@ Because of that, Options::disable_seek_compaction is now obsolete. It is still a * Supports several more DB properties: compaction-pending, background-errors and cur-size-active-mem-table. ### New Features + * If we find one truncated record at the end of the MANIFEST or WAL files, we will ignore it. We assume that writers of these records were interrupted and that we can safely ignore it. diff --git a/cache/cache.cc b/cache/cache.cc index a7d9f86bfb1..c5271659f46 100644 --- a/cache/cache.cc +++ b/cache/cache.cc @@ -33,6 +33,10 @@ static std::unordered_map {offsetof(struct LRUCacheOptions, high_pri_pool_ratio), OptionType::kDouble, OptionVerificationType::kNormal, OptionTypeFlags::kMutable}}, + {"low_pri_pool_ratio", + {offsetof(struct LRUCacheOptions, low_pri_pool_ratio), + OptionType::kDouble, OptionVerificationType::kNormal, + OptionTypeFlags::kMutable}}, }; #endif // ROCKSDB_LITE diff --git a/cache/lru_cache.cc b/cache/lru_cache.cc index 60cfb345885..4622f9c2a88 100644 --- a/cache/lru_cache.cc +++ b/cache/lru_cache.cc @@ -111,14 +111,17 @@ void LRUHandleTable::Resize() { LRUCacheShard::LRUCacheShard( size_t capacity, bool strict_capacity_limit, double high_pri_pool_ratio, - bool use_adaptive_mutex, CacheMetadataChargePolicy metadata_charge_policy, - int max_upper_hash_bits, + double low_pri_pool_ratio, bool use_adaptive_mutex, + CacheMetadataChargePolicy metadata_charge_policy, int max_upper_hash_bits, const std::shared_ptr& secondary_cache) : capacity_(0), high_pri_pool_usage_(0), + low_pri_pool_usage_(0), strict_capacity_limit_(strict_capacity_limit), high_pri_pool_ratio_(high_pri_pool_ratio), high_pri_pool_capacity_(0), + low_pri_pool_ratio_(low_pri_pool_ratio), + low_pri_pool_capacity_(0), table_(max_upper_hash_bits), usage_(0), lru_usage_(0), @@ -129,6 +132,7 @@ LRUCacheShard::LRUCacheShard( lru_.next = &lru_; lru_.prev = &lru_; lru_low_pri_ = &lru_; + lru_bottom_pri_ = &lru_; SetCapacity(capacity); } @@ -191,10 +195,12 @@ void LRUCacheShard::ApplyToSomeEntries( index_begin, index_end); } -void LRUCacheShard::TEST_GetLRUList(LRUHandle** lru, LRUHandle** lru_low_pri) { +void LRUCacheShard::TEST_GetLRUList(LRUHandle** lru, LRUHandle** lru_low_pri, + LRUHandle** lru_bottom_pri) { MutexLock l(&mutex_); *lru = &lru_; *lru_low_pri = lru_low_pri_; + *lru_bottom_pri = lru_bottom_pri_; } size_t LRUCacheShard::TEST_GetLRUSize() { @@ -213,12 +219,20 @@ double LRUCacheShard::GetHighPriPoolRatio() { return high_pri_pool_ratio_; } +double LRUCacheShard::GetLowPriPoolRatio() { + MutexLock l(&mutex_); + return low_pri_pool_ratio_; +} + void LRUCacheShard::LRU_Remove(LRUHandle* e) { assert(e->next != nullptr); assert(e->prev != nullptr); if (lru_low_pri_ == e) { lru_low_pri_ = e->prev; } + if (lru_bottom_pri_ == e) { + lru_bottom_pri_ = e->prev; + } e->next->prev = e->prev; e->prev->next = e->next; e->prev = e->next = nullptr; @@ -228,6 +242,9 @@ void LRUCacheShard::LRU_Remove(LRUHandle* e) { if (e->InHighPriPool()) { assert(high_pri_pool_usage_ >= total_charge); high_pri_pool_usage_ -= total_charge; + } else if (e->InLowPriPool()) { + assert(low_pri_pool_usage_ >= total_charge); + low_pri_pool_usage_ -= total_charge; } } @@ -242,17 +259,34 @@ void LRUCacheShard::LRU_Insert(LRUHandle* e) { e->prev->next = e; e->next->prev = e; e->SetInHighPriPool(true); + e->SetInLowPriPool(false); high_pri_pool_usage_ += total_charge; MaintainPoolSize(); - } else { - // Insert "e" to the head of low-pri pool. Note that when - // high_pri_pool_ratio is 0, head of low-pri pool is also head of LRU list. + } else if (low_pri_pool_ratio_ > 0 && + (e->IsHighPri() || e->IsLowPri() || e->HasHit())) { + // Insert "e" to the head of low-pri pool. e->next = lru_low_pri_->next; e->prev = lru_low_pri_; e->prev->next = e; e->next->prev = e; e->SetInHighPriPool(false); + e->SetInLowPriPool(true); + low_pri_pool_usage_ += total_charge; + MaintainPoolSize(); lru_low_pri_ = e; + } else { + // Insert "e" to the head of bottom-pri pool. + e->next = lru_bottom_pri_->next; + e->prev = lru_bottom_pri_; + e->prev->next = e; + e->next->prev = e; + e->SetInHighPriPool(false); + e->SetInLowPriPool(false); + // if the low-pri pool is empty, lru_low_pri_ also needs to be updated. + if (lru_bottom_pri_ == lru_low_pri_) { + lru_low_pri_ = e; + } + lru_bottom_pri_ = e; } lru_usage_ += total_charge; } @@ -263,10 +297,24 @@ void LRUCacheShard::MaintainPoolSize() { lru_low_pri_ = lru_low_pri_->next; assert(lru_low_pri_ != &lru_); lru_low_pri_->SetInHighPriPool(false); + lru_low_pri_->SetInLowPriPool(true); size_t total_charge = lru_low_pri_->CalcTotalCharge(metadata_charge_policy_); assert(high_pri_pool_usage_ >= total_charge); high_pri_pool_usage_ -= total_charge; + low_pri_pool_usage_ += total_charge; + } + + while (low_pri_pool_usage_ > low_pri_pool_capacity_) { + // Overflow last entry in low-pri pool to bottom-pri pool. + lru_bottom_pri_ = lru_bottom_pri_->next; + assert(lru_bottom_pri_ != &lru_); + lru_bottom_pri_->SetInHighPriPool(false); + lru_bottom_pri_->SetInLowPriPool(false); + size_t total_charge = + lru_bottom_pri_->CalcTotalCharge(metadata_charge_policy_); + assert(low_pri_pool_usage_ >= total_charge); + low_pri_pool_usage_ -= total_charge; } } @@ -292,6 +340,7 @@ void LRUCacheShard::SetCapacity(size_t capacity) { MutexLock l(&mutex_); capacity_ = capacity; high_pri_pool_capacity_ = capacity_ * high_pri_pool_ratio_; + low_pri_pool_capacity_ = capacity_ * low_pri_pool_ratio_; EvictFromLRU(0, &last_reference_list); } @@ -507,6 +556,13 @@ void LRUCacheShard::SetHighPriorityPoolRatio(double high_pri_pool_ratio) { MaintainPoolSize(); } +void LRUCacheShard::SetLowPriorityPoolRatio(double low_pri_pool_ratio) { + MutexLock l(&mutex_); + low_pri_pool_ratio_ = low_pri_pool_ratio; + low_pri_pool_capacity_ = capacity_ * low_pri_pool_ratio_; + MaintainPoolSize(); +} + bool LRUCacheShard::Release(Cache::Handle* handle, bool force_erase) { if (handle == nullptr) { return false; @@ -640,12 +696,15 @@ std::string LRUCacheShard::GetPrintableOptions() const { MutexLock l(&mutex_); snprintf(buffer, kBufferSize, " high_pri_pool_ratio: %.3lf\n", high_pri_pool_ratio_); + snprintf(buffer + strlen(buffer), kBufferSize - strlen(buffer), + " low_pri_pool_ratio: %.3lf\n", low_pri_pool_ratio_); } return std::string(buffer); } LRUCache::LRUCache(size_t capacity, int num_shard_bits, bool strict_capacity_limit, double high_pri_pool_ratio, + double low_pri_pool_ratio, std::shared_ptr allocator, bool use_adaptive_mutex, CacheMetadataChargePolicy metadata_charge_policy, @@ -659,7 +718,7 @@ LRUCache::LRUCache(size_t capacity, int num_shard_bits, for (int i = 0; i < num_shards_; i++) { new (&shards_[i]) LRUCacheShard( per_shard, strict_capacity_limit, high_pri_pool_ratio, - use_adaptive_mutex, metadata_charge_policy, + low_pri_pool_ratio, use_adaptive_mutex, metadata_charge_policy, /* max_upper_hash_bits */ 32 - num_shard_bits, secondary_cache); } secondary_cache_ = secondary_cache; @@ -763,7 +822,8 @@ std::shared_ptr NewLRUCache( double high_pri_pool_ratio, std::shared_ptr memory_allocator, bool use_adaptive_mutex, CacheMetadataChargePolicy metadata_charge_policy, - const std::shared_ptr& secondary_cache) { + const std::shared_ptr& secondary_cache, + double low_pri_pool_ratio) { if (num_shard_bits >= 20) { return nullptr; // the cache cannot be sharded into too many fine pieces } @@ -771,30 +831,40 @@ std::shared_ptr NewLRUCache( // invalid high_pri_pool_ratio return nullptr; } + if (low_pri_pool_ratio < 0.0 || low_pri_pool_ratio > 1.0) { + // Invalid high_pri_pool_ratio + return nullptr; + } + if (low_pri_pool_ratio + high_pri_pool_ratio > 1.0) { + // Invalid high_pri_pool_ratio and low_pri_pool_ratio combination + return nullptr; + } if (num_shard_bits < 0) { num_shard_bits = GetDefaultCacheShardBits(capacity); } return std::make_shared( capacity, num_shard_bits, strict_capacity_limit, high_pri_pool_ratio, - std::move(memory_allocator), use_adaptive_mutex, metadata_charge_policy, - secondary_cache); + low_pri_pool_ratio, std::move(memory_allocator), use_adaptive_mutex, + metadata_charge_policy, secondary_cache); } std::shared_ptr NewLRUCache(const LRUCacheOptions& cache_opts) { - return NewLRUCache( - cache_opts.capacity, cache_opts.num_shard_bits, - cache_opts.strict_capacity_limit, cache_opts.high_pri_pool_ratio, - cache_opts.memory_allocator, cache_opts.use_adaptive_mutex, - cache_opts.metadata_charge_policy, cache_opts.secondary_cache); + return NewLRUCache(cache_opts.capacity, cache_opts.num_shard_bits, + cache_opts.strict_capacity_limit, + cache_opts.high_pri_pool_ratio, + cache_opts.memory_allocator, cache_opts.use_adaptive_mutex, + cache_opts.metadata_charge_policy, + cache_opts.secondary_cache, cache_opts.low_pri_pool_ratio); } std::shared_ptr NewLRUCache( size_t capacity, int num_shard_bits, bool strict_capacity_limit, double high_pri_pool_ratio, std::shared_ptr memory_allocator, bool use_adaptive_mutex, - CacheMetadataChargePolicy metadata_charge_policy) { + CacheMetadataChargePolicy metadata_charge_policy, + double low_pri_pool_ratio) { return NewLRUCache(capacity, num_shard_bits, strict_capacity_limit, high_pri_pool_ratio, memory_allocator, use_adaptive_mutex, - metadata_charge_policy, nullptr); + metadata_charge_policy, nullptr, low_pri_pool_ratio); } } // namespace ROCKSDB_NAMESPACE diff --git a/cache/lru_cache.h b/cache/lru_cache.h index 7013c932840..9db3fbea99d 100644 --- a/cache/lru_cache.h +++ b/cache/lru_cache.h @@ -72,7 +72,7 @@ struct LRUHandle { // The number of external refs to this entry. The cache itself is not counted. uint32_t refs; - enum Flags : uint8_t { + enum Flags : uint16_t { // Whether this entry is referenced by the hash table. IN_CACHE = (1 << 0), // Whether this entry is high priority entry. @@ -87,9 +87,13 @@ struct LRUHandle { IS_PENDING = (1 << 5), // Has the item been promoted from a lower tier IS_PROMOTED = (1 << 6), + // Whether this entry is low priority entry. + IS_LOW_PRI = (1 << 7), + // Whether this entry is in low-pri pool. + IN_LOW_PRI_POOL = (1 << 8), }; - uint8_t flags; + uint16_t flags; #ifdef __SANITIZE_THREAD__ // TSAN can report a false data race on flags, where one thread is writing @@ -120,6 +124,8 @@ struct LRUHandle { bool InCache() const { return flags & IN_CACHE; } bool IsHighPri() const { return flags & IS_HIGH_PRI; } bool InHighPriPool() const { return flags & IN_HIGH_PRI_POOL; } + bool IsLowPri() const { return flags & IS_LOW_PRI; } + bool InLowPriPool() const { return flags & IN_LOW_PRI_POOL; } bool HasHit() const { return flags & HAS_HIT; } bool IsSecondaryCacheCompatible() const { #ifdef __SANITIZE_THREAD__ @@ -142,8 +148,13 @@ struct LRUHandle { void SetPriority(Cache::Priority priority) { if (priority == Cache::Priority::HIGH) { flags |= IS_HIGH_PRI; + flags &= ~IS_LOW_PRI; + } else if (priority == Cache::Priority::LOW) { + flags &= ~IS_HIGH_PRI; + flags |= IS_LOW_PRI; } else { flags &= ~IS_HIGH_PRI; + flags &= ~IS_LOW_PRI; } } @@ -155,6 +166,14 @@ struct LRUHandle { } } + void SetInLowPriPool(bool in_low_pri_pool) { + if (in_low_pri_pool) { + flags |= IN_LOW_PRI_POOL; + } else { + flags &= ~IN_LOW_PRI_POOL; + } + } + void SetHit() { flags |= HAS_HIT; } void SetSecondaryCacheCompatible(bool compat) { @@ -283,7 +302,8 @@ class LRUHandleTable { class ALIGN_AS(CACHE_LINE_SIZE) LRUCacheShard final : public CacheShard { public: LRUCacheShard(size_t capacity, bool strict_capacity_limit, - double high_pri_pool_ratio, bool use_adaptive_mutex, + double high_pri_pool_ratio, double low_pri_pool_ratio, + bool use_adaptive_mutex, CacheMetadataChargePolicy metadata_charge_policy, int max_upper_hash_bits, const std::shared_ptr& secondary_cache); @@ -300,6 +320,9 @@ class ALIGN_AS(CACHE_LINE_SIZE) LRUCacheShard final : public CacheShard { // Set percentage of capacity reserved for high-pri cache entries. void SetHighPriorityPoolRatio(double high_pri_pool_ratio); + // Set percentage of capacity reserved for low-pri cache entries. + void SetLowPriorityPoolRatio(double low_pri_pool_ratio); + // Like Cache methods, but with an extra "hash" parameter. virtual Status Insert(const Slice& key, uint32_t hash, void* value, size_t charge, Cache::DeleterFn deleter, @@ -352,15 +375,19 @@ class ALIGN_AS(CACHE_LINE_SIZE) LRUCacheShard final : public CacheShard { virtual std::string GetPrintableOptions() const override; - void TEST_GetLRUList(LRUHandle** lru, LRUHandle** lru_low_pri); + void TEST_GetLRUList(LRUHandle** lru, LRUHandle** lru_low_pri, + LRUHandle** lru_bottom_pri); - // Retrieves number of elements in LRU, for unit test purpose only - // not threadsafe + // Retrieves number of elements in LRU, for unit test purpose only. + // Not threadsafe. size_t TEST_GetLRUSize(); - // Retrieves high pri pool ratio + // Retrieves high pri pool ratio double GetHighPriPoolRatio(); + // Retrieves low pri pool ratio + double GetLowPriPoolRatio(); + private: friend class LRUCache; // Insert an item into the hash table and, if handle is null, insert into @@ -398,6 +425,9 @@ class ALIGN_AS(CACHE_LINE_SIZE) LRUCacheShard final : public CacheShard { // Memory size for entries in high-pri pool. size_t high_pri_pool_usage_; + // Memory size for entries in low-pri pool. + size_t low_pri_pool_usage_; + // Whether to reject insertion if cache reaches its full capacity. bool strict_capacity_limit_; @@ -408,6 +438,13 @@ class ALIGN_AS(CACHE_LINE_SIZE) LRUCacheShard final : public CacheShard { // Remember the value to avoid recomputing each time. double high_pri_pool_capacity_; + // Ratio of capacity reserved for low priority cache entries. + double low_pri_pool_ratio_; + + // Low-pri pool size, equals to capacity * low_pri_pool_ratio. + // Remember the value to avoid recomputing each time. + double low_pri_pool_capacity_; + // Dummy head of LRU list. // lru.prev is newest entry, lru.next is oldest entry. // LRU contains items which can be evicted, ie reference only by cache @@ -416,6 +453,9 @@ class ALIGN_AS(CACHE_LINE_SIZE) LRUCacheShard final : public CacheShard { // Pointer to head of low-pri pool in LRU list. LRUHandle* lru_low_pri_; + // Pointer to head of bottom-pri pool in LRU list. + LRUHandle* lru_bottom_pri_; + // ------------^^^^^^^^^^^^^----------- // Not frequently modified data members // ------------------------------------ @@ -450,7 +490,7 @@ class LRUCache : public ShardedCache { public: LRUCache(size_t capacity, int num_shard_bits, bool strict_capacity_limit, - double high_pri_pool_ratio, + double high_pri_pool_ratio, double low_pri_pool_ratio, std::shared_ptr memory_allocator = nullptr, bool use_adaptive_mutex = kDefaultToAdaptiveMutex, CacheMetadataChargePolicy metadata_charge_policy = diff --git a/cache/lru_cache_test.cc b/cache/lru_cache_test.cc index 67552dc4dc2..a7a76c475bd 100644 --- a/cache/lru_cache_test.cc +++ b/cache/lru_cache_test.cc @@ -39,13 +39,14 @@ class LRUCacheTest : public testing::Test { } void NewCache(size_t capacity, double high_pri_pool_ratio = 0.0, + double low_pri_pool_ratio = 1.0, bool use_adaptive_mutex = kDefaultToAdaptiveMutex) { DeleteCache(); cache_ = reinterpret_cast( port::cacheline_aligned_alloc(sizeof(LRUCacheShard))); new (cache_) LRUCacheShard( capacity, false /*strict_capcity_limit*/, high_pri_pool_ratio, - use_adaptive_mutex, kDontChargeCacheMetadata, + low_pri_pool_ratio, use_adaptive_mutex, kDontChargeCacheMetadata, 24 /*max_upper_hash_bits*/, nullptr /*secondary_cache*/); } @@ -74,32 +75,66 @@ class LRUCacheTest : public testing::Test { void Erase(const std::string& key) { cache_->Erase(key, 0 /*hash*/); } void ValidateLRUList(std::vector keys, - size_t num_high_pri_pool_keys = 0) { + size_t num_high_pri_pool_keys = 0, + size_t num_low_pri_pool_keys = 0, + size_t num_bottom_pri_pool_keys = 0) { LRUHandle* lru; LRUHandle* lru_low_pri; - cache_->TEST_GetLRUList(&lru, &lru_low_pri); + LRUHandle* lru_bottom_pri; + cache_->TEST_GetLRUList(&lru, &lru_low_pri, &lru_bottom_pri); + LRUHandle* iter = lru; + + bool in_low_pri_pool = false; bool in_high_pri_pool = false; + size_t high_pri_pool_keys = 0; + size_t low_pri_pool_keys = 0; + size_t bottom_pri_pool_keys = 0; + + if (iter == lru_bottom_pri) { + in_low_pri_pool = true; + in_high_pri_pool = false; + } if (iter == lru_low_pri) { + in_low_pri_pool = false; in_high_pri_pool = true; } + for (const auto& key : keys) { iter = iter->next; ASSERT_NE(lru, iter); ASSERT_EQ(key, iter->key().ToString()); ASSERT_EQ(in_high_pri_pool, iter->InHighPriPool()); + ASSERT_EQ(in_low_pri_pool, iter->InLowPriPool()); if (in_high_pri_pool) { + ASSERT_FALSE(iter->InLowPriPool()); high_pri_pool_keys++; + } else if (in_low_pri_pool) { + ASSERT_FALSE(iter->InHighPriPool()); + low_pri_pool_keys++; + } else { + bottom_pri_pool_keys++; + } + if (iter == lru_bottom_pri) { + ASSERT_FALSE(in_low_pri_pool); + ASSERT_FALSE(in_high_pri_pool); + in_low_pri_pool = true; + in_high_pri_pool = false; } if (iter == lru_low_pri) { + ASSERT_TRUE(in_low_pri_pool); ASSERT_FALSE(in_high_pri_pool); + in_low_pri_pool = false; in_high_pri_pool = true; } } ASSERT_EQ(lru, iter->next); + ASSERT_FALSE(in_low_pri_pool); ASSERT_TRUE(in_high_pri_pool); ASSERT_EQ(num_high_pri_pool_keys, high_pri_pool_keys); + ASSERT_EQ(num_low_pri_pool_keys, low_pri_pool_keys); + ASSERT_EQ(num_bottom_pri_pool_keys, bottom_pri_pool_keys); } private: @@ -111,98 +146,219 @@ TEST_F(LRUCacheTest, BasicLRU) { for (char ch = 'a'; ch <= 'e'; ch++) { Insert(ch); } - ValidateLRUList({"a", "b", "c", "d", "e"}); + ValidateLRUList({"a", "b", "c", "d", "e"}, 0, 5); for (char ch = 'x'; ch <= 'z'; ch++) { Insert(ch); } - ValidateLRUList({"d", "e", "x", "y", "z"}); + ValidateLRUList({"d", "e", "x", "y", "z"}, 0, 5); ASSERT_FALSE(Lookup("b")); - ValidateLRUList({"d", "e", "x", "y", "z"}); + ValidateLRUList({"d", "e", "x", "y", "z"}, 0, 5); ASSERT_TRUE(Lookup("e")); - ValidateLRUList({"d", "x", "y", "z", "e"}); + ValidateLRUList({"d", "x", "y", "z", "e"}, 0, 5); ASSERT_TRUE(Lookup("z")); - ValidateLRUList({"d", "x", "y", "e", "z"}); + ValidateLRUList({"d", "x", "y", "e", "z"}, 0, 5); Erase("x"); - ValidateLRUList({"d", "y", "e", "z"}); + ValidateLRUList({"d", "y", "e", "z"}, 0, 4); ASSERT_TRUE(Lookup("d")); - ValidateLRUList({"y", "e", "z", "d"}); + ValidateLRUList({"y", "e", "z", "d"}, 0, 4); Insert("u"); - ValidateLRUList({"y", "e", "z", "d", "u"}); + ValidateLRUList({"y", "e", "z", "d", "u"}, 0, 5); Insert("v"); - ValidateLRUList({"e", "z", "d", "u", "v"}); + ValidateLRUList({"e", "z", "d", "u", "v"}, 0, 5); } -TEST_F(LRUCacheTest, MidpointInsertion) { - // Allocate 2 cache entries to high-pri pool. - NewCache(5, 0.45); +TEST_F(LRUCacheTest, LowPriorityMidpointInsertion) { + // Allocate 2 cache entries to high-pri pool and 3 to low-pri pool. + NewCache(5, /* high_pri_pool_ratio */ 0.40, /* low_pri_pool_ratio */ 0.60); Insert("a", Cache::Priority::LOW); Insert("b", Cache::Priority::LOW); Insert("c", Cache::Priority::LOW); Insert("x", Cache::Priority::HIGH); Insert("y", Cache::Priority::HIGH); - ValidateLRUList({"a", "b", "c", "x", "y"}, 2); + ValidateLRUList({"a", "b", "c", "x", "y"}, 2, 3); // Low-pri entries inserted to the tail of low-pri list (the midpoint). // After lookup, it will move to the tail of the full list. Insert("d", Cache::Priority::LOW); - ValidateLRUList({"b", "c", "d", "x", "y"}, 2); + ValidateLRUList({"b", "c", "d", "x", "y"}, 2, 3); ASSERT_TRUE(Lookup("d")); - ValidateLRUList({"b", "c", "x", "y", "d"}, 2); + ValidateLRUList({"b", "c", "x", "y", "d"}, 2, 3); // High-pri entries will be inserted to the tail of full list. Insert("z", Cache::Priority::HIGH); - ValidateLRUList({"c", "x", "y", "d", "z"}, 2); + ValidateLRUList({"c", "x", "y", "d", "z"}, 2, 3); +} + +TEST_F(LRUCacheTest, BottomPriorityMidpointInsertion) { + // Allocate 2 cache entries to high-pri pool and 2 to low-pri pool. + NewCache(6, /* high_pri_pool_ratio */ 0.35, /* low_pri_pool_ratio */ 0.35); + + Insert("a", Cache::Priority::BOTTOM); + Insert("b", Cache::Priority::BOTTOM); + Insert("i", Cache::Priority::LOW); + Insert("j", Cache::Priority::LOW); + Insert("x", Cache::Priority::HIGH); + Insert("y", Cache::Priority::HIGH); + ValidateLRUList({"a", "b", "i", "j", "x", "y"}, 2, 2, 2); + + // Low-pri entries will be inserted to the tail of low-pri list (the + // midpoint). After lookup, 'k' will move to the tail of the full list, and + // 'x' will spill over to the low-pri pool. + Insert("k", Cache::Priority::LOW); + ValidateLRUList({"b", "i", "j", "k", "x", "y"}, 2, 2, 2); + ASSERT_TRUE(Lookup("k")); + ValidateLRUList({"b", "i", "j", "x", "y", "k"}, 2, 2, 2); + + // High-pri entries will be inserted to the tail of full list. Although y was + // inserted with high priority, it got spilled over to the low-pri pool. As + // a result, j also got spilled over to the bottom-pri pool. + Insert("z", Cache::Priority::HIGH); + ValidateLRUList({"i", "j", "x", "y", "k", "z"}, 2, 2, 2); + Erase("x"); + ValidateLRUList({"i", "j", "y", "k", "z"}, 2, 1, 2); + Erase("y"); + ValidateLRUList({"i", "j", "k", "z"}, 2, 0, 2); + + // Bottom-pri entries will be inserted to the tail of bottom-pri list. + Insert("c", Cache::Priority::BOTTOM); + ValidateLRUList({"i", "j", "c", "k", "z"}, 2, 0, 3); + Insert("d", Cache::Priority::BOTTOM); + ValidateLRUList({"i", "j", "c", "d", "k", "z"}, 2, 0, 4); + Insert("e", Cache::Priority::BOTTOM); + ValidateLRUList({"j", "c", "d", "e", "k", "z"}, 2, 0, 4); + + // Low-pri entries will be inserted to the tail of low-pri list (the + // midpoint). + Insert("l", Cache::Priority::LOW); + ValidateLRUList({"c", "d", "e", "l", "k", "z"}, 2, 1, 3); + Insert("m", Cache::Priority::LOW); + ValidateLRUList({"d", "e", "l", "m", "k", "z"}, 2, 2, 2); + + Erase("k"); + ValidateLRUList({"d", "e", "l", "m", "z"}, 1, 2, 2); + Erase("z"); + ValidateLRUList({"d", "e", "l", "m"}, 0, 2, 2); + + // Bottom-pri entries will be inserted to the tail of bottom-pri list. + Insert("f", Cache::Priority::BOTTOM); + ValidateLRUList({"d", "e", "f", "l", "m"}, 0, 2, 3); + Insert("g", Cache::Priority::BOTTOM); + ValidateLRUList({"d", "e", "f", "g", "l", "m"}, 0, 2, 4); + + // High-pri entries will be inserted to the tail of full list. + Insert("o", Cache::Priority::HIGH); + ValidateLRUList({"e", "f", "g", "l", "m", "o"}, 1, 2, 3); + Insert("p", Cache::Priority::HIGH); + ValidateLRUList({"f", "g", "l", "m", "o", "p"}, 2, 2, 2); } TEST_F(LRUCacheTest, EntriesWithPriority) { - // Allocate 2 cache entries to high-pri pool. - NewCache(5, 0.45); + // Allocate 2 cache entries to high-pri pool and 2 to low-pri pool. + NewCache(6, /* high_pri_pool_ratio */ 0.35, /* low_pri_pool_ratio */ 0.35); Insert("a", Cache::Priority::LOW); Insert("b", Cache::Priority::LOW); + ValidateLRUList({"a", "b"}, 0, 2, 0); + // Low-pri entries can overflow to bottom-pri pool. Insert("c", Cache::Priority::LOW); - ValidateLRUList({"a", "b", "c"}, 0); + ValidateLRUList({"a", "b", "c"}, 0, 2, 1); - // Low-pri entries can take high-pri pool capacity if available + // Bottom-pri entries can take high-pri pool capacity if available + Insert("t", Cache::Priority::LOW); Insert("u", Cache::Priority::LOW); + ValidateLRUList({"a", "b", "c", "t", "u"}, 0, 2, 3); Insert("v", Cache::Priority::LOW); - ValidateLRUList({"a", "b", "c", "u", "v"}, 0); + ValidateLRUList({"a", "b", "c", "t", "u", "v"}, 0, 2, 4); + Insert("w", Cache::Priority::LOW); + ValidateLRUList({"b", "c", "t", "u", "v", "w"}, 0, 2, 4); Insert("X", Cache::Priority::HIGH); Insert("Y", Cache::Priority::HIGH); - ValidateLRUList({"c", "u", "v", "X", "Y"}, 2); + ValidateLRUList({"t", "u", "v", "w", "X", "Y"}, 2, 2, 2); - // High-pri entries can overflow to low-pri pool. + // After lookup, the high-pri entry 'X' got spilled over to the low-pri pool. + // The low-pri entry 'v' got spilled over to the bottom-pri pool. Insert("Z", Cache::Priority::HIGH); - ValidateLRUList({"u", "v", "X", "Y", "Z"}, 2); + ValidateLRUList({"u", "v", "w", "X", "Y", "Z"}, 2, 2, 2); // Low-pri entries will be inserted to head of low-pri pool. Insert("a", Cache::Priority::LOW); - ValidateLRUList({"v", "X", "a", "Y", "Z"}, 2); + ValidateLRUList({"v", "w", "X", "a", "Y", "Z"}, 2, 2, 2); - // Low-pri entries will be inserted to head of high-pri pool after lookup. + // After lookup, the high-pri entry 'Y' got spilled over to the low-pri pool. + // The low-pri entry 'X' got spilled over to the bottom-pri pool. ASSERT_TRUE(Lookup("v")); - ValidateLRUList({"X", "a", "Y", "Z", "v"}, 2); + ValidateLRUList({"w", "X", "a", "Y", "Z", "v"}, 2, 2, 2); - // High-pri entries will be inserted to the head of the list after lookup. + // After lookup, the high-pri entry 'Z' got spilled over to the low-pri pool. + // The low-pri entry 'a' got spilled over to the bottom-pri pool. ASSERT_TRUE(Lookup("X")); - ValidateLRUList({"a", "Y", "Z", "v", "X"}, 2); + ValidateLRUList({"w", "a", "Y", "Z", "v", "X"}, 2, 2, 2); + + // After lookup, the low pri entry 'Z' got promoted back to high-pri pool. The + // high-pri entry 'v' got spilled over to the low-pri pool. ASSERT_TRUE(Lookup("Z")); - ValidateLRUList({"a", "Y", "v", "X", "Z"}, 2); + ValidateLRUList({"w", "a", "Y", "v", "X", "Z"}, 2, 2, 2); Erase("Y"); - ValidateLRUList({"a", "v", "X", "Z"}, 2); + ValidateLRUList({"w", "a", "v", "X", "Z"}, 2, 1, 2); Erase("X"); - ValidateLRUList({"a", "v", "Z"}, 1); + ValidateLRUList({"w", "a", "v", "Z"}, 1, 1, 2); + Insert("d", Cache::Priority::LOW); Insert("e", Cache::Priority::LOW); - ValidateLRUList({"a", "v", "d", "e", "Z"}, 1); + ValidateLRUList({"w", "a", "v", "d", "e", "Z"}, 1, 2, 3); + Insert("f", Cache::Priority::LOW); Insert("g", Cache::Priority::LOW); - ValidateLRUList({"d", "e", "f", "g", "Z"}, 1); + ValidateLRUList({"v", "d", "e", "f", "g", "Z"}, 1, 2, 3); + ASSERT_TRUE(Lookup("d")); + ValidateLRUList({"v", "e", "f", "g", "Z", "d"}, 2, 2, 2); + + // Erase some entries. + Erase("e"); + Erase("f"); + Erase("Z"); + ValidateLRUList({"v", "g", "d"}, 1, 1, 1); + + // Bottom-pri entries can take low- and high-pri pool capacity if available + Insert("o", Cache::Priority::BOTTOM); + ValidateLRUList({"v", "o", "g", "d"}, 1, 1, 2); + Insert("p", Cache::Priority::BOTTOM); + ValidateLRUList({"v", "o", "p", "g", "d"}, 1, 1, 3); + Insert("q", Cache::Priority::BOTTOM); + ValidateLRUList({"v", "o", "p", "q", "g", "d"}, 1, 1, 4); + + // High-pri entries can overflow to low-pri pool, and bottom-pri entries will + // be evicted. + Insert("x", Cache::Priority::HIGH); + ValidateLRUList({"o", "p", "q", "g", "d", "x"}, 2, 1, 3); + Insert("y", Cache::Priority::HIGH); + ValidateLRUList({"p", "q", "g", "d", "x", "y"}, 2, 2, 2); + Insert("z", Cache::Priority::HIGH); + ValidateLRUList({"q", "g", "d", "x", "y", "z"}, 2, 2, 2); + + // 'g' is bottom-pri before this lookup, it will be inserted to head of + // high-pri pool after lookup. + ASSERT_TRUE(Lookup("g")); + ValidateLRUList({"q", "d", "x", "y", "z", "g"}, 2, 2, 2); + + // High-pri entries will be inserted to head of high-pri pool after lookup. + ASSERT_TRUE(Lookup("z")); + ValidateLRUList({"q", "d", "x", "y", "g", "z"}, 2, 2, 2); + + // Bottom-pri entries will be inserted to head of high-pri pool after lookup. ASSERT_TRUE(Lookup("d")); - ValidateLRUList({"e", "f", "g", "Z", "d"}, 2); + ValidateLRUList({"q", "x", "y", "g", "z", "d"}, 2, 2, 2); + + // Bottom-pri entries will be inserted to the tail of bottom-pri list. + Insert("m", Cache::Priority::BOTTOM); + ValidateLRUList({"x", "m", "y", "g", "z", "d"}, 2, 2, 2); + + // Bottom-pri entries will be inserted to head of high-pri pool after lookup. + ASSERT_TRUE(Lookup("m")); + ValidateLRUList({"x", "y", "g", "z", "d", "m"}, 2, 2, 2); } class TestSecondaryCache : public SecondaryCache { @@ -222,8 +378,9 @@ class TestSecondaryCache : public SecondaryCache { explicit TestSecondaryCache(size_t capacity) : num_inserts_(0), num_lookups_(0), inject_failure_(false) { - cache_ = NewLRUCache(capacity, 0, false, 0.5, nullptr, - kDefaultToAdaptiveMutex, kDontChargeCacheMetadata); + cache_ = + NewLRUCache(capacity, 0, false, 0.5 /* high_pri_pool_ratio */, nullptr, + kDefaultToAdaptiveMutex, kDontChargeCacheMetadata); } ~TestSecondaryCache() override { cache_.reset(); } @@ -458,7 +615,10 @@ Cache::CacheItemHelper LRUSecondaryCacheTest::helper_fail_( LRUSecondaryCacheTest::DeletionCallback); TEST_F(LRUSecondaryCacheTest, BasicTest) { - LRUCacheOptions opts(1024, 0, false, 0.5, nullptr, kDefaultToAdaptiveMutex, + LRUCacheOptions opts(1024 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */, + 0.5 /* high_pri_pool_ratio */, + nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, kDontChargeCacheMetadata); std::shared_ptr secondary_cache = std::make_shared(2048); @@ -502,7 +662,10 @@ TEST_F(LRUSecondaryCacheTest, BasicTest) { } TEST_F(LRUSecondaryCacheTest, BasicFailTest) { - LRUCacheOptions opts(1024, 0, false, 0.5, nullptr, kDefaultToAdaptiveMutex, + LRUCacheOptions opts(1024 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */, + 0.5 /* high_pri_pool_ratio */, + nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, kDontChargeCacheMetadata); std::shared_ptr secondary_cache = std::make_shared(2048); @@ -529,7 +692,10 @@ TEST_F(LRUSecondaryCacheTest, BasicFailTest) { } TEST_F(LRUSecondaryCacheTest, SaveFailTest) { - LRUCacheOptions opts(1024, 0, false, 0.5, nullptr, kDefaultToAdaptiveMutex, + LRUCacheOptions opts(1024 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */, + 0.5 /* high_pri_pool_ratio */, + nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, kDontChargeCacheMetadata); std::shared_ptr secondary_cache = std::make_shared(2048); @@ -569,7 +735,10 @@ TEST_F(LRUSecondaryCacheTest, SaveFailTest) { } TEST_F(LRUSecondaryCacheTest, CreateFailTest) { - LRUCacheOptions opts(1024, 0, false, 0.5, nullptr, kDefaultToAdaptiveMutex, + LRUCacheOptions opts(1024 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */, + 0.5 /* high_pri_pool_ratio */, + nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, kDontChargeCacheMetadata); std::shared_ptr secondary_cache = std::make_shared(2048); @@ -610,8 +779,11 @@ TEST_F(LRUSecondaryCacheTest, CreateFailTest) { } TEST_F(LRUSecondaryCacheTest, FullCapacityTest) { - LRUCacheOptions opts(1024, 0, /*_strict_capacity_limit=*/true, 0.5, nullptr, - kDefaultToAdaptiveMutex, kDontChargeCacheMetadata); + LRUCacheOptions opts(1024 /* capacity */, 0 /* num_shard_bits */, + true /* strict_capacity_limit */, + 0.5 /* high_pri_pool_ratio */, + nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, + kDontChargeCacheMetadata); std::shared_ptr secondary_cache = std::make_shared(2048); opts.secondary_cache = secondary_cache; @@ -659,8 +831,11 @@ TEST_F(LRUSecondaryCacheTest, FullCapacityTest) { // if we try to insert block_1 to the block cache, it will always fails. Only // block_2 will be successfully inserted into the block cache. TEST_F(DBSecondaryCacheTest, TestSecondaryCacheCorrectness1) { - LRUCacheOptions opts(4 * 1024, 0, false, 0.5, nullptr, - kDefaultToAdaptiveMutex, kDontChargeCacheMetadata); + LRUCacheOptions opts(4 * 1024 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */, + 0.5 /* high_pri_pool_ratio */, + nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, + kDontChargeCacheMetadata); std::shared_ptr secondary_cache( new TestSecondaryCache(2048 * 1024)); opts.secondary_cache = secondary_cache; @@ -756,7 +931,10 @@ TEST_F(DBSecondaryCacheTest, TestSecondaryCacheCorrectness1) { // insert and cache block_1 in the block cache (this is the different place // from TestSecondaryCacheCorrectness1) TEST_F(DBSecondaryCacheTest, TestSecondaryCacheCorrectness2) { - LRUCacheOptions opts(6100, 0, false, 0.5, nullptr, kDefaultToAdaptiveMutex, + LRUCacheOptions opts(6100 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */, + 0.5 /* high_pri_pool_ratio */, + nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, kDontChargeCacheMetadata); std::shared_ptr secondary_cache( new TestSecondaryCache(2048 * 1024)); @@ -849,8 +1027,11 @@ TEST_F(DBSecondaryCacheTest, TestSecondaryCacheCorrectness2) { // cache all the blocks in the block cache and there is not secondary cache // insertion. 2 lookup is needed for the blocks. TEST_F(DBSecondaryCacheTest, NoSecondaryCacheInsertion) { - LRUCacheOptions opts(1024 * 1024, 0, false, 0.5, nullptr, - kDefaultToAdaptiveMutex, kDontChargeCacheMetadata); + LRUCacheOptions opts(1024 * 1024 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */, + 0.5 /* high_pri_pool_ratio */, + nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, + kDontChargeCacheMetadata); std::shared_ptr secondary_cache( new TestSecondaryCache(2048 * 1024)); opts.secondary_cache = secondary_cache; @@ -903,8 +1084,11 @@ TEST_F(DBSecondaryCacheTest, NoSecondaryCacheInsertion) { } TEST_F(DBSecondaryCacheTest, SecondaryCacheIntensiveTesting) { - LRUCacheOptions opts(8 * 1024, 0, false, 0.5, nullptr, - kDefaultToAdaptiveMutex, kDontChargeCacheMetadata); + LRUCacheOptions opts(8 * 1024 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */, + 0.5 /* high_pri_pool_ratio */, + nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, + kDontChargeCacheMetadata); std::shared_ptr secondary_cache( new TestSecondaryCache(2048 * 1024)); opts.secondary_cache = secondary_cache; @@ -952,8 +1136,11 @@ TEST_F(DBSecondaryCacheTest, SecondaryCacheIntensiveTesting) { // if we try to insert block_1 to the block cache, it will always fails. Only // block_2 will be successfully inserted into the block cache. TEST_F(DBSecondaryCacheTest, SecondaryCacheFailureTest) { - LRUCacheOptions opts(4 * 1024, 0, false, 0.5, nullptr, - kDefaultToAdaptiveMutex, kDontChargeCacheMetadata); + LRUCacheOptions opts(4 * 1024 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */, + 0.5 /* high_pri_pool_ratio */, + nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, + kDontChargeCacheMetadata); std::shared_ptr secondary_cache( new TestSecondaryCache(2048 * 1024)); opts.secondary_cache = secondary_cache; @@ -1044,7 +1231,10 @@ TEST_F(DBSecondaryCacheTest, SecondaryCacheFailureTest) { } TEST_F(LRUSecondaryCacheTest, BasicWaitAllTest) { - LRUCacheOptions opts(1024, 2, false, 0.5, nullptr, kDefaultToAdaptiveMutex, + LRUCacheOptions opts(1024 /* capacity */, 2 /* num_shard_bits */, + false /* strict_capacity_limit */, + 0.5 /* high_pri_pool_ratio */, + nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, kDontChargeCacheMetadata); std::shared_ptr secondary_cache = std::make_shared(32 * 1024); @@ -1099,7 +1289,10 @@ TEST_F(LRUSecondaryCacheTest, BasicWaitAllTest) { // a sync point callback in TestSecondaryCache::Lookup. We then control the // lookup result by setting the ResultMap. TEST_F(DBSecondaryCacheTest, TestSecondaryCacheMultiGet) { - LRUCacheOptions opts(1 << 20, 0, false, 0.5, nullptr, kDefaultToAdaptiveMutex, + LRUCacheOptions opts(1 << 20 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */, + 0.5 /* high_pri_pool_ratio */, + nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, kDontChargeCacheMetadata); std::shared_ptr secondary_cache( new TestSecondaryCache(2048 * 1024)); @@ -1182,15 +1375,16 @@ class LRUCacheWithStat : public LRUCache { public: LRUCacheWithStat( size_t _capacity, int _num_shard_bits, bool _strict_capacity_limit, - double _high_pri_pool_ratio, + double _high_pri_pool_ratio, double _low_pri_pool_ratio, std::shared_ptr _memory_allocator = nullptr, bool _use_adaptive_mutex = kDefaultToAdaptiveMutex, CacheMetadataChargePolicy _metadata_charge_policy = kDontChargeCacheMetadata, const std::shared_ptr& _secondary_cache = nullptr) : LRUCache(_capacity, _num_shard_bits, _strict_capacity_limit, - _high_pri_pool_ratio, _memory_allocator, _use_adaptive_mutex, - _metadata_charge_policy, _secondary_cache) { + _high_pri_pool_ratio, _low_pri_pool_ratio, _memory_allocator, + _use_adaptive_mutex, _metadata_charge_policy, + _secondary_cache) { insert_count_ = 0; lookup_count_ = 0; } @@ -1233,13 +1427,17 @@ class LRUCacheWithStat : public LRUCache { #ifndef ROCKSDB_LITE TEST_F(DBSecondaryCacheTest, LRUCacheDumpLoadBasic) { - LRUCacheOptions cache_opts(1024 * 1024, 0, false, 0.5, nullptr, + LRUCacheOptions cache_opts(1024 * 1024 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */, + 0.5 /* high_pri_pool_ratio */, + nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, kDontChargeCacheMetadata); LRUCacheWithStat* tmp_cache = new LRUCacheWithStat( cache_opts.capacity, cache_opts.num_shard_bits, cache_opts.strict_capacity_limit, cache_opts.high_pri_pool_ratio, - cache_opts.memory_allocator, cache_opts.use_adaptive_mutex, - cache_opts.metadata_charge_policy, cache_opts.secondary_cache); + cache_opts.low_pri_pool_ratio, cache_opts.memory_allocator, + cache_opts.use_adaptive_mutex, cache_opts.metadata_charge_policy, + cache_opts.secondary_cache); std::shared_ptr cache(tmp_cache); BlockBasedTableOptions table_options; table_options.block_cache = cache; @@ -1310,8 +1508,9 @@ TEST_F(DBSecondaryCacheTest, LRUCacheDumpLoadBasic) { tmp_cache = new LRUCacheWithStat( cache_opts.capacity, cache_opts.num_shard_bits, cache_opts.strict_capacity_limit, cache_opts.high_pri_pool_ratio, - cache_opts.memory_allocator, cache_opts.use_adaptive_mutex, - cache_opts.metadata_charge_policy, cache_opts.secondary_cache); + cache_opts.low_pri_pool_ratio, cache_opts.memory_allocator, + cache_opts.use_adaptive_mutex, cache_opts.metadata_charge_policy, + cache_opts.secondary_cache); std::shared_ptr cache_new(tmp_cache); table_options.block_cache = cache_new; table_options.block_size = 4 * 1024; @@ -1368,13 +1567,17 @@ TEST_F(DBSecondaryCacheTest, LRUCacheDumpLoadBasic) { } TEST_F(DBSecondaryCacheTest, LRUCacheDumpLoadWithFilter) { - LRUCacheOptions cache_opts(1024 * 1024, 0, false, 0.5, nullptr, + LRUCacheOptions cache_opts(1024 * 1024 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */, + 0.5 /* high_pri_pool_ratio */, + nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, kDontChargeCacheMetadata); LRUCacheWithStat* tmp_cache = new LRUCacheWithStat( cache_opts.capacity, cache_opts.num_shard_bits, cache_opts.strict_capacity_limit, cache_opts.high_pri_pool_ratio, - cache_opts.memory_allocator, cache_opts.use_adaptive_mutex, - cache_opts.metadata_charge_policy, cache_opts.secondary_cache); + cache_opts.low_pri_pool_ratio, cache_opts.memory_allocator, + cache_opts.use_adaptive_mutex, cache_opts.metadata_charge_policy, + cache_opts.secondary_cache); std::shared_ptr cache(tmp_cache); BlockBasedTableOptions table_options; table_options.block_cache = cache; @@ -1472,8 +1675,9 @@ TEST_F(DBSecondaryCacheTest, LRUCacheDumpLoadWithFilter) { tmp_cache = new LRUCacheWithStat( cache_opts.capacity, cache_opts.num_shard_bits, cache_opts.strict_capacity_limit, cache_opts.high_pri_pool_ratio, - cache_opts.memory_allocator, cache_opts.use_adaptive_mutex, - cache_opts.metadata_charge_policy, cache_opts.secondary_cache); + cache_opts.low_pri_pool_ratio, cache_opts.memory_allocator, + cache_opts.use_adaptive_mutex, cache_opts.metadata_charge_policy, + cache_opts.secondary_cache); std::shared_ptr cache_new(tmp_cache); table_options.block_cache = cache_new; table_options.block_size = 4 * 1024; @@ -1539,8 +1743,11 @@ TEST_F(DBSecondaryCacheTest, LRUCacheDumpLoadWithFilter) { // Test the option not to use the secondary cache in a certain DB. TEST_F(DBSecondaryCacheTest, TestSecondaryCacheOptionBasic) { - LRUCacheOptions opts(4 * 1024, 0, false, 0.5, nullptr, - kDefaultToAdaptiveMutex, kDontChargeCacheMetadata); + LRUCacheOptions opts(4 * 1024 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */, + 0.5 /* high_pri_pool_ratio */, + nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, + kDontChargeCacheMetadata); std::shared_ptr secondary_cache( new TestSecondaryCache(2048 * 1024)); opts.secondary_cache = secondary_cache; @@ -1634,8 +1841,11 @@ TEST_F(DBSecondaryCacheTest, TestSecondaryCacheOptionBasic) { // with new options, which set the lowest_used_cache_tier to // kNonVolatileBlockTier. So secondary cache will be used. TEST_F(DBSecondaryCacheTest, TestSecondaryCacheOptionChange) { - LRUCacheOptions opts(4 * 1024, 0, false, 0.5, nullptr, - kDefaultToAdaptiveMutex, kDontChargeCacheMetadata); + LRUCacheOptions opts(4 * 1024 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */, + 0.5 /* high_pri_pool_ratio */, + nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, + kDontChargeCacheMetadata); std::shared_ptr secondary_cache( new TestSecondaryCache(2048 * 1024)); opts.secondary_cache = secondary_cache; @@ -1729,8 +1939,11 @@ TEST_F(DBSecondaryCacheTest, TestSecondaryCacheOptionChange) { // Two DB test. We create 2 DBs sharing the same block cache and secondary // cache. We diable the secondary cache option for DB2. TEST_F(DBSecondaryCacheTest, TestSecondaryCacheOptionTwoDB) { - LRUCacheOptions opts(4 * 1024, 0, false, 0.5, nullptr, - kDefaultToAdaptiveMutex, kDontChargeCacheMetadata); + LRUCacheOptions opts(4 * 1024 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */, + 0.5 /* high_pri_pool_ratio */, + nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, + kDontChargeCacheMetadata); std::shared_ptr secondary_cache( new TestSecondaryCache(2048 * 1024)); opts.secondary_cache = secondary_cache; diff --git a/db/compact_files_test.cc b/db/compact_files_test.cc index da325a8a2de..edd48c37db1 100644 --- a/db/compact_files_test.cc +++ b/db/compact_files_test.cc @@ -421,9 +421,7 @@ TEST_F(CompactFilesTest, CompactionFilterWithGetSv) { return true; } - void SetDB(DB* db) { - db_ = db; - } + void SetDB(DB* db) { db_ = db; } const char* Name() const override { return "FilterWithGet"; } @@ -431,7 +429,6 @@ TEST_F(CompactFilesTest, CompactionFilterWithGetSv) { DB* db_; }; - std::shared_ptr cf(new FilterWithGet()); Options options; @@ -458,7 +455,6 @@ TEST_F(CompactFilesTest, CompactionFilterWithGetSv) { db->CompactFiles(ROCKSDB_NAMESPACE::CompactionOptions(), {fname}, 0)); } - delete db; } @@ -473,10 +469,9 @@ TEST_F(CompactFilesTest, SentinelCompressionType) { } // Check that passing `CompressionType::kDisableCompressionOption` to // `CompactFiles` causes it to use the column family compression options. - for (auto compaction_style : - {CompactionStyle::kCompactionStyleLevel, - CompactionStyle::kCompactionStyleUniversal, - CompactionStyle::kCompactionStyleNone}) { + for (auto compaction_style : {CompactionStyle::kCompactionStyleLevel, + CompactionStyle::kCompactionStyleUniversal, + CompactionStyle::kCompactionStyleNone}) { DestroyDB(db_name_, Options()); Options options; options.compaction_style = compaction_style; @@ -607,7 +602,8 @@ TEST_F(CompactFilesTest, IsWriteStalled) { for (int j = 0; j < 100; ++j) { char key[16]; bzero(key, 16); - sprintf(key, "foo%.2d", j); + snprintf(key, sizeof(key), "foo%.2d", j); + ASSERT_OK(wb.Put(handles[0], key, "bar")); } diff --git a/db/db_block_cache_test.cc b/db/db_block_cache_test.cc index 71cf49f1542..992e03ee21a 100644 --- a/db/db_block_cache_test.cc +++ b/db/db_block_cache_test.cc @@ -808,8 +808,8 @@ class MockCache : public LRUCache { MockCache() : LRUCache((size_t)1 << 25 /*capacity*/, 0 /*num_shard_bits*/, - false /*strict_capacity_limit*/, 0.0 /*high_pri_pool_ratio*/) { - } + false /*strict_capacity_limit*/, 0.0 /*high_pri_pool_ratio*/, + 0.0 /*low_pri_pool_ratio*/) {} using ShardedCache::Insert; diff --git a/db/db_compaction_test.cc b/db/db_compaction_test.cc index 33d047d97d0..5f28147aa08 100644 --- a/db/db_compaction_test.cc +++ b/db/db_compaction_test.cc @@ -93,7 +93,7 @@ class SplitAllPartitioner : public SstPartitioner { return PartitionerResult::kRequired; } - bool CanDoTrivialMove(const Slice&, const Slice&) { return true; } + bool CanDoTrivialMove(const Slice&, const Slice&) override { return true; } }; class SplitAllPatitionerFactory : public SstPartitionerFactory { @@ -140,9 +140,10 @@ class FlushedFileCollector : public EventListener { }; class CompactionStatsCollector : public EventListener { -public: + public: CompactionStatsCollector() - : compaction_completed_(static_cast(CompactionReason::kNumOfReasons)) { + : compaction_completed_( + static_cast(CompactionReason::kNumOfReasons)) { for (auto& v : compaction_completed_) { v.store(0); } @@ -176,7 +177,7 @@ class CompactionStatsCollector : public EventListener { return compaction_completed_.at(k).load(); } -private: + private: std::vector> compaction_completed_; }; @@ -215,9 +216,8 @@ Options DeletionTriggerOptions(Options options) { return options; } -bool HaveOverlappingKeyRanges( - const Comparator* c, - const SstFileMetaData& a, const SstFileMetaData& b) { +bool HaveOverlappingKeyRanges(const Comparator* c, const SstFileMetaData& a, + const SstFileMetaData& b) { if (c->CompareWithoutTimestamp(a.smallestkey, b.smallestkey) >= 0) { if (c->CompareWithoutTimestamp(a.smallestkey, b.largestkey) <= 0) { // b.smallestkey <= a.smallestkey <= b.largestkey @@ -242,18 +242,15 @@ bool HaveOverlappingKeyRanges( // Identifies all files between level "min_level" and "max_level" // which has overlapping key range with "input_file_meta". void GetOverlappingFileNumbersForLevelCompaction( - const ColumnFamilyMetaData& cf_meta, - const Comparator* comparator, - int min_level, int max_level, - const SstFileMetaData* input_file_meta, + const ColumnFamilyMetaData& cf_meta, const Comparator* comparator, + int min_level, int max_level, const SstFileMetaData* input_file_meta, std::set* overlapping_file_names) { std::set overlapping_files; overlapping_files.insert(input_file_meta); for (int m = min_level; m <= max_level; ++m) { for (auto& file : cf_meta.levels[m].files) { for (auto* included_file : overlapping_files) { - if (HaveOverlappingKeyRanges( - comparator, *included_file, file)) { + if (HaveOverlappingKeyRanges(comparator, *included_file, file)) { overlapping_files.insert(&file); overlapping_file_names->insert(file.name); break; @@ -284,7 +281,7 @@ void VerifyCompactionResult( * 2) stat.counts[i] == collector.NumberOfCompactions(i) */ void VerifyCompactionStats(ColumnFamilyData& cfd, - const CompactionStatsCollector& collector) { + const CompactionStatsCollector& collector) { #ifndef NDEBUG InternalStats* internal_stats_ptr = cfd.internal_stats(); ASSERT_NE(internal_stats_ptr, nullptr); @@ -305,17 +302,15 @@ void VerifyCompactionStats(ColumnFamilyData& cfd, // Verify InternalStats bookkeeping matches that of CompactionStatsCollector, // assuming that all compactions complete. for (int i = 0; i < num_of_reasons; i++) { - ASSERT_EQ(collector.NumberOfCompactions(static_cast(i)), counts[i]); + ASSERT_EQ(collector.NumberOfCompactions(static_cast(i)), + counts[i]); } #endif /* NDEBUG */ } -const SstFileMetaData* PickFileRandomly( - const ColumnFamilyMetaData& cf_meta, - Random* rand, - int* level = nullptr) { - auto file_id = rand->Uniform(static_cast( - cf_meta.file_count)) + 1; +const SstFileMetaData* PickFileRandomly(const ColumnFamilyMetaData& cf_meta, + Random* rand, int* level = nullptr) { + auto file_id = rand->Uniform(static_cast(cf_meta.file_count)) + 1; for (auto& level_meta : cf_meta.levels) { if (file_id <= level_meta.files.size()) { if (level != nullptr) { @@ -742,7 +737,6 @@ TEST_F(DBCompactionTest, DisableStatsUpdateReopen) { } } - TEST_P(DBCompactionTestWithParam, CompactionTrigger) { const int kNumKeysPerFile = 100; @@ -885,7 +879,7 @@ TEST_F(DBCompactionTest, BGCompactionsAllowed) { TEST_P(DBCompactionTestWithParam, CompactionsGenerateMultipleFiles) { Options options = CurrentOptions(); - options.write_buffer_size = 100000000; // Large write buffer + options.write_buffer_size = 100000000; // Large write buffer options.max_subcompactions = max_subcompactions_; CreateAndReopenWithCF({"pikachu"}, options); @@ -1146,7 +1140,7 @@ TEST_F(DBCompactionTest, ZeroSeqIdCompaction) { compact_opt.compression = kNoCompression; compact_opt.output_file_size_limit = 4096; const size_t key_len = - static_cast(compact_opt.output_file_size_limit) / 5; + static_cast(compact_opt.output_file_size_limit) / 5; DestroyAndReopen(options); @@ -1320,14 +1314,8 @@ TEST_P(DBCompactionTestWithParam, TrivialMoveNonOverlappingFiles) { DestroyAndReopen(options); // non overlapping ranges std::vector> ranges = { - {100, 199}, - {300, 399}, - {0, 99}, - {200, 299}, - {600, 699}, - {400, 499}, - {500, 550}, - {551, 599}, + {100, 199}, {300, 399}, {0, 99}, {200, 299}, + {600, 699}, {400, 499}, {500, 550}, {551, 599}, }; int32_t value_size = 10 * 1024; // 10 KB @@ -1370,14 +1358,15 @@ TEST_P(DBCompactionTestWithParam, TrivialMoveNonOverlappingFiles) { DestroyAndReopen(options); // Same ranges as above but overlapping ranges = { - {100, 199}, - {300, 399}, - {0, 99}, - {200, 299}, - {600, 699}, - {400, 499}, - {500, 560}, // this range overlap with the next one - {551, 599}, + {100, 199}, + {300, 399}, + {0, 99}, + {200, 299}, + {600, 699}, + {400, 499}, + {500, 560}, // this range overlap with the next + // one + {551, 599}, }; for (size_t i = 0; i < ranges.size(); i++) { for (int32_t j = ranges[i].first; j <= ranges[i].second; j++) { @@ -1912,7 +1901,7 @@ TEST_F(DBCompactionTest, DeleteFilesInRanges) { ASSERT_EQ("0,0,10", FilesPerLevel(0)); // file [0 => 100), [200 => 300), ... [800, 900) - for (auto i = 0; i < 10; i+=2) { + for (auto i = 0; i < 10; i += 2) { for (auto j = 0; j < 100; j++) { auto k = i * 100 + j; ASSERT_OK(Put(Key(k), values[k])); @@ -2354,14 +2343,14 @@ TEST_P(DBCompactionTestWithParam, LevelCompactionCFPathUse) { cf_opt1.cf_paths.emplace_back(dbname_ + "cf1_2", 4 * 1024 * 1024); cf_opt1.cf_paths.emplace_back(dbname_ + "cf1_3", 1024 * 1024 * 1024); option_vector.emplace_back(DBOptions(options), cf_opt1); - CreateColumnFamilies({"one"},option_vector[1]); + CreateColumnFamilies({"one"}, option_vector[1]); // Configure CF2 specific paths. cf_opt2.cf_paths.emplace_back(dbname_ + "cf2", 500 * 1024); cf_opt2.cf_paths.emplace_back(dbname_ + "cf2_2", 4 * 1024 * 1024); cf_opt2.cf_paths.emplace_back(dbname_ + "cf2_3", 1024 * 1024 * 1024); option_vector.emplace_back(DBOptions(options), cf_opt2); - CreateColumnFamilies({"two"},option_vector[2]); + CreateColumnFamilies({"two"}, option_vector[2]); ReopenWithColumnFamilies({"default", "one", "two"}, option_vector); @@ -2709,7 +2698,6 @@ TEST_P(DBCompactionTestWithParam, ManualCompaction) { } } - TEST_P(DBCompactionTestWithParam, ManualLevelCompactionOutputPathId) { Options options = CurrentOptions(); options.db_paths.emplace_back(dbname_ + "_2", 2 * 10485760); @@ -2846,14 +2834,13 @@ TEST_P(DBCompactionTestWithParam, DISABLED_CompactFilesOnLevelCompaction) { auto file_meta = PickFileRandomly(cf_meta, &rnd, &level); compaction_input_file_names.push_back(file_meta->name); GetOverlappingFileNumbersForLevelCompaction( - cf_meta, options.comparator, level, output_level, - file_meta, &overlapping_file_names); + cf_meta, options.comparator, level, output_level, file_meta, + &overlapping_file_names); } - ASSERT_OK(dbfull()->CompactFiles( - CompactionOptions(), handles_[1], - compaction_input_file_names, - output_level)); + ASSERT_OK(dbfull()->CompactFiles(CompactionOptions(), handles_[1], + compaction_input_file_names, + output_level)); // Make sure all overlapping files do not exist after compaction dbfull()->GetColumnFamilyMetaData(handles_[1], &cf_meta); @@ -2876,8 +2863,7 @@ TEST_P(DBCompactionTestWithParam, PartialCompactionFailure) { options.write_buffer_size = kKeysPerBuffer * kKvSize; options.max_write_buffer_number = 2; options.target_file_size_base = - options.write_buffer_size * - (options.max_write_buffer_number - 1); + options.write_buffer_size * (options.max_write_buffer_number - 1); options.level0_file_num_compaction_trigger = kNumL1Files; options.max_bytes_for_level_base = options.level0_file_num_compaction_trigger * @@ -2897,10 +2883,9 @@ TEST_P(DBCompactionTestWithParam, PartialCompactionFailure) { DestroyAndReopen(options); - const int kNumInsertedKeys = - options.level0_file_num_compaction_trigger * - (options.max_write_buffer_number - 1) * - kKeysPerBuffer; + const int kNumInsertedKeys = options.level0_file_num_compaction_trigger * + (options.max_write_buffer_number - 1) * + kKeysPerBuffer; Random rnd(301); std::vector keys; @@ -3598,9 +3583,8 @@ TEST_F(DBCompactionTest, CompactFilesPendingL0Bug) { ASSERT_EQ(kNumL0Files, cf_meta.levels[0].files.size()); std::vector input_filenames; input_filenames.push_back(cf_meta.levels[0].files.front().name); - ASSERT_OK(dbfull() - ->CompactFiles(CompactionOptions(), input_filenames, - 0 /* output_level */)); + ASSERT_OK(dbfull()->CompactFiles(CompactionOptions(), input_filenames, + 0 /* output_level */)); TEST_SYNC_POINT("DBCompactionTest::CompactFilesPendingL0Bug:ManualCompacted"); ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing(); } @@ -4247,9 +4231,9 @@ TEST_F(DBCompactionTest, LevelPeriodicAndTtlCompaction) { const int kValueSize = 100; Options options = CurrentOptions(); - options.ttl = 10 * 60 * 60; // 10 hours + options.ttl = 10 * 60 * 60; // 10 hours options.periodic_compaction_seconds = 48 * 60 * 60; // 2 days - options.max_open_files = -1; // needed for both periodic and ttl compactions + options.max_open_files = -1; // needed for both periodic and ttl compactions env_->SetMockSleep(); options.env = env_; @@ -4664,7 +4648,7 @@ TEST_F(DBCompactionTest, CompactRangeSkipFlushAfterDelay) { {"DBImpl::FlushMemTable:StallWaitDone", "CompactionJob::Run():End"}}); ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing(); - //used for the delayable flushes + // used for the delayable flushes FlushOptions flush_opts; flush_opts.allow_write_stall = true; for (int i = 0; i < kNumL0FilesLimit - 1; ++i) { @@ -4683,7 +4667,8 @@ TEST_F(DBCompactionTest, CompactRangeSkipFlushAfterDelay) { ASSERT_OK(Put(ToString(0), rnd.RandomString(1024))); ASSERT_OK(dbfull()->Flush(flush_opts)); ASSERT_OK(Put(ToString(0), rnd.RandomString(1024))); - TEST_SYNC_POINT("DBCompactionTest::CompactRangeSkipFlushAfterDelay:PostFlush"); + TEST_SYNC_POINT( + "DBCompactionTest::CompactRangeSkipFlushAfterDelay:PostFlush"); manual_compaction_thread.join(); // If CompactRange's flush was skipped, the final Put above will still be @@ -4884,10 +4869,10 @@ TEST_F(DBCompactionTest, CompactionLimiter) { } std::shared_ptr unique_limiter( - NewConcurrentTaskLimiter("unique_limiter", -1)); + NewConcurrentTaskLimiter("unique_limiter", -1)); - const char* cf_names[] = {"default", "0", "1", "2", "3", "4", "5", - "6", "7", "8", "9", "a", "b", "c", "d", "e", "f" }; + const char* cf_names[] = {"default", "0", "1", "2", "3", "4", "5", "6", "7", + "8", "9", "a", "b", "c", "d", "e", "f"}; const unsigned int cf_count = sizeof cf_names / sizeof cf_names[0]; std::unordered_map cf_to_limiter; @@ -4899,10 +4884,10 @@ TEST_F(DBCompactionTest, CompactionLimiter) { options.level0_file_num_compaction_trigger = 4; options.level0_slowdown_writes_trigger = 64; options.level0_stop_writes_trigger = 64; - options.max_background_jobs = kMaxBackgroundThreads; // Enough threads + options.max_background_jobs = kMaxBackgroundThreads; // Enough threads options.memtable_factory.reset( test::NewSpecialSkipListFactory(kNumKeysPerFile)); - options.max_write_buffer_number = 10; // Enough memtables + options.max_write_buffer_number = 10; // Enough memtables DestroyAndReopen(options); std::vector option_vector; @@ -4930,9 +4915,8 @@ TEST_F(DBCompactionTest, CompactionLimiter) { CreateColumnFamilies({cf_names[cf]}, option_vector[cf]); } - ReopenWithColumnFamilies(std::vector(cf_names, - cf_names + cf_count), - option_vector); + ReopenWithColumnFamilies( + std::vector(cf_names, cf_names + cf_count), option_vector); port::Mutex mutex; @@ -4994,7 +4978,7 @@ TEST_F(DBCompactionTest, CompactionLimiter) { // Enough L0 files to trigger compaction for (unsigned int cf = 0; cf < cf_count; cf++) { ASSERT_EQ(NumTableFilesAtLevel(0, cf), - options.level0_file_num_compaction_trigger); + options.level0_file_num_compaction_trigger); } // Create more files for one column family, which triggers speed up @@ -5037,7 +5021,7 @@ TEST_F(DBCompactionTest, CompactionLimiter) { // flush one more file to cf 1 for (int i = 0; i < kNumKeysPerFile; i++) { - ASSERT_OK(Put(cf_test, Key(keyIndex++), "")); + ASSERT_OK(Put(cf_test, Key(keyIndex++), "")); } // put extra key to trigger flush ASSERT_OK(Put(cf_test, "", "")); @@ -5072,9 +5056,7 @@ TEST_P(DBCompactionDirectIOTest, DirectIO) { }); if (options.use_direct_io_for_flush_and_compaction) { SyncPoint::GetInstance()->SetCallBack( - "SanitizeOptions:direct_io", [&](void* /*arg*/) { - readahead = true; - }); + "SanitizeOptions:direct_io", [&](void* /*arg*/) { readahead = true; }); } SyncPoint::GetInstance()->EnableProcessing(); CreateAndReopenWithCF({"pikachu"}, options); @@ -7403,8 +7385,8 @@ int main(int argc, char** argv) { ::testing::InitGoogleTest(&argc, argv); return RUN_ALL_TESTS(); #else - (void) argc; - (void) argv; + (void)argc; + (void)argv; return 0; #endif } diff --git a/db/db_test.cc b/db/db_test.cc index f2373ad1459..bbf63cb43ed 100644 --- a/db/db_test.cc +++ b/db/db_test.cc @@ -3298,10 +3298,8 @@ static bool CompareIterators(int step, DB* model, DB* db, options.snapshot = db_snap; Iterator* dbiter = db->NewIterator(options); bool ok = true; - int count = 0; for (miter->SeekToFirst(), dbiter->SeekToFirst(); ok && miter->Valid() && dbiter->Valid(); miter->Next(), dbiter->Next()) { - count++; if (miter->key().compare(dbiter->key()) != 0) { fprintf(stderr, "step %d: Key mismatch: '%s' vs. '%s'\n", step, EscapeString(miter->key()).c_str(), diff --git a/db/db_test2.cc b/db/db_test2.cc index 9dc0b4282c2..9654fae4c93 100644 --- a/db/db_test2.cc +++ b/db/db_test2.cc @@ -757,8 +757,12 @@ TEST_F(DBTest2, SharedWriteBufferLimitAcrossDB_RankByAge) { TEST_F(DBTest2, TestWriteBufferNoLimitWithCache) { Options options = CurrentOptions(); options.arena_block_size = 4096; - std::shared_ptr cache = - NewLRUCache(LRUCacheOptions(10000000, 1, false, 0.0)); + std::shared_ptr cache = NewLRUCache(LRUCacheOptions( + 10000000 /* capacity */, 1 /* num_shard_bits */, + false /* strict_capacity_limit */, 0.0 /* high_pri_pool_ratio */, + nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, + kDontChargeCacheMetadata)); + options.write_buffer_size = 50000; // this is never hit // Use a write buffer total size so that the soft limit is about // 105000. diff --git a/file/prefetch_test.cc b/file/prefetch_test.cc index 28f1a1b942c..33ab213c2b3 100644 --- a/file/prefetch_test.cc +++ b/file/prefetch_test.cc @@ -169,6 +169,7 @@ TEST_P(PrefetchTest, Basic) { for (iter->SeekToFirst(); iter->Valid(); iter->Next()) { num_keys++; } + ASSERT_EQ(num_keys, kNumKeys); } // Make sure prefetch is called only if file system support prefetch. @@ -767,6 +768,7 @@ TEST_P(PrefetchTest1, DBIterLevelReadAhead) { for (iter->SeekToFirst(); iter->Valid(); iter->Next()) { num_keys++; } + ASSERT_EQ(num_keys, 5 * kNumKeys); ASSERT_GT(buff_prefetch_count, 0); buff_prefetch_count = 0; diff --git a/include/rocksdb/cache.h b/include/rocksdb/cache.h index 772977df334..a3cdda4f1a0 100644 --- a/include/rocksdb/cache.h +++ b/include/rocksdb/cache.h @@ -72,6 +72,17 @@ struct LRUCacheOptions { // BlockBasedTableOptions::cache_index_and_filter_blocks_with_high_priority. double high_pri_pool_ratio = 0.5; + // Percentage of cache reserved for low priority entries. + // If greater than zero, the LRU list will be split into a high-pri list, a + // low-pri list and a bottom-pri list. High-pri entries will be inserted to + // the tail of high-pri list, while low-pri entries will be first inserted to + // the low-pri list (the midpoint) and bottom-pri entries will be first + // inserted to the bottom-pri list. + // + // + // See also high_pri_pool_ratio. + double low_pri_pool_ratio = 0.0; + // If non-nullptr will use this allocator instead of system allocator when // allocating memory for cache blocks. Call this method before you start using // the cache! @@ -99,11 +110,13 @@ struct LRUCacheOptions { std::shared_ptr _memory_allocator = nullptr, bool _use_adaptive_mutex = kDefaultToAdaptiveMutex, CacheMetadataChargePolicy _metadata_charge_policy = - kDefaultCacheMetadataChargePolicy) + kDefaultCacheMetadataChargePolicy, + double _low_pri_pool_ratio = 0.0) : capacity(_capacity), num_shard_bits(_num_shard_bits), strict_capacity_limit(_strict_capacity_limit), high_pri_pool_ratio(_high_pri_pool_ratio), + low_pri_pool_ratio(_low_pri_pool_ratio), memory_allocator(std::move(_memory_allocator)), use_adaptive_mutex(_use_adaptive_mutex), metadata_charge_policy(_metadata_charge_policy) {} @@ -123,7 +136,8 @@ extern std::shared_ptr NewLRUCache( std::shared_ptr memory_allocator = nullptr, bool use_adaptive_mutex = kDefaultToAdaptiveMutex, CacheMetadataChargePolicy metadata_charge_policy = - kDefaultCacheMetadataChargePolicy); + kDefaultCacheMetadataChargePolicy, + double low_pri_pool_ratio = 0.0); extern std::shared_ptr NewLRUCache(const LRUCacheOptions& cache_opts); @@ -145,7 +159,17 @@ class Cache { public: // Depending on implementation, cache entries with high priority could be less // likely to get evicted than low priority entries. - enum class Priority { HIGH, LOW }; + // + // The BOTTOM priority is mainly used for blob caching. Blobs are typically + // lower-value targets for caching than data blocks, since 1) with BlobDB, + // data blocks containing blob references conceptually form an index structure + // which has to be consulted before we can read the blob value, and 2) cached + // blobs represent only a single key-value, while cached data blocks generally + // contain multiple KVs. Since we would like to make it possible to use the + // same backing cache for the block cache and the blob cache, it would make + // sense to add a new, bottom cache priority level for blobs so data blocks + // are prioritized over them. + enum class Priority { HIGH, LOW, BOTTOM }; // A set of callbacks to allow objects in the primary block cache to be // be persisted in a secondary cache. The purpose of the secondary cache @@ -348,8 +372,8 @@ class Cache { // memory - call this only if you're shutting down the process. // Any attempts of using cache after this call will fail terribly. // Always delete the DB object before calling this method! - virtual void DisownData(){ - // default implementation is noop + virtual void DisownData() { + // default implementation is noop } struct ApplyToAllEntriesOptions { diff --git a/java/rocksjni/lru_cache.cc b/java/rocksjni/lru_cache.cc index cfdcb525bb7..393ec502956 100644 --- a/java/rocksjni/lru_cache.cc +++ b/java/rocksjni/lru_cache.cc @@ -20,12 +20,16 @@ jlong Java_org_rocksdb_LRUCache_newLRUCache(JNIEnv* /*env*/, jclass /*jcls*/, jlong jcapacity, jint jnum_shard_bits, jboolean jstrict_capacity_limit, - jdouble jhigh_pri_pool_ratio) { + jdouble jhigh_pri_pool_ratio, + jdouble jlow_pri_pool_ratio) { auto* sptr_lru_cache = new std::shared_ptr( ROCKSDB_NAMESPACE::NewLRUCache( static_cast(jcapacity), static_cast(jnum_shard_bits), static_cast(jstrict_capacity_limit), - static_cast(jhigh_pri_pool_ratio))); + static_cast(jhigh_pri_pool_ratio), + nullptr /* memory_allocator */, rocksdb::kDefaultToAdaptiveMutex, + rocksdb::kDontChargeCacheMetadata, + static_cast(jlow_pri_pool_ratio))); return reinterpret_cast(sptr_lru_cache); } diff --git a/java/src/main/java/org/rocksdb/LRUCache.java b/java/src/main/java/org/rocksdb/LRUCache.java index 5e5bdeea277..db90b17c5b2 100644 --- a/java/src/main/java/org/rocksdb/LRUCache.java +++ b/java/src/main/java/org/rocksdb/LRUCache.java @@ -16,7 +16,7 @@ public class LRUCache extends Cache { * @param capacity The fixed size capacity of the cache */ public LRUCache(final long capacity) { - this(capacity, -1, false, 0.0); + this(capacity, -1, false, 0.0, 0.0); } /** @@ -31,7 +31,7 @@ public LRUCache(final long capacity) { * by hash of the key */ public LRUCache(final long capacity, final int numShardBits) { - super(newLRUCache(capacity, numShardBits, false,0.0)); + super(newLRUCache(capacity, numShardBits, false, 0.0, 0.0)); } /** @@ -49,7 +49,7 @@ public LRUCache(final long capacity, final int numShardBits) { */ public LRUCache(final long capacity, final int numShardBits, final boolean strictCapacityLimit) { - super(newLRUCache(capacity, numShardBits, strictCapacityLimit,0.0)); + super(newLRUCache(capacity, numShardBits, strictCapacityLimit, 0.0, 0.0)); } /** @@ -69,14 +69,38 @@ public LRUCache(final long capacity, final int numShardBits, * @param highPriPoolRatio percentage of the cache reserves for high priority * entries */ - public LRUCache(final long capacity, final int numShardBits, - final boolean strictCapacityLimit, final double highPriPoolRatio) { - super(newLRUCache(capacity, numShardBits, strictCapacityLimit, - highPriPoolRatio)); + public LRUCache(final long capacity, final int numShardBits, final boolean strictCapacityLimit, + final double highPriPoolRatio) { + super(newLRUCache(capacity, numShardBits, strictCapacityLimit, highPriPoolRatio, 0.0)); + } + + /** + * Create a new cache with a fixed size capacity. The cache is sharded + * to 2^numShardBits shards, by hash of the key. The total capacity + * is divided and evenly assigned to each shard. If strictCapacityLimit + * is set, insert to the cache will fail when cache is full. User can also + * set percentage of the cache reserves for high priority entries and low + * priority entries via highPriPoolRatio and lowPriPoolRatio. + * numShardBits = -1 means it is automatically determined: every shard + * will be at least 512KB and number of shard bits will not exceed 6. + * + * @param capacity The fixed size capacity of the cache + * @param numShardBits The cache is sharded to 2^numShardBits shards, + * by hash of the key + * @param strictCapacityLimit insert to the cache will fail when cache is full + * @param highPriPoolRatio percentage of the cache reserves for high priority + * entries + * @param lowPriPoolRatio percentage of the cache reserves for low priority + * entries + */ + public LRUCache(final long capacity, final int numShardBits, final boolean strictCapacityLimit, + final double highPriPoolRatio, final double lowPriPoolRatio) { + super(newLRUCache( + capacity, numShardBits, strictCapacityLimit, highPriPoolRatio, lowPriPoolRatio)); } - private native static long newLRUCache(final long capacity, - final int numShardBits, final boolean strictCapacityLimit, - final double highPriPoolRatio); + private native static long newLRUCache(final long capacity, final int numShardBits, + final boolean strictCapacityLimit, final double highPriPoolRatio, + final double lowPriPoolRatio); @Override protected final native void disposeInternal(final long handle); } diff --git a/java/src/test/java/org/rocksdb/LRUCacheTest.java b/java/src/test/java/org/rocksdb/LRUCacheTest.java index 275cb560a13..4d194e71218 100644 --- a/java/src/test/java/org/rocksdb/LRUCacheTest.java +++ b/java/src/test/java/org/rocksdb/LRUCacheTest.java @@ -20,9 +20,10 @@ public void newLRUCache() { final long capacity = 80000000; final int numShardBits = 16; final boolean strictCapacityLimit = true; - final double highPriPoolRatio = 0.05; - try(final Cache lruCache = new LRUCache(capacity, - numShardBits, strictCapacityLimit, highPriPoolRatio)) { + final double highPriPoolRatio = 0.5; + final double lowPriPoolRatio = 0.5; + try (final Cache lruCache = new LRUCache( + capacity, numShardBits, strictCapacityLimit, highPriPoolRatio, lowPriPoolRatio)) { //no op assertThat(lruCache.getUsage()).isGreaterThanOrEqualTo(0); assertThat(lruCache.getPinnedUsage()).isGreaterThanOrEqualTo(0); diff --git a/memory/memory_allocator_test.cc b/memory/memory_allocator_test.cc index 90aed448b61..658cf4133ef 100644 --- a/memory/memory_allocator_test.cc +++ b/memory/memory_allocator_test.cc @@ -87,7 +87,7 @@ TEST_P(MemoryAllocatorTest, DatabaseBlockCache) { options.create_if_missing = true; BlockBasedTableOptions table_options; - auto cache = NewLRUCache(1024 * 1024, 6, false, false, allocator_); + auto cache = NewLRUCache(1024 * 1024, 6, false, 0.0, allocator_); table_options.block_cache = cache; options.table_factory.reset(NewBlockBasedTableFactory(table_options)); DB* db = nullptr; diff --git a/memtable/inlineskiplist_test.cc b/memtable/inlineskiplist_test.cc index 8fb0cb1d9da..32e4c6c7d80 100644 --- a/memtable/inlineskiplist_test.cc +++ b/memtable/inlineskiplist_test.cc @@ -609,11 +609,9 @@ class TestStateImpl : public TestState { static void ConcurrentReader(void* arg) { TestState* state = reinterpret_cast(arg); Random rnd(state->seed_); - int64_t reads = 0; state->Change(TestState::RUNNING); while (!state->quit_flag_.load(std::memory_order_acquire)) { state->ReadStep(&rnd); - ++reads; } state->Change(TestState::DONE); } diff --git a/memtable/skiplist_test.cc b/memtable/skiplist_test.cc index d35bc856d8b..5f43a348c37 100644 --- a/memtable/skiplist_test.cc +++ b/memtable/skiplist_test.cc @@ -169,7 +169,7 @@ class ConcurrentTest { static uint64_t hash(Key key) { return key & 0xff; } static uint64_t HashNumbers(uint64_t k, uint64_t g) { - uint64_t data[2] = { k, g }; + uint64_t data[2] = {k, g}; return Hash(reinterpret_cast(data), sizeof(data), 0); } @@ -311,11 +311,7 @@ class TestState { int seed_; std::atomic quit_flag_; - enum ReaderState { - STARTING, - RUNNING, - DONE - }; + enum ReaderState { STARTING, RUNNING, DONE }; explicit TestState(int s) : seed_(s), quit_flag_(false), state_(STARTING), state_cv_(&mu_) {} @@ -344,11 +340,9 @@ class TestState { static void ConcurrentReader(void* arg) { TestState* state = reinterpret_cast(arg); Random rnd(state->seed_); - int64_t reads = 0; state->Change(TestState::RUNNING); while (!state->quit_flag_.load(std::memory_order_acquire)) { state->t_.ReadStep(&rnd); - ++reads; } state->Change(TestState::DONE); } diff --git a/table/block_based/block_based_table_factory.cc b/table/block_based/block_based_table_factory.cc index 5b1bd2e6861..7881db95a33 100644 --- a/table/block_based/block_based_table_factory.cc +++ b/table/block_based/block_based_table_factory.cc @@ -464,6 +464,7 @@ void BlockBasedTableFactory::InitializeOptions() { // It makes little sense to pay overhead for mid-point insertion while the // block size is only 8MB. co.high_pri_pool_ratio = 0.0; + co.low_pri_pool_ratio = 0.0; table_options_.block_cache = NewLRUCache(co); } if (table_options_.block_size_deviation < 0 || diff --git a/table/block_based/block_based_table_reader_test.cc b/table/block_based/block_based_table_reader_test.cc index 305986eca5e..2936c885519 100644 --- a/table/block_based/block_based_table_reader_test.cc +++ b/table/block_based/block_based_table_reader_test.cc @@ -163,7 +163,7 @@ TEST_P(BlockBasedTableReaderTest, MultiGet) { // Internal key is constructed directly from this key, // and internal key size is required to be >= 8 bytes, // so use %08u as the format string. - sprintf(k, "%08u", key); + snprintf(k, sizeof(k), "%08u", key); std::string v; if (block % 2) { v = rnd.HumanReadableString(256); @@ -270,7 +270,7 @@ TEST_P(BlockBasedTableReaderTestVerifyChecksum, ChecksumMismatch) { // Internal key is constructed directly from this key, // and internal key size is required to be >= 8 bytes, // so use %08u as the format string. - sprintf(k, "%08u", key); + snprintf(k, sizeof(k), "%08u", key); std::string v = rnd.RandomString(256); kv[std::string(k)] = v; key++; diff --git a/tools/db_bench_tool.cc b/tools/db_bench_tool.cc index d747a8236f3..1056f7f40a5 100644 --- a/tools/db_bench_tool.cc +++ b/tools/db_bench_tool.cc @@ -277,20 +277,24 @@ DEFINE_string(column_family_distribution, "", "and `num_hot_column_families=0`, a valid list could be " "\"10,20,30,40\"."); -DEFINE_int64(reads, -1, "Number of read operations to do. " +DEFINE_int64(reads, -1, + "Number of read operations to do. " "If negative, do FLAGS_num reads."); -DEFINE_int64(deletes, -1, "Number of delete operations to do. " +DEFINE_int64(deletes, -1, + "Number of delete operations to do. " "If negative, do FLAGS_num deletions."); DEFINE_int32(bloom_locality, 0, "Control bloom filter probes locality"); -DEFINE_int64(seed, 0, "Seed base for random number generators. " +DEFINE_int64(seed, 0, + "Seed base for random number generators. " "When 0 it is deterministic."); DEFINE_int32(threads, 1, "Number of concurrent threads to run."); -DEFINE_int32(duration, 0, "Time in seconds for the random-ops tests to run." +DEFINE_int32(duration, 0, + "Time in seconds for the random-ops tests to run." " When 0 then num & reads determine the test duration"); DEFINE_string(value_size_distribution_type, "fixed", @@ -341,7 +345,8 @@ DEFINE_int32(user_timestamp_size, 0, DEFINE_int32(num_multi_db, 0, "Number of DBs used in the benchmark. 0 means single DB."); -DEFINE_double(compression_ratio, 0.5, "Arrange to generate values that shrink" +DEFINE_double(compression_ratio, 0.5, + "Arrange to generate values that shrink" " to this fraction of their original size after compression"); DEFINE_double( @@ -500,9 +505,8 @@ DEFINE_int32(base_background_compactions, -1, "DEPRECATED"); DEFINE_uint64(subcompactions, 1, "Maximum number of subcompactions to divide L0-L1 compactions " "into."); -static const bool FLAGS_subcompactions_dummy - __attribute__((__unused__)) = RegisterFlagValidator(&FLAGS_subcompactions, - &ValidateUint32Range); +static const bool FLAGS_subcompactions_dummy __attribute__((__unused__)) = + RegisterFlagValidator(&FLAGS_subcompactions, &ValidateUint32Range); DEFINE_int32(max_background_flushes, ROCKSDB_NAMESPACE::Options().max_background_flushes, @@ -523,10 +527,12 @@ DEFINE_int32(universal_size_ratio, 0, "Percentage flexibility while comparing file size" " (for universal compaction only)."); -DEFINE_int32(universal_min_merge_width, 0, "The minimum number of files in a" +DEFINE_int32(universal_min_merge_width, 0, + "The minimum number of files in a" " single compaction run (for universal compaction only)."); -DEFINE_int32(universal_max_merge_width, 0, "The max number of files to compact" +DEFINE_int32(universal_max_merge_width, 0, + "The max number of files to compact" " in universal style compaction"); DEFINE_int32(universal_max_size_amplification_percent, 0, @@ -555,6 +561,9 @@ DEFINE_double(cache_high_pri_pool_ratio, 0.0, "If > 0.0, we also enable " "cache_index_and_filter_blocks_with_high_priority."); +DEFINE_double(cache_low_pri_pool_ratio, 0.0, + "Ratio of block cache reserve for low pri blocks."); + DEFINE_bool(use_clock_cache, false, "Replace default LRU block cache with clock cache."); @@ -666,7 +675,7 @@ DEFINE_int32(file_opening_threads, "threads that will be used to open files during DB::Open()"); DEFINE_bool(new_table_reader_for_compaction_inputs, true, - "If true, uses a separate file handle for compaction inputs"); + "If true, uses a separate file handle for compaction inputs"); DEFINE_int32(compaction_readahead_size, 0, "Compaction readahead size"); @@ -692,7 +701,8 @@ DEFINE_bool(memtable_whole_key_filtering, false, DEFINE_bool(memtable_use_huge_page, false, "Try to use huge page in memtables."); -DEFINE_bool(use_existing_db, false, "If true, do not destroy the existing" +DEFINE_bool(use_existing_db, false, + "If true, do not destroy the existing" " database. If you set this flag and also specify a benchmark that" " wants a fresh database, that benchmark will fail."); @@ -729,8 +739,8 @@ DEFINE_bool(use_keep_filter, false, "Whether to use a noop compaction filter"); static bool ValidateCacheNumshardbits(const char* flagname, int32_t value) { if (value >= 20) { - fprintf(stderr, "Invalid value for --%s: %d, must be < 20\n", - flagname, value); + fprintf(stderr, "Invalid value for --%s: %d, must be < 20\n", flagname, + value); return false; } return true; @@ -750,10 +760,12 @@ DEFINE_int32(stats_level, ROCKSDB_NAMESPACE::StatsLevel::kExceptDetailedTimers, DEFINE_string(statistics_string, "", "Serialized statistics string"); static class std::shared_ptr dbstats; -DEFINE_int64(writes, -1, "Number of write operations to do. If negative, do" +DEFINE_int64(writes, -1, + "Number of write operations to do. If negative, do" " --num reads."); -DEFINE_bool(finish_after_writes, false, "Write thread terminates after all writes are finished"); +DEFINE_bool(finish_after_writes, false, + "Write thread terminates after all writes are finished"); DEFINE_bool(sync, false, "Sync all writes to disk"); @@ -816,24 +828,27 @@ DEFINE_uint64(periodic_compaction_seconds, DEFINE_uint64(ttl_seconds, ROCKSDB_NAMESPACE::Options().ttl, "Set options.ttl"); static bool ValidateInt32Percent(const char* flagname, int32_t value) { - if (value <= 0 || value>=100) { - fprintf(stderr, "Invalid value for --%s: %d, 0< pct <100 \n", - flagname, value); + if (value <= 0 || value >= 100) { + fprintf(stderr, "Invalid value for --%s: %d, 0< pct <100 \n", flagname, + value); return false; } return true; } -DEFINE_int32(readwritepercent, 90, "Ratio of reads to reads/writes (expressed" +DEFINE_int32(readwritepercent, 90, + "Ratio of reads to reads/writes (expressed" " as percentage) for the ReadRandomWriteRandom workload. The " "default value 90 means 90% operations out of all reads and writes" " operations are reads. In other words, 9 gets for every 1 put."); -DEFINE_int32(mergereadpercent, 70, "Ratio of merges to merges&reads (expressed" +DEFINE_int32(mergereadpercent, 70, + "Ratio of merges to merges&reads (expressed" " as percentage) for the ReadRandomMergeRandom workload. The" " default value 70 means 70% out of all read and merge operations" " are merges. In other words, 7 merges for every 3 gets."); -DEFINE_int32(deletepercent, 2, "Percentage of deletes out of reads/writes/" +DEFINE_int32(deletepercent, 2, + "Percentage of deletes out of reads/writes/" "deletes (used in RandomWithVerify only). RandomWithVerify " "calculates writepercent as (100 - FLAGS_readwritepercent - " "deletepercent), so deletepercent must be smaller than (100 - " @@ -1134,7 +1149,8 @@ DEFINE_int32(compression_zstd_max_train_bytes, "Maximum size of training data passed to zstd's dictionary " "trainer."); -DEFINE_int32(min_level_to_compress, -1, "If non-negative, compression starts" +DEFINE_int32(min_level_to_compress, -1, + "If non-negative, compression starts" " from this level. Levels with number < min_level_to_compress are" " not compressed. Otherwise, apply compression_type to " "all levels."); @@ -1182,13 +1198,16 @@ static std::shared_ptr env_guard; static ROCKSDB_NAMESPACE::Env* FLAGS_env = ROCKSDB_NAMESPACE::Env::Default(); -DEFINE_int64(stats_interval, 0, "Stats are reported every N operations when " +DEFINE_int64(stats_interval, 0, + "Stats are reported every N operations when " "this is greater than zero. When 0 the interval grows over time."); -DEFINE_int64(stats_interval_seconds, 0, "Report stats every N seconds. This " +DEFINE_int64(stats_interval_seconds, 0, + "Report stats every N seconds. This " "overrides stats_interval when both are > 0."); -DEFINE_int32(stats_per_interval, 0, "Reports additional stats per interval when" +DEFINE_int32(stats_per_interval, 0, + "Reports additional stats per interval when" " this is greater than 0."); DEFINE_int64(report_interval_seconds, 0, @@ -1208,7 +1227,7 @@ DEFINE_int32(perf_level, ROCKSDB_NAMESPACE::PerfLevel::kDisable, static bool ValidateRateLimit(const char* flagname, double value) { const double EPSILON = 1e-10; - if ( value < -EPSILON ) { + if (value < -EPSILON) { fprintf(stderr, "Invalid value for --%s: %12.6f, must be >= 0.0\n", flagname, value); return false; @@ -1277,24 +1296,19 @@ DEFINE_bool(rate_limiter_auto_tuned, false, "Enable dynamic adjustment of rate limit according to demand for " "background I/O"); +DEFINE_bool(sine_write_rate, false, "Use a sine wave write_rate_limit"); -DEFINE_bool(sine_write_rate, false, - "Use a sine wave write_rate_limit"); - -DEFINE_uint64(sine_write_rate_interval_milliseconds, 10000, - "Interval of which the sine wave write_rate_limit is recalculated"); +DEFINE_uint64( + sine_write_rate_interval_milliseconds, 10000, + "Interval of which the sine wave write_rate_limit is recalculated"); -DEFINE_double(sine_a, 1, - "A in f(x) = A sin(bx + c) + d"); +DEFINE_double(sine_a, 1, "A in f(x) = A sin(bx + c) + d"); -DEFINE_double(sine_b, 1, - "B in f(x) = A sin(bx + c) + d"); +DEFINE_double(sine_b, 1, "B in f(x) = A sin(bx + c) + d"); -DEFINE_double(sine_c, 0, - "C in f(x) = A sin(bx + c) + d"); +DEFINE_double(sine_c, 0, "C in f(x) = A sin(bx + c) + d"); -DEFINE_double(sine_d, 1, - "D in f(x) = A sin(bx + c) + d"); +DEFINE_double(sine_d, 1, "D in f(x) = A sin(bx + c) + d"); DEFINE_bool(rate_limit_bg_reads, false, "Use options.rate_limiter on compaction reads"); @@ -1383,7 +1397,8 @@ DEFINE_bool(print_malloc_stats, false, DEFINE_bool(disable_auto_compactions, false, "Do not auto trigger compactions"); DEFINE_uint64(wal_ttl_seconds, 0, "Set the TTL for the WAL Files in seconds."); -DEFINE_uint64(wal_size_limit_MB, 0, "Set the size limit for the WAL Files" +DEFINE_uint64(wal_size_limit_MB, 0, + "Set the size limit for the WAL Files" " in MB."); DEFINE_uint64(max_total_wal_size, 0, "Set total max WAL size"); @@ -1451,11 +1466,12 @@ DEFINE_int32(num_deletion_threads, 1, "Number of threads to do deletion (used in TimeSeries and delete " "expire_style only)."); -DEFINE_int32(max_successive_merges, 0, "Maximum number of successive merge" +DEFINE_int32(max_successive_merges, 0, + "Maximum number of successive merge" " operations on a key in the memtable"); static bool ValidatePrefixSize(const char* flagname, int32_t value) { - if (value < 0 || value>=2000000000) { + if (value < 0 || value >= 2000000000) { fprintf(stderr, "Invalid value for --%s: %d. 0<= PrefixSize <=2000000000\n", flagname, value); return false; @@ -1463,9 +1479,11 @@ static bool ValidatePrefixSize(const char* flagname, int32_t value) { return true; } -DEFINE_int32(prefix_size, 0, "control the prefix size for HashSkipList and " +DEFINE_int32(prefix_size, 0, + "control the prefix size for HashSkipList and " "plain table"); -DEFINE_int64(keys_per_prefix, 0, "control average number of keys generated " +DEFINE_int64(keys_per_prefix, 0, + "control average number of keys generated " "per prefix, 0 means no special handling of the prefix, " "i.e. use the prefix comes with the generated random number."); DEFINE_bool(total_order_seek, false, @@ -1479,11 +1497,14 @@ DEFINE_bool( DEFINE_int32(memtable_insert_with_hint_prefix_size, 0, "If non-zero, enable " "memtable insert with hint with the given prefix size."); -DEFINE_bool(enable_io_prio, false, "Lower the background flush/compaction " +DEFINE_bool(enable_io_prio, false, + "Lower the background flush/compaction " "threads' IO priority"); -DEFINE_bool(enable_cpu_prio, false, "Lower the background flush/compaction " +DEFINE_bool(enable_cpu_prio, false, + "Lower the background flush/compaction " "threads' CPU priority"); -DEFINE_bool(identity_as_first_hash, false, "the first hash function of cuckoo " +DEFINE_bool(identity_as_first_hash, false, + "the first hash function of cuckoo " "table becomes an identity function. This is only valid when key " "is 8 bytes"); DEFINE_bool(dump_malloc_stats, true, "Dump malloc stats in LOG "); @@ -1505,24 +1526,30 @@ DEFINE_bool(multiread_batched, false, "Use the new MultiGet API"); DEFINE_string(memtablerep, "skip_list", ""); DEFINE_int64(hash_bucket_count, 1024 * 1024, "hash bucket count"); -DEFINE_bool(use_plain_table, false, "if use plain table " +DEFINE_bool(use_plain_table, false, + "if use plain table " "instead of block-based table format"); DEFINE_bool(use_cuckoo_table, false, "if use cuckoo table format"); DEFINE_double(cuckoo_hash_ratio, 0.9, "Hash ratio for Cuckoo SST table."); -DEFINE_bool(use_hash_search, false, "if use kHashSearch " +DEFINE_bool(use_hash_search, false, + "if use kHashSearch " "instead of kBinarySearch. " "This is valid if only we use BlockTable"); -DEFINE_bool(use_block_based_filter, false, "if use kBlockBasedFilter " +DEFINE_bool(use_block_based_filter, false, + "if use kBlockBasedFilter " "instead of kFullFilter for filter block. " "This is valid if only we use BlockTable"); -DEFINE_string(merge_operator, "", "The merge operator to use with the database." +DEFINE_string(merge_operator, "", + "The merge operator to use with the database." "If a new merge operator is specified, be sure to use fresh" " database The possible merge operators are defined in" " utilities/merge_operators.h"); -DEFINE_int32(skip_list_lookahead, 0, "Used with skip_list memtablerep; try " +DEFINE_int32(skip_list_lookahead, 0, + "Used with skip_list memtablerep; try " "linear search first for this many steps from the previous " "position"); -DEFINE_bool(report_file_operations, false, "if report number of file " +DEFINE_bool(report_file_operations, false, + "if report number of file " "operations"); DEFINE_bool(report_open_timing, false, "if report open timing"); DEFINE_int32(readahead_size, 0, "Iterator readahead size"); @@ -1842,11 +1869,7 @@ class ReportFileOpEnv : public EnvWrapper { } // namespace -enum DistributionType : unsigned char { - kFixed = 0, - kUniform, - kNormal -}; +enum DistributionType : unsigned char { kFixed = 0, kUniform, kNormal }; static enum DistributionType FLAGS_value_size_distribution_type_e = kFixed; @@ -1878,33 +1901,27 @@ class BaseDistribution { } return val; } + private: virtual unsigned int Get() = 0; - virtual bool NeedTruncate() { - return true; - } + virtual bool NeedTruncate() { return true; } unsigned int min_value_size_; unsigned int max_value_size_; }; -class FixedDistribution : public BaseDistribution -{ +class FixedDistribution : public BaseDistribution { public: - FixedDistribution(unsigned int size) : - BaseDistribution(size, size), - size_(size) {} + FixedDistribution(unsigned int size) + : BaseDistribution(size, size), size_(size) {} + private: - virtual unsigned int Get() override { - return size_; - } - virtual bool NeedTruncate() override { - return false; - } + virtual unsigned int Get() override { return size_; } + virtual bool NeedTruncate() override { return false; } unsigned int size_; }; -class NormalDistribution - : public BaseDistribution, public std::normal_distribution { +class NormalDistribution : public BaseDistribution, + public std::normal_distribution { public: NormalDistribution(unsigned int _min, unsigned int _max) : BaseDistribution(_min, _max), @@ -1922,9 +1939,8 @@ class NormalDistribution std::mt19937 gen_; }; -class UniformDistribution - : public BaseDistribution, - public std::uniform_int_distribution { +class UniformDistribution : public BaseDistribution, + public std::uniform_int_distribution { public: UniformDistribution(unsigned int _min, unsigned int _max) : BaseDistribution(_min, _max), @@ -1932,12 +1948,8 @@ class UniformDistribution gen_(rd_()) {} private: - virtual unsigned int Get() override { - return (*this)(gen_); - } - virtual bool NeedTruncate() override { - return false; - } + virtual unsigned int Get() override { return (*this)(gen_); } + virtual bool NeedTruncate() override { return false; } std::random_device rd_; std::mt19937 gen_; }; @@ -1950,7 +1962,6 @@ class RandomGenerator { std::unique_ptr dist_; public: - RandomGenerator() { auto max_value_size = FLAGS_value_size_max; switch (FLAGS_value_size_distribution_type_e) { @@ -1959,8 +1970,8 @@ class RandomGenerator { FLAGS_value_size_max)); break; case kNormal: - dist_.reset(new NormalDistribution(FLAGS_value_size_min, - FLAGS_value_size_max)); + dist_.reset( + new NormalDistribution(FLAGS_value_size_min, FLAGS_value_size_max)); break; case kFixed: default: @@ -2009,7 +2020,7 @@ struct DBWithColumnFamilies { DB* db; #ifndef ROCKSDB_LITE OptimisticTransactionDB* opt_txn_db; -#endif // ROCKSDB_LITE +#endif // ROCKSDB_LITE std::atomic num_created; // Need to be updated after all the // new entries in cfh are set. size_t num_hot; // Number of column families to be queried at each moment. @@ -2022,7 +2033,8 @@ struct DBWithColumnFamilies { DBWithColumnFamilies() : db(nullptr) #ifndef ROCKSDB_LITE - , opt_txn_db(nullptr) + , + opt_txn_db(nullptr) #endif // ROCKSDB_LITE { cfh.clear(); @@ -2205,19 +2217,12 @@ enum OperationType : unsigned char { }; static std::unordered_map> - OperationTypeString = { - {kRead, "read"}, - {kWrite, "write"}, - {kDelete, "delete"}, - {kSeek, "seek"}, - {kMerge, "merge"}, - {kUpdate, "update"}, - {kCompress, "compress"}, - {kCompress, "uncompress"}, - {kCrc, "crc"}, - {kHash, "hash"}, - {kOthers, "op"} -}; + OperationTypeString = {{kRead, "read"}, {kWrite, "write"}, + {kDelete, "delete"}, {kSeek, "seek"}, + {kMerge, "merge"}, {kUpdate, "update"}, + {kCompress, "compress"}, {kCompress, "uncompress"}, + {kCrc, "crc"}, {kHash, "hash"}, + {kOthers, "op"}}; class CombinedStats; class Stats { @@ -2235,7 +2240,8 @@ class Stats { uint64_t last_op_finish_; uint64_t last_report_finish_; std::unordered_map, - std::hash> hist_; + std::hash> + hist_; std::string message_; bool exclude_from_merge_; ReporterAgent* reporter_agent_; // does not own @@ -2267,15 +2273,14 @@ class Stats { } void Merge(const Stats& other) { - if (other.exclude_from_merge_) - return; + if (other.exclude_from_merge_) return; for (auto it = other.hist_.begin(); it != other.hist_.end(); ++it) { auto this_it = hist_.find(it->first); if (this_it != hist_.end()) { this_it->second->Merge(*(other.hist_.at(it->first))); } else { - hist_.insert({ it->first, it->second }); + hist_.insert({it->first, it->second}); } } @@ -2294,9 +2299,7 @@ class Stats { seconds_ = (finish_ - start_) * 1e-6; } - void AddMessage(Slice msg) { - AppendWithSpace(&message_, msg); - } + void AddMessage(Slice msg) { AppendWithSpace(&message_, msg); } void SetId(int id) { id_ = id; } void SetExcludeFromMerge() { exclude_from_merge_ = true; } @@ -2305,27 +2308,27 @@ class Stats { std::vector thread_list; FLAGS_env->GetThreadList(&thread_list); - fprintf(stderr, "\n%18s %10s %12s %20s %13s %45s %12s %s\n", - "ThreadID", "ThreadType", "cfName", "Operation", - "ElapsedTime", "Stage", "State", "OperationProperties"); + fprintf(stderr, "\n%18s %10s %12s %20s %13s %45s %12s %s\n", "ThreadID", + "ThreadType", "cfName", "Operation", "ElapsedTime", "Stage", + "State", "OperationProperties"); int64_t current_time = 0; clock_->GetCurrentTime(¤t_time).PermitUncheckedError(); for (auto ts : thread_list) { fprintf(stderr, "%18" PRIu64 " %10s %12s %20s %13s %45s %12s", - ts.thread_id, - ThreadStatus::GetThreadTypeName(ts.thread_type).c_str(), - ts.cf_name.c_str(), - ThreadStatus::GetOperationName(ts.operation_type).c_str(), - ThreadStatus::MicrosToString(ts.op_elapsed_micros).c_str(), - ThreadStatus::GetOperationStageName(ts.operation_stage).c_str(), - ThreadStatus::GetStateName(ts.state_type).c_str()); + ts.thread_id, + ThreadStatus::GetThreadTypeName(ts.thread_type).c_str(), + ts.cf_name.c_str(), + ThreadStatus::GetOperationName(ts.operation_type).c_str(), + ThreadStatus::MicrosToString(ts.op_elapsed_micros).c_str(), + ThreadStatus::GetOperationStageName(ts.operation_stage).c_str(), + ThreadStatus::GetStateName(ts.state_type).c_str()); auto op_properties = ThreadStatus::InterpretOperationProperties( ts.operation_type, ts.op_properties); for (const auto& op_prop : op_properties) { - fprintf(stderr, " %s %" PRIu64" |", - op_prop.first.c_str(), op_prop.second); + fprintf(stderr, " %s %" PRIu64 " |", op_prop.first.c_str(), + op_prop.second); } fprintf(stderr, "\n"); } @@ -2333,13 +2336,9 @@ class Stats { void ResetSineInterval() { sine_interval_ = clock_->NowMicros(); } - uint64_t GetSineInterval() { - return sine_interval_; - } + uint64_t GetSineInterval() { return sine_interval_; } - uint64_t GetStart() { - return start_; - } + uint64_t GetStart() { return start_; } void ResetLastOpTime() { // Set to now to avoid latency from calls to SleepForMicroseconds @@ -2355,8 +2354,7 @@ class Stats { uint64_t now = clock_->NowMicros(); uint64_t micros = now - last_op_finish_; - if (hist_.find(op_type) == hist_.end()) - { + if (hist_.find(op_type) == hist_.end()) { auto hist_temp = std::make_shared(); hist_.insert({op_type, std::move(hist_temp)}); } @@ -2372,13 +2370,20 @@ class Stats { done_ += num_ops; if (done_ >= next_report_) { if (!FLAGS_stats_interval) { - if (next_report_ < 1000) next_report_ += 100; - else if (next_report_ < 5000) next_report_ += 500; - else if (next_report_ < 10000) next_report_ += 1000; - else if (next_report_ < 50000) next_report_ += 5000; - else if (next_report_ < 100000) next_report_ += 10000; - else if (next_report_ < 500000) next_report_ += 50000; - else next_report_ += 100000; + if (next_report_ < 1000) + next_report_ += 100; + else if (next_report_ < 5000) + next_report_ += 500; + else if (next_report_ < 10000) + next_report_ += 1000; + else if (next_report_ < 50000) + next_report_ += 5000; + else if (next_report_ < 100000) + next_report_ += 10000; + else if (next_report_ < 500000) + next_report_ += 50000; + else + next_report_ += 100000; fprintf(stderr, "... finished %" PRIu64 " ops%30s\r", done_, ""); } else { uint64_t now = clock_->NowMicros(); @@ -2458,9 +2463,7 @@ class Stats { } } - void AddBytes(int64_t n) { - bytes_ += n; - } + void AddBytes(int64_t n) { bytes_ += n; } void Report(const Slice& name) { // Pretend at least one op was done in case we are running a benchmark @@ -2479,14 +2482,11 @@ class Stats { } AppendWithSpace(&extra, message_); double elapsed = (finish_ - start_) * 1e-6; - double throughput = (double)done_/elapsed; + double throughput = (double)done_ / elapsed; fprintf(stdout, "%-12s : %11.3f micros/op %ld ops/sec;%s%s\n", - name.ToString().c_str(), - seconds_ * 1e6 / done_, - (long)throughput, - (extra.empty() ? "" : " "), - extra.c_str()); + name.ToString().c_str(), seconds_ * 1e6 / done_, (long)throughput, + (extra.empty() ? "" : " "), extra.c_str()); if (FLAGS_histogram) { for (auto it = hist_.begin(); it != hist_.end(); ++it) { fprintf(stdout, "Microseconds per %s:\n%s\n", @@ -2642,13 +2642,13 @@ struct SharedState { long num_done; bool start; - SharedState() : cv(&mu), perf_level(FLAGS_perf_level) { } + SharedState() : cv(&mu), perf_level(FLAGS_perf_level) {} }; // Per-thread state for concurrent executions of the same benchmark. struct ThreadState { - int tid; // 0..n-1 when running in n threads - Random64 rand; // Has different seeds for different threads + int tid; // 0..n-1 when running in n threads + Random64 rand; // Has different seeds for different threads Stats stats; SharedState* shared; @@ -2660,7 +2660,7 @@ class Duration { public: Duration(uint64_t max_seconds, int64_t max_ops, int64_t ops_per_stage = 0) { max_seconds_ = max_seconds; - max_ops_= max_ops; + max_ops_ = max_ops; ops_per_stage_ = (ops_per_stage > 0) ? ops_per_stage : max_ops; ops_ = 0; start_at_ = FLAGS_env->NowMicros(); @@ -2669,7 +2669,7 @@ class Duration { int64_t GetStage() { return std::min(ops_, max_ops_ - 1) / ops_per_stage_; } bool Done(int64_t increment) { - if (increment <= 0) increment = 1; // avoid Done(0) and infinite loops + if (increment <= 0) increment = 1; // avoid Done(0) and infinite loops ops_ += increment; if (max_seconds_) { @@ -2808,28 +2808,30 @@ class Benchmark { FLAGS_key_size, FLAGS_user_timestamp_size); auto avg_value_size = FLAGS_value_size; if (FLAGS_value_size_distribution_type_e == kFixed) { - fprintf(stdout, "Values: %d bytes each (%d bytes after compression)\n", + fprintf(stdout, + "Values: %d bytes each (%d bytes after compression)\n", avg_value_size, static_cast(avg_value_size * FLAGS_compression_ratio + 0.5)); } else { avg_value_size = (FLAGS_value_size_min + FLAGS_value_size_max) / 2; - fprintf(stdout, "Values: %d avg bytes each (%d bytes after compression)\n", + fprintf(stdout, + "Values: %d avg bytes each (%d bytes after compression)\n", avg_value_size, static_cast(avg_value_size * FLAGS_compression_ratio + 0.5)); fprintf(stdout, "Values Distribution: %s (min: %d, max: %d)\n", - FLAGS_value_size_distribution_type.c_str(), - FLAGS_value_size_min, FLAGS_value_size_max); + FLAGS_value_size_distribution_type.c_str(), FLAGS_value_size_min, + FLAGS_value_size_max); } fprintf(stdout, "Entries: %" PRIu64 "\n", num_); fprintf(stdout, "Prefix: %d bytes\n", FLAGS_prefix_size); fprintf(stdout, "Keys per prefix: %" PRIu64 "\n", keys_per_prefix_); fprintf(stdout, "RawSize: %.1f MB (estimated)\n", - ((static_cast(FLAGS_key_size + avg_value_size) * num_) - / 1048576.0)); - fprintf(stdout, "FileSize: %.1f MB (estimated)\n", - (((FLAGS_key_size + avg_value_size * FLAGS_compression_ratio) - * num_) - / 1048576.0)); + ((static_cast(FLAGS_key_size + avg_value_size) * num_) / + 1048576.0)); + fprintf( + stdout, "FileSize: %.1f MB (estimated)\n", + (((FLAGS_key_size + avg_value_size * FLAGS_compression_ratio) * num_) / + 1048576.0)); fprintf(stdout, "Write rate: %" PRIu64 " bytes/second\n", FLAGS_benchmark_write_rate_limit); fprintf(stdout, "Read rate: %" PRIu64 " ops/second\n", @@ -2863,9 +2865,9 @@ class Benchmark { void PrintWarnings(const char* compression) { #if defined(__GNUC__) && !defined(__OPTIMIZE__) - fprintf(stdout, - "WARNING: Optimization is disabled: benchmarks unnecessarily slow\n" - ); + fprintf( + stdout, + "WARNING: Optimization is disabled: benchmarks unnecessarily slow\n"); #endif #ifndef NDEBUG fprintf(stdout, @@ -2901,7 +2903,7 @@ class Benchmark { start++; } unsigned int limit = static_cast(s.size()); - while (limit > start && isspace(s[limit-1])) { + while (limit > start && isspace(s[limit - 1])) { limit--; } return Slice(s.data() + start, limit - start); @@ -2909,8 +2911,8 @@ class Benchmark { #endif void PrintEnvironment() { - fprintf(stderr, "RocksDB: version %d.%d\n", - kMajorVersion, kMinorVersion); + fprintf(stderr, "RocksDB: version %d.%d\n", kMajorVersion, + kMinorVersion); #if defined(__linux) || defined(__APPLE__) || defined(__FreeBSD__) time_t now = time(nullptr); @@ -3050,11 +3052,12 @@ class Benchmark { #ifdef MEMKIND FLAGS_use_cache_memkind_kmem_allocator ? std::make_shared() - : nullptr + : nullptr, #else - nullptr + nullptr, #endif - ); + kDefaultToAdaptiveMutex, kDefaultCacheMetadataChargePolicy, + FLAGS_cache_low_pri_pool_ratio); if (FLAGS_use_cache_memkind_kmem_allocator) { #ifndef MEMKIND fprintf(stderr, "Memkind library is not linked with the binary."); @@ -3797,7 +3800,7 @@ class Benchmark { } } - SetPerfLevel(static_cast (shared->perf_level)); + SetPerfLevel(static_cast(shared->perf_level)); perf_context.EnablePerLevelPerfContext(); thread->stats.Start(thread->tid); (arg->bm->*(arg->method))(thread); @@ -3894,7 +3897,7 @@ class Benchmark { template static inline void ChecksumBenchmark(FnType fn, ThreadState* thread, Args... args) { - const int size = FLAGS_block_size; // use --block_size option for db_bench + const int size = FLAGS_block_size; // use --block_size option for db_bench std::string labels = "(" + ToString(FLAGS_block_size) + " per op)"; const char* label = labels.c_str(); @@ -3933,7 +3936,7 @@ class Benchmark { int dummy; std::atomic ap(&dummy); int count = 0; - void *ptr = nullptr; + void* ptr = nullptr; thread->stats.AddMessage("(each op is 1000 loads)"); while (count < 100000) { for (int i = 0; i < 1000; i++) { @@ -3945,7 +3948,7 @@ class Benchmark { if (ptr == nullptr) exit(1); // Disable unused variable warning. } - void Compress(ThreadState *thread) { + void Compress(ThreadState* thread) { RandomGenerator gen; Slice input = gen.Generate(FLAGS_block_size); int64_t bytes = 0; @@ -3977,7 +3980,7 @@ class Benchmark { } } - void Uncompress(ThreadState *thread) { + void Uncompress(ThreadState* thread) { RandomGenerator gen; Slice input = gen.Generate(FLAGS_block_size); std::string compressed; @@ -4057,7 +4060,7 @@ class Benchmark { options.write_buffer_size = FLAGS_write_buffer_size; options.max_write_buffer_number = FLAGS_max_write_buffer_number; options.min_write_buffer_number_to_merge = - FLAGS_min_write_buffer_number_to_merge; + FLAGS_min_write_buffer_number_to_merge; options.max_write_buffer_number_to_maintain = FLAGS_max_write_buffer_number_to_maintain; options.max_write_buffer_size_to_maintain = @@ -4129,8 +4132,9 @@ class Benchmark { } else if ((FLAGS_prefix_size == 0) && (options.memtable_factory->IsInstanceOf("prefix_hash") || options.memtable_factory->IsInstanceOf("hash_linkedlist"))) { - fprintf(stderr, "prefix_size should be non-zero if PrefixHash or " - "HashLinkedList memtablerep is used\n"); + fprintf(stderr, + "prefix_size should be non-zero if PrefixHash or " + "HashLinkedList memtablerep is used\n"); exit(1); } if (FLAGS_use_plain_table) { @@ -4171,8 +4175,8 @@ class Benchmark { ROCKSDB_NAMESPACE::CuckooTableOptions table_options; table_options.hash_table_ratio = FLAGS_cuckoo_hash_ratio; table_options.identity_as_first_hash = FLAGS_identity_as_first_hash; - options.table_factory = std::shared_ptr( - NewCuckooTableFactory(table_options)); + options.table_factory = + std::shared_ptr(NewCuckooTableFactory(table_options)); #else fprintf(stderr, "Cuckoo table is not supported in lite mode\n"); exit(1); @@ -4184,7 +4188,7 @@ class Benchmark { if (FLAGS_use_hash_search) { if (FLAGS_prefix_size == 0) { fprintf(stderr, - "prefix_size not assigned when enable use_hash_search \n"); + "prefix_size not assigned when enable use_hash_search \n"); exit(1); } block_based_options.index_type = BlockBasedTableOptions::kHashSearch; @@ -4246,6 +4250,12 @@ class Benchmark { block_based_options.cache_index_and_filter_blocks_with_high_priority = true; } + if (FLAGS_cache_high_pri_pool_ratio + FLAGS_cache_low_pri_pool_ratio > + 1.0) { + fprintf(stderr, + "Sum of high_pri_pool_ratio and low_pri_pool_ratio " + "cannot exceed 1.0.\n"); + } block_based_options.block_cache = cache_; block_based_options.block_cache_compressed = compressed_cache_; block_based_options.block_size = FLAGS_block_size; @@ -4333,13 +4343,13 @@ class Benchmark { exit(1); } options.max_bytes_for_level_multiplier_additional = - FLAGS_max_bytes_for_level_multiplier_additional_v; + FLAGS_max_bytes_for_level_multiplier_additional_v; } options.level0_stop_writes_trigger = FLAGS_level0_stop_writes_trigger; options.level0_file_num_compaction_trigger = FLAGS_level0_file_num_compaction_trigger; options.level0_slowdown_writes_trigger = - FLAGS_level0_slowdown_writes_trigger; + FLAGS_level0_slowdown_writes_trigger; options.compression = FLAGS_compression_type_e; if (FLAGS_simulate_hybrid_fs_file != "") { options.bottommost_temperature = Temperature::kWarm; @@ -4355,8 +4365,7 @@ class Benchmark { for (int i = 0; i < FLAGS_min_level_to_compress; i++) { options.compression_per_level[i] = kNoCompression; } - for (int i = FLAGS_min_level_to_compress; - i < FLAGS_num_levels; i++) { + for (int i = FLAGS_min_level_to_compress; i < FLAGS_num_levels; i++) { options.compression_per_level[i] = FLAGS_compression_type_e; } } @@ -4380,7 +4389,7 @@ class Benchmark { options.write_thread_max_yield_usec = FLAGS_write_thread_max_yield_usec; options.write_thread_slow_yield_usec = FLAGS_write_thread_slow_yield_usec; options.rate_limit_delay_max_milliseconds = - FLAGS_rate_limit_delay_max_milliseconds; + FLAGS_rate_limit_delay_max_milliseconds; options.table_cache_numshardbits = FLAGS_table_cache_numshardbits; options.max_compaction_bytes = FLAGS_max_compaction_bytes; options.disable_auto_compactions = FLAGS_disable_auto_compactions; @@ -4414,23 +4423,23 @@ class Benchmark { // set universal style compaction configurations, if applicable if (FLAGS_universal_size_ratio != 0) { options.compaction_options_universal.size_ratio = - FLAGS_universal_size_ratio; + FLAGS_universal_size_ratio; } if (FLAGS_universal_min_merge_width != 0) { options.compaction_options_universal.min_merge_width = - FLAGS_universal_min_merge_width; + FLAGS_universal_min_merge_width; } if (FLAGS_universal_max_merge_width != 0) { options.compaction_options_universal.max_merge_width = - FLAGS_universal_max_merge_width; + FLAGS_universal_max_merge_width; } if (FLAGS_universal_max_size_amplification_percent != 0) { options.compaction_options_universal.max_size_amplification_percent = - FLAGS_universal_max_size_amplification_percent; + FLAGS_universal_max_size_amplification_percent; } if (FLAGS_universal_compression_size_percent != -1) { options.compaction_options_universal.compression_size_percent = - FLAGS_universal_compression_size_percent; + FLAGS_universal_compression_size_percent; } options.compaction_options_universal.allow_trivial_move = FLAGS_universal_allow_trivial_move; @@ -4477,7 +4486,6 @@ class Benchmark { exit(1); } #endif // ROCKSDB_LITE - } void InitializeOptionsGeneral(Options* opts) { @@ -4606,7 +4614,7 @@ class Benchmark { } void OpenDb(Options options, const std::string& db_name, - DBWithColumnFamilies* db) { + DBWithColumnFamilies* db) { uint64_t open_start = FLAGS_report_open_timing ? FLAGS_env->NowNanos() : 0; Status s; if (use_multi_write_) { @@ -4624,7 +4632,7 @@ class Benchmark { std::vector column_families; for (size_t i = 0; i < num_hot; i++) { column_families.push_back(ColumnFamilyDescriptor( - ColumnFamilyName(i), ColumnFamilyOptions(options))); + ColumnFamilyName(i), ColumnFamilyOptions(options))); } std::vector cfh_idx_to_prob; if (!FLAGS_column_family_distribution.empty()) { @@ -4650,8 +4658,8 @@ class Benchmark { } #ifndef ROCKSDB_LITE if (FLAGS_readonly) { - s = DB::OpenForReadOnly(options, db_name, column_families, - &db->cfh, &db->db); + s = DB::OpenForReadOnly(options, db_name, column_families, &db->cfh, + &db->db); } else if (FLAGS_optimistic_transaction_db) { s = OptimisticTransactionDB::Open(options, db_name, column_families, &db->cfh, &db->opt_txn_db); @@ -4762,9 +4770,7 @@ class Benchmark { } } - enum WriteMode { - RANDOM, SEQUENTIAL, UNIQUE_RANDOM - }; + enum WriteMode { RANDOM, SEQUENTIAL, UNIQUE_RANDOM }; void WriteSeqDeterministic(ThreadState* thread) { DoDeterministicCompact(thread, open_options_.compaction_style, SEQUENTIAL); @@ -4775,13 +4781,9 @@ class Benchmark { UNIQUE_RANDOM); } - void WriteSeq(ThreadState* thread) { - DoWrite(thread, SEQUENTIAL); - } + void WriteSeq(ThreadState* thread) { DoWrite(thread, SEQUENTIAL); } - void WriteRandom(ThreadState* thread) { - DoWrite(thread, RANDOM); - } + void WriteRandom(ThreadState* thread) { DoWrite(thread, RANDOM); } void WriteUniqueRandom(ThreadState* thread) { DoWrite(thread, UNIQUE_RANDOM); @@ -4835,9 +4837,7 @@ class Benchmark { std::vector values_; }; - DB* SelectDB(ThreadState* thread) { - return SelectDBWithCfh(thread)->db; - } + DB* SelectDB(ThreadState* thread) { return SelectDBWithCfh(thread)->db; } DBWithColumnFamilies* SelectDBWithCfh(ThreadState* thread) { return SelectDBWithCfh(thread->rand.Next()); @@ -4846,13 +4846,13 @@ class Benchmark { DBWithColumnFamilies* SelectDBWithCfh(uint64_t rand_int) { if (db_.db != nullptr) { return &db_; - } else { + } else { return &multi_dbs_[rand_int % multi_dbs_.size()]; } } double SineRate(double x) { - return FLAGS_sine_a*sin((FLAGS_sine_b*x) + FLAGS_sine_c) + FLAGS_sine_d; + return FLAGS_sine_a * sin((FLAGS_sine_b * x) + FLAGS_sine_c) + FLAGS_sine_d; } void DoWrite(ThreadState* thread, WriteMode write_mode) { @@ -5138,8 +5138,7 @@ class Benchmark { // We use same rand_num as seed for key and column family so that we // can deterministically find the cfh corresponding to a particular // key while reading the key. - batch.Put(db_with_cfh->GetCfh(rand_num), key, - val); + batch.Put(db_with_cfh->GetCfh(rand_num), key, val); } batch_bytes += val.size() + key_size_ + user_timestamp_size_; bytes += val.size() + key_size_ + user_timestamp_size_; @@ -5212,8 +5211,8 @@ class Benchmark { } if (thread->shared->write_rate_limiter.get() != nullptr) { thread->shared->write_rate_limiter->Request( - batch_bytes, Env::IO_HIGH, - nullptr /* stats */, RateLimiter::OpType::kWrite); + batch_bytes, Env::IO_HIGH, nullptr /* stats */, + RateLimiter::OpType::kWrite); // Set time at which last op finished to Now() to hide latency and // sleep from rate limiter. Also, do the check once per batch, not // once per write. @@ -5251,12 +5250,12 @@ class Benchmark { if (usecs_since_last > (FLAGS_sine_write_rate_interval_milliseconds * uint64_t{1000})) { double usecs_since_start = - static_cast(now - thread->stats.GetStart()); + static_cast(now - thread->stats.GetStart()); thread->stats.ResetSineInterval(); uint64_t write_rate = - static_cast(SineRate(usecs_since_start / 1000000.0)); + static_cast(SineRate(usecs_since_start / 1000000.0)); thread->shared->write_rate_limiter.reset( - NewGenericRateLimiter(write_rate)); + NewGenericRateLimiter(write_rate)); } } if (!s.ok()) { @@ -5348,11 +5347,13 @@ class Benchmark { continue; } } - writes_ /= static_cast(open_options_.max_bytes_for_level_multiplier); + writes_ /= + static_cast(open_options_.max_bytes_for_level_multiplier); } for (size_t i = 0; i < num_db; i++) { if (sorted_runs[i].size() < num_levels - 1) { - fprintf(stderr, "n is too small to fill %" ROCKSDB_PRIszt " levels\n", num_levels); + fprintf(stderr, "n is too small to fill %" ROCKSDB_PRIszt " levels\n", + num_levels); exit(1); } } @@ -5363,13 +5364,14 @@ class Benchmark { auto options = db->GetOptions(); MutableCFOptions mutable_cf_options(options); for (size_t j = 0; j < sorted_runs[i].size(); j++) { - compactionOptions.output_file_size_limit = - MaxFileSizeForLevel(mutable_cf_options, - static_cast(output_level), compaction_style); + compactionOptions.output_file_size_limit = MaxFileSizeForLevel( + mutable_cf_options, static_cast(output_level), + compaction_style); std::cout << sorted_runs[i][j].size() << std::endl; - db->CompactFiles(compactionOptions, {sorted_runs[i][j].back().name, - sorted_runs[i][j].front().name}, - static_cast(output_level - j) /*level*/); + db->CompactFiles( + compactionOptions, + {sorted_runs[i][j].back().name, sorted_runs[i][j].front().name}, + static_cast(output_level - j) /*level*/); } } } else if (compaction_style == kCompactionStyleUniversal) { @@ -5400,11 +5402,13 @@ class Benchmark { } num_files_at_level0[i] = meta.levels[0].files.size(); } - writes_ = static_cast(writes_* static_cast(100) / (ratio + 200)); + writes_ = static_cast(writes_ * static_cast(100) / + (ratio + 200)); } for (size_t i = 0; i < num_db; i++) { if (sorted_runs[i].size() < num_levels) { - fprintf(stderr, "n is too small to fill %" ROCKSDB_PRIszt " levels\n", num_levels); + fprintf(stderr, "n is too small to fill %" ROCKSDB_PRIszt " levels\n", + num_levels); exit(1); } } @@ -5415,9 +5419,9 @@ class Benchmark { auto options = db->GetOptions(); MutableCFOptions mutable_cf_options(options); for (size_t j = 0; j < sorted_runs[i].size(); j++) { - compactionOptions.output_file_size_limit = - MaxFileSizeForLevel(mutable_cf_options, - static_cast(output_level), compaction_style); + compactionOptions.output_file_size_limit = MaxFileSizeForLevel( + mutable_cf_options, static_cast(output_level), + compaction_style); db->CompactFiles( compactionOptions, {sorted_runs[i][j].back().name, sorted_runs[i][j].front().name}, @@ -5428,7 +5432,7 @@ class Benchmark { } else if (compaction_style == kCompactionStyleFIFO) { if (num_levels != 1) { return Status::InvalidArgument( - "num_levels should be 1 for FIFO compaction"); + "num_levels should be 1 for FIFO compaction"); } if (FLAGS_num_multi_db != 0) { return Status::InvalidArgument("Doesn't support multiDB"); @@ -5445,7 +5449,7 @@ class Benchmark { db->GetColumnFamilyMetaData(&meta); auto total_size = meta.levels[0].size; if (total_size >= - db->GetOptions().compaction_options_fifo.max_table_files_size) { + db->GetOptions().compaction_options_fifo.max_table_files_size) { for (auto file_meta : meta.levels[0].files) { file_names.emplace_back(file_meta.name); } @@ -5482,8 +5486,8 @@ class Benchmark { db->GetColumnFamilyMetaData(&meta); auto total_size = meta.levels[0].size; assert(total_size <= - db->GetOptions().compaction_options_fifo.max_table_files_size); - break; + db->GetOptions().compaction_options_fifo.max_table_files_size); + break; } // verify smallest/largest seqno and key range of each sorted run @@ -5549,7 +5553,9 @@ class Benchmark { for (size_t k = 0; k < num_db; k++) { auto db = db_list[k]; fprintf(stdout, - "---------------------- DB %" ROCKSDB_PRIszt " LSM ---------------------\n", k); + "---------------------- DB %" ROCKSDB_PRIszt + " LSM ---------------------\n", + k); db->GetColumnFamilyMetaData(&meta); for (auto& levelMeta : meta.levels) { if (levelMeta.files.empty()) { @@ -5778,7 +5784,9 @@ class Benchmark { } while (!duration.Done(100)); char msg[100]; - snprintf(msg, sizeof(msg), "(%" PRIu64 " of %" PRIu64 " found, " + snprintf(msg, sizeof(msg), + "(%" PRIu64 " of %" PRIu64 + " found, " "issued %" PRIu64 " non-exist keys)\n", found, read, nonexist); @@ -5884,8 +5892,8 @@ class Benchmark { } char msg[100]; - snprintf(msg, sizeof(msg), "(%" PRIu64 " of %" PRIu64 " found)\n", - found, read); + snprintf(msg, sizeof(msg), "(%" PRIu64 " of %" PRIu64 " found)\n", found, + read); thread->stats.AddBytes(bytes); thread->stats.AddMessage(msg); @@ -5905,7 +5913,7 @@ class Benchmark { int64_t found = 0; ReadOptions options(FLAGS_verify_checksum, true); std::vector keys; - std::vector > key_guards; + std::vector> key_guards; std::vector values(entries_per_batch_); PinnableSlice* pin_values = new PinnableSlice[entries_per_batch_]; std::unique_ptr pin_values_guard(pin_values); @@ -5989,8 +5997,8 @@ class Benchmark { } char msg[100]; - snprintf(msg, sizeof(msg), "(%" PRIu64 " of %" PRIu64 " found)", - found, read); + snprintf(msg, sizeof(msg), "(%" PRIu64 " of %" PRIu64 " found)", found, + read); thread->stats.AddBytes(bytes); thread->stats.AddMessage(msg); } @@ -6393,8 +6401,8 @@ class Benchmark { } else if (query_type == 1) { // the Put query puts++; - int64_t val_size = ParetoCdfInversion( - u, FLAGS_value_theta, FLAGS_value_k, FLAGS_value_sigma); + int64_t val_size = ParetoCdfInversion(u, FLAGS_value_theta, + FLAGS_value_k, FLAGS_value_sigma); if (val_size < 0) { val_size = 10; } else if (val_size > value_max) { @@ -6445,9 +6453,9 @@ class Benchmark { } char msg[256]; snprintf(msg, sizeof(msg), - "( Gets:%" PRIu64 " Puts:%" PRIu64 " Seek:%" PRIu64 " of %" PRIu64 - " in %" PRIu64 " found)\n", - gets, puts, seek, found, read); + "( Gets:%" PRIu64 " Puts:%" PRIu64 " Seek:%" PRIu64 + " SeekFound:%" PRIu64 " of %" PRIu64 " in %" PRIu64 " found)\n", + gets, puts, seek, seek_found, found, read); thread->stats.AddBytes(bytes); thread->stats.AddMessage(msg); @@ -6597,8 +6605,8 @@ class Benchmark { } char msg[100]; - snprintf(msg, sizeof(msg), "(%" PRIu64 " of %" PRIu64 " found)\n", - found, read); + snprintf(msg, sizeof(msg), "(%" PRIu64 " of %" PRIu64 " found)\n", found, + read); thread->stats.AddBytes(bytes); thread->stats.AddMessage(msg); if (FLAGS_perf_level > ROCKSDB_NAMESPACE::PerfLevel::kDisable) { @@ -6664,13 +6672,9 @@ class Benchmark { } } - void DeleteSeq(ThreadState* thread) { - DoDelete(thread, true); - } + void DeleteSeq(ThreadState* thread) { DoDelete(thread, true); } - void DeleteRandom(ThreadState* thread) { - DoDelete(thread, false); - } + void DeleteRandom(ThreadState* thread) { DoDelete(thread, false); } void ReadWhileWriting(ThreadState* thread) { if (thread->tid > 0) { @@ -6763,9 +6767,9 @@ class Benchmark { thread->stats.FinishedOps(&db_, db_.db, 1, kWrite); if (FLAGS_benchmark_write_rate_limit > 0) { - write_rate_limiter->Request( - key.size() + val.size(), Env::IO_HIGH, - nullptr /* stats */, RateLimiter::OpType::kWrite); + write_rate_limiter->Request(key.size() + val.size(), Env::IO_HIGH, + nullptr /* stats */, + RateLimiter::OpType::kWrite); } } thread->stats.AddBytes(bytes); @@ -6798,19 +6802,15 @@ class Benchmark { fprintf(stderr, "num reads to do %" PRIu64 "\n", reads_); Duration duration(FLAGS_duration, reads_); - uint64_t num_seek_to_first = 0; - uint64_t num_next = 0; while (!duration.Done(1)) { if (!iter->Valid()) { iter->SeekToFirst(); - num_seek_to_first++; } else if (!iter->status().ok()) { fprintf(stderr, "Iterator error: %s\n", iter->status().ToString().c_str()); abort(); } else { iter->Next(); - num_next++; } thread->stats.FinishedOps(&db_, db_.db, 1, kSeek); @@ -6850,7 +6850,6 @@ class Benchmark { return s; } - // Given a key K, this deletes (K+"0", V), (K+"1", V), (K+"2", V) // in DB atomically i.e in a single batch. Also refer GetMany. Status DeleteMany(DB* db, const WriteOptions& writeoptions, @@ -6964,7 +6963,7 @@ class Benchmark { put_weight = 100 - get_weight - delete_weight; } GenerateKeyFromInt(thread->rand.Next() % FLAGS_numdistinct, - FLAGS_numdistinct, &key); + FLAGS_numdistinct, &key); if (get_weight > 0) { // do all the gets first Status s = GetMany(db, options, key, &value); @@ -7002,8 +7001,8 @@ class Benchmark { } char msg[128]; snprintf(msg, sizeof(msg), - "( get:%" PRIu64 " put:%" PRIu64 " del:%" PRIu64 " total:%" \ - PRIu64 " found:%" PRIu64 ")", + "( get:%" PRIu64 " put:%" PRIu64 " del:%" PRIu64 " total:%" PRIu64 + " found:%" PRIu64 ")", gets_done, puts_done, deletes_done, readwrites_, found); thread->stats.AddMessage(msg); } @@ -7057,7 +7056,7 @@ class Benchmark { get_weight--; reads_done++; thread->stats.FinishedOps(nullptr, db, 1, kRead); - } else if (put_weight > 0) { + } else if (put_weight > 0) { // then do all the corresponding number of puts // for all the gets we have done earlier Status s; @@ -7077,8 +7076,9 @@ class Benchmark { } } char msg[100]; - snprintf(msg, sizeof(msg), "( reads:%" PRIu64 " writes:%" PRIu64 \ - " total:%" PRIu64 " found:%" PRIu64 ")", + snprintf(msg, sizeof(msg), + "( reads:%" PRIu64 " writes:%" PRIu64 " total:%" PRIu64 + " found:%" PRIu64 ")", reads_done, writes_done, readwrites_, found); thread->stats.AddMessage(msg); } @@ -7142,8 +7142,8 @@ class Benchmark { thread->stats.FinishedOps(nullptr, db, 1, kUpdate); } char msg[100]; - snprintf(msg, sizeof(msg), - "( updates:%" PRIu64 " found:%" PRIu64 ")", readwrites_, found); + snprintf(msg, sizeof(msg), "( updates:%" PRIu64 " found:%" PRIu64 ")", + readwrites_, found); thread->stats.AddBytes(bytes); thread->stats.AddMessage(msg); } @@ -7186,7 +7186,8 @@ class Benchmark { exit(1); } - Slice value = gen.Generate(static_cast(existing_value.size())); + Slice value = + gen.Generate(static_cast(existing_value.size())); std::string new_value; if (status.ok()) { @@ -7210,8 +7211,8 @@ class Benchmark { thread->stats.FinishedOps(nullptr, db, 1); } char msg[100]; - snprintf(msg, sizeof(msg), - "( updates:%" PRIu64 " found:%" PRIu64 ")", readwrites_, found); + snprintf(msg, sizeof(msg), "( updates:%" PRIu64 " found:%" PRIu64 ")", + readwrites_, found); thread->stats.AddMessage(msg); } @@ -7259,7 +7260,7 @@ class Benchmark { Slice operand = gen.Generate(); if (value.size() > 0) { // Use a delimiter to match the semantics for StringAppendOperator - value.append(1,','); + value.append(1, ','); } value.append(operand.data(), operand.size()); @@ -7281,7 +7282,7 @@ class Benchmark { char msg[100]; snprintf(msg, sizeof(msg), "( updates:%" PRIu64 " found:%" PRIu64 ")", - readwrites_, found); + readwrites_, found); thread->stats.AddBytes(bytes); thread->stats.AddMessage(msg); } @@ -7312,12 +7313,10 @@ class Benchmark { Slice val = gen.Generate(); if (FLAGS_num_column_families > 1) { s = db_with_cfh->db->Merge(write_options_, - db_with_cfh->GetCfh(key_rand), key, - val); + db_with_cfh->GetCfh(key_rand), key, val); } else { - s = db_with_cfh->db->Merge(write_options_, - db_with_cfh->db->DefaultColumnFamily(), key, - val); + s = db_with_cfh->db->Merge( + write_options_, db_with_cfh->db->DefaultColumnFamily(), key, val); } if (!s.ok()) { @@ -7371,8 +7370,7 @@ class Benchmark { thread->stats.FinishedOps(nullptr, db, 1, kMerge); } else { Status s = db->Get(options, key, &value); - if (value.length() > max_length) - max_length = value.length(); + if (value.length() > max_length) max_length = value.length(); if (!s.ok() && !s.IsNotFound()) { fprintf(stderr, "get error: %s\n", s.ToString().c_str()); @@ -7618,9 +7616,8 @@ class Benchmark { return; } - Status s = - RandomTransactionInserter::Verify(db_.db, - static_cast(FLAGS_transaction_sets)); + Status s = RandomTransactionInserter::Verify( + db_.db, static_cast(FLAGS_transaction_sets)); if (s.ok()) { fprintf(stdout, "RandomTransactionVerify Success.\n"); @@ -7845,9 +7842,9 @@ class Benchmark { thread->stats.AddBytes(bytes); if (FLAGS_benchmark_write_rate_limit > 0) { - write_rate_limiter->Request( - key.size() + val.size(), Env::IO_HIGH, - nullptr /* stats */, RateLimiter::OpType::kWrite); + write_rate_limiter->Request(key.size() + val.size(), Env::IO_HIGH, + nullptr /* stats */, + RateLimiter::OpType::kWrite); } } } @@ -8230,12 +8227,12 @@ int db_bench_tool(int argc, char** argv) { } FLAGS_compression_type_e = - StringToCompressionType(FLAGS_compression_type.c_str()); + StringToCompressionType(FLAGS_compression_type.c_str()); #ifndef ROCKSDB_LITE // Stacked BlobDB FLAGS_blob_db_compression_type_e = - StringToCompressionType(FLAGS_blob_db_compression_type.c_str()); + StringToCompressionType(FLAGS_blob_db_compression_type.c_str()); int env_opts = !FLAGS_hdfs.empty() + !FLAGS_env_uri.empty() + !FLAGS_fs_uri.empty(); @@ -8314,7 +8311,7 @@ int db_bench_tool(int argc, char** argv) { } FLAGS_value_size_distribution_type_e = - StringToDistributionType(FLAGS_value_size_distribution_type.c_str()); + StringToDistributionType(FLAGS_value_size_distribution_type.c_str()); // Note options sanitization may increase thread pool sizes according to // max_background_flushes/max_background_compactions/max_background_jobs