Summary:
- Make MemoryAllocator and its implementations into a Customizable class.
- Added a "DefaultMemoryAllocator" which uses new and delete
- Added a "CountedMemoryAllocator" that counts the number of allocs and free
- Updated the existing tests to use these new allocators
- Changed the memkind allocator test into a generic test that can test the various allocators.
- Added tests for creating all of the allocators
- Added tests to verify/create the JemallocNodumpAllocator using its options.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/8980
Reviewed By: zhichao-cao
Differential Revision: D32990403
Pulled By: mrambacher
fbshipit-source-id: 6fdfe8218c10dd8dfef34344a08201be1fa95c76
Summary:
This header file was including everything and the kitchen sink when it did not need to. This resulted in many places including this header when they needed other pieces instead.
Cleaned up this header to only include what was needed and fixed up the remaining code to include what was now missing.
Hopefully, this sort of code hygiene cleanup will speed up the builds...
Pull Request resolved: https://github.com/facebook/rocksdb/pull/8930
Reviewed By: pdillinger
Differential Revision: D31142788
Pulled By: mrambacher
fbshipit-source-id: 6b45de3f300750c79f751f6227dece9cfd44085d
Summary:
Previously, the `MemPurge` sampling function was assessing whether a random entry from a memtable was garbage or not by simply querying the given memtable (see https://github.com/facebook/rocksdb/issues/8628 for more details).
In this diff, I am updating the sampling function by querying not only the memtable the entry was drawn from, but also all subsequent memtables that have a greater memtable ID.
I also added the size of the value for KV entries in the payload/useful payload estimates (which was also one of the reasons why sampling was not as good as mempurging all the time in terms of L0 SST files reduction).
Once these changes were made, I was able to clean obsolete objects and functions from the `MemtableList` struct, and did a bit of cleanup everywhere.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/8656
Reviewed By: pdillinger
Differential Revision: D30288583
Pulled By: bjlemaire
fbshipit-source-id: 7646a545ec56f4715949daa59ab5eee74540feb3
Summary:
PR https://github.com/facebook/rocksdb/issues/5908 added `flush_jobs_info_` to `FlushJob` to make sure
`OnFlushCompleted()` is called after committing flush results to
MANIFEST. However, `flush_jobs_info_` is not updated in atomic
flush, causing `NotifyOnFlushCompleted()` to skip `OnFlushCompleted()`.
This PR fixes this, in a similar way to https://github.com/facebook/rocksdb/issues/5908 that handles regular flush.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/8585
Test Plan: make check
Reviewed By: jay-zhuang
Differential Revision: D29913720
Pulled By: riversand963
fbshipit-source-id: 4ff023c98372fa2c93188d4a5c8a4e9ffa0f4dda
Summary:
The db_stress crash was caused by a call to `IsFlushPending()` made by a stats function which triggered an `assert([false])`, which I didn't plan when I created the `trigger_flush` bool. It turns out that this bool variable is not useful: I created it because I thought the `imm_flush_needed` atomic bool would actually trigger a flush.
It turns out that this bool is only checked in `IsFlushPending` - this is its only use - and a flush is triggered by either a background thread checking on the imm array, or by an explicit call to `SchedulePendingFlush` which creates a flush request, that is then added to a flush request queue.
In this PR, I reverted the MemtableList::Add function to what it was before my changes.
I tested the fix by running the exact command line that deterministically triggered the assert error (see below), which confirmed that this is where the error was coming from.
I also run `db_crashtest.py whitebox` and `blackbox` for a couple hours locally before committing this PR.
Experiment run:
```./db_stress --acquire_snapshot_one_in=0 --allow_concurrent_memtable_write=1 --avoid_flush_during_recovery=0 --avoid_unnecessary_blocking_io=1 --backup_max_size=104857600 --backup_one_in=100000 --batch_protection_bytes_per_key=0 --block_size=16384 --bloom_bits=76.90653425292307 --bottommost_compression_type=disable --cache_index_and_filter_blocks=1 --cache_size=1048576 --checkpoint_one_in=1000000 --checksum_type=kCRC32c --clear_column_family_one_in=0 --column_families=1 --compact_files_one_in=1000000 --compact_range_one_in=0 --compaction_ttl=2 --compression_max_dict_buffer_bytes=0 --compression_max_dict_bytes=0 --compression_parallel_threads=1 --compression_type=zstd --compression_zstd_max_train_bytes=0 --continuous_verification_interval=0 --db=/dev/shm/rocksdb/rocksdb_crashtest_blackbox --db_write_buffer_size=0 --delpercent=4 --delrangepercent=1 --destroy_db_initially=0 --enable_compaction_filter=1 --enable_pipelined_write=0 --expected_values_path=/dev/shm/rocksdb/rocksdb_crashtest_expected --experimental_allow_mempurge=1 --experimental_mempurge_policy=kAlternate --fail_if_options_file_error=1 --file_checksum_impl=none --flush_one_in=1000000 --format_version=2 --get_current_wal_file_one_in=0 --get_live_files_one_in=1000000 --get_property_one_in=1000000 --get_sorted_wal_files_one_in=0 --index_block_restart_interval=14 --index_type=0 --iterpercent=0 --key_len_percent_dist=1,30,69 --level_compaction_dynamic_level_bytes=False --long_running_snapshots=1 --mark_for_compaction_one_file_in=10 --max_background_compactions=1 --max_bytes_for_level_base=67108864 --max_key=100000000 --max_key_len=3 --max_manifest_file_size=1073741824 --max_write_batch_group_size_bytes=64 --max_write_buffer_number=3 --max_write_buffer_size_to_maintain=0 --memtablerep=skip_list --mmap_read=0 --mock_direct_io=True --nooverwritepercent=1 --open_files=-1 --open_metadata_write_fault_one_in=8 --open_read_fault_one_in=32 --open_write_fault_one_in=16 --ops_per_thread=100000000 --optimize_filters_for_memory=1 --paranoid_file_checks=0 --partition_filters=0 --partition_pinning=0 --pause_background_one_in=1000000 --periodic_compaction_seconds=1000 --prefix_size=-1 --prefixpercent=0 --progress_reports=0 --read_fault_one_in=0 --readpercent=60 --recycle_log_file_num=1 --reopen=20 --set_options_one_in=0 --snapshot_hold_ops=100000 --sst_file_manager_bytes_per_sec=104857600 --sst_file_manager_bytes_per_truncate=0 --subcompactions=3 --sync=1 --sync_fault_injection=False --target_file_size_base=16777216 --target_file_size_multiplier=1 --test_batches_snapshots=0 --top_level_index_pinning=1 --unpartitioned_pinning=3 --use_clock_cache=0 --use_direct_io_for_flush_and_compaction=1 --use_direct_reads=0 --use_full_merge_v1=1 --use_merge=0 --use_multiget=0 --use_ribbon_filter=1 --user_timestamp_size=0 --verify_checksum=1 --verify_checksum_one_in=1000000 --verify_db_one_in=100000 --write_buffer_size=33554432 --write_dbid_to_manifest=1 --writepercent=35```
Pull Request resolved: https://github.com/facebook/rocksdb/pull/8604
Reviewed By: pdillinger
Differential Revision: D30047295
Pulled By: bjlemaire
fbshipit-source-id: b9e379bfa3d6b9bd2b275725fb0bca4bd81a3dbe
Summary:
The main challenge to make the memtable garbage collection prototype (nicknamed `mempurge`) was to not get rid of WAL files that contain unflushed (but mempurged) data. That was successfully guaranteed by not writing the VersionEdit to the MANIFEST file after a successful mempurge.
By not writing VersionEdits to the `MANIFEST` file after a succesful mempurge operation, we do not change the earliest log file number that contains unflushed data: `cfd->GetLogNumber()` (`cfd->SetLogNumber()` is only called in `VersionSet::ProcessManifestWrites`). As a result, a number of functions introduced earlier just for the mempurge operation are not obscolete/redundant. (e.g.: `FlushJob::ExtractEarliestLogFileNumber`), and this PR aims at cleaning up all these now-unnecessary functions. In particular, we no longer need to store the earliest log file number in the `MemTable` struct itself. This PR therefore also reverts the `MemTable` struct to its original form.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/8558
Test Plan: Already included in `db_flush_test.cc`.
Reviewed By: anand1976
Differential Revision: D29764351
Pulled By: bjlemaire
fbshipit-source-id: 0f43b260fa270251862512f397d3f24ee62e8437
Summary:
In this PR, `mempurge` is made compatible with the Write Ahead Log: in case of recovery, the DB is now capable of recovering the data that was "mempurged" and kept in the `imm()` list of immutable memtables.
The twist was to add a uint64_t to the `memtable` struct to store the number of the earliest log file containing entries from the `memtable`. When a `Flush` operation is replaced with a `MemPurge`, the `VersionEdit` (which usually contains the new min log file number to pick up for recovery and the level 0 file path of the newly created SST file) is no longer appended to the manifest log, and every time the `deleteWal` method is called, a check is made on the list of immutable memtables.
This PR also includes a unit test that verifies that no data is lost upon Reopening of the database when the mempurge feature is activated. This extensive unit test includes two column families, with valid data contained in the imm() at time of "crash"/reopening (recovery).
Pull Request resolved: https://github.com/facebook/rocksdb/pull/8528
Reviewed By: pdillinger
Differential Revision: D29701097
Pulled By: bjlemaire
fbshipit-source-id: 072a900fb6ccc1edcf5eef6caf88f3060238edf9
Summary:
In https://github.com/facebook/rocksdb/issues/8454, I introduced a new process baptized `MemPurge` (memtable garbage collection). This new PR is built upon this past mempurge prototype.
In this PR, I made the `mempurge` process a background task, which provides superior performance since the mempurge process does not cling on the db_mutex anymore, and addresses severe restrictions from the past iteration (including a scenario where the past mempurge was failling, when a memtable was mempurged but was still referred to by an iterator/snapshot/...).
Now the mempurge process ressembles an in-memory compaction process: the stack of immutable memtables is filtered out, and the useful payload is used to populate an output memtable. If the output memtable is filled at more than 60% capacity (arbitrary heuristic) the mempurge process is aborted and a regular flush process takes place, else the output memtable is kept in the immutable memtable stack. Note that adding this output memtable to the `imm()` memtable stack does not trigger another flush process, so that the flush thread can go to sleep at the end of a successful mempurge.
MemPurge is activated by making the `experimental_allow_mempurge` flag `true`. When activated, the `MemPurge` process will always happen when the flush reason is `kWriteBufferFull`.
The 3 unit tests confirm that this process supports `Put`, `Get`, `Delete`, `DeleteRange` operators and is compatible with `Iterators` and `CompactionFilters`.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/8505
Reviewed By: pdillinger
Differential Revision: D29619283
Pulled By: bjlemaire
fbshipit-source-id: 8a99bee76b63a8211bff1a00e0ae32360aaece95
Summary:
The patch adds initial support for reading blobs to the batched `MultiGet` API.
The current implementation simply retrieves the blob values as the blob indexes
are encountered; that is, reads from blob files are currently not batched. (This
will be optimized in a separate phase.) In addition, the patch removes some dead
code related to BlobDB from the batched `MultiGet` implementation, namely the
`is_blob` / `is_blob_index` flags that are passed around in `DBImpl` and `MemTable` /
`MemTableListVersion`. These were never hooked up to anything and wouldn't
work anyways, since a single flag is not sufficient to communicate the "blobness"
of multiple key-values.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7766
Test Plan: `make check`
Reviewed By: jay-zhuang
Differential Revision: D25479290
Pulled By: ltamasi
fbshipit-source-id: 7aba2d290e31876ee592bcf1adfd1018713a8000
Summary:
min_wal_number_to_keep should not be decreasing, if it does not increase, then there is no need to log the WAL obsoletions in MANIFEST since a previous one has been logged.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7765
Test Plan: watch existing tests and stress tests to pass
Reviewed By: pdillinger
Differential Revision: D25462542
Pulled By: cheng-chang
fbshipit-source-id: 0085fcb6edf5cf2b0fc32f9932a7566f508768ff
Summary:
Currently, when a WAL becomes obsolete after flushing, if VersionSet::WalSet does not contain the WAL, we do not track the WAL obsoletion event in MANIFEST.
But consider this case:
* WAL 10 is synced, a VersionEdit is LogAndApplied to MANIFEST to log this WAL addition event, but the VersionEdit is not applied to WalSet yet since its corresponding ManifestWriter is still pending in the write queue;
* Since the above ManifestWriter is blocking, the LogAndApply will block on a conditional variable and release the db mutex, so another LogAndApply can proceed to enqueue other VersionEdits concurrently;
* Now flush happens, and WAL 10 becomes obsolete, although WalSet does not contain WAL 10 yet, we should call LogAndApply to enqueue a VersionEdit to indicate the obsoletion of WAL 10;
* otherwise, when the queued edit indicating WAL 10 addition is logged to MANIFEST, and DB crashes and reopens, the WAL 10 might have been removed from disk, but it still exists in MANIFEST.
This PR changes the behavior to: always `LogAndApply` any WAL addition or obsoletion event, without considering the order issues caused by concurrency, but when applying the edits to `WalSet`, do not add the WALs if they are already obsolete. In this approach, the logical events of WAL addition and obsoletion are always tracked in MANIFEST, so we can inspect the MANIFEST and know all the previous WAL events, but we choose to ignore certain events due to the concurrency issues such as the case above, or the case in https://github.com/facebook/rocksdb/pull/7725.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7759
Test Plan: make check
Reviewed By: pdillinger
Differential Revision: D25423089
Pulled By: cheng-chang
fbshipit-source-id: 9cb9a7fbc1875bf954f2a42f9b6cfd6d49a7b21c
Summary:
Consider the case:
1. All column families are flushed, so all WALs become obsolete, but no WAL is removed from disk yet because the removal is asynchronous, a VersionEdit is written to MANIFEST indicating that WALs before a certain WAL number are obsolete, let's say this number is 3;
2. `SyncWAL` is called, so all the on-disk WALs are synced, and if track_and_verify_wal_in_manifest=true, the WALs will be tracked in MANIFEST, let's say the WAL numbers are 1 and 2;
3. DB crashes;
4. During DB recovery, when replaying MANIFEST, we first see that WAL with number < 3 are obsolete, then we see that WAL 1 and 2 are synced, so according to current implementation of `WalSet`, the `WalSet` will be recovered to include WAL 1 and 2;
5. WAL 1 and 2 are asynchronously deleted from disk, then the WAL verification algorithm fails with `Corruption: missing WAL`.
The above case is reproduced in a new unit test `DBBasicTestTrackWal::DoNotTrackObsoleteWal`.
The fix is to maintain the upper bound of the obsolete WAL numbers, any WAL with number less than the maintained number is considered to be obsolete, so shouldn't be tracked even if they are later synced. The number is maintained in `WalSet`.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7725
Test Plan:
1. a new unit test `DBBasicTestTrackWal::DoNotTrackObsoleteWal` is added.
2. run `make crash_test` on devserver.
Reviewed By: riversand963
Differential Revision: D25238914
Pulled By: cheng-chang
fbshipit-source-id: f5dccd57c3d89f19565ec5731f2d42f06d272b72
Summary:
When 2 phase commit is enabled, if there are prepared data in a WAL, the WAL should be kept, the minimum log number for such a WAL is written to MANIFEST during flush. In atomic flush, such information is not written to MANIFEST.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7570
Test Plan: Added a new unit test `DBAtomicFlushTest.ManualFlushUnder2PC`, this test fails in atomic flush without this PR, after this PR, it succeeds.
Reviewed By: riversand963
Differential Revision: D24394222
Pulled By: cheng-chang
fbshipit-source-id: 60ce74b21b704804943be40c8de01b41269cf116
Summary:
https://github.com/facebook/rocksdb/issues/7340 reports and reproduces an assertion failure caused by a combination of the following:
- atomic flush is disabled.
- a column family can appear multiple times in the flush queue at the same time. This behavior was introduced in release 5.17.
Consequently, it is possible that two flushes race with each other. One bg flush thread flushes all memtables. The other thread calls `FlushMemTableToOutputFile()` afterwards, and hits the assertion error below.
```
assert(cfd->imm()->NumNotFlushed() != 0);
assert(cfd->imm()->IsFlushPending());
```
Fix this by reverting the behavior. In non-atomic-flush case, a column family can appear in the flush queue at most once at the same time.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7362
Test Plan:
make check
Also run stress test successfully for 10 times.
```
make crash_test
```
Reviewed By: ajkr
Differential Revision: D25172996
Pulled By: riversand963
fbshipit-source-id: f1559b6366cc609e961e3fc83fae548f1fad08ce
Summary:
The logic for computing min_log_number_to_keep in atomic flush was incorrect.
For example, when all column families are flushed, the min_log_number_to_keep should be the latest new log. But the incorrect logic calls `PrecomputeMinLogNumberToKeepNon2PC` for each column family, and returns the minimum of them. However, `PrecomputeMinLogNumberToKeepNon2PC(cf)` assumes column families other than `cf` are flushed, but in case all column families are flushed, this assumption is incorrect.
Without this fix, the WAL referenced by the computed min_log_number_to_keep may actually contain no unflushed data, so the WAL might have actually been deleted from disk on recovery, then an incorrect error `Corruption: missing WAL` will be reported.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7660
Test Plan:
run `make crash_test_with_atomic_flush` on devserver
added a unit test in `db_flush_test`
Reviewed By: riversand963
Differential Revision: D24906265
Pulled By: cheng-chang
fbshipit-source-id: 08deda62e71f67f59e3b7925cdd86dd09bd4f430
Summary:
When a WAL is synced, an edit is written to MANIFEST.
After flushing memtables, the obsoleted WALs are piggybacked to MANIFEST while writing the new L0 files to MANIFEST.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7601
Test Plan:
`track_and_verify_wals_in_manifest` is enabled by default for all tests extending `DBBasicTest`, and in db_stress_test.
Unit test `wal_edit_test`, `version_edit_test`, and `version_set_test` are also updated.
Watch all tests to pass.
Reviewed By: ltamasi
Differential Revision: D24553957
Pulled By: cheng-chang
fbshipit-source-id: 66a569ff1bdced38e22900bd240b73113906e040
Summary:
Currently, the following interleaving of events can lead to SuperVersion containing both immutable memtables as well as the resulting L0. This can cause Get to return incorrect result if there are merge operands. This may also affect other operations such as single deletes.
```
time main_thr bg_flush_thr bg_compact_thr compact_thr set_opts_thr
0 | WriteManifest:0
1 | issue compact
2 | wait
3 | Merge(counter)
4 | issue flush
5 | wait
6 | WriteManifest:1
7 | wake up
8 | write manifest
9 | wake up
10 | Get(counter)
11 | remove imm
V
```
The reason behind is that: one bg flush thread's installing new `Version` can be batched and performed by another thread that is the "leader" MANIFEST writer. This bg thread removes the memtables from current super version only after `LogAndApply` returns. After the leader MANIFEST writer signals (releasing mutex) this bg flush thread, it is possible that another thread sees this cf with both memtables (whose data have been flushed to the newest L0) and the L0 before this bg flush thread removes the memtables.
To address this issue, each bg flush thread can pass a callback function to `LogAndApply`. The callback is responsible for removing the memtables. Therefore, the leader MANIFEST writer can call this callback and remove the memtables before releasing the mutex.
Test plan (devserver)
```
$make merge_test
$./merge_test --gtest_filter=MergeTest.MergeWithCompactionAndFlush
$make check
```
Pull Request resolved: https://github.com/facebook/rocksdb/pull/6069
Reviewed By: cheng-chang
Differential Revision: D18790894
Pulled By: riversand963
fbshipit-source-id: e41bd600c0448b4f4b2deb3f7677f95e3076b4ed
Summary:
The patch adds support for writing blob files during flush by integrating
`BlobFileBuilder` with the flush logic, most importantly, `BuildTable` and
`CompactionIterator`. If `enable_blob_files` is set, large values are extracted
to blob files and replaced with references. The resulting blob files are then
logged to the MANIFEST as part of the flush job's `VersionEdit` and
added to the `Version`, similarly to table files. Errors related to writing
blob files fail the flush, and any blob files written by such jobs are immediately
deleted (again, similarly to how SST files are handled). In addition, the patch
extends the logging and statistics around flushes to account for the presence
of blob files (e.g. `InternalStats::CompactionStats::bytes_written`, which is
used for calculating write amplification, now considers the blob files as well).
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7345
Test Plan: Tested using `make check` and `db_bench`.
Reviewed By: riversand963
Differential Revision: D23506369
Pulled By: ltamasi
fbshipit-source-id: 646885f22dfbe063f650d38a1fedc132f499a159
Summary:
When a memtable is trimmed in MemTableListVersion, the memtable
is only added to delete list if it is
the last reference. However it is not the last reference as it is held
by the super version. But the super version would not be switched if the
delete list is empty. So the memtable is never destroyed and memory
usage increases beyond write_buffer_size +
max_write_buffer_size_to_maintain.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7296
Test Plan:
1. ./db_bench -benchmarks=randomtransaction
-optimistic_transaction_db=1 -statistics -stats_interval_seconds=1
-duration=90 -num=500000 --max_write_buffer_size_to_maintain=16000000
--transaction_set_snapshot
Reviewed By: ltamasi
Differential Revision: D23267395
Pulled By: akankshamahajan15
fbshipit-source-id: 3a8d437fe9f4015f851ff84c0e29528aa946b650
Summary:
After https://github.com/facebook/rocksdb/issues/6949 , VersionSet::io_status_ can be concurrently accessed by multiple
threads without lock, causing tsan test to fail. For example, a bg flush thread
resets io_status_ before calling LogAndApply(), while another thread already in
the process of LogAndApply() reads io_status_. This is a bug.
We do not have to reset io_status_ each time we call LogAndApply(). io_status_
is part of the state of VersionSet, and it indicates the outcome of preceding
MANIFEST/CURRENT files IO operations. Its value should be updated only when:
1. MANIFEST/CURRENT files IO fail for the first time.
2. MANIFEST/CURRENT files IO succeed as part of recovering from a prior
failure without process restart, e.g. calling Resume().
Test Plan (devserver):
COMPILE_WITH_TSAN=1 make check
COMPILE_WITH_TSAN=1 make db_test2
./db_test2 --gtest_filter=DBTest2.CompactionStall
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7034
Reviewed By: zhichao-cao
Differential Revision: D22247137
Pulled By: riversand963
fbshipit-source-id: 77b83e05390f3ee3cd2d96d3fdd6fe4f225e3216
Summary:
This PR provides preliminary support for handling IO error during MANIFEST write.
File write/sync is not guaranteed to be atomic. If we encounter an IOError while writing/syncing to the MANIFEST file, we cannot be sure about the state of the MANIFEST file. The version edits may or may not have reached the file. During cleanup, if we delete the newly-generated SST files referenced by the pending version edit(s), but the version edit(s) actually are persistent in the MANIFEST, then next recovery attempt will process the version edits(s) and then fail since the SST files have already been deleted.
One approach is to truncate the MANIFEST after write/sync error, so that it is safe to delete the SST files. However, file truncation may not be supported on certain file systems. Therefore, we take the following approach.
If an IOError is detected during MANIFEST write/sync, we disable file deletions for the faulty database. Depending on whether the IOError is retryable (set by underlying file system), either RocksDB or application can call `DB::Resume()`, or simply shutdown and restart. During `Resume()`, RocksDB will try to switch to a new MANIFEST and write all existing in-memory version storage in the new file. If this succeeds, then RocksDB may proceed. If all recovery is completed, then file deletions will be re-enabled.
Note that multiple threads can call `LogAndApply()` at the same time, though only one of them will be going through the process MANIFEST write, possibly batching the version edits of other threads. When the leading MANIFEST writer finishes, all of the MANIFEST writing threads in this batch will have the same IOError. They will all call `ErrorHandler::SetBGError()` in which file deletion will be disabled.
Possible future directions:
- Add an `ErrorContext` structure so that it is easier to pass more info to `ErrorHandler`. Currently, as in this example, a new `BackgroundErrorReason` has to be added.
Test plan (dev server):
make check
Pull Request resolved: https://github.com/facebook/rocksdb/pull/6949
Reviewed By: anand1976
Differential Revision: D22026020
Pulled By: riversand963
fbshipit-source-id: f3c68a2ef45d9b505d0d625c7c5e0c88495b91c8
Summary:
Confusing checks for null that are never null
Pull Request resolved: https://github.com/facebook/rocksdb/pull/6933
Test Plan: make check
Reviewed By: cheng-chang
Differential Revision: D21885466
Pulled By: pdillinger
fbshipit-source-id: 4b48e03c2a33727f2702b0d12292f9fda5a3c475
Summary:
In the current code base, we use Status to get and store the returned status from the call. Specifically, for IO related functions, the current Status cannot reflect the IO Error details such as error scope, error retryable attribute, and others. With the implementation of https://github.com/facebook/rocksdb/issues/5761, we have the new Wrapper for IO, which returns IOStatus instead of Status. However, the IOStatus is purged at the lower level of write path and transferred to Status.
The first job of this PR is to pass the IOStatus to the write path (flush, WAL write, and Compaction). The second job is to identify the Retryable IO Error as HardError, and set the bg_error_ as HardError. In this case, the DB Instance becomes read only. User is informed of the Status and need to take actions to deal with it (e.g., call db->Resume()).
Pull Request resolved: https://github.com/facebook/rocksdb/pull/6487
Test Plan: Added the testing case to error_handler_fs_test. Pass make asan_check
Reviewed By: anand1976
Differential Revision: D20685017
Pulled By: zhichao-cao
fbshipit-source-id: ff85f042896243abcd6ef37877834e26f36b6eb0
Summary:
In the current code base, we can use Directory from Env to manage directory (e.g, Fsync()). The PR https://github.com/facebook/rocksdb/issues/5761 introduce the File System as a new Env API. So we further replace the Directory class in DB with FSDirectory such that we can have more IO information from IOStatus returned by FSDirectory.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/6468
Test Plan: pass make asan_check
Differential Revision: D20195261
Pulled By: zhichao-cao
fbshipit-source-id: 93962cb9436852bfcfb76e086d9e7babd461cbe1
Summary:
Added new Get() methods that return timestamp. Dummy implementation is given so that classes derived from DB don't need to be touched to provide their implementation. MultiGet is not included.
ReadRandom perf test (10 minutes) on the same development machine ram drive with the same DB data shows no regression (within marge of error). The test is adapted from https://github.com/facebook/rocksdb/wiki/RocksDB-In-Memory-Workload-Performance-Benchmarks.
base line (commit 72ee067b9):
101.712 micros/op 314602 ops/sec; 36.0 MB/s (5658999 of 5658999 found)
This PR:
100.288 micros/op 319071 ops/sec; 36.5 MB/s (5674999 of 5674999 found)
./db_bench --db=r:\rocksdb.github --num_levels=6 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --cache_size=2147483648 --cache_numshardbits=6 --compression_type=none --compression_ratio=1 --min_level_to_compress=-1 --disable_seek_compaction=1 --hard_rate_limit=2 --write_buffer_size=134217728 --max_write_buffer_number=2 --level0_file_num_compaction_trigger=8 --target_file_size_base=134217728 --max_bytes_for_level_base=1073741824 --disable_wal=0 --wal_dir=r:\rocksdb.github\WAL_LOG --sync=0 --verify_checksum=1 --delete_obsolete_files_period_micros=314572800 --max_background_compactions=4 --max_background_flushes=0 --level0_slowdown_writes_trigger=16 --level0_stop_writes_trigger=24 --statistics=0 --stats_per_interval=0 --stats_interval=1048576 --histogram=0 --use_plain_table=1 --open_files=-1 --mmap_read=1 --mmap_write=0 --memtablerep=prefix_hash --bloom_bits=10 --bloom_locality=1 --duration=600 --benchmarks=readrandom --use_existing_db=1 --num=25000000 --threads=32
Pull Request resolved: https://github.com/facebook/rocksdb/pull/6409
Differential Revision: D20200086
Pulled By: riversand963
fbshipit-source-id: 490edd74d924f62bd8ae9c29c2a6bbbb8410ca50
Summary:
When dynamically linking two binaries together, different builds of RocksDB from two sources might cause errors. To provide a tool for user to solve the problem, the RocksDB namespace is changed to a flag which can be overridden in build time.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/6433
Test Plan: Build release, all and jtest. Try to build with ROCKSDB_NAMESPACE with another flag.
Differential Revision: D19977691
fbshipit-source-id: aa7f2d0972e1c31d75339ac48478f34f6cfcfb3e
Summary:
https://github.com/facebook/rocksdb/pull/6177 introduced a data race
involving `MemTableList::InstallNewVersion` and `MemTableList::NumFlushed`.
The patch fixes this by caching whether the current version has any
memtable history (i.e. flushed memtables that are kept around for
transaction conflict checking) in an `std::atomic<bool>` member called
`current_has_history_`, similarly to how `current_memory_usage_excluding_last_`
is handled.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/6187
Test Plan:
```
make clean
COMPILE_WITH_TSAN=1 make db_test -j24
./db_test
```
Differential Revision: D19084059
Pulled By: ltamasi
fbshipit-source-id: 327a5af9700fb7102baea2cc8903c085f69543b9
Summary:
We have observed an increase in CPU load caused by frequent calls to
`ColumnFamilyData::InstallSuperVersion` from `DBImpl::TrimMemtableHistory`
when using `max_write_buffer_size_to_maintain` to limit the amount of
memtable history maintained for transaction conflict checking. Part of the issue
is that trimming can potentially be scheduled even if there is no memtable
history. The patch adds a check that fixes this.
See also https://github.com/facebook/rocksdb/pull/6169.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/6177
Test Plan:
Compared `perf` output for
```
./db_bench -benchmarks=randomtransaction -optimistic_transaction_db=1 -statistics -stats_interval_seconds=1 -duration=90 -num=500000 --max_write_buffer_size_to_maintain=16000000 --transaction_set_snapshot=1 --threads=32
```
before and after the change. There is a significant reduction for the call chain
`rocksdb::DBImpl::TrimMemtableHistory` -> `rocksdb::ColumnFamilyData::InstallSuperVersion` ->
`rocksdb::ThreadLocalPtr::StaticMeta::Scrape` even without https://github.com/facebook/rocksdb/pull/6169.
Differential Revision: D19057445
Pulled By: ltamasi
fbshipit-source-id: dff81882d7b280e17eda7d9b072a2d4882c50f79
Summary:
Read keys from a snapshot that a range deletion were added after the snapshot was created and this range deletion was inside an immutable memtable, we will get wrong key set.
More detail rest in codes.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/6062
Differential Revision: D18966785
Pulled By: pdillinger
fbshipit-source-id: 38a60bb1e2d0a1dbfc8ec641617200b6a02b86c3
Summary:
Fix issue https://github.com/facebook/rocksdb/issues/6012.
I found that it may be caused by the following codes in function _RemoveOldMemTables()_ in **db/memtable_list.cc** :
```
for (auto it = memlist.rbegin(); it != memlist.rend(); ++it) {
MemTable* mem = *it;
if (mem->GetNextLogNumber() > log_number) {
break;
}
current_->Remove(mem, to_delete);
```
The iterator **it** turns invalid after `current_->Remove(mem, to_delete);`
Pull Request resolved: https://github.com/facebook/rocksdb/pull/6013
Test Plan:
```
make check
```
Differential Revision: D18401107
Pulled By: riversand963
fbshipit-source-id: bf0da3b868ed70f7aff24cf7b3e2049c0c5c7a4e
Summary:
When there are concurrent flush job on the same CF, `OnFlushCompleted` can be called before the flush result being install to LSM. Fixing the issue by passing `FlushJobInfo` through `MemTable`, and the thread who commit the flush result can fetch the `FlushJobInfo` and fire `OnFlushCompleted` on behave of the thread actually writing the SST.
Fix https://github.com/facebook/rocksdb/issues/5892
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5908
Test Plan: Add new test. The test will fail without the fix.
Differential Revision: D17916144
Pulled By: riversand963
fbshipit-source-id: e18df67d9533b5baee52ae3605026cdeb05cbe10
Summary:
RocksDB has a MultiGet() API that implements batched key lookup for higher performance (https://github.com/facebook/rocksdb/blob/master/include/rocksdb/db.h#L468). Currently, batching is implemented in BlockBasedTableReader::MultiGet() for SST file lookups. One of the ways it improves performance is by pipelining bloom filter lookups (by prefetching required cachelines for all the keys in the batch, and then doing the probe) and thus hiding the cache miss latency. The same concept can be extended to the memtable as well. This PR involves implementing a pipelined bloom filter lookup in DynamicBloom, and implementing MemTable::MultiGet() that can leverage it.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5818
Test Plan:
Existing tests
Performance Test:
Ran the below command which fills up the memtable and makes sure there are no flushes and then call multiget. Ran it on master and on the new change and see atleast 1% performance improvement across all the test runs I did. Sometimes the improvement was upto 5%.
TEST_TMPDIR=/data/users/$USER/benchmarks/feature/ numactl -C 10 ./db_bench -benchmarks="fillseq,multireadrandom" -num=600000 -compression_type="none" -level_compaction_dynamic_level_bytes -write_buffer_size=200000000 -target_file_size_base=200000000 -max_bytes_for_level_base=16777216 -reads=90000 -threads=1 -compression_type=none -cache_size=4194304000 -batch_size=32 -disable_auto_compactions=true -bloom_bits=10 -cache_index_and_filter_blocks=true -pin_l0_filter_and_index_blocks_in_cache=true -multiread_batched=true -multiread_stride=4 -statistics -memtable_whole_key_filtering=true -memtable_bloom_size_ratio=10
Differential Revision: D17578869
Pulled By: vjnadimpalli
fbshipit-source-id: 23dc651d9bf49db11d22375bf435708875a1f192
Summary:
MyRocks currently sets `max_write_buffer_number_to_maintain` in order to maintain enough history for transaction conflict checking. The effectiveness of this approach depends on the size of memtables. When memtables are small, it may not keep enough history; when memtables are large, this may consume too much memory.
We are proposing a new way to configure memtable list history: by limiting the memory usage of immutable memtables. The new option is `max_write_buffer_size_to_maintain` and it will take precedence over the old `max_write_buffer_number_to_maintain` if they are both set to non-zero values. The new option accounts for the total memory usage of flushed immutable memtables and mutable memtable. When the total usage exceeds the limit, RocksDB may start dropping immutable memtables (which is also called trimming history), starting from the oldest one.
The semantics of the old option actually works both as an upper bound and lower bound. History trimming will start if number of immutable memtables exceeds the limit, but it will never go below (limit-1) due to history trimming.
In order the mimic the behavior with the new option, history trimming will stop if dropping the next immutable memtable causes the total memory usage go below the size limit. For example, assuming the size limit is set to 64MB, and there are 3 immutable memtables with sizes of 20, 30, 30. Although the total memory usage is 80MB > 64MB, dropping the oldest memtable will reduce the memory usage to 60MB < 64MB, so in this case no memtable will be dropped.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5022
Differential Revision: D14394062
Pulled By: miasantreble
fbshipit-source-id: 60457a509c6af89d0993f988c9b5c2aa9e45f5c5
Summary:
This is a new API added to db.h to allow for fetching all merge operands associated with a Key. The main motivation for this API is to support use cases where doing a full online merge is not necessary as it is performance sensitive. Example use-cases:
1. Update subset of columns and read subset of columns -
Imagine a SQL Table, a row is encoded as a K/V pair (as it is done in MyRocks). If there are many columns and users only updated one of them, we can use merge operator to reduce write amplification. While users only read one or two columns in the read query, this feature can avoid a full merging of the whole row, and save some CPU.
2. Updating very few attributes in a value which is a JSON-like document -
Updating one attribute can be done efficiently using merge operator, while reading back one attribute can be done more efficiently if we don't need to do a full merge.
----------------------------------------------------------------------------------------------------
API :
Status GetMergeOperands(
const ReadOptions& options, ColumnFamilyHandle* column_family,
const Slice& key, PinnableSlice* merge_operands,
GetMergeOperandsOptions* get_merge_operands_options,
int* number_of_operands)
Example usage :
int size = 100;
int number_of_operands = 0;
std::vector<PinnableSlice> values(size);
GetMergeOperandsOptions merge_operands_info;
db_->GetMergeOperands(ReadOptions(), db_->DefaultColumnFamily(), "k1", values.data(), merge_operands_info, &number_of_operands);
Description :
Returns all the merge operands corresponding to the key. If the number of merge operands in DB is greater than merge_operands_options.expected_max_number_of_operands no merge operands are returned and status is Incomplete. Merge operands returned are in the order of insertion.
merge_operands-> Points to an array of at-least merge_operands_options.expected_max_number_of_operands and the caller is responsible for allocating it. If the status returned is Incomplete then number_of_operands will contain the total number of merge operands found in DB for key.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5604
Test Plan:
Added unit test and perf test in db_bench that can be run using the command:
./db_bench -benchmarks=getmergeoperands --merge_operator=sortlist
Differential Revision: D16657366
Pulled By: vjnadimpalli
fbshipit-source-id: 0faadd752351745224ee12d4ae9ef3cb529951bf
Summary:
When using `PRIu64` type of printf specifier, current code base does the following:
```
#ifndef __STDC_FORMAT_MACROS
#define __STDC_FORMAT_MACROS
#endif
#include <inttypes.h>
```
However, this can be simplified to
```
#include <cinttypes>
```
as long as flag `-std=c++11` is used.
This should solve issues like https://github.com/facebook/rocksdb/issues/5159
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5402
Differential Revision: D15701195
Pulled By: miasantreble
fbshipit-source-id: 6dac0a05f52aadb55e9728038599d3d2e4b59d03
Summary:
Many logging related source files are under util/. It will be more structured if they are together.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5387
Differential Revision: D15579036
Pulled By: siying
fbshipit-source-id: 3850134ed50b8c0bb40a0c8ae1f184fa4081303f
Summary:
There are too many types of files under util/. Some test related files don't belong to there or just are just loosely related. Mo
ve them to a new directory test_util/, so that util/ is cleaner.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5377
Differential Revision: D15551366
Pulled By: siying
fbshipit-source-id: 0f5c8653832354ef8caa31749c0143815d719e2c
Summary:
RocksDB secondary can replay both MANIFEST and WAL now.
On the one hand, the memory usage by memtables will grow after replaying WAL for sometime. On the other hand, replaying the MANIFEST can bring the database persistent data to a more recent point in time, giving us the opportunity to discard some memtables containing out-dated data.
This PR coordinates the MANIFEST and WAL replay, using the updates from MANIFEST replay to update the active memtable and immutable memtable list of each column family.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5305
Differential Revision: D15386512
Pulled By: riversand963
fbshipit-source-id: a3ea6fc415f8382d8cf624f52a71ebdcffa3e355
Summary:
Previous code may call `~ColumnFamilyData` in `DBImpl::AtomicFlushMemTablesToOutputFiles` if the column family is dropped or `cfd->IsFlushPending() == false`. In `~ColumnFamilyData`, the db mutex is released briefly and re-acquired. This can cause correctness issue. The reason is as follows.
Assume there are more bg flush threads. After bg_flush_thr1 releases the db mutex, bg_flush_thr2 can grab it and pop an element from the flush queue. This will cause bg_flush_thr2 to accidentally pick some memtables which should have been picked by bg_flush_thr1. To make the matter worse, bg_flush_thr2 can clear `flush_requested_` flag for the memtable list, causing a subsequent call to `MemTableList::IsFlushPending()` by bg_flush_thr1 to return false, which is wrong.
The fix is to delay `ColumnFamilyData::Unref` and `~ColumnFamilyData` for column families not selected for flush until `AtomicFlushMemTablesToOutputFiles` returns. Furthermore, a bg flush thread should not clear `MemTableList::flush_requested_` in `MemTableList::PickMemtablesToFlush` unless atomic flush is not used **or** the memtable list does not have unpicked memtables.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5294
Differential Revision: D15295297
Pulled By: riversand963
fbshipit-source-id: 03b101205ca22c242647cbf488bcf0ed80b2ecbd
Summary:
if an operation just involves a single column family, then we do
not have to set the kInAtomicGroup tag when writing to MANIFEST. This change
can fix a compatibility test failure, i.e. 5.15 and earlier cannot recognize
kInAtomicGroup tag.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4981
Differential Revision: D14072687
Pulled By: riversand963
fbshipit-source-id: 46b0c61e399f16c6b7169de0b33430d0ed90d6d4
Summary:
1. this commit fixes our handling of a combination of two separate edge
cases. If a flush job does not pick any memtable to flush (because another
flush job has already picked the same memtables), and the column family
assigned to the flush job is dropped right before RocksDB calls
rocksdb::InstallMemtableAtomicFlushResults, our original code passes
a FileMetaData object whose file number is 0, failing the assertion in
rocksdb::InstallMemtableAtomicFlushResults (assert(m->GetFileNumber() > 0)).
2. Also piggyback a small change: since we already create a local copy of column family's mutable CF options to eliminate potential race condition with `SetOptions` call, we might as well use the local copy in other function calls in the same scope.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4932
Differential Revision: D13901322
Pulled By: riversand963
fbshipit-source-id: b936580af7c127ea0c6c19ea10cd5fcede9fb0f9
Summary:
as titled.
Since different bg flush threads can flush different sets of column families
(due to column family creation and drop), we decide not to let one thread
perform atomic flush result installation for other threads. Bg flush threads
will install their atomic flush results sequentially to MANIFEST, using
a conditional variable, i.e. atomic_flush_install_cv_ to coordinate.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4791
Differential Revision: D13498930
Pulled By: riversand963
fbshipit-source-id: dd7482fc41f4bd22dad1e1ef7d4764ef424688d7
Summary:
Now that v2 is fully functional, the v1 aggregator is removed.
The v2 aggregator has been renamed.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4778
Differential Revision: D13495930
Pulled By: abhimadan
fbshipit-source-id: 9d69500a60a283e79b6c4fa938fc68a8aa4d40d6
Summary:
If one column family is dropped, we should simply skip it and continue to flush
other active ones.
Currently we use Status::ShutdownInProgress to notify caller of column families
being dropped. In the future, we should consider using a different Status code.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4708
Differential Revision: D13378954
Pulled By: riversand963
fbshipit-source-id: 42f248cdf2d32d4c0f677cd39012694b8f1328ca