Commit graph

477 commits

Author SHA1 Message Date
Peter Dillinger 9f4c597d83 FaultInjectionTestFS read unsynced data by default (#12729)
Summary:
In places (e.g. GetSortedWals()) RocksDB relies on querying the file size or even reading the contents of files currently open for writing, and as in POSIX semantics, expects to see the flushed size and contents regardless of what has been synced. FaultInjectionTestFS historically did not emulate this behavior, only showing synced data from such read operations. (Different from FaultInjectionTestEnv--sigh.)

This change makes the "proper" behavior the default behavior, at least for GetFileSize and FSSequentialFile. However, this new functionality is disabled in db_stress because of undiagnosed, unresolved issues.

Also removes unused and confusing field `pos_at_last_flush_`

This change is needed to support testing a relevant bug fix (in a follow-up diff).  Other suggested follow-up:
* Fix db_stress not to rely on the old behavior, and fix a related FIXME in db_stress_test_base.cc in LockWAL testing.
* Fill in some corner cases in the FileSystem API for reading unsynced data (see new TODO items).
* Consider deprecating and removing Flush() API functions from FileSystem APIs. It is not clear to me that there is a supported scenario in which they do anything but confuse API users and developers. If there is a use for them, it doesn't appear to be tested.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12729

Test Plan: applies to all unit tests successfully, just updating the unit test from https://github.com/facebook/rocksdb/issues/12556 due to relying on the errant behavior. Also added a specific unit test

Reviewed By: hx235

Differential Revision: D58091835

Pulled By: pdillinger

fbshipit-source-id: f47a63b2b000f5875b6293a98577bff663d7fd33
2024-06-04 15:25:23 -07:00
Po-Chuan Hsieh b03d415660 Fix build on i386 (#12719)
Summary:
Cited from https://pkg-status.freebsd.org/beefy21/data/140i386-default/02faf78f4c9b/logs/rocksdb-9.2.1.log
The error message is as follows:
```
mkdir -p db_stress_tool && clang++ -O2 -pipe  -DOS_FREEBSD -fstack-protector-strong -isystem /usr/local/include -fno-strict-aliasing   -Wno-inconsistent-missing-override -Wno-unused-parameter -Wno-unused-variable -Wno-unused-private-field -isystem /usr/local/include -std=c++17   -fPIC -DROCKSDB_DLL -DROCKSDB_USE_RTTI   -g -W -Wextra -Wall -Wsign-compare -Wshadow -Wunused-parameter -Werror -I. -I./include -std=c++17 -O2 -pipe  -DOS_FREEBSD -fstack-protector-strong -isystem /usr/local/include -fno-strict-aliasing   -Wno-inconsistent-missing-override -Wno-unused-parameter -Wno-unused-variable -Wno-unused-private-field -isystem /usr/local/include -std=c++17  -faligned-new -DHAVE_ALIGNED_NEW -DROCKSDB_PLATFORM_POSIX -DROCKSDB_LIB_IO_POSIX -O2 -pipe  -DOS_FREEBSD -fstack-protector-strong -isystem /usr/local/include -fno-strict-aliasing  -D_REENTRANT -DOS_FREEBSD -DSNAPPY -DGFLAGS=1 -DZLIB -DBZIP2 -DLZ4 -DZSTD -DROCKSDB_MALLOC_USABLE_SIZE -DROCKSDB_PTHREAD_ADAPTIVE_MUTEX -DROCKSDB_BACKTRACE -DROCKSDB_SCHED_GETCPU_PRESENT   -isystem third-party/gtest-1.8.1/fused-src -O2 -fno-omit-frame-pointer -momit-leaf-frame-pointer -DNDEBUG -Woverloaded-virtual -Wnon-virtual-dtor -Wno-missing-field-initializers -Wno-invalid-offsetof -c db_stress_tool/db_stress_common.cc -o db_stress_tool/db_stress_common.o
db_stress_tool/db_stress_common.cc:204:17: error: format specifies type 'unsigned long' but the argument has type 'size_t' (aka 'unsigned int') [-Werror,-Wformat]
                block_cache->GetCapacity());
                ^~~~~~~~~~~~~~~~~~~~~~~~~~
1 error generated.
```

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12719

Reviewed By: hx235

Differential Revision: D58093539

Pulled By: jaykorean

fbshipit-source-id: 400cae3a4b0d23b168937a5388065ef1c4b8b56e
2024-06-04 09:36:00 -07:00
Levi Tamasi 023a808417 Disable iterator refresh for CoalescingIterator in TestIterateAgainstExpected (#12723)
Summary:
Pull Request resolved: https://github.com/facebook/rocksdb/pull/12723

`CoalescingIterator` doesn't support `Refresh` currently; the patch adds a check that was missing from https://github.com/facebook/rocksdb/pull/12721 to disable this operation when multi-CF iterators are in use in the stress test.

Reviewed By: jaykorean

Differential Revision: D58053334

fbshipit-source-id: 3146f0e7e87230b49b244cecdfcee345c0ce78fa
2024-06-01 09:52:48 -07:00
Jay Huh f3b7e959b3 Add CoalescingIterator to TestIterateAgainstExpected (#12721)
Summary:
Continuing from https://github.com/facebook/rocksdb/pull/12706. Adding the CoalescingIterator to `TestIterateAgainstExpected` as well when `use_multi_cf_iterator` is set True

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12721

Test Plan:
```
python3 tools/db_crashtest.py blackbox --simple --max_key=25000000 --write_buffer_size=4194304 --use_attribute_group=0 --use_put_entity_one_in=1 --use_multi_get=1 --use_multi_cf_iterator=1 --verify_iterator_with_expected_state_one_in=2
```

Reviewed By: ltamasi

Differential Revision: D58033811

Pulled By: jaykorean

fbshipit-source-id: 7caf39883e277e695b653e295ad72b1004169ca0
2024-05-31 15:17:06 -07:00
Levi Tamasi 6f17056e40 Add transactional/read-your-own-write MultiGetEntity stress test (#12717)
Summary:
Pull Request resolved: https://github.com/facebook/rocksdb/pull/12717

The PR adds `Transaction::MultiGetEntity` to the stress tests. Similarly to what we do for `Transaction::MultiGet`, in this mode we open a transaction and randomly add writes for some of the queried keys to it while keeping track of the values written on a per-key basis. The results of `Transaction::MultiGetEntity` can then be validated against these expected values (in order to test the read-your-own-writes functionality) as well as the results returned by `Transaction::GetEntity` for the same keys.

Reviewed By: jaykorean

Differential Revision: D57990210

fbshipit-source-id: 9bf3bb292051c2c57757f86b517919197b03c524
2024-05-31 12:01:59 -07:00
Jay Huh a901ef48f0 Introduce use_multi_cf_iterator in stress test (#12706)
Summary:
Introduce `use_multi_cf_iterator`, and when it's set, use `CoalescingIterator` in `TestIterate()`. Because all the column families contain the same data in today's Stress Test, we can compare `CoalescingIterator` against any `DBIter` from any of the column families. Currently, coalescing logic verification is done by unit tests, but we can extend the stress test to support different data in different column families in the future.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12706

Test Plan:
```
python3 tools/db_crashtest.py blackbox --simple --max_key=25000000 --write_buffer_size=4194304 --use_attribute_group=0 --use_put_entity_one_in=1 --use_multi_get=1 --use_multi_cf_iterator=1
```

**More PRs to come**
- Use `AttributeGroupIterator` when both `use_multi_cf_iterator` and `use_attribute_group` are true
- Support `Refresh()` in `CoalescingIterator`
- Extend Stress Test to support different data in different CFs (Long-term)

Reviewed By: ltamasi

Differential Revision: D58020247

Pulled By: jaykorean

fbshipit-source-id: 8e2483b85cf2bb0f5a9bb44851601bbf063484ec
2024-05-31 10:50:15 -07:00
Levi Tamasi 01179678b2 Refactor the non-attribute-group/attribute-group code paths in NonBatchedOpsStressTest::TestMultiGetEntity (#12715)
Summary:
Pull Request resolved: https://github.com/facebook/rocksdb/pull/12715

The patch refactors/deduplicates the non-attribute-group and attribute-group code paths in `NonBatchedOpsStressTest::TestMultiGetEntity` by introducing two new generic lambdas `verify_expected_errors` and `check_results` (the latter of which subsumes the existing `handle_results`) that can handle both types of APIs. This change also serves as groundwork for the upcoming transactional `MultiGetEntity` stress tests.

Reviewed By: jaykorean

Differential Revision: D57977700

fbshipit-source-id: 83a18a9e57f46ea92ba07b2f0dca3e9bc353f257
2024-05-30 13:31:25 -07:00
Levi Tamasi b6ea246333 Fix NonBatchOpsStressTest::TestGetEntity by adding fuzzy match (#12711)
Summary:
Pull Request resolved: https://github.com/facebook/rocksdb/pull/12711

The patch adds the missing other half of https://github.com/facebook/rocksdb/pull/12709: when there is no locking in a read test, we have to be more permissive when it comes to values returned by queries. In particular, any expected state value in a small window around the read call should be allowed, and discrepancies in the presence/absence of a key should only be treated as a failure if the key is guaranteed to have not existed/existed during the above window.

Reviewed By: hx235

Differential Revision: D57938678

fbshipit-source-id: cd5c8bc2e014ec12ea4daf441965f3ec2115663e
2024-05-29 17:00:11 -07:00
Levi Tamasi d9316260e4 Remove unneccessary MutexLock from NonBatchedOpStressTest::TestGetEntity (#12709)
Summary:
Pull Request resolved: https://github.com/facebook/rocksdb/pull/12709

This is most likely copypasta from `TestGet` from before https://github.com/facebook/rocksdb/pull/11058 . There is no need to lock the mutex for the key for reads; in fact, doing so is detrimental to test coverage since it locks out concurrent writers.

Reviewed By: jowlyzhang

Differential Revision: D57915207

fbshipit-source-id: eb0dbf6b84e5408b87d96dd47597511996e206a7
2024-05-29 10:16:37 -07:00
Levi Tamasi 5cec4bbcab Support PutEntity as write method in the transactional MultiGet stress test (#12699)
Summary:
Pull Request resolved: https://github.com/facebook/rocksdb/pull/12699

The patch adds `PutEntity` to the potential write operations used in the read-your-own-writes tests for `Transaction::MultiGet`. Note that since the stress test generates wide-column structures which have the value returned by `GenerateValue` in the default column, this does not affect the results returned by the `MultiGet` API (unless we have a bug).

The wide-column entity is generated according to the usual rules based on the value base and the `use_put_entity_one_in` flag. The entire entity structure will be validated by the upcoming stress test for `Transaction::MultiGetEntity`, where we also plan to leverage this logic.

Reviewed By: jowlyzhang

Differential Revision: D57799075

fbshipit-source-id: 5f86c2b2b3ceee8e1b8bf7453c02f1f1b1b00751
2024-05-28 16:54:10 -07:00
Peter Dillinger d2ef70872f Rename, deprecate LogFile and VectorLogPtr (#12695)
Summary:
These names are confusing with `Logger` etc. so moving to `WalFile` etc.

Other small, related name refactorings.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12695

Test Plan: Left most unit tests using old names as an API compatibility test. Non-test code compiles with deprecated names removed. No functional changes.

Reviewed By: ajkr

Differential Revision: D57747458

Pulled By: pdillinger

fbshipit-source-id: 7b77596b9c20d865d43b9dc66c30c8bd2b3b424f
2024-05-28 09:24:49 -07:00
Levi Tamasi bd801bd98c Factor out the RYW transaction building logic into a helper (#12697)
Summary:
Pull Request resolved: https://github.com/facebook/rocksdb/pull/12697

As groundwork for stress testing `Transaction::MultiGetEntity`, the patch factors out the logic for adding transactional writes for some of the keys in a `MultiGet` batch into a separate helper method called `MaybeAddKeyToTxnForRYW`.

Reviewed By: jowlyzhang

Differential Revision: D57791830

fbshipit-source-id: ef347ba6e6e82dfe5cedb4cf67dd6d1503901d89
2024-05-24 14:24:17 -07:00
Changyu Bi fecb10c2fa Improve universal compaction sorted-run trigger (#12477)
Summary:
Universal compaction currently uses `level0_file_num_compaction_trigger` for two purposes:
1. the trigger for checking if there is any compaction to do, and
2. the limit on the number of sorted runs. RocksDB will do compaction to keep the number of sorted runs no more than the value of this option.

This can make the option inflexible. A value that is too small causes higher write amp: more compactions to reduce the number of sorted runs. A value that is too big delays potential compaction work and causes worse read performance. This PR introduce an option `CompactionOptionsUniversal::max_read_amp` for only the second purpose: to specify
the hard limit on the number of sorted runs.

For backward compatibility, `max_read_amp = -1` by default, which means to fallback to the current behavior.
When `max_read_amp > 0`,`level0_file_num_compaction_trigger` will only serve as a trigger to find potential compaction.
When `max_read_amp = 0`, RocksDB will auto-tune the limit on the number of sorted runs. The estimation is based on DB size, write_buffer_size and size_ratio, so it is adaptive to the size change of the DB. See more in `UniversalCompactionBuilder::PickCompaction()`.
Alternatively, users now can configure `max_read_amp` to a very big value and keep `level0_file_num_compaction_trigger` small. This will allow `size_ratio` and `max_size_amplification_percent` to control the number of sorted runs. This essentially disables compactions with reason kUniversalSortedRunNum.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12477

Test Plan:
* new unit test
* existing unit test for default behavior
* updated crash test with the new option
* benchmark:
  * Create a DB that is roughly 24GB in the last level. When `max_read_amp = 0`, we estimate that the DB needs 9 levels to avoid excessive compactions to reduce the number of sorted runs.
  * We then run fillrandom to ingest another 24GB data to compare write amp.
     * case 1: small level0 trigger: `level0_file_num_compaction_trigger=5, max_read_amp=-1`
       * write-amp: 4.8
     * case 2: auto-tune: `level0_file_num_compaction_trigger=5, max_read_amp=0`
       *  write-amp: 3.6
     * case 3: auto-tune with minimal trigger: `level0_file_num_compaction_trigger=1, max_read_amp=0`
       *  write-amp: 3.8
     * case 4: hard-code a good value for trigger: `level0_file_num_compaction_trigger=9`
       * write-amp: 2.8
```
Case 1:
** Compaction Stats [default] **
Level    Files   Size     Score Read(GB)  Rn(GB) Rnp1(GB) Write(GB) Wnew(GB) Moved(GB) W-Amp Rd(MB/s) Wr(MB/s) Comp(sec) CompMergeCPU(sec) Comp(cnt) Avg(sec) KeyIn KeyDrop Rblob(GB) Wblob(GB)
------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
  L0      0/0    0.00 KB   1.0      0.0     0.0      0.0      22.6     22.6       0.0   1.0      0.0    163.2    141.94            111.10       108    1.314       0      0       0.0       0.0
 L45      8/0    1.81 GB   0.0     39.6    11.1     28.5      39.3     10.8       0.0   3.5    209.0    207.3    194.25            191.29        43    4.517    348M  2498K       0.0       0.0
 L46     13/0    3.12 GB   0.0     15.3     9.5      5.8      15.0      9.3       0.0   1.6    203.1    199.3     77.13             75.88        16    4.821    134M  2362K       0.0       0.0
 L47     19/0    4.68 GB   0.0     15.4    10.5      4.9      14.7      9.8       0.0   1.4    204.0    194.9     77.38             76.15         8    9.673    135M  5920K       0.0       0.0
 L48     38/0    9.42 GB   0.0     19.6    11.7      7.9      17.3      9.4       0.0   1.5    206.5    182.3     97.15             95.02         4   24.287    172M    20M       0.0       0.0
 L49     91/0   22.70 GB   0.0      0.0     0.0      0.0       0.0      0.0       0.0   0.0      0.0      0.0      0.00              0.00         0    0.000       0      0       0.0       0.0
 Sum    169/0   41.74 GB   0.0     89.9    42.9     47.0     109.0     61.9       0.0   4.8    156.7    189.8    587.85            549.45       179    3.284    791M    31M       0.0       0.0

Case 2:
** Compaction Stats [default] **
Level    Files   Size     Score Read(GB)  Rn(GB) Rnp1(GB) Write(GB) Wnew(GB) Moved(GB) W-Amp Rd(MB/s) Wr(MB/s) Comp(sec) CompMergeCPU(sec) Comp(cnt) Avg(sec) KeyIn KeyDrop Rblob(GB) Wblob(GB)
------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
  L0      1/0   214.47 MB   1.2      0.0     0.0      0.0      22.6     22.6       0.0   1.0      0.0    164.5    140.81            109.98       108    1.304       0      0       0.0       0.0
 L44      0/0    0.00 KB   0.0      1.3     1.3      0.0       1.2      1.2       0.0   1.0    206.1    204.9      6.24              5.98         3    2.081     11M    51K       0.0       0.0
 L45      4/0   844.36 MB   0.0      7.1     5.4      1.7       7.0      5.4       0.0   1.3    194.6    192.9     37.41             36.00        13    2.878     62M   489K       0.0       0.0
 L46     11/0    2.57 GB   0.0     14.6     9.8      4.8      14.3      9.5       0.0   1.5    193.7    189.8     77.09             73.54        17    4.535    128M  2411K       0.0       0.0
 L47     24/0    5.81 GB   0.0     19.8    12.0      7.8      18.8     11.0       0.0   1.6    191.4    181.1    106.19            101.21         9   11.799    174M  9166K       0.0       0.0
 L48     38/0    9.42 GB   0.0     19.6    11.8      7.9      17.3      9.4       0.0   1.5    197.3    173.6    101.97             97.23         4   25.491    172M    20M       0.0       0.0
 L49     91/0   22.70 GB   0.0      0.0     0.0      0.0       0.0      0.0       0.0   0.0      0.0      0.0      0.00              0.00         0    0.000       0      0       0.0       0.0
 Sum    169/0   41.54 GB   0.0     62.4    40.3     22.1      81.3     59.2       0.0   3.6    136.1    177.2    469.71            423.94       154    3.050    549M    32M       0.0       0.0

Case 3:
** Compaction Stats [default] **
Level    Files   Size     Score Read(GB)  Rn(GB) Rnp1(GB) Write(GB) Wnew(GB) Moved(GB) W-Amp Rd(MB/s) Wr(MB/s) Comp(sec) CompMergeCPU(sec) Comp(cnt) Avg(sec) KeyIn KeyDrop Rblob(GB) Wblob(GB)
------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
  L0      0/0    0.00 KB   5.0      0.0     0.0      0.0      22.6     22.6       0.0   1.0      0.0    163.8    141.43            111.13       108    1.310       0      0       0.0       0.0
 L44      0/0    0.00 KB   0.0      0.8     0.8      0.0       0.8      0.8       0.0   1.0    201.4    200.2      4.26              4.19         2    2.130   7360K    33K       0.0       0.0
 L45      4/0   844.38 MB   0.0      6.3     5.0      1.2       6.2      5.0       0.0   1.2    202.0    200.3     31.81             31.50        12    2.651     55M   403K       0.0       0.0
 L46      7/0    1.62 GB   0.0     13.3     8.8      4.6      13.1      8.6       0.0   1.5    198.9    195.7     68.72             67.89        17    4.042    117M  1696K       0.0       0.0
 L47     24/0    5.81 GB   0.0     21.7    12.9      8.8      20.6     11.8       0.0   1.6    198.5    188.6    112.04            109.97        12    9.336    191M  9352K       0.0       0.0
 L48     41/0   10.14 GB   0.0     24.8    13.0     11.8      21.9     10.1       0.0   1.7    198.6    175.6    127.88            125.36         6   21.313    218M    25M       0.0       0.0
 L49     91/0   22.70 GB   0.0      0.0     0.0      0.0       0.0      0.0       0.0   0.0      0.0      0.0      0.00              0.00         0    0.000       0      0       0.0       0.0
 Sum    167/0   41.10 GB   0.0     67.0    40.5     26.4      85.4     58.9       0.0   3.8    141.1    179.8    486.13            450.04       157    3.096    589M    36M       0.0       0.0

Case 4:
** Compaction Stats [default] **
Level    Files   Size     Score Read(GB)  Rn(GB) Rnp1(GB) Write(GB) Wnew(GB) Moved(GB) W-Amp Rd(MB/s) Wr(MB/s) Comp(sec) CompMergeCPU(sec) Comp(cnt) Avg(sec) KeyIn KeyDrop Rblob(GB) Wblob(GB)
------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
  L0      0/0    0.00 KB   0.7      0.0     0.0      0.0      22.6     22.6       0.0   1.0      0.0    158.6    146.02            114.68       108    1.352       0      0       0.0       0.0
 L42      0/0    0.00 KB   0.0      1.7     1.7      0.0       1.7      1.7       0.0   1.0    185.4    184.3      9.25              8.96         4    2.314     14M    67K       0.0       0.0
 L43      0/0    0.00 KB   0.0      2.5     2.5      0.0       2.5      2.5       0.0   1.0    197.8    195.6     13.01             12.65         4    3.253     22M   202K       0.0       0.0
 L44      4/0   844.40 MB   0.0      4.2     4.2      0.0       4.1      4.1       0.0   1.0    188.1    185.1     22.81             21.89         5    4.562     36M   503K       0.0       0.0
 L45     13/0    3.12 GB   0.0      7.5     6.5      1.0       7.2      6.2       0.0   1.1    188.7    181.8     40.69             39.32         5    8.138     65M  2282K       0.0       0.0
 L46     17/0    4.18 GB   0.0      8.3     7.1      1.2       7.9      6.6       0.0   1.1    192.2    181.8     44.23             43.06         4   11.058     73M  3846K       0.0       0.0
 L47     22/0    5.34 GB   0.0      8.9     7.5      1.4       8.2      6.8       0.0   1.1    189.1    174.1     48.12             45.37         3   16.041     78M  6098K       0.0       0.0
 L48     27/0    6.58 GB   0.0      9.2     7.6      1.6       8.2      6.6       0.0   1.1    195.2    172.9     48.52             47.11         2   24.262     81M  9217K       0.0       0.0
 L49     91/0   22.70 GB   0.0      0.0     0.0      0.0       0.0      0.0       0.0   0.0      0.0      0.0      0.00              0.00         0    0.000       0      0       0.0       0.0
 Sum    174/0   42.74 GB   0.0     42.3    37.0      5.3      62.4     57.1       0.0   2.8    116.3    171.3    372.66            333.04       135    2.760    372M    22M       0.0       0.0

setup:
./db_bench --benchmarks=fillseq,compactall,waitforcompaction --num=200000000 --compression_type=none --disable_wal=1 --compaction_style=1 --num_levels=50 --target_file_size_base=268435456 --max_compaction_bytes=6710886400 --level0_file_num_compaction_trigger=10 --write_buffer_size=268435456 --seed 1708494134896523

benchmark:
./db_bench --benchmarks=overwrite,waitforcompaction,stats --num=200000000 --compression_type=none --disable_wal=1 --compaction_style=1 --write_buffer_size=268435456 --level0_file_num_compaction_trigger=5 --target_file_size_base=268435456 --use_existing_db=1 --num_levels=50 --writes=200000000 --universal_max_read_amp=-1 --seed=1716488324800233

```

Reviewed By: ajkr

Differential Revision: D55370922

Pulled By: cbi42

fbshipit-source-id: 9be69979126b840d08e93e7059260e76a878bb2a
2024-05-24 10:10:31 -07:00
Levi Tamasi f044b6a6ad Fix a couple of issues in the stress test for Transaction::MultiGet (#12696)
Summary:
Pull Request resolved: https://github.com/facebook/rocksdb/pull/12696

Two fixes:
1) `Random::Uniform(n)` returns an integer from the interval [0, n - 1], so `Uniform(2)` returns 0 or 1, which means is that we have apparently never covered transactions with deletions in the test. (To prevent similar issues, the patch cleans this write logic up a bit using an `enum class` for the type of write.)
2) The keys passed in to `TestMultiGet` can have duplicates. What this boils down to is that we have to keep track of the latest expected values for read-your-own-writes on a per-key basis.

Reviewed By: jowlyzhang

Differential Revision: D57750212

fbshipit-source-id: e8ab603252c32331f8db0dfb2affcca1e188c790
2024-05-23 16:47:39 -07:00
Levi Tamasi db0960800a Add Transaction::PutEntity to the stress tests (#12688)
Summary:
Pull Request resolved: https://github.com/facebook/rocksdb/pull/12688

As a first step of covering the wide-column transaction APIs, the patch adds `PutEntity` to the optimistic and pessimistic transaction stress tests (for the latter, only when the WriteCommitted policy is utilized). Other APIs and the multi-operation transaction test will be covered by subsequent PRs.

Reviewed By: jaykorean

Differential Revision: D57675781

fbshipit-source-id: bfe062ec5f6ab48641cd99a70f239ce4aa39299c
2024-05-22 11:30:33 -07:00
Peter Dillinger d89ab23bec Disallow memtable flush and sst ingest while WAL is locked (#12652)
Summary:
We recently noticed that some memtable flushed and file
ingestions could proceed during LockWAL, in violation of its stated
contract. (Note: we aren't 100% sure its actually needed by MySQL, but
we want it to be in a clean state nonetheless.)

Despite earlier skepticism that this could be done safely (https://github.com/facebook/rocksdb/issues/12666), I
found a place to wait to wait for LockWAL to be cleared before allowing
these operations to proceed: WaitForPendingWrites()

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12652

Test Plan:
Added to unit tests. Extended how db_stress validates LockWAL
and re-enabled combination of ingestion and LockWAL in crash test, in
follow-up to https://github.com/facebook/rocksdb/issues/12642

Ran blackbox_crash_test for a long while with relevant features
amplified.

Suggested follow-up: fix FaultInjectionTestFS to report file sizes
consistent with what the user has requested to be flushed.

Reviewed By: jowlyzhang

Differential Revision: D57622142

Pulled By: pdillinger

fbshipit-source-id: aef265fce69465618974b4ec47f4636257c676ce
2024-05-21 10:17:34 -07:00
Hui Xiao d7b938882e Sync WAL during db Close() (#12556)
Summary:
**Context/Summary:**
Below crash test found out we don't sync WAL upon DB close, which can lead to unsynced data loss. This PR syncs it.
```
./db_stress --threads=1 --disable_auto_compactions=1 --WAL_size_limit_MB=0 --WAL_ttl_seconds=0 --acquire_snapshot_one_in=0 --adaptive_readahead=0 --adm_policy=1 --advise_random_on_open=1 --allow_concurrent_memtable_write=1 --allow_data_in_errors=True --allow_fallocate=0 --async_io=0 --auto_readahead_size=0 --avoid_flush_during_recovery=1 --avoid_flush_during_shutdown=0 --avoid_unnecessary_blocking_io=1 --backup_max_size=104857600 --backup_one_in=0 --batch_protection_bytes_per_key=0 --bgerror_resume_retry_interval=1000000 --block_align=0 --block_protection_bytes_per_key=2 --block_size=16384 --bloom_before_level=1 --bloom_bits=29.895303579352174 --bottommost_compression_type=disable --bottommost_file_compaction_delay=0 --bytes_per_sync=0 --cache_index_and_filter_blocks=0 --cache_index_and_filter_blocks_with_high_priority=1 --cache_size=33554432 --cache_type=lru_cache --charge_compression_dictionary_building_buffer=1 --charge_file_metadata=0 --charge_filter_construction=1 --charge_table_reader=1 --checkpoint_one_in=0 --checksum_type=kxxHash64 --clear_column_family_one_in=0 --column_families=1 --compact_files_one_in=0 --compact_range_one_in=0 --compaction_pri=0 --compaction_readahead_size=0 --compaction_style=0 --compaction_ttl=0 --compress_format_version=2 --compressed_secondary_cache_ratio=0 --compressed_secondary_cache_size=0 --compression_checksum=1 --compression_max_dict_buffer_bytes=0 --compression_max_dict_bytes=0 --compression_parallel_threads=4 --compression_type=zstd --compression_use_zstd_dict_trainer=1 --compression_zstd_max_train_bytes=0 --continuous_verification_interval=0 --data_block_index_type=0 --db=/dev/shm/rocksdb_test/rocksdb_crashtest_whitebox --db_write_buffer_size=0 --default_temperature=kUnknown --default_write_temperature=kUnknown --delete_obsolete_files_period_micros=0 --delpercent=0 --delrangepercent=0 --destroy_db_initially=1 --detect_filter_construct_corruption=1 --disable_wal=0 --dump_malloc_stats=0 --enable_checksum_handoff=0 --enable_compaction_filter=0 --enable_custom_split_merge=0 --enable_do_not_compress_roles=1 --enable_index_compression=1 --enable_memtable_insert_with_hint_prefix_extractor=0 --enable_pipelined_write=0 --enable_sst_partitioner_factory=0 --enable_thread_tracking=1 --enable_write_thread_adaptive_yield=0 --expected_values_dir=/dev/shm/rocksdb_test/rocksdb_crashtest_expected --fail_if_options_file_error=0 --fifo_allow_compaction=1 --file_checksum_impl=none --fill_cache=0 --flush_one_in=1000 --format_version=5 --get_current_wal_file_one_in=0 --get_live_files_one_in=0 --get_property_one_in=0 --get_sorted_wal_files_one_in=0 --hard_pending_compaction_bytes_limit=274877906944 --high_pri_pool_ratio=0 --index_block_restart_interval=6 --index_shortening=0 --index_type=0 --ingest_external_file_one_in=0 --initial_auto_readahead_size=16384 --iterpercent=0 --key_len_percent_dist=1,30,69 --last_level_temperature=kUnknown --level_compaction_dynamic_level_bytes=1 --lock_wal_one_in=0 --log2_keys_per_lock=10 --log_file_time_to_roll=0 --log_readahead_size=16777216 --long_running_snapshots=0 --low_pri_pool_ratio=0 --lowest_used_cache_tier=0 --manifest_preallocation_size=5120 --manual_wal_flush_one_in=0 --mark_for_compaction_one_file_in=0 --max_auto_readahead_size=0 --max_background_compactions=1 --max_bytes_for_level_base=67108864 --max_key=2500000 --max_key_len=3 --max_log_file_size=0 --max_manifest_file_size=1073741824 --max_sequential_skip_in_iterations=8 --max_total_wal_size=0 --max_write_batch_group_size_bytes=64 --max_write_buffer_number=10 --max_write_buffer_size_to_maintain=0 --memtable_insert_hint_per_batch=0 --memtable_max_range_deletions=0 --memtable_prefix_bloom_size_ratio=0.5 --memtable_protection_bytes_per_key=1 --memtable_whole_key_filtering=1 --memtablerep=skip_list --metadata_charge_policy=0 --min_write_buffer_number_to_merge=1 --mmap_read=0 --mock_direct_io=True --nooverwritepercent=1 --num_file_reads_for_auto_readahead=0 --num_levels=1 --open_files=-1 --open_metadata_write_fault_one_in=0 --open_read_fault_one_in=0 --open_write_fault_one_in=0 --ops_per_thread=3 --optimize_filters_for_hits=1 --optimize_filters_for_memory=1 --optimize_multiget_for_io=0 --paranoid_file_checks=0 --partition_filters=0 --partition_pinning=1 --pause_background_one_in=0 --periodic_compaction_seconds=0 --prefix_size=1 --prefixpercent=0 --prepopulate_block_cache=0 --preserve_internal_time_seconds=3600 --progress_reports=0 --read_amp_bytes_per_bit=0 --read_fault_one_in=0 --readahead_size=16384 --readpercent=0 --recycle_log_file_num=0 --reopen=2 --report_bg_io_stats=1 --sample_for_compression=5 --secondary_cache_fault_one_in=0 --secondary_cache_uri= --skip_stats_update_on_db_open=1 --snapshot_hold_ops=0 --soft_pending_compaction_bytes_limit=68719476736 --sst_file_manager_bytes_per_sec=0 --sst_file_manager_bytes_per_truncate=0 --stats_dump_period_sec=10 --stats_history_buffer_size=1048576 --strict_bytes_per_sync=0 --subcompactions=3 --sync=0 --sync_fault_injection=1 --table_cache_numshardbits=6 --target_file_size_base=16777216 --target_file_size_multiplier=1 --test_batches_snapshots=0 --top_level_index_pinning=0 --unpartitioned_pinning=3 --use_adaptive_mutex=1 --use_adaptive_mutex_lru=0 --use_delta_encoding=1 --use_direct_io_for_flush_and_compaction=0 --use_direct_reads=0 --use_full_merge_v1=0 --use_get_entity=0 --use_merge=0 --use_multi_get_entity=0 --use_multiget=1 --use_put_entity_one_in=0 --use_write_buffer_manager=0 --user_timestamp_size=0 --value_size_mult=32 --verification_only=0 --verify_checksum=1 --verify_checksum_one_in=1000 --verify_compression=0 --verify_db_one_in=100000 --verify_file_checksums_one_in=0 --verify_iterator_with_expected_state_one_in=5 --verify_sst_unique_id_in_manifest=1 --wal_bytes_per_sync=0 --wal_compression=zstd --write_buffer_size=33554432 --write_dbid_to_manifest=0 --write_fault_one_in=0 --writepercent=100

 Verification failed for column family 0 key 000000000000B9D1000000000000012B000000000000017D (4756691): value_from_db: , value_from_expected: 010000000504070609080B0A0D0C0F0E111013121514171619181B1A1D1C1F1E212023222524272629282B2A2D2C2F2E313033323534373639383B3A3D3C3F3E, msg: Iterator verification: Value not found: NotFound:
Verification failed :(
```

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12556

Test Plan:
- New UT
- Same stress test command failed before this fix but pass after
- CI

Reviewed By: ajkr

Differential Revision: D56267964

Pulled By: hx235

fbshipit-source-id: af1b7e8769c129f64ba1c7f1ff17102f1239b929
2024-05-20 17:33:43 -07:00
Changyu Bi 2eb404de13 Print non-ok status for multi_ops_txns_stress test (#12660)
Summary:
Currently `assert(s.ok())` does not offer much information to debug.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12660

Test Plan:
revert https://github.com/facebook/rocksdb/issues/12639 and run `python3 ./tools/db_crashtest.py blackbox --write_policy write_prepared --recycle_log_file_num=1 --threads=1 --test_multiops_txn`

before this PR:
```
Exit Before Killing
stdout:

stderr:
 db_stress: db_stress_tool/multi_ops_txns_stress.cc:1529: void rocksdb::MultiOpsTxnsStressTest::PreloadDb(rocksdb::SharedState*, int, uint32_t, uint32_t, uint32_t, uint32_t): Assertion `s.ok()' failed.
```

after this PR:
```
Exit Before Killing
stdout:

stderr:
 Verification failed: PreloadDB failed: Invalid argument: WriteOptions::disableWAL option is not supported if DBOptions::recycle_log_file_num > 0
db_stress: db_stress_tool/db_stress_test_base.cc:517: void rocksdb::StressTest::ProcessStatus(rocksdb::SharedState*, std::string, rocksdb::Status, bool) const: Assertion `false' failed.
Received signal 6 (Aborted)
```

Reviewed By: ajkr

Differential Revision: D57410819

Pulled By: cbi42

fbshipit-source-id: a03c2202c3fd666eb2f58bae24e0c9e3e6ed4265
2024-05-15 21:14:41 -07:00
Jay Huh b4c6956a59 Add MultiGetEntity AttributeGroup API to stress test (#12640)
Summary:
Continuing from https://github.com/facebook/rocksdb/pull/12605, adding AttributeGroup `MultiGetEntity` API to stress tests.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12640

Test Plan:
**AttributeGroup Tests**

NonBatchOps
```
python3 tools/db_crashtest.py blackbox --simple --max_key=25000000 --write_buffer_size=4194304 --use_attribute_group=1 --use_put_entity_one_in=1 --use_multi_get=1
```

BatchOps
```
python3 tools/db_crashtest.py blackbox  --test_batches_snapshots=1 --max_key=25000000 --write_buffer_size=4194304 --use_attribute_group=1 --use_put_entity_one_in=1 --use_multi_get=1
```

CfConsistency Test
```
python3 tools/db_crashtest.py blackbox --cf_consistency --max_key=25000000 --write_buffer_size=4194304 --use_attribute_group=1 --use_put_entity_one_in=1 --use_multi_get=1
```

**Non-AttributeGroup Tests**

NonBatchOps
```
python3 tools/db_crashtest.py blackbox --simple --max_key=25000000 --write_buffer_size=4194304 --use_attribute_group=0 --use_put_entity_one_in=1 --use_multi_get=1
```

BatchOps
```
python3 tools/db_crashtest.py blackbox  --test_batches_snapshots=1 --max_key=25000000 --write_buffer_size=4194304 --use_attribute_group=0 --use_put_entity_one_in=1 --use_multi_get=1
```

CfConsistency Test
```
python3 tools/db_crashtest.py blackbox --cf_consistency --max_key=25000000 --write_buffer_size=4194304 --use_attribute_group=0 --use_put_entity_one_in=1 --use_multi_get=1
```

Reviewed By: ltamasi

Differential Revision: D57233931

Pulled By: jaykorean

fbshipit-source-id: 8cea771ac2e5749050bf5319360c6c5aa476d7d5
2024-05-14 16:33:44 -07:00
Yu Zhang 7d9642d876 Add logging for read timestamp during VerifyDB (#12638)
Summary:
As titled. To help debug some verification failures.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12638

Test Plan: manually tested

Reviewed By: ajkr

Differential Revision: D57219549

Pulled By: jowlyzhang

fbshipit-source-id: 59c05ac85fb1c24449e7394ea04172c855d86420
2024-05-10 12:34:53 -07:00
Jay Huh 1f2715d1d2 AttributeGroup APIs in stress test - PutEntity and GetEntity (#12605)
Summary:
Adding AttributeGroup APIs in stress test. This contains the following changes only. More PRs to follow.

- Introduce `use_attribute_group` flag
- AttributeGroup `PutEntity()` and `GetEntity()` are now used per `use_attribute_group` flag in BatchOps, NonBatchOps and CfConsistency tests

In the next PRs I plan to add
- AttributeGroup `MultiGetEntity()` in Stress Test
- AttributeGroupIterator in Stress Test (along with CoalescingIterator)

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12605

Test Plan:
NonBatchOps
```
python3 tools/db_crashtest.py blackbox --simple --max_key=25000000 --write_buffer_size=4194304 --use_attribute_group=1 --use_put_entity_one_in=1
```

BatchOps
```
python3 tools/db_crashtest.py blackbox --test_batches_snapshots=1 --max_key=25000000 --write_buffer_size=4194304 --use_attribute_group=1 --use_put_entity_one_in=1
```

CfConsistency Test
```
python3 tools/db_crashtest.py blackbox --cf_consistency --max_key=25000000 --write_buffer_size=4194304 --use_attribute_group=1 --use_put_entity_one_in=1
```

Reviewed By: ltamasi

Differential Revision: D56916768

Pulled By: jaykorean

fbshipit-source-id: 8555d9e0d05927740a10e4e8301e44beec59a6f5
2024-05-09 16:40:22 -07:00
Hui Xiao 9bddac0dcf Add more public APIs to crash test (#12617)
Summary:
**Context/Summary:**
As titled. Bonus: found that PromoteL0 called with other concurrent PromoteL0 will return non-okay error so clarify the API.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12617

Test Plan: CI

Reviewed By: ajkr

Differential Revision: D56954428

Pulled By: hx235

fbshipit-source-id: 0e056153c515003fd241ffec59b0d8a27529db4c
2024-05-09 15:37:38 -07:00
Andrew Kryczka 933ac0e05c Fix locking for ColumnFamilyOptions::inplace_update_support (#12624)
Summary:
In `SaveValue()`, the read lock needs to be obtained before `VerifyEntryChecksum()` because the KV checksum verification reads the entire value metadata+data, which is all mutable when `ColumnFamilyOptions::inplace_update_support == true`.

In `MemTable::Update()`, the write lock needs to be obtained before mutating the value metadata (changing the value size) because it can be read concurrently.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12624

Test Plan:
```
$ make COMPILE_WITH_TSAN=1 -j56 db_stress
...
$ python3 tools/db_crashtest.py blackbox --simple --max_key=10 --inplace_update_support=1 --interval=10 --allow_concurrent_memtable_write=0
```

Reviewed By: cbi42

Differential Revision: D57034571

Pulled By: ajkr

fbshipit-source-id: 3dddf881ad87923143acdf6bfec12ce47bb13a48
2024-05-08 08:30:12 -07:00
Changyu Bi 5bf2c00a35 Clarify manual compaction and file ingestion behavior with FIFO compaction (#12618)
Summary:
For manual compaction, FIFO compaction will always skip key range overlapping checking with SST files. If CompactRange() is called with CompactionRangeOptions::change_level=true, a CF with FIFO compaction will now return Status::NotSupported.

For file ingestion, we will always ingest into L0. Previously, it's possible to ingest files into non-L0 levels with FIFO compaction.

These changes also help to fix [this](a178d15baf/db/db_impl/db_impl_compaction_flush.cc (L1269)) assertion failure in crash tests.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12618

Test Plan: added unit tests to verify the new behavior.

Reviewed By: hx235

Differential Revision: D56962401

Pulled By: cbi42

fbshipit-source-id: 19812a1509650b4162b379ca5bee02f2e9d9569d
2024-05-07 12:00:15 -07:00
Hui Xiao b312dbe920 Remove duplicate inplace_update_support crash/stress test flag (#12610)
Summary:
**Context/Summary:**
As titled. There were two flags serving the same purpose so removed one of them.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12610

Test Plan: CI

Reviewed By: jaykorean, ajkr

Differential Revision: D56916119

Pulled By: hx235

fbshipit-source-id: 011140a7945782cc613ca86d4b542db0cf7fb444
2024-05-03 11:33:33 -07:00
Yu Zhang 8b3d9e6bfe Add TimedPut to stress test (#12559)
Summary:
This also updates WriteBatch's protection info to include write time since there are several places in memtable that by default protects the whole value slice.

This PR is stacked on https://github.com/facebook/rocksdb/issues/12543

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12559

Reviewed By: pdillinger

Differential Revision: D56308285

Pulled By: jowlyzhang

fbshipit-source-id: 5524339fe0dd6c918dc940ca2f0657b5f2111c56
2024-04-30 15:40:35 -07:00
Hui Xiao 24a35b6e57 Add more public APIs to crash/stress test (#12541)
Summary:
**Context/Summary:**
This PR includes some public DB APIs not tested in crash/stress yet can be added in a straightforward way.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12541

Test Plan:
- Locally run crash test heavily stressing on these new APIs
- CI

Reviewed By: jowlyzhang

Differential Revision: D56164892

Pulled By: hx235

fbshipit-source-id: 8bb568c3e65aec39d642987033f1d76c52f69bd8
2024-04-16 15:43:26 -07:00
Jay Huh dfdc3b158e Add offpeak feature to crash test (#12549)
Summary:
As title. Add offpeak feature in stress test.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12549

Test Plan:
Ran stress test locally with the flag set
```
Running db_stress with pid=701060: ./db_stress ... --daily_offpeak_time_utc=04:00-08:00 ... --periodic_compaction_seconds=10 ...
...
KILLED 701060

stdout:
 Choosing random keys with no overwrite
Creating 6250000 locks
2024/04/16-11:38:19  Initializing db_stress
RocksDB version           : 9.2
Format version            : 5
TransactionDB             : false
Stacked BlobDB            : false
Read only mode            : false
Atomic flush              : false
Manual WAL flush          : true
Column families           : 1
Clear CFs one in          : 0
Number of threads         : 32
Ops per thread            : 100000000
Time to live(sec)         : unused
Read percentage           : 60%
Prefix percentage         : 0%
Write percentage          : 35%
Delete percentage         : 4%
Delete range percentage   : 1%
No overwrite percentage   : 1%
Iterate percentage        : 0%
Custom ops percentage     : 0%
DB-write-buffer-size      : 0
Write-buffer-size         : 4194304
Iterations                : 10
Max key                   : 25000000
Ratio #ops/#keys          : 128.000000
Num times DB reopens      : 0
Batches/snapshots         : 0
Do update in place        : 0
Num keys per lock         : 4
Compression               : LZ4
Bottommost Compression    : DisableOption
Checksum type             : kxxHash
File checksum impl        : none
Bloom bits / key          : 18.000000
Max subcompactions        : 4
Use MultiGet              : false
Use GetEntity             : false
Use MultiGetEntity        : false
Verification only         : false
Memtablerep               : skip_list
Test kill odd             : 0
Periodic Compaction Secs  : 10
Daily Offpeak UTC         : 04:00-08:00  <<<<<<<<<<<<<<< Newly added
Compaction TTL            : 0
Compaction Pri            : kMinOverlappingRatio
Background Purge          : 0
Write DB ID to manifest   : 0
Max Write Batch Group Size: 16
Use dynamic level         : 1
Read fault one in         : 0
Write fault one in        : 1000
Open metadata write fault one in:
                            8
Sync fault injection      : 0
Best efforts recovery     : 0
Fail if OPTIONS file error: 0
User timestamp size bytes : 0
Persist user defined timestamps : 1
WAL compression           : zstd
Try verify sst unique id  : 1
------------------------------------------------
```

Reviewed By: hx235

Differential Revision: D56203102

Pulled By: jaykorean

fbshipit-source-id: 11a9be7362b3b26940d74d41c8bf4ebac3f03a2d
2024-04-16 12:44:44 -07:00
Hui Xiao d41e568b1c Add inplace_update_support to crash/stress test (#12535)
Summary:
**Context/Summary:**
`inplace_update_support=true` is not tested in crash/stress test. Since it's not compatible with snapshots like compaction_filter, we need to sanitize its value in presence of snapshots-related options. A minor refactoring is added to centralize such sanitization in db_crashtest.py - see `check_multiget_consistency` and `check_multiget_entity_consistency`

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12535

Test Plan: CI

Reviewed By: ajkr

Differential Revision: D56102978

Pulled By: hx235

fbshipit-source-id: 2e2ab6685a65123b14a321b99f45f60bc6509c6b
2024-04-15 16:11:58 -07:00
Hui Xiao ef26d68e8d Renable kAdmPolicyThreeQueue in crash test (#12524)
Summary:
Context/Summary:

We need a `nvm_sec_cache` when `kAdmPolicyThreeQueue` is used otherwise a nullptr cache will be accessed causing us segfault in https://github.com/facebook/rocksdb/pull/12521

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12524

Test Plan: - Re-enabled `kAdmPolicyThreeQueue` and rehearsed stress test that failed before this fix and pass after

Reviewed By: jowlyzhang

Differential Revision: D55997093

Pulled By: hx235

fbshipit-source-id: e1c6f1015091b4cff0ce6a3fff981d5dece52a62
2024-04-11 14:53:11 -07:00
Hui Xiao 72c1376fcf Fix "assertion failed - iter != ROCKSDB_NAMESPACE::OptionsHelper::temperature_to_string.end()" (#12519)
Summary:
Context/Summary: for unknown reason, calling a db stress common function in db stress flag file for temperature-related flags will cause some weird behavior in some compilation/build.
```
assertion failed - iter != ROCKSDB_NAMESPACE::OptionsHelper::temperature_to_string.end()
```
For now, we decide not to call such function by hard-coding their default stress test values.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12519

Test Plan: - Run a rehearsal stress test with this fix and weird behavior is gone.

Reviewed By: jowlyzhang

Differential Revision: D55884693

Pulled By: hx235

fbshipit-source-id: ba5135f5b37a9fa686b3ccae8d3f77e62d6562c9
2024-04-08 13:45:41 -07:00
Hui Xiao ad423abbd1 Add more missing options in crash test (#12508)
Summary:
**Context/Summary:**
This is to improve our crash test coverage.

Bonus change:
- Added the missing Options string mapping for `CacheTier::kVolatileCompressedTier`
- Deprecated crash test options `enable_tiered_storage` mainly for setting `last_level_temperature` which is now covered in crash test by itself
- Intensified `verify_checksum_one_in\verify_file_checksums_one_in` as I found out these together with new coverage surface more issues

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12508

Test Plan: CI to look out for trivial failures

Reviewed By: jowlyzhang

Differential Revision: D55768594

Pulled By: hx235

fbshipit-source-id: 9b829da0309a7db3fcdb17992a524dd64498325c
2024-04-08 09:48:03 -07:00
Hui Xiao d985902ef4 Disallow refitting more than 1 file from non-L0 to L0 (#12481)
Summary:
**Context/Summary:**
We recently discovered that `CompactRange(change_level=true, target_level=0)` can possibly refit more than 1 files to L0. This refitting can cause read performance regression as we need to go through every file in L0, corruption in some edge case and false positive corruption caught by force consistency check. We decided to explicitly disallow such behavior.

A related change to OptionChangeMigration():
- When migrating to FIFO with `compaction_options_fifo.max_table_files_size > 0`, RocksDB will [CompactRange() all the to-be-migrate data into a couple L0 files](https://github.com/facebook/rocksdb/blob/main/utilities/option_change_migration/option_change_migration.cc#L164-L169) to avoid dropping all the data upon migration finishes when the migrated data is larger than max_table_files_size. This is achieved by first compacting all the data into a couple non-L0 files and refitting those files from non-L0 to L0 if needed. In that way, only some data instead of all data will be dropped immediately after migration to FIFO with a max_table_files_size.
- Since this type of refitting behavior is disallowed from now on, we won't do this trick anymore and explicitly state such risk in API comment.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12481

Test Plan:
- New UT
- Modified UT

Reviewed By: cbi42

Differential Revision: D55351178

Pulled By: hx235

fbshipit-source-id: 9d8854f2f81d7e8aff859c3a4e53b7d688048e80
2024-03-29 10:52:36 -07:00
Changyu Bi 3d5be596a5 Fix a bug in iterator with UDT + ReadOptions::pin_data (#12451)
Summary:
with https://github.com/facebook/rocksdb/issues/12414 enabling `ReadOptions::pin_data`, this bug surfaced as corrupted per key-value checksum during crash test. `saved_key_.GetUserKey()` could be pinned user key, so DBIter should not overwrite it.

In one case, it only surfaces when iterator skips many keys of the same user key. To stress that code path, this PR also added `max_sequential_skip_in_iterations` to crash test.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12451

Test Plan:
- Set ReadOptions::pin_data to true, the bug can be reproed quickly with `./db_stress --persist_user_defined_timestamps=1 --user_timestamp_size=8 --writepercent=35 --delpercent=4 --delrangepercent=1 --iterpercent=20 --nooverwritepercent=1 --prefix_size=8 --prefixpercent=10 --readpercent=30 --memtable_protection_bytes_per_key=8 --block_protection_bytes_per_key=2 --clear_column_family_one_in=0`.
    - Set max_sequential_skip_in_iterations to 1 for the other occurrence of the bug.

Reviewed By: jowlyzhang

Differential Revision: D55003766

Pulled By: cbi42

fbshipit-source-id: 23e1049129456684dafb028b6132b70e0afc07fb
2024-03-18 09:05:11 -07:00
Hui Xiao fa4978c566 Re-suppress tolerable manual compaction stress test failures (#12437)
Summary:
**Context/Summary:**
Previously manual compaction stress test failures won't terminate stress test. https://github.com/facebook/rocksdb/pull/12414 made more manual compaction failures terminate the stress test for signal boosting. A downside to that PR: some tolerable manual compaction stress test failures also unnecessarily terminate stress test.

Ideally we should exclude exactly those tolerable errors (left as a TODO) from being able to terminate. For now we approximate those errors by Aborted(), InvalidArgument(), NotSupported() etc. It's still an improvement to pre-https://github.com/facebook/rocksdb/pull/12414 situation.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12437

Test Plan: - No more tolerable manual compaction stress test failures terminating stress test.

Reviewed By: cbi42

Differential Revision: D54913010

Pulled By: hx235

fbshipit-source-id: c43fa79d8f9c1c8b4f8786f8f46508b0ad619a9e
2024-03-14 14:50:56 -07:00
Hui Xiao 30243c6573 Add missing db crash options (#12414)
Summary:
**Context/Summary:**
We are doing a sweep in all public options, including but not limited to the `Options`, `Read/WriteOptions`, `IngestExternalFileOptions`, cache options.., to find and add the uncovered ones into db crash. The options included in this PR require minimum changes to db crash other than adding the options themselves.

A bonus change: to surface new issues by improved coverage in stderror, we decided to fail/terminate crash test for manual compactions (CompactFiles, CompactRange()) on meaningful errors. See https://github.com/facebook/rocksdb/pull/12414/files#diff-5c4ced6afb6a90e27fec18ab03b2cd89e8f99db87791b4ecc6fa2694284d50c0R2528-R2532, https://github.com/facebook/rocksdb/pull/12414/files#diff-5c4ced6afb6a90e27fec18ab03b2cd89e8f99db87791b4ecc6fa2694284d50c0R2330-R2336 for more.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12414

Test Plan:
- Run `python3 ./tools/db_crashtest.py --simple blackbox` for 10 minutes to ensure no trivial failure
- Run `python3 tools/db_crashtest.py --simple blackbox --compact_files_one_in=1 --compact_range_one_in=1 --read_fault_one_in=1 --write_fault_one_in=1 --interval=50` for a while to ensure the bonus change does not result in trivial crash/termination of stress test

Reviewed By: ajkr, jowlyzhang, cbi42

Differential Revision: D54691774

Pulled By: hx235

fbshipit-source-id: 50443dfb6aaabd8e24c79a2e42b68c6de877be88
2024-03-12 17:24:12 -07:00
Peter Dillinger a53ed91691 Fix/improve temperature handling for file ingestion (#12402)
Summary:
Partly following up on leftovers from https://github.com/facebook/rocksdb/issues/12388

In terms of public API:
* Make it clear that IngestExternalFileArg::file_temperature is just a hint for opening the existing file, though it was previously used for both copy-from temp hint and copy-to temp, which was bizarre.
* Specify how IngestExternalFile assigns temperature to file ingested into DB. (See details in comments.) This approach is not perfect in terms of matching how the DB assigns temperatures, but was the simplest way to get close. The key complication for matching DB temperature assignments is that ingestion files are copied (to a destination temp) before their target level is determined (in general).
* Add a temperature option to SstFileWriter::Open so that files intended for ingestion can be initially written to a chosen temperature.
* Note that "fail_if_not_bottommost_level" is obsolete/confusing use of "bottommost"

In terms of the implementation, there was a similar bit of oddness with the internal CopyFile API, which only took one temperature, ambiguously applicable to the source, destination, or both. This is also fixed.

Eventual suggested follow-up:
* Before copying files for ingestion, determine a tentative level assignment to use for destination temperature, and keep that even if final level assignment happens to be different at commit time (rare).
* More temperature handling for CreateColumnFamilyWithImport and Checkpoints.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12402

Test Plan:
Deeply revamped
ExternalSSTFileBasicTest.IngestWithTemperature to test the new changes. Previously this test was insufficient because it was only looking at temperatures according to the DB manifest. Incorporating FileTemperatureTestFS allows us to also test the temperatures in the storage layer.

Used macros instead of functions for better tracing to critical source location on test failures.

Some enhancements to FileTemperatureTestFS in the process of developing the revamped test.

Reviewed By: jowlyzhang

Differential Revision: D54442794

Pulled By: pdillinger

fbshipit-source-id: 41d9d0afdc073e6a983304c10bbc07c70cc7e995
2024-03-05 16:56:08 -08:00
Peter Dillinger d780e7a561 Remove bottommost_temperature (#12389)
Summary:
deprecated option already replaced by `last_level_temperature`. (Keeping recognition of the option in old options files.)

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12389

Test Plan: tests updated

Reviewed By: jowlyzhang, cbi42

Differential Revision: D54267946

Pulled By: pdillinger

fbshipit-source-id: 65c49b15e7394829c1f3b44edd4179d2daff6017
2024-02-27 14:48:00 -08:00
Andrew Kryczka 8e29f243c9 No filesystem reads during Merge() writes (#12365)
Summary:
This occasional filesystem read in the write path has caused user pain. It doesn't seem very useful considering it only limits one component's merge chain length, and only helps merge uncached (i.e., infrequently read) values. This PR proposes allowing `max_successive_merges` to be exceeded when the value cannot be read from in-memory components. I included a rollback flag (`strict_max_successive_merges`) just in case.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12365

Test Plan:
"rocksdb.block.cache.data.add" is number of data blocks read from filesystem. Since the benchmark is write-only, compaction is disabled, and flush doesn't read data blocks, any nonzero value means the user write issued the read.

```
$ for s in false true; do echo -n "strict_max_successive_merges=$s: " && ./db_bench -value_size=64 -write_buffer_size=131072 -writes=128 -num=1 -benchmarks=mergerandom,flush,mergerandom -merge_operator=stringappend -disable_auto_compactions=true -compression_type=none -strict_max_successive_merges=$s -max_successive_merges=100 -statistics=true |& grep 'block.cache.data.add COUNT' ; done
strict_max_successive_merges=false: rocksdb.block.cache.data.add COUNT : 0
strict_max_successive_merges=true: rocksdb.block.cache.data.add COUNT : 1
```

Reviewed By: hx235

Differential Revision: D53982520

Pulled By: ajkr

fbshipit-source-id: e40f761a60bd601f232417ac0058e4a33ee9c0f4
2024-02-21 13:15:27 -08:00
Akanksha Mahajan 956f1dfde3 Change ReadAsync callback API to remove const from FSReadRequest (#11649)
Summary:
Modify ReadAsync callback API to remove const from FSReadRequest as const doesn't let to fs_scratch to move the ownership.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11649

Test Plan: CircleCI jobs

Reviewed By: anand1976

Differential Revision: D53585309

Pulled By: akankshamahajan15

fbshipit-source-id: 3bff9035db0e6fbbe34721a5963443355807420d
2024-02-16 09:14:55 -08:00
Changyu Bi 42a8e583c9 Print zstd warning to stdout in stress test (#12338)
Summary:
so the stress test does not fail.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12338

Reviewed By: jaykorean

Differential Revision: D53542941

Pulled By: cbi42

fbshipit-source-id: 83b2eb3cb5cc4c5a268da386c22c4aadeb039a74
2024-02-07 14:17:51 -08:00
Peter Dillinger 54cb9c77d9 Prefer static_cast in place of most reinterpret_cast (#12308)
Summary:
The following are risks associated with pointer-to-pointer reinterpret_cast:
* Can produce the "wrong result" (crash or memory corruption). IIRC, in theory this can happen for any up-cast or down-cast for a non-standard-layout type, though in practice would only happen for multiple inheritance cases (where the base class pointer might be "inside" the derived object). We don't use multiple inheritance a lot, but we do.
* Can mask useful compiler errors upon code change, including converting between unrelated pointer types that you are expecting to be related, and converting between pointer and scalar types unintentionally.

I can only think of some obscure cases where static_cast could be troublesome when it compiles as a replacement:
* Going through `void*` could plausibly cause unnecessary or broken pointer arithmetic. Suppose we have
`struct Derived: public Base1, public Base2`.  If we have `Derived*` -> `void*` -> `Base2*` -> `Derived*` through reinterpret casts, this could plausibly work (though technical UB) assuming the `Base2*` is not dereferenced. Changing to static cast could introduce breaking pointer arithmetic.
* Unnecessary (but safe) pointer arithmetic could arise in a case like `Derived*` -> `Base2*` -> `Derived*` where before the Base2 pointer might not have been dereferenced. This could potentially affect performance.

With some light scripting, I tried replacing pointer-to-pointer reinterpret_casts with static_cast and kept the cases that still compile. Most occurrences of reinterpret_cast have successfully been changed (except for java/ and third-party/). 294 changed, 257 remain.

A couple of related interventions included here:
* Previously Cache::Handle was not actually derived from in the implementations and just used as a `void*` stand-in with reinterpret_cast. Now there is a relationship to allow static_cast. In theory, this could introduce pointer arithmetic (as described above) but is unlikely without multiple inheritance AND non-empty Cache::Handle.
* Remove some unnecessary casts to void* as this is allowed to be implicit (for better or worse).

Most of the remaining reinterpret_casts are for converting to/from raw bytes of objects. We could consider better idioms for these patterns in follow-up work.

I wish there were a way to implement a template variant of static_cast that would only compile if no pointer arithmetic is generated, but best I can tell, this is not possible. AFAIK the best you could do is a dynamic check that the void* conversion after the static cast is unchanged.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12308

Test Plan: existing tests, CI

Reviewed By: ltamasi

Differential Revision: D53204947

Pulled By: pdillinger

fbshipit-source-id: 9de23e618263b0d5b9820f4e15966876888a16e2
2024-02-07 10:44:11 -08:00
Peter Dillinger 76c834e441 Remove 'virtual' when implied by 'override' (#12319)
Summary:
... to follow modern C++ style / idioms.

Used this hack:
```
for FILE in `cat my_list_of_files`; do perl -pi -e 'BEGIN{undef $/;} s/ virtual( [^;{]* override)/$1/smg' $FILE; done
```

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12319

Test Plan: existing tests, CI

Reviewed By: jaykorean

Differential Revision: D53275303

Pulled By: pdillinger

fbshipit-source-id: bc0881af270aa8ef4d0ae4f44c5a6614b6407377
2024-01-31 13:14:42 -08:00
Akanksha Mahajan 95d582e0cc Enable io_uring in stress test (#12313)
Summary:
Enable io_uring in stress test

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12313

Test Plan: Crash test

Reviewed By: anand1976

Differential Revision: D53238319

Pulled By: akankshamahajan15

fbshipit-source-id: c0c8e6a6479f6977210370606e9d551c1299ba62
2024-01-31 12:37:42 -08:00
Peter Dillinger acf77e1bfe Fix possible crash test segfault in FileExpectedStateManager::Restore() (#12314)
Summary:
`replayer` could be `nullptr` if `!s.ok()` from an earlier failure. Also consider status returned from `record->Accept()`.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12314

Test Plan: blackbox_crash_test run

Reviewed By: hx235

Differential Revision: D53241506

Pulled By: pdillinger

fbshipit-source-id: fd330417c23391ca819c3ee0f69e4156d81934dc
2024-01-30 16:16:04 -08:00
Jay Huh 0d68aff3a1 StressTest - Move some stderr messages to stdout (#12304)
Summary:
Moving some of the messages that we print out in `stderr` to `stdout` to make `stderr` more strictly related to errors.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12304

Test Plan:
```
$> python3 tools/db_crashtest.py blackbox --simple --max_key=25000000 --write_buffer_size=4194304
```

**Before**
```
...
stderr:
WARNING: prefix_size is non-zero but memtablerep != prefix_hash
Error : jewoongh injected test error This is not a real failure.
Verification failed :(
```

**After**

No longer seeing the `WARNING: prefix_size is non-zero but memtablerep != prefix_hash` message in stderr, but still appears in stdout.

```
WARNING: prefix_size is non-zero but memtablerep != prefix_hash
Integrated BlobDB: blob files enabled 0, min blob size 0, blob file size 268435456, blob compression type NoCompression, blob GC enabled 0, cutoff 0.250000, force threshold 1.000000, blob compaction readahead size 0, blob file starting level 0
Integrated BlobDB: blob cache disabled
DB path: [/tmp/jewoongh/rocksdb_crashtest_blackboxzztp281q]
(Re-)verified 0 unique IDs
2024/01/29-11:57:51  Initializing worker threads
Crash-recovery verification passed :)
2024/01/29-11:57:58  Starting database operations
2024/01/29-11:57:58  Starting verification
Stress Test : 245.167 micros/op 10221 ops/sec
            : Wrote 0.00 MB (0.10 MB/sec) (16% of 6 ops)
            : Wrote 1 times
            : Deleted 0 times
            : Single deleted 0 times
            : 4 read and 0 found the key
            : Prefix scanned 0 times
            : Iterator size sum is 0
            : Iterated 3 times
            : Deleted 0 key-ranges
            : Range deletions covered 0 keys
            : Got errors 0 times
            : 0 CompactFiles() succeed
            : 0 CompactFiles() did not succeed

stderr:
Error : jewoongh injected test error This is not a real failure.
Error : jewoongh injected test error This is not a real failure.
Error : jewoongh injected test error This is not a real failure.
Verification failed :(
```

Reviewed By: akankshamahajan15

Differential Revision: D53193587

Pulled By: jaykorean

fbshipit-source-id: 40d59f4c993c5ce043c571a207ccc9b74a0180c6
2024-01-29 12:52:59 -08:00
Peter Dillinger 4e60663b31 Remove unnecessary, confusing 'extern' (#12300)
Summary:
In C++, `extern` is redundant in a number of cases:
* "Global" function declarations and definitions
* "Global" variable definitions when already declared `extern`

For consistency and simplicity, I've removed these in code that *we own*. In a couple of cases, I removed obsolete declarations, and for MagicNumber constants, I have consolidated the declarations into a header file (format.h)
as standard best practice would prescribe.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12300

Test Plan: no functional changes, CI

Reviewed By: ajkr

Differential Revision: D53148629

Pulled By: pdillinger

fbshipit-source-id: fb8d927959892e03af09b0c0d542b0a3b38fd886
2024-01-29 10:38:08 -08:00
Hui Xiao 438fc3d9b7 No consistency check when compaction filter is enabled in stress test (#12291)
Summary:
**Context/Summary:**
[Consistency check between Multiget and Get](d82d179a5e/db_stress_tool/no_batched_ops_stress.cc (L585-L591)) requires snapshot to be repeatable, that is, no keys being protected by this snapshot is deleted. However compaction filter can remove keys under snapshot, e,g,[DBStressCompactionFilter](d82d179a5e/db_stress_tool/db_stress_compaction_filter.h (L59)), which makes consistency check fail meaninglessly. This is noted in https://github.com/facebook/rocksdb/wiki/Compaction-Filter - "Since release 6.0, with compaction filter enabled, RocksDB always invoke filtering for any key, even if it knows it will make a snapshot not repeatable."

This PR makes consistency check happens only when compaction filter is not enabled in stress test

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12291

Test Plan:
- Make the stress test command that fails on consistency check pass
```
 ./db_stress --preserve_unverified_changes=1 --acquire_snapshot_one_in=0 --adaptive_readahead=0 --allow_concurrent_memtable_write=0 --allow_data_in_errors=True --async_io=0 --auto_readahead_size=0 --avoid_flush_during_recovery=0 --avoid_unnecessary_blocking_io=1 --backup_max_size=104857600 --backup_one_in=0 --batch_protection_bytes_per_key=0 --block_protection_bytes_per_key=0 --block_size=16384 --bloom_before_level=2147483647 --bloom_bits=30.729729833325962 --bottommost_compression_type=disable --bottommost_file_compaction_delay=0 --bytes_per_sync=0 --cache_index_and_filter_blocks=1 --cache_size=33554432 --cache_type=lru_cache --charge_compression_dictionary_building_buffer=0 --charge_file_metadata=1 --charge_filter_construction=0 --charge_table_reader=1 --checkpoint_one_in=0 --checksum_type=kCRC32c --clear_column_family_one_in=0 --column_families=1 --compact_files_one_in=0 --compact_range_one_in=0 --compaction_pri=4 --compaction_readahead_size=0 --compaction_ttl=0 --compressed_secondary_cache_size=8388608 --compression_checksum=0 --compression_max_dict_buffer_bytes=0 --compression_max_dict_bytes=0 --compression_parallel_threads=1 --compression_type=zlib --compression_use_zstd_dict_trainer=0 --compression_zstd_max_train_bytes=0 --continuous_verification_interval=0 --data_block_index_type=0 --db=$db --db_write_buffer_size=0 --delpercent=0 --delrangepercent=50 --destroy_db_initially=0 --detect_filter_construct_corruption=1 --disable_wal=0 --enable_compaction_filter=1 --disable_auto_compactions=1 --enable_pipelined_write=0 --enable_thread_tracking=0 --expected_values_dir=$expected --fail_if_options_file_error=1 --fifo_allow_compaction=0 --file_checksum_impl=xxh64 --flush_one_in=0 --format_version=5 --get_current_wal_file_one_in=0 --get_live_files_one_in=0 --get_property_one_in=0 --get_sorted_wal_files_one_in=0 --index_block_restart_interval=3 --index_type=0 --ingest_external_file_one_in=100 --initial_auto_readahead_size=0 --iterpercent=0 --key_len_percent_dist=1,30,69 --level_compaction_dynamic_level_bytes=1 --lock_wal_one_in=0 --long_running_snapshots=0 --manual_wal_flush_one_in=0 --mark_for_compaction_one_file_in=0 --max_auto_readahead_size=0 --max_background_compactions=1 --max_bytes_for_level_base=67108864 --max_key=1000 --max_key_len=3 --max_manifest_file_size=1073741824 --max_write_batch_group_size_bytes=64 --max_write_buffer_number=10 --max_write_buffer_size_to_maintain=2097152 --memtable_max_range_deletions=0 --memtable_prefix_bloom_size_ratio=0.01 --memtable_protection_bytes_per_key=8 --memtable_whole_key_filtering=0 --memtablerep=skip_list --min_write_buffer_number_to_merge=1 --mmap_read=0 --mock_direct_io=True --nooverwritepercent=1 --num_file_reads_for_auto_readahead=1 --open_files=-1 --open_metadata_write_fault_one_in=0 --open_read_fault_one_in=0 --open_write_fault_one_in=0 --ops_per_thread=100000 --optimize_filters_for_memory=0 --paranoid_file_checks=0 --partition_filters=0 --partition_pinning=2 --pause_background_one_in=0 --periodic_compaction_seconds=0 --prefix_size=8 --prefixpercent=0 --prepopulate_block_cache=0 --preserve_internal_time_seconds=0 --progress_reports=0 --read_fault_one_in=0 --readahead_size=0 --readpercent=45 --recycle_log_file_num=0 --reopen=0 --secondary_cache_fault_one_in=0 --secondary_cache_uri= --set_options_one_in=0 --snapshot_hold_ops=0 --sst_file_manager_bytes_per_sec=104857600 --sst_file_manager_bytes_per_truncate=1048576 --stats_dump_period_sec=0 --subcompactions=1 --sync=0 --sync_fault_injection=0 --target_file_size_base=167772 --target_file_size_multiplier=1 --test_batches_snapshots=0 --top_level_index_pinning=0 --unpartitioned_pinning=3 --use_direct_io_for_flush_and_compaction=0 --use_direct_reads=0 --use_full_merge_v1=0 --use_get_entity=0 --use_merge=0 --use_multi_get_entity=0 --use_multiget=1 --use_put_entity_one_in=0 --use_write_buffer_manager=0 --user_timestamp_size=0 --value_size_mult=32 --verification_only=0 --verify_checksum=0 --verify_checksum_one_in=0 --verify_db_one_in=0 --verify_file_checksums_one_in=0 --verify_iterator_with_expected_state_one_in=0 --verify_sst_unique_id_in_manifest=0 --wal_bytes_per_sync=0 --wal_compression=none --write_buffer_size=335544 --write_dbid_to_manifest=1 --write_fault_one_in=0 --writepercent=5
 ```

Reviewed By: jaykorean, ajkr

Differential Revision: D53075223

Pulled By: hx235

fbshipit-source-id: 61aa4a79de5d123a55eb5ac08b449a8362cc91ae
2024-01-25 09:58:25 -08:00
Yu Zhang 9243f1b668 Ensures PendingExpectedValue either Commit or Rollback (#12244)
Summary:
This PR adds automatic checks in the `PendingExpectedValue` class to make sure it's either committed or rolled back before being destructed.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12244

Reviewed By: hx235

Differential Revision: D52853794

Pulled By: jowlyzhang

fbshipit-source-id: 1dcd7695f2c52b79695be0abe11e861047637dc4
2024-01-24 11:04:40 -08:00
Changyu Bi a29db3048f Fix TestGetEntity failure with UDT (#12264)
Summary:
Use the read option with right timestamp and skip verification when using old timestamps.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12264

Test Plan:
I can repro with small keyspace:
```
./db_stress --acquire_snapshot_one_in=10000 --adaptive_readahead=0 --allow_data_in_errors=True --async_io=0 --auto_readahead_size=1 --avoid_flush_during_recovery=0 --avoid_unnecessary_blocking_io=0 --backup_max_size=104857600 --backup_one_in=1000 --batch_protection_bytes_per_key=0 --block_protection_bytes_per_key=4 --block_size=16384 --bloom_before_level=7 --bloom_bits=15 --bottommost_compression_type=xpress --bottommost_file_compaction_delay=0 --bytes_per_sync=262144 --cache_index_and_filter_blocks=1 --cache_size=33554432 --cache_type=fixed_hyper_clock_cache --charge_compression_dictionary_building_buffer=1 --charge_file_metadata=0 --charge_filter_construction=1 --charge_table_reader=1 --checkpoint_one_in=10000 --checksum_type=kXXH3 --clear_column_family_one_in=0 --compact_files_one_in=1000000 --compact_range_one_in=1000000 --compaction_pri=4 --compaction_readahead_size=0 --compaction_style=1 --compaction_ttl=0 --compressed_secondary_cache_size=16777216 --compression_checksum=1 --compression_max_dict_buffer_bytes=0 --compression_max_dict_bytes=0 --compression_parallel_threads=1 --compression_type=snappy --compression_use_zstd_dict_trainer=0 --compression_zstd_max_train_bytes=0 --continuous_verification_interval=0 --data_block_index_type=1 --db_write_buffer_size=0 --delpercent=4 --delrangepercent=1 --destroy_db_initially=1 --detect_filter_construct_corruption=1 --disable_wal=0 --enable_compaction_filter=0 --enable_pipelined_write=1 --enable_thread_tracking=0 --fail_if_options_file_error=1 --fifo_allow_compaction=1 --file_checksum_impl=big --flush_one_in=1000 --format_version=2 --get_current_wal_file_one_in=0 --get_live_files_one_in=10000 --get_property_one_in=100000 --get_sorted_wal_files_one_in=0 --index_block_restart_interval=13 --index_type=0 --ingest_external_file_one_in=0 --initial_auto_readahead_size=16384 --iterpercent=10 --key_len_percent_dist=1,30,69 --level_compaction_dynamic_level_bytes=0 --lock_wal_one_in=10000 --log2_keys_per_lock=10 --long_running_snapshots=0 --manual_wal_flush_one_in=1000 --mark_for_compaction_one_file_in=0 --max_auto_readahead_size=0 --max_background_compactions=20 --max_bytes_for_level_base=10485760 --max_key=1000 --max_key_len=3 --max_manifest_file_size=16384 --max_write_batch_group_size_bytes=1048576 --max_write_buffer_number=3 --max_write_buffer_size_to_maintain=2097152 --memtable_max_range_deletions=0 --memtable_prefix_bloom_size_ratio=0.01 --memtable_protection_bytes_per_key=0 --memtable_whole_key_filtering=0 --memtablerep=skip_list --min_write_buffer_number_to_merge=2 --mmap_read=1 --mock_direct_io=False --nooverwritepercent=1 --num_file_reads_for_auto_readahead=1 --open_files=100 --open_metadata_write_fault_one_in=0 --open_read_fault_one_in=32 --open_write_fault_one_in=16 --ops_per_thread=200000 --optimize_filters_for_memory=0 --paranoid_file_checks=0 --partition_filters=0 --partition_pinning=2 --pause_background_one_in=1000000 --periodic_compaction_seconds=0 --persist_user_defined_timestamps=1 --prefix_size=8 --prefixpercent=5 --prepopulate_block_cache=1 --preserve_internal_time_seconds=3600 --progress_reports=0 --read_fault_one_in=0 --readahead_size=16384 --readpercent=45 --recycle_log_file_num=1 --reopen=20 --secondary_cache_fault_one_in=32 --secondary_cache_uri= --snapshot_hold_ops=100000 --sst_file_manager_bytes_per_sec=0 --sst_file_manager_bytes_per_truncate=0 --stats_dump_period_sec=0 --subcompactions=3 --sync=0 --sync_fault_injection=0 --target_file_size_base=524288 --target_file_size_multiplier=2 --test_batches_snapshots=0 --test_cf_consistency=0 --top_level_index_pinning=1 --unpartitioned_pinning=1 --use_direct_io_for_flush_and_compaction=0 --use_direct_reads=0 --use_full_merge_v1=0 --use_get_entity=1 --use_merge=0 --use_multi_get_entity=0 --use_multiget=1 --use_put_entity_one_in=0 --use_txn=0 --use_write_buffer_manager=0 --user_timestamp_size=8 --value_size_mult=32 --verification_only=0 --verify_checksum=1 --verify_checksum_one_in=1000000 --verify_db_one_in=100000 --verify_file_checksums_one_in=100000 --verify_iterator_with_expected_state_one_in=5 --verify_sst_unique_id_in_manifest=1 --wal_bytes_per_sync=524288 --wal_compression=zstd --write_buffer_size=4194304 --write_dbid_to_manifest=0 --write_fault_one_in=0 --writepercent=35 --db=/dev/shm/rocksdb_test/rocksdb_crashtest_whitebox --expected_values_dir=/dev/shm/rocksdb_test/rocksdb_crashtest_expected

Errors when run with main:
error : inconsistent values for key 0x00000000000000E5000000000000012B000000000000014D: expected state has the key, GetEntity returns NotFound.

error : inconsistent values for key 0x0000000000000009000000000000012B0000000000000254: GetEntity returns :0x010000000504070609080B0A0D0C0F0E111013121514171619181B1A1D1C1F1E212023222524272629282B2A2D2C2F2E313033323534373639383B3A3D3C3F3E, expected state does not have the key.
```

Reviewed By: jaykorean

Differential Revision: D52966251

Pulled By: cbi42

fbshipit-source-id: 09436a1b747f1ac545140fc83a2fa4555fef51c1
2024-01-22 12:15:17 -08:00
Yu Zhang c4228abdc0 Fix backup/checkpoint stress test failure (#12227)
Summary:
This PR fixes this type of stress test failure that could happen in either checkpoint or backup. Example failure messages are like this:

`Failure in a backup/restore operation with: Corruption: 0x00000000000001D5000000000000012B00000000000000FD exists in original db but not in restore`

`A checkpoint operation failed with: Corruption: 0x0000000000000365000000000000012B0000000000000067 exists in original db but not in checkpoint /...`

The internal task has an example test command to quickly reproduce this type of error.

The common symptom of these test failures are these expected keys do not exist in the original db either. The root cause is `TestCheckpoint` and `TestBackupRestore` both use the expected state as a proxy for the state of the original db when it comes to check a key's existence. 0758271d51/db_stress_tool/db_stress_test_base.cc (L1838)

This `ExpectedState::Exists` API returns true if a key has a pending write, such as a pending put. In usual case, this pending put should either soon materialize to an actual write when `PendingExpectedValue::Commit` is called to reflect a successful write to the DB, or test should be safely terminated if write to DB fails. All of which happens while a key is locked. So checkpoint and backup usually won't see the discrepancy between db and expected state caused by pending writes. However, the external file ingestion test currently has a path that will proceed the test after a failed ingestion caused by injected errors, leaving the pending put in the expected state. 0758271d51/db_stress_tool/no_batched_ops_stress.cc (L1577-L1589)

I think a proper and future proof fix for this is to explicitly rollback a pending state when a db write operation failed so that expected state do not diverge from db in the first place. I added a `PendingExpectedValue::Rollback` API so that we don't implicitly depend on thread termination to prevent test failures. Another place that could cause same divergence as external file ingestion is `PreloadDbAndReopenAsReadOnly`.
0758271d51/db_stress_tool/db_stress_test_base.cc (L616-L619)

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12227

Reviewed By: hx235

Differential Revision: D52705470

Pulled By: jowlyzhang

fbshipit-source-id: b21586b037caeeba29a2cff8c2fdc6f1d0bda9cf
2024-01-16 13:28:51 -08:00
Yu Zhang 8d0c09d7e6 Abort verification when expected state has pending writes / db return non OK(NotFound) status (#12232)
Summary:
In the current flow, the verification will pass and continue the test when db return non Ok(NotFound) status while expected state has pending writes.

fdfd044bb2/db_stress_tool/no_batched_ops_stress.cc (L2054-L2065)

We can just abort when such a db status is ever encountered. This can prevent follow up tests like `TestCheckpoint` and `TestBackupRestore` to consider such a key as existing in the db via the `ExpectedState::Exists` API. This could be a reason for some recent test failures in this path.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12232

Reviewed By: cbi42

Differential Revision: D52737393

Pulled By: jowlyzhang

fbshipit-source-id: f2658c5332ccd42f6190783960e2dc6fcd81ccc5
2024-01-12 12:23:09 -08:00
Jay Huh fdfd044bb2 Logging for test failure due to get/multiget inconsistency (#12228)
Summary:
Additional logging for debugging purpose

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12228

Test Plan: CI

Reviewed By: cbi42

Differential Revision: D52713401

Pulled By: jaykorean

fbshipit-source-id: 535972d60debb70c220887f0f4c06a32f7668f72
2024-01-11 18:15:17 -08:00
Jay Huh 0758271d51 Fix TestGetEntity in stress test when UDT is enabled (#12222)
Summary:
Similar to https://github.com/facebook/rocksdb/issues/11249 , we started to get failures from `TestGetEntity` when the User-defined-timestamp was enabled. Applying the same fix as the `TestGet`

_Scenario copied from  #11249_

<table>
  <tr>
    <th>TestGet thread</th>
    <th> A writing thread</th>
  </tr>
  <tr>
    <td>read_opts.timestamp = GetNow()</td>
    <td></td>
  </tr>
  <tr>
    <td></td>
    <td>Lock key, do write</td>
  </tr>
  <tr>
    <td>Lock key, read(read_opts) return NotFound</td>
    <td></td>
  </tr>
</table>

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12222

Reviewed By: jowlyzhang

Differential Revision: D52678830

Pulled By: jaykorean

fbshipit-source-id: 6e154f67bb32968add8fea0b7ae7c4858ea64ee7
2024-01-10 16:35:54 -08:00
Changyu Bi cd15331711 Print status when VerifyOrSyncValue() fails with non-OK status (#12217)
Summary:
This should print more helpful message when a non-ok status like Corruption is returned.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12217

Test Plan: CI passes.

Reviewed By: jaykorean

Differential Revision: D52637595

Pulled By: cbi42

fbshipit-source-id: e810eeb4cba633d4d4c5d198da4468995e4ed427
2024-01-09 14:20:08 -08:00
Hui Xiao 06e593376c Group SST write in flush, compaction and db open with new stats (#11910)
Summary:
## Context/Summary
Similar to https://github.com/facebook/rocksdb/pull/11288, https://github.com/facebook/rocksdb/pull/11444, categorizing SST/blob file write according to different io activities allows more insight into the activity.

For that, this PR does the following:
- Tag different write IOs by passing down and converting WriteOptions to IOOptions
- Add new SST_WRITE_MICROS histogram in WritableFileWriter::Append() and breakdown FILE_WRITE_{FLUSH|COMPACTION|DB_OPEN}_MICROS

Some related code refactory to make implementation cleaner:
- Blob stats
   - Replace high-level write measurement with low-level WritableFileWriter::Append() measurement for BLOB_DB_BLOB_FILE_WRITE_MICROS. This is to make FILE_WRITE_{FLUSH|COMPACTION|DB_OPEN}_MICROS  include blob file. As a consequence, this introduces some behavioral changes on it, see HISTORY and db bench test plan below for more info.
   - Fix bugs where BLOB_DB_BLOB_FILE_SYNCED/BLOB_DB_BLOB_FILE_BYTES_WRITTEN include file failed to sync and bytes failed to write.
- Refactor WriteOptions constructor for easier construction with io_activity and rate_limiter_priority
- Refactor DBImpl::~DBImpl()/BlobDBImpl::Close() to bypass thread op verification
- Build table
   - TableBuilderOptions now includes Read/WriteOpitons so BuildTable() do not need to take these two variables
   - Replace the io_priority passed into BuildTable() with TableBuilderOptions::WriteOpitons::rate_limiter_priority. Similar for BlobFileBuilder.
This parameter is used for dynamically changing file io priority for flush, see  https://github.com/facebook/rocksdb/pull/9988?fbclid=IwAR1DtKel6c-bRJAdesGo0jsbztRtciByNlvokbxkV6h_L-AE9MACzqRTT5s for more
   - Update ThreadStatus::FLUSH_BYTES_WRITTEN to use io_activity to track flush IO in flush job and db open instead of io_priority

## Test
### db bench

Flush
```
./db_bench --statistics=1 --benchmarks=fillseq --num=100000 --write_buffer_size=100

rocksdb.sst.write.micros P50 : 1.830863 P95 : 4.094720 P99 : 6.578947 P100 : 26.000000 COUNT : 7875 SUM : 20377
rocksdb.file.write.flush.micros P50 : 1.830863 P95 : 4.094720 P99 : 6.578947 P100 : 26.000000 COUNT : 7875 SUM : 20377
rocksdb.file.write.compaction.micros P50 : 0.000000 P95 : 0.000000 P99 : 0.000000 P100 : 0.000000 COUNT : 0 SUM : 0
rocksdb.file.write.db.open.micros P50 : 0.000000 P95 : 0.000000 P99 : 0.000000 P100 : 0.000000 COUNT : 0 SUM : 0
```

compaction, db oopen
```
Setup: ./db_bench --statistics=1 --benchmarks=fillseq --num=10000 --disable_auto_compactions=1 -write_buffer_size=100 --db=../db_bench

Run:./db_bench --statistics=1 --benchmarks=compact  --db=../db_bench --use_existing_db=1

rocksdb.sst.write.micros P50 : 2.675325 P95 : 9.578788 P99 : 18.780000 P100 : 314.000000 COUNT : 638 SUM : 3279
rocksdb.file.write.flush.micros P50 : 0.000000 P95 : 0.000000 P99 : 0.000000 P100 : 0.000000 COUNT : 0 SUM : 0
rocksdb.file.write.compaction.micros P50 : 2.757353 P95 : 9.610687 P99 : 19.316667 P100 : 314.000000 COUNT : 615 SUM : 3213
rocksdb.file.write.db.open.micros P50 : 2.055556 P95 : 3.925000 P99 : 9.000000 P100 : 9.000000 COUNT : 23 SUM : 66
```

blob stats - just to make sure they aren't broken by this PR
```
Integrated Blob DB

Setup: ./db_bench --enable_blob_files=1 --statistics=1 --benchmarks=fillseq --num=10000 --disable_auto_compactions=1 -write_buffer_size=100 --db=../db_bench

Run:./db_bench --enable_blob_files=1 --statistics=1 --benchmarks=compact  --db=../db_bench --use_existing_db=1

pre-PR:
rocksdb.blobdb.blob.file.write.micros P50 : 7.298246 P95 : 9.771930 P99 : 9.991813 P100 : 16.000000 COUNT : 235 SUM : 1600
rocksdb.blobdb.blob.file.synced COUNT : 1
rocksdb.blobdb.blob.file.bytes.written COUNT : 34842

post-PR:
rocksdb.blobdb.blob.file.write.micros P50 : 2.000000 P95 : 2.829360 P99 : 2.993779 P100 : 9.000000 COUNT : 707 SUM : 1614
- COUNT is higher and values are smaller as it includes header and footer write
- COUNT is 3X higher due to each Append() count as one post-PR, while in pre-PR, 3 Append()s counts as one. See https://github.com/facebook/rocksdb/pull/11910/files#diff-32b811c0a1c000768cfb2532052b44dc0b3bf82253f3eab078e15ff201a0dabfL157-L164

rocksdb.blobdb.blob.file.synced COUNT : 1 (stay the same)
rocksdb.blobdb.blob.file.bytes.written COUNT : 34842 (stay the same)
```

```
Stacked Blob DB

Run: ./db_bench --use_blob_db=1 --statistics=1 --benchmarks=fillseq --num=10000 --disable_auto_compactions=1 -write_buffer_size=100 --db=../db_bench

pre-PR:
rocksdb.blobdb.blob.file.write.micros P50 : 12.808042 P95 : 19.674497 P99 : 28.539683 P100 : 51.000000 COUNT : 10000 SUM : 140876
rocksdb.blobdb.blob.file.synced COUNT : 8
rocksdb.blobdb.blob.file.bytes.written COUNT : 1043445

post-PR:
rocksdb.blobdb.blob.file.write.micros P50 : 1.657370 P95 : 2.952175 P99 : 3.877519 P100 : 24.000000 COUNT : 30001 SUM : 67924
- COUNT is higher and values are smaller as it includes header and footer write
- COUNT is 3X higher due to each Append() count as one post-PR, while in pre-PR, 3 Append()s counts as one. See https://github.com/facebook/rocksdb/pull/11910/files#diff-32b811c0a1c000768cfb2532052b44dc0b3bf82253f3eab078e15ff201a0dabfL157-L164

rocksdb.blobdb.blob.file.synced COUNT : 8 (stay the same)
rocksdb.blobdb.blob.file.bytes.written COUNT : 1043445 (stay the same)
```

###  Rehearsal CI stress test
Trigger 3 full runs of all our CI stress tests

###  Performance

Flush
```
TEST_TMPDIR=/dev/shm ./db_basic_bench_pre_pr --benchmark_filter=ManualFlush/key_num:524288/per_key_size:256 --benchmark_repetitions=1000
-- default: 1 thread is used to run benchmark; enable_statistics = true

Pre-pr: avg 507515519.3 ns
497686074,499444327,500862543,501389862,502994471,503744435,504142123,504224056,505724198,506610393,506837742,506955122,507695561,507929036,508307733,508312691,508999120,509963561,510142147,510698091,510743096,510769317,510957074,511053311,511371367,511409911,511432960,511642385,511691964,511730908,

Post-pr: avg 511971266.5 ns, regressed 0.88%
502744835,506502498,507735420,507929724,508313335,509548582,509994942,510107257,510715603,511046955,511352639,511458478,512117521,512317380,512766303,512972652,513059586,513804934,513808980,514059409,514187369,514389494,514447762,514616464,514622882,514641763,514666265,514716377,514990179,515502408,
```

Compaction
```
TEST_TMPDIR=/dev/shm ./db_basic_bench_{pre|post}_pr --benchmark_filter=ManualCompaction/comp_style:0/max_data:134217728/per_key_size:256/enable_statistics:1  --benchmark_repetitions=1000
-- default: 1 thread is used to run benchmark

Pre-pr: avg 495346098.30 ns
492118301,493203526,494201411,494336607,495269217,495404950,496402598,497012157,497358370,498153846

Post-pr: avg 504528077.20, regressed 1.85%. "ManualCompaction" include flush so the isolated regression for compaction should be around 1.85-0.88 = 0.97%
502465338,502485945,502541789,502909283,503438601,504143885,506113087,506629423,507160414,507393007
```

Put with WAL (in case passing WriteOptions slows down this path even without collecting SST write stats)
```
TEST_TMPDIR=/dev/shm ./db_basic_bench_pre_pr --benchmark_filter=DBPut/comp_style:0/max_data:107374182400/per_key_size:256/enable_statistics:1/wal:1  --benchmark_repetitions=1000
-- default: 1 thread is used to run benchmark

Pre-pr: avg 3848.10 ns
3814,3838,3839,3848,3854,3854,3854,3860,3860,3860

Post-pr: avg 3874.20 ns, regressed 0.68%
3863,3867,3871,3874,3875,3877,3877,3877,3880,3881
```

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11910

Reviewed By: ajkr

Differential Revision: D49788060

Pulled By: hx235

fbshipit-source-id: 79e73699cda5be3b66461687e5147c2484fc5eff
2023-12-29 15:29:23 -08:00
Yu Zhang c2ab4e754b Add initial support to stress test persist_user_defined_timestamps (#12124)
Summary:
This PR adds initial stress testing for the user-defined timestamps in memtable only feature. Each flavor of the `*_ts` crash test get a 1 in 3 chance to run with timestamps not persisted, this setting is initialized once and kept consistent across the following re-runs.

This initial stress test included these things besides disabling incompatible feature combinations to make the test run more stably:
1) It currently only run test methods that validates db state with expected state. Not the ones that validate db state by comparing result from one API to another API. Such as `TestMultiGet` (compared with `Get`), similarly `TestMultiGetEntity`, `TestIterate` (compare src iterator to a control iterator). Due to timestamps being removed, results from one API to another API is not directly comparable as it is now. More test logic to handle that need to be added, will do that in a follow up.

2) Even when comparing db state to expected state, sometimes the db can receive `InvalidArgument` too due to timestamps getting flushed and removed. Added some logic to handle that.

3) When timestamps are not persisted, we don't try to read with older timestamp. Since that's making it easier to get `InvalidArgument`. And this capability is not yet needed by our customer so it's disabled for now.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12124

Test Plan: running multiple flavor of this test on continuous run for sometime before checkin

Reviewed By: ltamasi

Differential Revision: D51916267

Pulled By: jowlyzhang

fbshipit-source-id: 3f3eb5f9618d05d296062820e0ef5cb8edc7c2b2
2023-12-12 09:35:29 -08:00
Yu Zhang ba8fa0f546 internal_repo_rocksdb (4372117296613874540) (#12117)
Summary: Pull Request resolved: https://github.com/facebook/rocksdb/pull/12117

Reviewed By: ajkr

Differential Revision: D51745846

Pulled By: jowlyzhang

fbshipit-source-id: 51c806a484b3b43d174b06d2cfe9499191d09914
2023-12-04 11:17:32 -08:00
anand76 4d04138512 Add dynamic disabling of compressed cache to db_stress (#12102)
Summary:
We now support re-enabling the compressed portion of the `TieredCache` after dynamically disabling it. Add it to db_stress for testing purposes.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12102

Reviewed By: akankshamahajan15

Differential Revision: D51594259

Pulled By: anand1976

fbshipit-source-id: ea544e30a5ebd6290fc9ed46a241f09634764d2a
2023-11-27 13:00:15 -08:00
anand76 2222caec9e Make CacheWithSecondaryAdapter reservation accounting more robust (#12059)
Summary:
`CacheWithSecondaryAdapter` can distribute placeholder reservations across the primary and secondary caches. The current implementation of the accounting is quite complicated in order to avoid using a mutex. This may cause the accounting to be slightly off after changes to the cache capacity and ratio, resulting in assertion failures. There's also a bug in the unlikely event that the total reservation exceeds the cache capacity. Furthermore, the current implementation is difficult to reason about.

This PR simplifies it by doing the accounting while holding a mutex. The reservations are processed in 1MB chunks in order to avoid taking a lock too frequently. As a side effect, this also removes the restriction of not allowing to increase the compressed secondary cache capacity after decreasing it to 0.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12059

Test Plan: Existing unit tests, and a new test for capacity increase from 0

Reviewed By: pdillinger

Differential Revision: D51278686

Pulled By: anand1976

fbshipit-source-id: 7e1ad2c50694772997072dd59cab35c93c12ba4f
2023-11-14 16:25:52 -08:00
Changyu Bi b48480cfd0 Enable TestIterateAgainstExpected() in more crash tests (#12040)
Summary:
db_stress flag `verify_iterator_with_expected_state_one_in` is only enabled for in crash test if --simple flag is set. This PR enables it for all supported crash tests by enabling it by default. This adds coverage for --txn and --enable_ts crash tests.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12040

Test Plan:
ran crash tests that disabled this flag before for a few hours
```
python3 ./tools/db_crashtest.py blackbox --verify_iterator_with_expected_state_one_in=1 --txn --txn_write_policy=[0,1,2]

python3 ./tools/db_crashtest.py blackbox --verify_iterator_with_expected_state_one_in=1 --enable_ts
```

Reviewed By: ajkr, hx235

Differential Revision: D50980001

Pulled By: cbi42

fbshipit-source-id: 3daf6b4c32bdddc5df057240068162aa1a907587
2023-11-03 16:27:11 -07:00
Changyu Bi 8505b26db1 Fix stress test error message for black/whitebox test to catch failures (#12039)
Summary:
black/whitebox crash test relies on error/fail keyword in stderr to catch stress test failure. If a db_stress run prints an error message without these keyword, and then is killed before it graceful exits and prints out "Verification failed" here (2648e0a747/db_stress_tool/db_stress_driver.cc (L256)), the error won't be caught. This is more likely to happen if db_stress is printing a stack trace. This PR fixes some error messages. Ideally in the future we should not rely on searching for keywords in stderr to determine failed stress tests.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12039

Test Plan:
```
Added the following change on top of this PR to simulate exit without relevant keyword:

@@ -1586,6 +1587,8 @@ class NonBatchedOpsStressTest : public StressTest {
     assert(thread);
     assert(!rand_column_families.empty());
     assert(!rand_keys.empty());
+    fprintf(stderr, "Inconsistency");
+    thread->shared->SafeTerminate();

python3 ./tools/db_crashtest.py blackbox --simple --verify_iterator_with_expected_state_one_in=1 --interval=10

will print a stack trace but continue to run db_stress.
```

Reviewed By: jaykorean

Differential Revision: D50960076

Pulled By: cbi42

fbshipit-source-id: 5c60a1be04ce4a43adbd33f040d54434f2ae24c9
2023-11-03 09:53:22 -07:00
Yu Zhang a42910537d Save the correct user comparator name in OPTIONS file (#12037)
Summary:
I noticed the user comparator name in OPTIONS file can be incorrect when working on a recent stress test failure. The name of the comparator retrieved via the "Comparator::GetRootComparator" API is saved in OPTIONS file as the user comparator. The intention was to get the user comparator wrapped in the internal comparator. However `ImmutableCFOptions.user_comparator` has always been a user comparator of type `Comparator`. The corresponding `GetRootComparator` API is also defined only for user comparator type `Comparator`, not the internal key comparator type `InternalKeyComparator`.

For built in comparator `BytewiseComparator` and `ReverseBytewiseComparator`, there is no difference between `Comparator::Name` and `Comparator::GetRootComparator::Name` because these built in comparators' root comparator is themselves. However, for built in comparator `BytewiseComparatorWithU64Ts` and `ReverseBytewiseComparatorWithU64Ts`, there are differences. So this change update the logic to persist the user comparator's name, not its root comparator's name.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12037

Test Plan:
The restore flow in stress test, which relies on converting Options object to string and back to Options object is updated to help validate comparator object can be correctly serialized and deserialized with the OPTIONS file mechanism

Updated unit test to use a comparator that has a root comparator that is not itself.

Reviewed By: cbi42

Differential Revision: D50909750

Pulled By: jowlyzhang

fbshipit-source-id: 9086d7135c7a6f4b5565fb47fce194ea0a024f52
2023-11-02 13:27:59 -07:00
Yu Zhang 0b057a7acc Initialize comparator explicitly in PrepareOptionsForRestoredDB() (#12034)
Summary:
This is to fix below error seeing in stress test:
```
Failure in DB::Open in backup/restore with: Invalid argument: Cannot open a column family and disable user-defined timestamps feature if its existing persist_user_defined_timestamps flag is not false.
```

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12034

Reviewed By: cbi42

Differential Revision: D50860689

Pulled By: jowlyzhang

fbshipit-source-id: ebc6cf0a75caa43d3d3bd58e3d5c2ac754cc637c
2023-10-31 16:10:48 -07:00
Changyu Bi 2818a74b95 Initialize merge operator explicitly in PrepareOptionsForRestoredDB() (#12033)
Summary:
We are seeing the following stress test failure: `Failure in DB::Get in backup/restore with: Invalid argument: merge_operator is not properly initialized. Verification failed: Backup/restore failed: Invalid argument: merge_operator is not properly initialized.`. The reason is likely that `GetColumnFamilyOptionsFromString()` does not set merge operator if it's a customized merge operator. Fixing it by initializing merge operator explicitly.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12033

Test Plan:
this repro gives the error consistently before this PR
```
./db_stress --acquire_snapshot_one_in=10000 --adaptive_readahead=0 --allow_concurrent_memtable_write=1 --allow_data_in_errors=True --async_io=0 --atomic_flush=1 --auto_readahead_size=1 --avoid_flush_during_recovery=0 --avoid_unnecessary_blocking_io=1 --backup_max_size=1048576000000 --backup_one_in=50 --batch_protection_bytes_per_key=8 --block_protection_bytes_per_key=2 --block_size=16384 --bloom_before_level=2147483646 --bloom_bits=31.014388066505518 --bottommost_compression_type=lz4hc --bottommost_file_compaction_delay=0 --bytes_per_sync=0 --cache_index_and_filter_blocks=0 --cache_size=33554432 --cache_type=fixed_hyper_clock_cache --charge_compression_dictionary_building_buffer=1 --charge_file_metadata=1 --charge_filter_construction=0 --charge_table_reader=1 --checkpoint_one_in=1000000 --checksum_type=kxxHash --clear_column_family_one_in=0 --column_families=1 --compact_files_one_in=1000000 --compact_range_one_in=1000000 --compaction_pri=3 --compaction_readahead_size=0 --compaction_ttl=10 --compressed_secondary_cache_ratio=0.0 --compressed_secondary_cache_size=0 --compression_checksum=1 --compression_max_dict_buffer_bytes=4095 --compression_max_dict_bytes=16384 --compression_parallel_threads=1 --compression_type=none --compression_use_zstd_dict_trainer=1 --compression_zstd_max_train_bytes=0 --continuous_verification_interval=0 --data_block_index_type=0 --db=/dev/shm/rocksdb_test/rocksdb_crashtest_blackbox --db_write_buffer_size=0 --delpercent=4 --delrangepercent=1 --destroy_db_initially=1 --detect_filter_construct_corruption=0 --disable_wal=1 --enable_compaction_filter=0 --enable_pipelined_write=0 --enable_thread_tracking=0 --expected_values_dir=/dev/shm/rocksdb_test/rocksdb_crashtest_expected --fail_if_options_file_error=1 --fifo_allow_compaction=0 --file_checksum_impl=xxh64 --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=10 --index_type=2 --ingest_external_file_one_in=0 --initial_auto_readahead_size=16384 --iterpercent=10 --key_len_percent_dist=1,30,69 --level_compaction_dynamic_level_bytes=1 --lock_wal_one_in=1000000 --long_running_snapshots=1 --manual_wal_flush_one_in=0 --mark_for_compaction_one_file_in=10 --max_auto_readahead_size=524288 --max_background_compactions=1 --max_bytes_for_level_base=67108864 --max_key=100 --max_key_len=3 --max_manifest_file_size=1073741824 --max_write_batch_group_size_bytes=1048576 --max_write_buffer_number=3 --max_write_buffer_size_to_maintain=8388608 --memtable_max_range_deletions=1000 --memtable_prefix_bloom_size_ratio=0 --memtable_protection_bytes_per_key=2 --memtable_whole_key_filtering=0 --memtablerep=skip_list --min_write_buffer_number_to_merge=2 --mmap_read=1 --mock_direct_io=False --nooverwritepercent=1 --num_file_reads_for_auto_readahead=0 --open_files=-1 --open_metadata_write_fault_one_in=0 --open_read_fault_one_in=0 --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=0 --prefix_size=-1 --prefixpercent=0 --prepopulate_block_cache=1 --preserve_internal_time_seconds=0 --progress_reports=0 --read_fault_one_in=0 --readahead_size=0 --readpercent=50 --recycle_log_file_num=0 --reopen=0 --secondary_cache_fault_one_in=0 --set_options_one_in=0 --snapshot_hold_ops=100000 --sst_file_manager_bytes_per_sec=0 --sst_file_manager_bytes_per_truncate=0 --stats_dump_period_sec=0 --subcompactions=1 --sync=0 --sync_fault_injection=1 --target_file_size_base=16777216 --target_file_size_multiplier=1 --test_batches_snapshots=0 --top_level_index_pinning=0 --unpartitioned_pinning=1 --use_direct_io_for_flush_and_compaction=0 --use_direct_reads=0 --use_full_merge_v1=0 --use_get_entity=0 --use_merge=1 --use_multi_get_entity=0 --use_multiget=1 --use_put_entity_one_in=10 --use_write_buffer_manager=0 --user_timestamp_size=0 --value_size_mult=32 --verification_only=0 --verify_checksum=1 --verify_checksum_one_in=1000000 --verify_db_one_in=100000 --verify_file_checksums_one_in=1000000 --verify_iterator_with_expected_state_one_in=5 --verify_sst_unique_id_in_manifest=1 --wal_bytes_per_sync=524288 --wal_compression=zstd --write_buffer_size=33554432 --write_dbid_to_manifest=0 --write_fault_one_in=0 --writepercent=35
```

Reviewed By: hx235

Differential Revision: D50825558

Pulled By: cbi42

fbshipit-source-id: 8468dc0444c112415a515af8291ef3abec8a42de
2023-10-31 07:39:41 -07:00
Hui Xiao 212b5bf826 Deep-copy Options in restored db for stress test to avoid race with SetOptions() (#12015)
Summary:
**Context**
DB open will persist the `Options` in memory to options file and verify the file right after the write. The verification is done by comparing the options from parsing the written options file against the `Options` object in memory. Upon inconsistency, corruption such as https://github.com/facebook/rocksdb/blob/main/options/options_parser.cc#L725 will be returned.

This verification assumes the `Options` object in memory is not changed from before the write till the verification. This assumption can break during [opening the restored db in stress test](0f141352d8/db_stress_tool/db_stress_test_base.cc (L1784-L1799)).

This [line](0f141352d8/db_stress_tool/db_stress_test_base.cc (L1770)) makes it shares some pointer options (e.g, `std::shared_ptr<const FilterPolicy> filter_policy`) with other threads (e.g, SetOptions()) in db stress.

And since https://github.com/facebook/rocksdb/pull/11838, filter_policy's field `bloom_before_level ` has now been mutable by SetOptions(). Therefore we started to see stress test failure like below:

```
Failure in DB::Open in backup/restore with: IO error: DB::Open() failed --- Unable to persist Options file: IO error: Unable to persist options.: Corruption: [RocksDBOptionsParser]:failed the verification on BlockBasedTable::: filter_policy.id

Verification failed: Backup/restore failed: IO error: DB::Open() failed --- Unable to persist Options file: IO error: Unable to persist options.: Corruption: [RocksDBOptionsParser]:failed the verification on BlockBasedTable::: filter_policy.id

db_stress: db_stress_tool/db_stress_test_base.cc:479: void rocksdb::StressTest::ProcessStatus(rocksdb::SharedState*, std::string, rocksdb::Status) const: Assertion `false' failed.
```

**Summary**
This PR uses "deep copy" of the `options_` by CreateXXXFromString() to avoid sharing pointer options.

**Test plan**
Run the below db stress command that failed before this PR and pass after
```
./db_stress --column_families=1 --threads=2 --preserve_unverified_changes=0 --acquire_snapshot_one_in=10000 --adaptive_readahead=0 --allow_data_in_errors=True --async_io=0 --auto_readahead_size=1 --avoid_flush_during_recovery=0 --avoid_unnecessary_blocking_io=0 --backup_max_size=104857600 --backup_one_in=10 --batch_protection_bytes_per_key=0 --block_protection_bytes_per_key=1 --block_size=16384 --bloom_before_level=2147483646 --bloom_bits=0 --bottommost_compression_type=disable --bottommost_file_compaction_delay=86400 --bytes_per_sync=0 --cache_index_and_filter_blocks=0 --cache_size=33554432 --cache_type=tiered_auto_hyper_clock_cache --charge_compression_dictionary_building_buffer=0 --charge_file_metadata=0 --charge_filter_construction=0 --charge_table_reader=1 --checkpoint_one_in=1000000 --checksum_type=kXXH3 --clear_column_family_one_in=0 --compact_files_one_in=1000000 --compact_range_one_in=1000000 --compaction_pri=2 --compaction_readahead_size=0 --compaction_ttl=0 --compressed_secondary_cache_ratio=0.3333333333333333 --compressed_secondary_cache_size=0 --compression_checksum=1 --compression_max_dict_buffer_bytes=0 --compression_max_dict_bytes=0 --compression_parallel_threads=1 --compression_type=lz4 --compression_use_zstd_dict_trainer=1 --compression_zstd_max_train_bytes=0 --continuous_verification_interval=0 --data_block_index_type=1 --db=/dev/shm/rocksdb_test/rocksdb_crashtest_blackbox --db_write_buffer_size=8388608 --delpercent=4 --delrangepercent=1 --destroy_db_initially=1 --detect_filter_construct_corruption=0 --disable_wal=0 --enable_compaction_filter=0 --enable_pipelined_write=0 --enable_thread_tracking=0 --expected_values_dir=/dev/shm/rocksdb_test/rocksdb_crashtest_expected --fail_if_options_file_error=1 --fifo_allow_compaction=1 --file_checksum_impl=big --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 --ingest_external_file_one_in=0 --initial_auto_readahead_size=524288 --iterpercent=10 --key_len_percent_dist=1,30,69 --level_compaction_dynamic_level_bytes=1 --lock_wal_one_in=1000000 --long_running_snapshots=1 --manual_wal_flush_one_in=1000 --mark_for_compaction_one_file_in=0 --max_auto_readahead_size=0 --max_background_compactions=20 --max_bytes_for_level_base=10485760 --max_key=2500 --max_key_len=3 --max_manifest_file_size=1073741824 --max_write_batch_group_size_bytes=16777216 --max_write_buffer_number=3 --max_write_buffer_size_to_maintain=4194304 --memtable_max_range_deletions=0 --memtable_prefix_bloom_size_ratio=0 --memtable_protection_bytes_per_key=0 --memtable_whole_key_filtering=0 --memtablerep=skip_list --min_write_buffer_number_to_merge=1 --mmap_read=1 --mock_direct_io=False --nooverwritepercent=1 --num_file_reads_for_auto_readahead=1 --open_files=500000 --open_metadata_write_fault_one_in=0 --open_read_fault_one_in=0 --open_write_fault_one_in=0 --ops_per_thread=100000000 --optimize_filters_for_memory=0 --paranoid_file_checks=1 --partition_filters=0 --partition_pinning=2 --pause_background_one_in=1000000 --periodic_compaction_seconds=0 --prefix_size=-1 --prefixpercent=0 --prepopulate_block_cache=0 --preserve_internal_time_seconds=3600 --progress_reports=0 --read_fault_one_in=1000 --readahead_size=0 --readpercent=50 --recycle_log_file_num=0 --reopen=0 --secondary_cache_fault_one_in=0 --secondary_cache_uri= --set_options_one_in=5 --snapshot_hold_ops=100000 --sst_file_manager_bytes_per_sec=104857600 --sst_file_manager_bytes_per_truncate=1048576 --stats_dump_period_sec=600 --subcompactions=3 --sync=0 --sync_fault_injection=0 --target_file_size_base=2097152 --target_file_size_multiplier=2 --test_batches_snapshots=0 --top_level_index_pinning=2 --unpartitioned_pinning=0 --use_direct_io_for_flush_and_compaction=0 --use_direct_reads=0 --use_full_merge_v1=0 --use_get_entity=0 --use_merge=1 --use_multi_get_entity=0 --use_multiget=1 --use_put_entity_one_in=0 --use_write_buffer_manager=0 --user_timestamp_size=0 --value_size_mult=32 --verification_only=0 --verify_checksum=1 --verify_checksum_one_in=1000000 --verify_db_one_in=100000 --verify_file_checksums_one_in=1000000 --verify_sst_unique_id_in_manifest=1 --wal_bytes_per_sync=524288 --wal_compression=zstd --write_buffer_size=4194304 --write_dbid_to_manifest=1 --write_fault_one_in=0 --writepercent=35
```

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12015

Reviewed By: pdillinger

Differential Revision: D50666136

Pulled By: hx235

fbshipit-source-id: 804acc23aecb4eedfe5c44f732e86291f2420b2b
2023-10-27 17:07:39 -07:00
Changyu Bi 9ded9f789f Fix db_stress FaultInjectionTestFS set up before DB open (#11958)
Summary:
We saw frequent stress test failures with error messages like:
```
Verification failed for column family 0 key ...: value_from_db: , value_from_expected: ..., msg: GetEntity verification: Value not found: NotFound:
```
One cause for this is that data in WAL is lost after a crash. We initialize FaultInjectionTestFS to be not direct writable when write_fault_injection is enabled (see code change). This can cause the first WAL created during DB open to be lost if a db_stress is killed before the first WAL is synced. This PR initializes FaultInjectionTestFS to be direct writable. Note that FaultInjectionTestFS will be configured propertly for write fault injection after DB open in `RunStressTestImpl()`. So this change should not affect write fault injection coverage.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11958

Test Plan:
a repro for the above bug:
```
Simulate crash before first WAL is sealed:

 --- a/db_stress_tool/db_stress_driver.cc
+++ b/db_stress_tool/db_stress_driver.cc
@@ -256,6 +256,7 @@ bool RunStressTestImpl(SharedState* shared) {
     fprintf(stderr, "Verification failed :(\n");
     return false;
   }
+  exit(1);
   return true;
 }

./db_stress --clear_column_family_one_in=0 --column_families=1 --preserve_internal_time_seconds=60 --destroy_db_initially=0 --db=/dev/shm/rocksdb_crashtest_blackbox --db_write_buffer_size=2097152 --destroy_db_initially=0 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --reopen=0 --test_batches_snapshots=0 --threads=1 --ops_per_thread=100 --write_fault_one_in=1000 --sync_fault_injection=0

./db_stress_main  --clear_column_family_one_in=0 --column_families=1 --preserve_internal_time_seconds=60 --destroy_db_initially=0 --db=/dev/shm/rocksdb_crashtest_blackbox --db_write_buffer_size=2097152 --destroy_db_initially=0 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --reopen=0 --test_batches_snapshots=0 --sync_fault_injection=1
```

Reviewed By: akankshamahajan15

Differential Revision: D50300347

Pulled By: cbi42

fbshipit-source-id: 3a4881d72197f5ece82364382a0100912e16c2d6
2023-10-14 13:33:55 -07:00
Changyu Bi 50b0879d50 Do not fail stress test when file ingestion return injected error (#11956)
Summary:
Currently, if file ingestion hit injected error, stress test is considered failed since it prints a message to stderr containing the keyword "error" and db_crashtest.py looks for it in stderr. This PR fixes it by print injected error to stdout.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11956

Test Plan: Check future stress test runs.

Reviewed By: akankshamahajan15

Differential Revision: D50293537

Pulled By: cbi42

fbshipit-source-id: e74915b1b3c6876a61ab6933c4529780362ec02b
2023-10-14 10:08:03 -07:00
Jay Huh d2daa10afc Fix crash_test_with_best_efforts_recovery (#11938)
Summary:
Thanks ltamasi and ajkr  for initial investigations on the test failure. Per the investigations, the following scenario is likely causing the test to fail.

1. Recovery is needed (could be any reason during crash test)
2. Trying to recover from the latest manifest fails (likely due to read error injection)
3. DB opens with recovery from the next manifest which is different from step 2.
4. Expected state is based on the manifest we tried and failed in step 2.
5. Two manifests used in step 2 and 3 are confirmed to have difference in LSM trees (Thanks ltamasi  again for the finding).

```
2023/10/05-11:24:18.942189 56341 [db/version_set.cc:6079] Trying to recover from manifest: /dev/shm/rocksdb_test/rocksdb_crashtest_blackbox/MANIFEST-007184
...
2023/10/05-11:24:18.978007 56341 [db/version_set.cc:6079] Trying to recover from manifest: /dev/shm/rocksdb_test/rocksdb_crashtest_blackbox/MANIFEST-007180
```
```
[ltamasi@devbig1024.prn1 /tmp/x]$ ldb manifest_dump --hex --path=MANIFEST-007184_renamed_ > 2
[ltamasi@devbig1024.prn1 /tmp/x]$ ldb manifest_dump --hex --path=MANIFEST-007180_renamed_ > 1
[ltamasi@devbig1024.prn1 /tmp/x]$ diff 1 2
 --- 1   2023-10-09 10:29:16.966215207 -0700
+++ 2   2023-10-09 10:29:11.984241645 -0700
@@ -13,7 +13,7 @@
  7174:3950254[1875617 .. 2203952]['000000000003415B000000000000012B000000000000007D' seq:1906214, type:1 .. '000000000003CA59000000000000012B000000000000005C' seq:2039838, type:1]
  7175:88060[2074748 .. 2203892]['000000000003CA6300000000000000CF78787878787878' seq:2167539, type:2 .. '000000000003D08F000000000000012B0000000000000130' seq:2112478, type:0]
 --- level 6 --- version# 1 ---
- 7057:3132633[0 .. 2046144]['0000000000000009000000000000000978' seq:0, type:1 .. '0000000000005F8B000000000000012B00000000000002AC' seq:0, type:1]
+ 7219:2135565[0 .. 2046144]['0000000000000009000000000000000978' seq:0, type:1 .. '0000000000005F8B000000000000012B00000000000002AC' seq:0, type:1]
  7061:827724[0 .. 2046131]['0000000000005F95000000000000000778787878787878' seq:0, type:1 .. '000000000000784F000000000000012B0000000000000113' seq:0, type:1]
  6763:1352[0 .. 0]['000000000000784F000000000000012B0000000000000129' seq:0, type:1 .. '000000000000784F000000000000012B0000000000000129' seq:0, type:1]
  7173:4812291[0 .. 2203957]['000000000000784F000000000000012B0000000000000138' seq:0, type:1 .. '0000000000020FAE787878787878' seq:0, type:1]
@@ -77,4 +77,4 @@
 --- level 61 --- version# 1 ---
 --- level 62 --- version# 1 ---
 --- level 63 --- version# 1 ---
-next_file_number 7182 last_sequence 2203963  prev_log_number 0 max_column_family 0 min_log_number_to_keep 7015
+next_file_number 7221 last_sequence 2203963  prev_log_number 0 max_column_family 0 min_log_number_to_keep 7015
```

We have two options to fix this. Either skip verification against expected state or disable read injection when BE recovery is enabled. I chose to skip verification against expected state per discussion. (See comments in this PR)

Please note that some linter changes were included in this PR.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11938

Test Plan:
```
TEST_TMPDIR=/dev/shm/rocksdb make crash_test_with_best_efforts_recovery
```

Reviewed By: ltamasi

Differential Revision: D50136341

Pulled By: jaykorean

fbshipit-source-id: ac7434d592aebc148bfc3a4fcaa34936f136b95c
2023-10-11 14:26:10 -07:00
anand76 20b4f1356e Enable write fault injection in db_stress (#11924)
Summary:
This PR depends on https://github.com/facebook/rocksdb/issues/11879 . Enable write fault injection for the basic whitebox, blackbox, and cf_consistency modes. For other test modes like multiops_txn, best_efforts_recovery etc., leave it disabled for now until we can do more testing.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11924

Reviewed By: ajkr

Differential Revision: D50178252

Pulled By: anand1976

fbshipit-source-id: 5794f81c14cded1eb28762b2de818dfff1c1a34c
2023-10-11 11:28:00 -07:00
anand76 5b11f5a3a2 Add TieredCache and compressed cache capacity change to db_stress (#11935)
Summary:
Add `TieredCache` to the cache types tested by db_stress. Also add compressed secondary cache capacity change, and `WriteBufferManager` integration with `TieredCache` for memory charging.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11935

Test Plan: Run whitebox/blackbox crash tests locally

Reviewed By: akankshamahajan15

Differential Revision: D50135365

Pulled By: anand1976

fbshipit-source-id: 7d73ed00c00a0953d86e49f35cce6bd550ba00f1
2023-10-10 13:12:18 -07:00
Andrew Kryczka 8a9cfd5292 Make stopped writes block on recovery (#11879)
Summary:
Relaxed the constraints for blocking when writes are stopped. When a recovery is already being attempted, we might as well let `!no_slowdown` writes wait on it in case it succeeds. This makes the user-visible behavior consistent across recovery flush and non-recovery flush.

This enables `db_stress` to inject retryable (soft) flush read errors without having to handle user write failures. I changed `db_stress` a bit to permit injected errors in much more foreground operations as more admin operations (like `GetLiveFiles()`) can fail on a retryable error during flush.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11879

Reviewed By: anand1976

Differential Revision: D49571196

Pulled By: ajkr

fbshipit-source-id: 5d516d6faf20d2c6bfe0594ab4f2706bca6d69b0
2023-10-10 06:29:01 -07:00
akankshamahajan 97f6f475bc Fix various failures in auto_readahead_size (#11884)
Summary:
1. **Error** in TestIterateAgainstExpected API - `Assertion index < pre_read_expected_values.size() && index < post_read_expected_values.size() failed.`
 **Fix** - `Prev` op is not supported with `auto_readahead_size`. So added support to Reseek in db_iter, if Prev is called. In BlockBasedTableIterator, index_iter_ already moves forward. So there is no way to do Prev from BlockBasedTableIterator.

2. **Error** - `void rocksdb::BlockBasedTableIterator::BlockCacheLookupForReadAheadSize(uint64_t, size_t, size_t&): Assertion index_iter_->value().handle.offset() == offset`
   **Fix** - Remove prefetch_buffer to be used when uncompressed dict is read.

3. ** Error in TestPrefixScan API - `db_stress: db/db_iter.cc:369: bool rocksdb::DBIter::FindNextUserEntryInternal(bool, const rocksdb::Slice*): Assertion !skipping_saved_key || CompareKeyForSkip(ikey_.user_key, saved_key_.GetUserKey()) > 0 failed.
Received signal 6 (Aborted)
Invoking GDB for stack trace...
db_stress: table/merging_iterator.cc:1036: bool rocksdb::MergingIterator::SkipNextDeleted(): Assertion comparator_->Compare(range_tombstone_iters_[i]->start_key(), pik) <= 0 failed`
  **Fix** -  SeekPrev also calls 1) SeekPrev , 2)Seek and then 3)Prev in some cases in db_iter.cc leading to failure of Prev operation. These backward operations also call Seek.  Added direction to disable lookup once direction is backwards in BlockBasedTableIterator.cc

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11884

Test Plan: Ran various flavors of crash tests locally for the whole duration

Reviewed By: anand1976

Differential Revision: D49834201

Pulled By: akankshamahajan15

fbshipit-source-id: 9a007b4d46a48002c43dc4623a400ecf47d997fe
2023-10-02 17:47:24 -07:00
Andrew Kryczka be879cc56b stress test verification value mismatch message (#11912)
Summary:
Separate the message for value mismatch from the message for an extra value in the DB

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11912

Reviewed By: hx235

Differential Revision: D49792137

Pulled By: ajkr

fbshipit-source-id: 311bc1801843a15367f409ead88ef755acbde468
2023-10-02 16:07:39 -07:00
Levi Tamasi 01e2d33565 Add the wide-column aware merge API to the stress tests (#11906)
Summary:
Pull Request resolved: https://github.com/facebook/rocksdb/pull/11906

The patch adds stress test coverage for the wide-column aware `FullMergeV3` API by implementing a new `DBStressWideMergeOperator`. This operator is similar to `PutOperator` / `PutOperatorV2` in the sense that its result is based on the last merge operand; however, the merge result can be either a plain value or a wide-column entity, depending on the value base encoded into the operand and the value of the `use_put_entity_one_in` stress test parameter. Following the same rule for merge results that we do for writes ensures that the queries issued by the validation logic receive the expected results. The new operator is used instead of `PutOperatorV2` whenever `use_put_entity_one_in` is positive. Note that the patch also makes it possible to set `use_put_entity_one_in` and `use_merge` (but not `use_full_merge_v1`) at the same time, giving `use_put_entity_one_in` precedence, so the stress test will use `PutEntity` for writes passing the `use_put_entity_one_in` check described above and `Merge` for any other writes.

Reviewed By: jaykorean

Differential Revision: D49760024

fbshipit-source-id: 3893602c3e7935381b484f4f5026f1983e3a04a9
2023-09-29 08:54:50 -07:00
akankshamahajan 3d67b5e8e5 Lookup ahead in block cache ahead to tune Readaheadsize (#11860)
Summary:
Implement block cache lookup to determine readahead_size during scans. It's enabled if auto_readahead_size, block_cache and iterate_upper_bound - all three are set.

Design -
1. Whenever there is a cache miss and FilePrefetchBuffer is called, a callback is made to determine readahead_size for that prefetching.
2. The callback iterates over index and do block cache lookup for each data block handle until existing readahead_size is reached. Then It removes the cache hit data blocks from end to calculate optimized readahead_size.
3. Since index_iter_ is moved, it stores block handles in a queue, and use that queue to get block handle instead of doing index_iter_->Next().
4. This is for Sync scans. Async scans support is in progress.

NOTE:
The issue right now is after Seek and Next, if Prev is called, there is no way to do Prev operation. index_iter_ is already pointing to a different block. So it returns "Not supported" in that case with error message - "auto tuning of readahead size is not supported with Prev op"

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11860

Test Plan:
- Added new unit test
- crash_tests
- Running scans locally to check for any regression

Reviewed By: anand1976

Differential Revision: D49548118

Pulled By: akankshamahajan15

fbshipit-source-id: f1aee409a71b4ad9e5bf3610f43edf30c6630c78
2023-09-22 18:12:08 -07:00
Changyu Bi cfe0e0b037 Initialize FaultInjectionTestFS DirectWritable field (#11862)
Summary:
FaultInjectionTestFS is not directly writable by default. Should set it to direct writable if there is no write fault injection.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11862

Test Plan: internal stress test failure reduces.

Reviewed By: jaykorean

Differential Revision: D49428108

Pulled By: cbi42

fbshipit-source-id: 5dfa1fbb454272a14f8228a5c496d480d7138ef1
2023-09-19 12:23:38 -07:00
Changyu Bi ba5897ada8 Fix stress test failure due to write fault injections and disable write fault injection (#11859)
Summary:
This PR contains two fixes:

1. disable write fault injection since it caused several other kinds of internal stress test failures. I'll try to fix those separately before enabling it again.
2. Fix segfault like
```
https://github.com/facebook/rocksdb/issues/5  0x000000000083dc43 in rocksdb::port::Mutex::Lock (this=0x30) at internal_repo_rocksdb/repo/port/port_posix.cc:80
80	internal_repo_rocksdb/repo/port/port_posix.cc: No such file or directory.
https://github.com/facebook/rocksdb/issues/6  0x0000000000465142 in rocksdb::MutexLock::MutexLock (mu=0x30, this=<optimized out>) at internal_repo_rocksdb/repo/util/mutexlock.h:37
37	internal_repo_rocksdb/repo/util/mutexlock.h: No such file or directory.
https://github.com/facebook/rocksdb/issues/7  rocksdb::FaultInjectionTestFS::DisableWriteErrorInjection (this=0x0) at internal_repo_rocksdb/repo/utilities/fault_injection_fs.h:505
505	internal_repo_rocksdb/repo/utilities/fault_injection_fs.h: No such file or directory.
```

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11859

Test Plan: db_stress with no fault injection: `./db_stress --write_fault_one_in=0 --read_fault_one_in=0 --open_metadata_write_fault_one_in=0 --open_read_fault_one_in=0 --open_write_fault_one_in=0 --sync_fault_injection=0`

Reviewed By: jaykorean

Differential Revision: D49408247

Pulled By: cbi42

fbshipit-source-id: 0ca01f20e6e81bf52af77818b50d562ef7462165
2023-09-19 08:33:05 -07:00
Jay Huh 20dbf51247 DB Stress Fix - Commit pending value after checking for status (#11856)
Summary:
We've seen occasional crash test failure in optimistic transaction DB with the following error message.

```
stderr:
WARNING: prefix_size is non-zero but memtablerep != prefix_hash
Verification failed for column family 0 key 0000000000001EDB0000000000000036787878787878 (789064): value_from_db: 010000000504070609080B0A0D0C0F0E111013121514171619181B1A1D1C1F1E212023222524272629282B2A2D2C2F2E313033323534373639383B3A3D3C3F3E, value_from_expected: , msg: MultiGet verification: Unexpected value found
Crash-recovery verification failed :(
No writes or ops?
Verification failed :(
```

There was a possibility if the db stress crashes after `pending_expected_value.Commit()` but before `thread->shared->SafeTerminate();`, we may have expected value committed while actual DB value was not.

Moving the `pending_expected_value.Commit()` after `s.ok()` check to fix the test.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11856

Test Plan:
Ran the following in a script with while loop. (It doesn't always repro the issue even without this fix, though..)
```
./db_stress --acquire_snapshot_one_in=10000 --adaptive_readahead=0 --allow_data_in_errors=True --async_io=1 --atomic_flush=1 --auto_readahead_size=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_protection_bytes_per_key=0 --block_size=16384 --bloom_before_level=8 --bloom_bits=11 --bottommost_compression_type=lz4 --bottommost_file_compaction_delay=0 --bytes_per_sync=0 --cache_index_and_filter_blocks=0 --cache_size=8388608 --cache_type=auto_hyper_clock_cache --charge_compression_dictionary_building_buffer=0 --charge_file_metadata=0 --charge_filter_construction=0 --charge_table_reader=0 --checkpoint_one_in=1000000 --checksum_type=kxxHash --clear_column_family_one_in=0 --compact_files_one_in=1000000 --compact_range_one_in=1000000 --compaction_pri=4 --compaction_readahead_size=0 --compaction_ttl=100 --compression_checksum=0 --compression_max_dict_buffer_bytes=2097151 --compression_max_dict_bytes=16384 --compression_parallel_threads=1 --compression_type=lz4 --compression_use_zstd_dict_trainer=0 --compression_zstd_max_train_bytes=0 --continuous_verification_interval=0 --data_block_index_type=0 --db=/dev/shm/rocksdb_test/rocksdb_crashtest_blackbox --db_write_buffer_size=1048576 --delpercent=5 --delrangepercent=0 --destroy_db_initially=0 --detect_filter_construct_corruption=0 --disable_wal=1 --enable_compaction_filter=0 --enable_pipelined_write=0 --enable_thread_tracking=1 --expected_values_dir=/dev/shm/rocksdb_test/rocksdb_crashtest_expected --fail_if_options_file_error=1 --fifo_allow_compaction=1 --file_checksum_impl=crc32c --flush_one_in=1000000 --format_version=5 --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=3 --index_type=2 --ingest_external_file_one_in=0 --initial_auto_readahead_size=524288 --iterpercent=10 --key_len_percent_dist=1,30,69 --level_compaction_dynamic_level_bytes=0 --lock_wal_one_in=1000000 --long_running_snapshots=0 --manual_wal_flush_one_in=0 --mark_for_compaction_one_file_in=10 --max_auto_readahead_size=524288 --max_background_compactions=20 --max_bytes_for_level_base=10485760 --max_key=25000000 --max_key_len=3 --max_manifest_file_size=1073741824 --max_write_batch_group_size_bytes=16777216 --max_write_buffer_number=3 --max_write_buffer_size_to_maintain=2097152 --memtable_max_range_deletions=0 --memtable_prefix_bloom_size_ratio=0.001 --memtable_protection_bytes_per_key=2 --memtable_whole_key_filtering=0 --memtablerep=skip_list --min_write_buffer_number_to_merge=1 --mmap_read=1 --mock_direct_io=False --nooverwritepercent=1 --num_file_reads_for_auto_readahead=1 --occ_lock_bucket_count=500 --occ_validation_policy=0 --open_files=-1 --open_metadata_write_fault_one_in=8 --open_read_fault_one_in=0 --open_write_fault_one_in=0 --ops_per_thread=100000000 --optimize_filters_for_memory=1 --paranoid_file_checks=1 --partition_filters=0 --partition_pinning=1 --pause_background_one_in=1000000 --periodic_compaction_seconds=2 --prefix_size=5 --prefixpercent=5 --prepopulate_block_cache=1 --preserve_internal_time_seconds=36000 --progress_reports=0 --read_fault_one_in=0 --readahead_size=0 --readpercent=45 --recycle_log_file_num=1 --reopen=0 --secondary_cache_fault_one_in=0 --secondary_cache_uri=compressed_secondary_cache://capacity=8388608;enable_custom_split_merge=true --set_options_one_in=10000 --share_occ_lock_buckets=0 --snapshot_hold_ops=100000 --sst_file_manager_bytes_per_sec=0 --sst_file_manager_bytes_per_truncate=0 --stats_dump_period_sec=0 --subcompactions=3 --sync=0 --sync_fault_injection=0 --target_file_size_base=2097152 --target_file_size_multiplier=2 --test_batches_snapshots=0 --top_level_index_pinning=3 --unpartitioned_pinning=2 --use_direct_io_for_flush_and_compaction=0 --use_direct_reads=0 --use_full_merge_v1=0 --use_get_entity=0 --use_merge=0 --use_multi_get_entity=0 --use_multiget=1 --use_optimistic_txn=1 --use_put_entity_one_in=0 --use_txn=1 --user_timestamp_size=0 --value_size_mult=32 --verification_only=0 --verify_checksum=1 --verify_checksum_one_in=1000000 --verify_db_one_in=100000 --verify_file_checksums_one_in=1000000 --verify_sst_unique_id_in_manifest=1 --wal_bytes_per_sync=0 --wal_compression=zstd --write_buffer_size=4194304 --write_dbid_to_manifest=1 --write_fault_one_in=0 --writepercent=35&
```
```
pid=$!
sleep 10
kill -9 $pid
sleep 1
```
```
db_stress --acquire_snapshot_one_in=10000 --adaptive_readahead=1 --allow_data_in_errors=True --async_io=0 --atomic_flush=1 --auto_readahead_size=1 --avoid_flush_during_recovery=0 --avoid_unnecessary_blocking_io=1 --backup_max_size=104857600 --backup_one_in=100000 --batch_protection_bytes_per_key=8 --block_protection_bytes_per_key=0 --block_size=16384 --bloom_before_level=2147483647 --bloom_bits=75.01353068032098 --bottommost_compression_type=xpress --bottommost_file_compaction_delay=3600 --bytes_per_sync=262144 --cache_index_and_filter_blocks=0 --cache_size=8388608 --cache_type=auto_hyper_clock_cache --charge_compression_dictionary_building_buffer=1 --charge_file_metadata=0 --charge_filter_construction=1 --charge_table_reader=0 --checkpoint_one_in=1000000 --checksum_type=kCRC32c --clear_column_family_one_in=0 --compact_files_one_in=1000000 --compact_range_one_in=1000000 --compaction_pri=4 --compaction_readahead_size=0 --compaction_ttl=0 --compression_checksum=0 --compression_max_dict_buffer_bytes=0 --compression_max_dict_bytes=0 --compression_parallel_threads=1 --compression_type=xpress --compression_use_zstd_dict_trainer=0 --compression_zstd_max_train_bytes=0 --continuous_verification_interval=0 --data_block_index_type=1 --db=/dev/shm/rocksdb_test/rocksdb_crashtest_blackbox --db_write_buffer_size=134217728 --delpercent=5 --delrangepercent=0 --destroy_db_initially=0 --detect_filter_construct_corruption=1 --disable_wal=1 --enable_compaction_filter=0 --enable_pipelined_write=0 --enable_thread_tracking=1 --expected_values_dir=/dev/shm/rocksdb_test/rocksdb_crashtest_expected --fail_if_options_file_error=1 --fifo_allow_compaction=0 --file_checksum_impl=none --flush_one_in=1000000 --format_version=5 --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=10 --index_type=0 --ingest_external_file_one_in=0 --initial_auto_readahead_size=524288 --iterpercent=10 --key_len_percent_dist=1,30,69 --level_compaction_dynamic_level_bytes=1 --lock_wal_one_in=1000000 --long_running_snapshots=0 --manual_wal_flush_one_in=0 --mark_for_compaction_one_file_in=10 --max_auto_readahead_size=524288 --max_background_compactions=20 --max_bytes_for_level_base=10485760 --max_key=25000000 --max_key_len=3 --max_manifest_file_size=1073741824 --max_write_batch_group_size_bytes=1048576 --max_write_buffer_number=3 --max_write_buffer_size_to_maintain=1048576 --memtable_max_range_deletions=100 --memtable_prefix_bloom_size_ratio=0.01 --memtable_protection_bytes_per_key=8 --memtable_whole_key_filtering=0 --memtablerep=skip_list --min_write_buffer_number_to_merge=2 --mmap_read=0 --mock_direct_io=True --nooverwritepercent=1 --num_file_reads_for_auto_readahead=2 --occ_lock_bucket_count=10 --occ_validation_policy=0 --open_files=-1 --open_metadata_write_fault_one_in=0 --open_read_fault_one_in=32 --open_write_fault_one_in=16 --ops_per_thread=100000000 --optimize_filters_for_memory=0 --paranoid_file_checks=1 --partition_filters=0 --partition_pinning=0 --pause_background_one_in=1000000 --periodic_compaction_seconds=0 --prefix_size=1 --prefixpercent=5 --prepopulate_block_cache=0 --preserve_internal_time_seconds=36000 --progress_reports=0 --read_fault_one_in=32 --readahead_size=0 --readpercent=45 --recycle_log_file_num=1 --reopen=0 --secondary_cache_fault_one_in=0 --secondary_cache_uri=compressed_secondary_cache://capacity=8388608;enable_custom_split_merge=true --set_options_one_in=10000 --share_occ_lock_buckets=0 --snapshot_hold_ops=100000 --sst_file_manager_bytes_per_sec=104857600 --sst_file_manager_bytes_per_truncate=0 --stats_dump_period_sec=0 --subcompactions=4 --sync=0 --sync_fault_injection=0 --target_file_size_base=2097152 --target_file_size_multiplier=2 --test_batches_snapshots=0 --top_level_index_pinning=1 --unpartitioned_pinning=1 --use_direct_io_for_flush_and_compaction=1 --use_direct_reads=0 --use_full_merge_v1=1 --use_get_entity=0 --use_merge=1 --use_multi_get_entity=0 --use_multiget=1 --use_optimistic_txn=1 --use_put_entity_one_in=0 --use_txn=1 --user_timestamp_size=0 --value_size_mult=32 --verification_only=0 --verify_checksum=1 --verify_checksum_one_in=1000000 --verify_db_one_in=100000 --verify_file_checksums_one_in=0 --verify_sst_unique_id_in_manifest=1 --wal_bytes_per_sync=524288 --wal_compression=zstd --write_buffer_size=4194304 --write_dbid_to_manifest=1 --write_fault_one_in=0 --writepercent=35
```

Reviewed By: cbi42

Differential Revision: D49403091

Pulled By: jaykorean

fbshipit-source-id: 5ee6136133bbdc46aa733e5101c1f998f658c200
2023-09-18 23:04:38 -07:00
Changyu Bi c90807d103 Inject retryable write IOError when writing to SST files in stress test (#11829)
Summary:
* db_crashtest.py now may set `write_fault_one_in` to 500 for blackbox and whitebox simple test.
* Error injection only applies to writing to SST files. Flush error will cause DB to pause background operations and auto-resume. Compaction error will just re-schedule later.
* File ingestion and back up tests are updated to check if the result status is due to an injected error.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11829

Test Plan:
a full round of whitebox simple and blackbox simple crash test
*  `python3 ./tools/db_crashtest.py whitebox/blackbox --simple  --write_fault_one_in=500`

Reviewed By: ajkr

Differential Revision: D49256962

Pulled By: cbi42

fbshipit-source-id: 68e0c9648d8e03bad39c7672b25d5500fc286d97
2023-09-18 16:23:26 -07:00
Jay Huh 99f8820054 Fix test on IOActivity check for MultiGetEntity (#11850)
Summary:
After https://github.com/facebook/rocksdb/issues/11842  merged, we started to see some crash_test failures.

There is a flow inside `TestMultiGetEntity()` that it calls `GetEntity()` to compare the result between `MultiGetEntity()` and `GetEntity()` 1c6faf3587/db_stress_tool/no_batched_ops_stress.cc (L1068-L1072)

However, IOActivity check inside DbStressRandomAccessFileWrapper was expecting IOActivity::MultiGet when GetEntity() was called. We are fixing the test by setting expected operation to be GetEntity before calling GetEntity()

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11850

Test Plan:
Error repro'ed by the following run before fix and no more error after the fix.

```
./db_stress --acquire_snapshot_one_in=10000 --adaptive_readahead=0 --allow_concurrent_memtable_write=0 --allow_data_in_errors=True --async_io=1 --auto_readahead_size=1 --avoid_flush_during_recovery=0 --avoid_unnecessary_blocking_io=1 --backup_max_size=104857600 --backup_one_in=100000 --batch_protection_bytes_per_key=8 --block_protection_bytes_per_key=0 --block_size=16384 --bloom_before_level=1 --bloom_bits=9.880688060667444 --bottommost_compression_type=zstd --bottommost_file_compaction_delay=86400 --bytes_per_sync=262144 --cache_index_and_filter_blocks=0 --cache_size=8388608 --cache_type=auto_hyper_clock_cache --charge_compression_dictionary_building_buffer=1 --charge_file_metadata=0 --charge_filter_construction=0 --charge_table_reader=1 --checkpoint_one_in=1000000 --checksum_type=kxxHash64 --clear_column_family_one_in=0 --column_families=1 --compact_files_one_in=1000000 --compact_range_one_in=1000000 --compaction_pri=3 --compaction_readahead_size=1048576 --compaction_ttl=0 --compression_checksum=0 --compression_max_dict_buffer_bytes=0 --compression_max_dict_bytes=0 --compression_parallel_threads=1 --compression_type=none --compression_use_zstd_dict_trainer=1 --compression_zstd_max_train_bytes=0 --continuous_verification_interval=0 --data_block_index_type=1 --db=/dev/shm/rocksdb_test/rocksdb_crashtest_blackbox --db_write_buffer_size=0 --delpercent=4 --delrangepercent=1 --destroy_db_initially=0 --detect_filter_construct_corruption=1 --disable_wal=0 --enable_compaction_filter=0 --enable_pipelined_write=0 --enable_thread_tracking=1 --expected_values_dir=/dev/shm/rocksdb_test/rocksdb_crashtest_expected --fail_if_options_file_error=0 --fifo_allow_compaction=1 --file_checksum_impl=big --flush_one_in=1000000 --format_version=6 --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=4 --index_type=0 --ingest_external_file_one_in=0 --initial_auto_readahead_size=0 --iterpercent=10 --key_len_percent_dist=1,30,69 --level_compaction_dynamic_level_bytes=0 --lock_wal_one_in=1000000 --long_running_snapshots=0 --manual_wal_flush_one_in=0 --mark_for_compaction_one_file_in=0 --max_auto_readahead_size=524288 --max_background_compactions=1 --max_bytes_for_level_base=67108864 --max_key=25000000 --max_key_len=3 --max_manifest_file_size=1073741824 --max_write_batch_group_size_bytes=16777216 --max_write_buffer_number=3 --max_write_buffer_size_to_maintain=2097152 --memtable_max_range_deletions=0 --memtable_prefix_bloom_size_ratio=0 --memtable_protection_bytes_per_key=2 --memtable_whole_key_filtering=0 --memtablerep=skip_list --min_write_buffer_number_to_merge=2 --mmap_read=1 --mock_direct_io=False --nooverwritepercent=1 --num_file_reads_for_auto_readahead=0 --open_files=-1 --open_metadata_write_fault_one_in=0 --open_read_fault_one_in=0 --open_write_fault_one_in=0 --ops_per_thread=100000000 --optimize_filters_for_memory=0 --paranoid_file_checks=1 --partition_filters=0 --partition_pinning=3 --pause_background_one_in=1000000 --periodic_compaction_seconds=1 --prefix_size=-1 --prefixpercent=0 --prepopulate_block_cache=0 --preserve_internal_time_seconds=0 --progress_reports=0 --read_fault_one_in=32 --readahead_size=16384 --readpercent=50 --recycle_log_file_num=1 --reopen=0 --secondary_cache_fault_one_in=0 --secondary_cache_uri=compressed_secondary_cache://capacity=8388608;enable_custom_split_merge=true --set_options_one_in=0 --snapshot_hold_ops=100000 --sst_file_manager_bytes_per_sec=104857600 --sst_file_manager_bytes_per_truncate=0 --stats_dump_period_sec=0 --subcompactions=2 --sync=0 --sync_fault_injection=1 --target_file_size_base=16777216 --target_file_size_multiplier=1 --test_batches_snapshots=0 --top_level_index_pinning=3 --unpartitioned_pinning=0 --use_direct_io_for_flush_and_compaction=0 --use_direct_reads=0 --use_full_merge_v1=0 --use_get_entity=0 --use_merge=1 --use_multi_get_entity=1 --use_multiget=1 --use_put_entity_one_in=0 --user_timestamp_size=0 --value_size_mult=32 --verification_only=0 --verify_checksum=1 --verify_checksum_one_in=1000000 --verify_db_one_in=100000 --verify_file_checksums_one_in=1000000 --verify_iterator_with_expected_state_one_in=5 --verify_sst_unique_id_in_manifest=1 --wal_bytes_per_sync=524288 --wal_compression=none --write_buffer_size=33554432 --write_dbid_to_manifest=0 --writepercent=35
```

Reviewed By: cbi42

Differential Revision: D49344996

Pulled By: jaykorean

fbshipit-source-id: 8059b8127c0e3cb8af96cf222f47398413c92c50
2023-09-15 22:50:49 -07:00
Peter Dillinger 1c6faf3587 Make RibbonFilterPolicy::bloom_before_level mutable (SetOptions()) (#11838)
Summary:
An internal user wants to be able to dynamically switch between Bloom and Ribbon filters, without a custom FilterPolicy. Making `filter_policy` mutable would actually make issue https://github.com/facebook/rocksdb/issues/10079 worse, because it would be a race on a pointer field, not just on scalars.

As a reasonable compromise until that is fixed, I am enabling dynamic control over Bloom vs. Ribbon choice by making
RibbonFilterPolicy::bloom_before_level mutable, and doing that safely by using an atomic.

I've also slightly tweaked the interpretation of that field so that setting it to INT_MAX really means "always Bloom."

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11838

Test Plan: unit tests added/extended. crash test updated for SetOptions call and tested under TSAN with amplified probability (lower set_options_one_in).

Reviewed By: ajkr

Differential Revision: D49296284

Pulled By: pdillinger

fbshipit-source-id: e4251c077510df9a9c719876f482448c0d15402a
2023-09-15 15:46:10 -07:00
Jay Huh cff6490bc4 Add IOActivity.kMultiGetEntity (#11842)
Summary:
- As a follow up from https://github.com/facebook/rocksdb/issues/11799, adding `Env::IOActivity::kMultiGetEntity` support to `DBImpl::MultiGetEntity()`.

## Minor Refactor
- Because both `DBImpl::MultiGet()` and `DBImpl::MultiGetEntity()` call `DBImpl::MultiGetCommon()` which later calls `DBImpl::MultiGetWithCallback()` where we check `Env::IOActivity::kMultiGet`, minor refactor was needed so that we don't check `Env::IOActivity::kMultiGet` for  `DBImpl::MultiGetEntity()`.
- I still see more areas for refactoring to avoid duplicate code of checking IOActivity and setting it when Unknown, but this will be addressed separately.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11842

Test Plan:
- Added the `ThreadStatus::OperationType::OP_MULTIGETENTITY` in `db_stress` to verify the pass-down IOActivity in a thread aligns with the actual activity the thread is doing.
```
python3 tools/db_crashtest.py blackbox --max_key=25000000 --write_buffer_size=4194304 --max_bytes_for_level_base=2097152 --target_file_size_base=2097152 --periodic_compaction_seconds=0 --use_put_entity_one_in=10 --use_get_entity=1 --duration=60 --interval=10
python3 tools/db_crashtest.py blackbox --simple --max_key=25000000 --write_buffer_size=4194304 --max_bytes_for_level_base=2097152 --target_file_size_base=2097152 --periodic_compaction_seconds=0 --use_put_entity_one_in=10 --use_get_entity=1 --duration=60 --interval=10
python3 tools/db_crashtest.py blackbox --cf_consistency --max_key=25000000 --write_buffer_size=4194304 --max_bytes_for_level_base=2097152 --target_file_size_base=2097152 --periodic_compaction_seconds=0 --use_put_entity_one_in=10 --use_get_entity=1 --duration=60 --interval=10
```

Reviewed By: ltamasi

Differential Revision: D49329575

Pulled By: jaykorean

fbshipit-source-id: 05198f1d3f92e6be3d42a3d184bacb3ab2ce6923
2023-09-15 15:34:04 -07:00
leipeng 68ce5d84f6 Add new Iterator API Refresh(const snapshot*) (#10594)
Summary:
This PR resolves https://github.com/facebook/rocksdb/issues/10487 & https://github.com/facebook/rocksdb/issues/10536, user code needs to call Refresh() periodically.

The main code change is to support range deletions. A range tombstone iterator uses a sequence number as upper bound to decide which range tombstones are effective. During Iterator refresh, this sequence number upper bound needs to be updated for all range tombstone iterators under DBIter and LevelIterator. LevelIterator may create new table iterators and range tombstone iterator during scanning, so it needs to be aware of iterator refresh. The code path that propagates this change is `db_iter_->set_sequence(read_seq)  -> MergingIterator::SetRangeDelReadSeqno() -> TruncatedRangeDelIterator::SetRangeDelReadSeqno() and LevelIterator::SetRangeDelReadSeqno()`.

This change also fixes an issue where range tombstone iterators created by LevelIterator may access ReadOptions::snapshot, even though we do not explicitly require users to keep a snapshot alive after creating an Iterator.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/10594

Test Plan:
* New unit tests.
* Add Iterator::Refresh(snapshot) to stress test. Note that this change only adds tests for refreshing to the same snapshot since this is the main target use case.

TODO in a following PR:
* Stress test Iterator::Refresh() to different snapshots or no snapshot.

Reviewed By: ajkr

Differential Revision: D48456896

Pulled By: cbi42

fbshipit-source-id: 2e642c04e91235cc9542ef4cd37b3c20823bd779
2023-09-15 10:44:43 -07:00
Jay Huh f2b623bcc1 GetEntity Support for ReadOnlyDB and SecondaryDB (#11799)
Summary:
`GetEntity` API support for ReadOnly DB and Secondary DB.
- Introduced `GetImpl()` with `GetImplOptions` in `db_impl_readonly` and refactored current `Get()` logic into `GetImpl()` so that look up logic can be reused for `GetEntity()` (Following the same pattern as `DBImpl::Get()` and `DBImpl::GetEntity()`)
- Introduced `GetImpl()` with `GetImplOptions` in `db_impl_secondary` and refactored current `GetImpl()` logic. This is to make `DBImplSecondary::Get/GetEntity` consistent with `DBImpl::Get/GetEntity` and `DBImplReadOnly::Get/GetEntity`
- `GetImpl()` in `db_impl` is now virtual. both `db_impl_readonly` and `db_impl_secondary`'s `Get()` override are no longer needed since all three dbs now have the same `Get()` which calls `GetImpl()` internally.
- `GetImpl()` in `DBImplReadOnly` and `DBImplSecondary` now pass in `columns` instead of `nullptr` in lookup functions like `memtable->get()`
- Introduced `GetEntity()` API in `DBImplReadOnly` and `DBImplSecondary` which simply calls `GetImpl()` with `columns` set in `GetImplOptions`.
- Introduced `Env::IOActivity::kGetEntity` and set read_options.io_activity to `Env::IOActivity::kGetEntity` for `GetEntity()` operations (in db_impl)

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11799

Test Plan:
**Unit Tests**
- Added verification in `DBWideBasicTest::PutEntity` by Reopening DB as ReadOnly with the same setup.
- Added verification in `DBSecondaryTest::ReopenAsSecondary` by calling `PutEntity()` and `GetEntity()` on top of existing `Put()` and `Get()`
- `make -j64 check`

**Crash Tests**
- `python3 tools/db_crashtest.py blackbox --max_key=25000000 --write_buffer_size=4194304 --max_bytes_for_level_base=2097152 --target_file_size_base=2097152 --periodic_compaction_seconds=0 --use_put_entity_one_in=10 --use_get_entity=1 --duration=60 --inter
val=10`
- `python3 tools/db_crashtest.py blackbox --simple --max_key=25000000 --write_buffer_size=4194304 --max_bytes_for_level_base=2097152 --target_file_size_base=2097152 --periodic_compaction_seconds=0 --use_put_entity_one_in=10 --use_get_entity=1 `
- `python3 tools/db_crashtest.py blackbox --cf_consistency --max_key=25000000 --write_buffer_size=4194304 --max_bytes_for_level_base=2097152 --target_file_size_base=2097152 --periodic_compaction_seconds=0 --use_put_entity_one_in=10 --use_get_entity=1 --duration=60 --inter
val=10`

Reviewed By: ltamasi

Differential Revision: D49037040

Pulled By: jaykorean

fbshipit-source-id: a0648253ded6e91af7953de364ed3c6bf163626b
2023-09-15 08:30:44 -07:00
Levi Tamasi 1e63fc9925 Add a helper method WideColumnsHelper::SortColumns (#11823)
Summary:
Pull Request resolved: https://github.com/facebook/rocksdb/pull/11823

Similarly to https://github.com/facebook/rocksdb/pull/11813, the patch is a small refactoring that eliminates some copy-paste around sorting the columns of entities by column name.

Reviewed By: jaykorean

Differential Revision: D49195504

fbshipit-source-id: d48c9f290e3203f838cc5949856c469ecf730008
2023-09-12 12:36:07 -07:00
Levi Tamasi 8fc78a3a9e Add helper methods WideColumnsHelper::{Has,Get}DefaultColumn (#11813)
Summary:
Pull Request resolved: https://github.com/facebook/rocksdb/pull/11813

The patch adds a couple of helper methods `WideColumnsHelper::{Has,Get}DefaultColumn` to eliminate some code duplication.

Reviewed By: jaykorean

Differential Revision: D49166682

fbshipit-source-id: f229ca5b94599f7445a0112b10f8317292505c82
2023-09-11 16:32:32 -07:00
anand76 137cd4bb75 Disable error injection after compaction completion (#11798)
Summary:
https://github.com/facebook/rocksdb/issues/11789 added error injection during compaction to db_stress. However, error injection was not disabled after compaction completion, which resulted in some test failures due to stale errors.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11798

Reviewed By: cbi42

Differential Revision: D49022821

Pulled By: anand1976

fbshipit-source-id: 3cbfe18d55bee393697e063d05e7a7a7f88b7635
2023-09-06 12:57:10 -07:00
Andrew Kryczka 392d6957cd Added compaction read errors to db_stress (#11789)
Summary:
- Fixed misspellings of "inject"
- Made user read errors retryable when `FLAGS_inject_error_severity == 1`
- Added compaction read errors when `FLAGS_read_fault_one_in > 0`. These are always retryable so that the DB will keep accepting writes
- Reenabled setting `compaction_readahead_size` in crash test. The reason for disabling it was to "keep the test clean", which is not a good enough reason to skip testing it

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11789

Test Plan:
With https://github.com/facebook/rocksdb/issues/11782 reverted, reproduced the bug:
- Build: `make -j56 db_stress`
- Command: `TEST_TMPDIR=/dev/shm python3 tools/db_crashtest.py blackbox --simple --write_buffer_size=524288 --target_file_size_base=524288 --max_bytes_for_level_base=2097152 --interval=10 --max_key=1000000`
- Output:
```
stderr has error message:
***put or merge error: Corruption: Compaction number of input keys does not match number of keys processed.***
```

Reviewed By: cbi42

Differential Revision: D48939994

Pulled By: ajkr

fbshipit-source-id: a1efb799efecdfd5d9cfd185e4a6321db8fccfbb
2023-09-05 10:41:29 -07:00
Jay Huh ea9a5b2914 Wide Column support in ldb (#11754)
Summary:
wide_columns can now be pretty-printed in the following commands
- `./ldb dump_wal`
- `./ldb dump`
- `./ldb idump`
- `./ldb dump_live_files`
- `./ldb scan`
- `./sst_dump --command=scan`

There are opportunities to refactor to reduce some nearly identical code. This PR is initial change to add wide column support in `ldb` and `sst_dump` tool. More PRs to come for the refactor.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11754

Test Plan:
**New Tests added**
- `WideColumnsHelperTest::DumpWideColumns`
- `WideColumnsHelperTest::DumpSliceAsWideColumns`

**Changes added to existing tests**
- `ExternalSSTFileTest::BasicMixed` added to cover mixed case (This test should have been added in https://github.com/facebook/rocksdb/issues/11688). This test does not verify the ldb or sst_dump output. This test was used to create test SST files having some rows with wide columns and some without and the generated SST files were used to manually test sst_dump_tool.
- `createSST()` in `sst_dump_test` now takes `wide_column_one_in` to add wide column value in SST

**dump_wal**
```
./ldb dump_wal --walfile=/tmp/rocksdbtest-226125/db_wide_basic_test_2675429_2308393776696827948/000004.log --print_value --header
```
```
Sequence,Count,ByteSize,Physical Offset,Key(s) : value
1,1,59,0,PUT_ENTITY(0) : 0x:0x68656C6C6F 0x617474725F6E616D6531:0x666F6F 0x617474725F6E616D6532:0x626172
2,1,34,42,PUT_ENTITY(0) : 0x617474725F6F6E65:0x74776F 0x617474725F7468726565:0x666F7572
3,1,17,7d,PUT(0) : 0x7468697264 : 0x62617A
```

**idump**
```
./ldb --db=/tmp/rocksdbtest-226125/db_wide_basic_test_3481961_2308393776696827948/ idump
```
```
'first' seq:1, type:22 => :hello attr_name1:foo attr_name2:bar
'second' seq:2, type:22 => attr_one:two attr_three:four
'third' seq:3, type:1 => baz
Internal keys in range: 3
```

**SST Dump from dump_live_files**
```
./ldb --db=/tmp/rocksdbtest-226125/db_wide_basic_test_3481961_2308393776696827948/ compact
./ldb --db=/tmp/rocksdbtest-226125/db_wide_basic_test_3481961_2308393776696827948/ dump_live_files
```
```
...
==============================
SST Files
==============================
/tmp/rocksdbtest-226125/db_wide_basic_test_3481961_2308393776696827948/000013.sst level:1
------------------------------
Process /tmp/rocksdbtest-226125/db_wide_basic_test_3481961_2308393776696827948/000013.sst
Sst file format: block-based
'first' seq:0, type:22 => :hello attr_name1:foo attr_name2:bar
'second' seq:0, type:22 => attr_one:two attr_three:four
'third' seq:0, type:1 => baz
...
```

**dump**
```
./ldb --db=/tmp/rocksdbtest-226125/db_wide_basic_test_3481961_2308393776696827948/ dump
```
```
first ==> :hello attr_name1:foo attr_name2:bar
second ==> attr_one:two attr_three:four
third ==> baz
Keys in range: 3
```

**scan**
```
./ldb --db=/tmp/rocksdbtest-226125/db_wide_basic_test_3481961_2308393776696827948/ scan
```
```
first : :hello attr_name1:foo attr_name2:bar
second : attr_one:two attr_three:four
third : baz
```

**sst_dump**
```
./sst_dump --file=/tmp/rocksdbtest-226125/db_wide_basic_test_3481961_2308393776696827948/000013.sst --command=scan
```
```
options.env is 0x7ff54b296000
Process /tmp/rocksdbtest-226125/db_wide_basic_test_3481961_2308393776696827948/000013.sst
Sst file format: block-based
from [] to []
'first' seq:0, type:22 => :hello attr_name1:foo attr_name2:bar
'second' seq:0, type:22 => attr_one:two attr_three:four
'third' seq:0, type:1 => baz
```

Reviewed By: ltamasi

Differential Revision: D48837999

Pulled By: jaykorean

fbshipit-source-id: b0280f0589d2b9716bb9b50530ffcabb397d140f
2023-08-30 12:45:52 -07:00
Akanksha Mahajan 6353c6e2fb Add new experimental ReadOption auto_readahead_size to db_bench and db_stress (#11729)
Summary:
Same as title

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11729

Test Plan: make crash_test -j32

Reviewed By: anand1976

Differential Revision: D48534820

Pulled By: akankshamahajan15

fbshipit-source-id: 3a2a28af98dfad164b82ddaaf9fddb94c53a652e
2023-08-24 14:58:27 -07:00
Fuat Basik bc448e9c89 Run db_stress for final time to ensure un-interrupted validation (#11592)
Summary:
In blackbox tests, db_stress command always run with timeout. Timeout can happen during validation, leaving some of the keys not checked. Since key validation is done in order, it is quite likely that keys those are towards to the end of the set are never validated. This PR adds a final execution, without timeout, to ensure validation is executed for all keys, at least once.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11592

Reviewed By: cbi42

Differential Revision: D48003998

Pulled By: hx235

fbshipit-source-id: 72543475a932f12cf0f57534b7e3b6e07e87080f
2023-08-23 15:24:23 -07:00
Changyu Bi 5e0584bd73 Do not drop unsynced data during reopen in stress test (#11731)
Summary:
Currently the stress test does not support restoring expected state (to a specific sequence number) when there is unsynced data loss during the reopen phase. This causes a few internal stress test failure with errors like inconsistent value. This PR disables dropping unsynced data during reopen to avoid failures due to this issue. We can re-enable later after we decide to support unsynced data loss during DB reopen in stress test.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11731

Test Plan:
* Running this test a few times can fail for inconsistent value before this change
```
./db_stress --acquire_snapshot_one_in=10000 --adaptive_readahead=1 --allow_concurrent_memtable_write=1 --allow_data_in_errors=True --async_io=0 --avoid_flush_during_recovery=0 --avoid_unnecessary_blocking_io=0 --backup_max_size=104857600 --backup_one_in=0 --batch_protection_bytes_per_key=0 --block_protection_bytes_per_key=8 --block_size=16384 --bloom_bits=20.57166126835524 --bottommost_compression_type=disable --bytes_per_sync=262144 --cache_index_and_filter_blocks=1 --cache_size=8388608 --cache_type=auto_hyper_clock_cache --charge_compression_dictionary_building_buffer=1 --charge_file_metadata=1 --charge_filter_construction=0 --charge_table_reader=1 --checkpoint_one_in=0 --checksum_type=kxxHash --clear_column_family_one_in=0 --column_families=1 --compact_files_one_in=1000000 --compact_range_one_in=1000000 --compaction_pri=3 --compaction_style=1 --compaction_ttl=100 --compression_max_dict_buffer_bytes=0 --compression_max_dict_bytes=0 --compression_parallel_threads=1 --compression_type=zstd --compression_use_zstd_dict_trainer=1 --compression_zstd_max_train_bytes=0 --continuous_verification_interval=0 --data_block_index_type=0 --db=/dev/shm/rocksdb_test/rocksdb_crashtest_whitebox --db_write_buffer_size=0 --delpercent=4 --delrangepercent=1 --destroy_db_initially=0 --detect_filter_construct_corruption=0 --disable_wal=0 --enable_compaction_filter=0 --enable_pipelined_write=1 --enable_thread_tracking=0 --expected_values_dir=/dev/shm/rocksdb_test/rocksdb_crashtest_expected --fail_if_options_file_error=1 --fifo_allow_compaction=1 --file_checksum_impl=big --flush_one_in=1000000 --format_version=3 --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=6 --index_type=3 --ingest_external_file_one_in=0 --initial_auto_readahead_size=16384 --iterpercent=10 --key_len_percent_dist=1,30,69 --level_compaction_dynamic_level_bytes=1 --lock_wal_one_in=1000000 --log2_keys_per_lock=10 --long_running_snapshots=1 --manual_wal_flush_one_in=1000000 --mark_for_compaction_one_file_in=10 --max_auto_readahead_size=0 --max_background_compactions=1 --max_bytes_for_level_base=67108864 --max_key=25000000 --max_key_len=3 --max_manifest_file_size=1073741824 --max_write_batch_group_size_bytes=16777216 --max_write_buffer_number=3 --max_write_buffer_size_to_maintain=0 --memtable_max_range_deletions=100 --memtable_prefix_bloom_size_ratio=0 --memtable_protection_bytes_per_key=1 --memtable_whole_key_filtering=0 --memtablerep=skip_list --min_write_buffer_number_to_merge=2 --mmap_read=0 --mock_direct_io=False --nooverwritepercent=1 --num_file_reads_for_auto_readahead=1 --open_files=-1 --open_metadata_write_fault_one_in=0 --open_read_fault_one_in=5 --open_write_fault_one_in=0 --ops_per_thread=200000 --optimize_filters_for_memory=0 --paranoid_file_checks=1 --partition_filters=0 --partition_pinning=3 --pause_background_one_in=1000000 --periodic_compaction_seconds=10 --prefix_size=-1 --prefixpercent=0 --prepopulate_block_cache=1 --preserve_internal_time_seconds=0 --progress_reports=0 --read_fault_one_in=1000 --readahead_size=524288 --readpercent=50 --recycle_log_file_num=0 --reopen=20 --ribbon_starting_level=0 --secondary_cache_fault_one_in=32 --snapshot_hold_ops=100000 --sst_file_manager_bytes_per_sec=0 --sst_file_manager_bytes_per_truncate=0 --stats_dump_period_sec=10 --subcompactions=3 --sync=0 --sync_fault_injection=1 --target_file_size_base=16777216 --target_file_size_multiplier=1 --test_batches_snapshots=0 --top_level_index_pinning=2 --unpartitioned_pinning=1 --use_direct_io_for_flush_and_compaction=0 --use_direct_reads=0 --use_full_merge_v1=0 --use_get_entity=1 --use_merge=0 --use_multi_get_entity=0 --use_multiget=1 --use_put_entity_one_in=1 --user_timestamp_size=0 --value_size_mult=32 --verify_checksum=1 --verify_checksum_one_in=1000000 --verify_db_one_in=100000 --verify_file_checksums_one_in=1000000 --verify_iterator_with_expected_state_one_in=5 --verify_sst_unique_id_in_manifest=1 --wal_bytes_per_sync=524288 --wal_compression=zstd --write_buffer_size=33554432 --write_dbid_to_manifest=1 --writepercent=35```

Reviewed By: hx235

Differential Revision: D48537494

Pulled By: cbi42

fbshipit-source-id: ddae21b9bb6ee8d67229121f58513e95f7ef6d8d
2023-08-22 09:47:04 -07:00
Changyu Bi c2aad555c3 Add CompressionOptions::checksum for enabling ZSTD checksum (#11666)
Summary:
Optionally enable zstd checksum flag (d857369028/lib/zstd.h (L428)) to detect corruption during decompression. Main changes are in compression.h:
* User can set CompressionOptions::checksum to true to enable this feature.
* We enable this feature in ZSTD by setting the checksum flag in ZSTD compression context: `ZSTD_CCtx`.
* Uses `ZSTD_compress2()` to do compression since it supports frame parameter like the checksum flag. Compression level is also set in compression context as a flag.
* Error handling during decompression to propagate error message from ZSTD.
* Updated microbench to test read performance impact.

About compatibility, the current compression decoders should continue to work with the data created by the new compression API `ZSTD_compress2()`: https://github.com/facebook/zstd/issues/3711.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11666

Test Plan:
* Existing unit tests for zstd compression
* Add unit test `DBTest2.ZSTDChecksum` to test the corruption case
* Manually tested that compression levels, parallel compression, dictionary compression, index compression all work with the new ZSTD_compress2() API.
* Manually tested with `sst_dump --command=recompress` that different compression levels and dictionary compression settings all work.
* Manually tested compiling with older versions of ZSTD: v1.3.8, v1.1.0, v0.6.2.
* Perf impact: from public benchmark data: http://fastcompression.blogspot.com/2019/03/presenting-xxh3.html for checksum and https://github.com/facebook/zstd#benchmarks, if decompression is 1700MB/s and checksum computation is 70000MB/s, checksum computation is an additional ~2.4% time for decompression. Compression is slower and checksumming should be less noticeable.
* Microbench:
```
TEST_TMPDIR=/dev/shm ./branch_db_basic_bench --benchmark_filter=DBGet/comp_style:0/max_data:1048576/per_key_size:256/enable_statistics:0/negative_query:0/enable_filter:0/mmap:0/compression_type:7/compression_checksum:1/no_blockcache:1/iterations:10000/threads:1 --benchmark_repetitions=100

Min out of 100 runs:
Main:
10390 10436 10456 10484 10499 10535 10544 10545 10565 10568

After this PR, checksum=false
10285 10397 10503 10508 10515 10557 10562 10635 10640 10660

After this PR, checksum=true
10827 10876 10925 10949 10971 11052 11061 11063 11100 11109
```
* db_bench:
```
Write perf
TEST_TMPDIR=/dev/shm/ ./db_bench_ichecksum --benchmarks=fillseq[-X10] --compression_type=zstd --num=10000000 --compression_checksum=..

[FillSeq checksum=0]
fillseq [AVG    10 runs] : 281635 (± 31711) ops/sec;   31.2 (± 3.5) MB/sec
fillseq [MEDIAN 10 runs] : 294027 ops/sec;   32.5 MB/sec

[FillSeq checksum=1]
fillseq [AVG    10 runs] : 286961 (± 34700) ops/sec;   31.7 (± 3.8) MB/sec
fillseq [MEDIAN 10 runs] : 283278 ops/sec;   31.3 MB/sec

Read perf
TEST_TMPDIR=/dev/shm ./db_bench_ichecksum --benchmarks=readrandom[-X20] --num=100000000 --reads=1000000 --use_existing_db=true --readonly=1

[Readrandom checksum=1]
readrandom [AVG    20 runs] : 360928 (± 3579) ops/sec;    4.0 (± 0.0) MB/sec
readrandom [MEDIAN 20 runs] : 362468 ops/sec;    4.0 MB/sec

[Readrandom checksum=0]
readrandom [AVG    20 runs] : 380365 (± 2384) ops/sec;    4.2 (± 0.0) MB/sec
readrandom [MEDIAN 20 runs] : 379800 ops/sec;    4.2 MB/sec

Compression
TEST_TMPDIR=/dev/shm ./db_bench_ichecksum --benchmarks=compress[-X20] --compression_type=zstd --num=100000000 --compression_checksum=1

checksum=1
compress [AVG    20 runs] : 54074 (± 634) ops/sec;  211.2 (± 2.5) MB/sec
compress [MEDIAN 20 runs] : 54396 ops/sec;  212.5 MB/sec

checksum=0
compress [AVG    20 runs] : 54598 (± 393) ops/sec;  213.3 (± 1.5) MB/sec
compress [MEDIAN 20 runs] : 54592 ops/sec;  213.3 MB/sec

Decompression:
TEST_TMPDIR=/dev/shm ./db_bench_ichecksum --benchmarks=uncompress[-X20] --compression_type=zstd --compression_checksum=1

checksum = 0
uncompress [AVG    20 runs] : 167499 (± 962) ops/sec;  654.3 (± 3.8) MB/sec
uncompress [MEDIAN 20 runs] : 167210 ops/sec;  653.2 MB/sec
checksum = 1
uncompress [AVG    20 runs] : 167980 (± 924) ops/sec;  656.2 (± 3.6) MB/sec
uncompress [MEDIAN 20 runs] : 168465 ops/sec;  658.1 MB/sec
```

Reviewed By: ajkr

Differential Revision: D48019378

Pulled By: cbi42

fbshipit-source-id: 674120c6e1853c2ced1436ac8138559d0204feba
2023-08-18 15:01:59 -07:00
Changyu Bi d1ff401472 Delay bottommost level single file compactions (#11701)
Summary:
For leveled compaction, RocksDB has a special kind of compaction with reason "kBottommmostFiles" that compacts bottommost level files to clear data held by snapshots (more detail in https://github.com/facebook/rocksdb/issues/3009). Such compactions can happen soon after a relevant snapshot is released. For some use cases, a bottommost file may contain only a small amount of keys that can be cleared, so compacting such a file has a high write amp. In addition, these bottommost files may be compacted in compactions with reason other than "kBottommmostFiles" if we wait for some time (so that enough data is ingested to trigger such a compaction). This PR introduces an option `bottommost_file_compaction_delay` to specify the delay of these bottommost level single file compactions.

* The main change is in `VersionStorageInfo::ComputeBottommostFilesMarkedForCompaction()` where we only add a file to `bottommost_files_marked_for_compaction_` if it oldest_snapshot is larger than its non-zero largest_seqno **and** the file is old enough. Note that if a file is not old enough but its largest_seqno is less than oldest_snapshot, we exclude it from the calculation of `bottommost_files_mark_threshold_`. This makes the change simpler, but such a file's eligibility for compaction will only be checked the next time `ComputeBottommostFilesMarkedForCompaction()` is called. This happens when a new Version is created (compaction, flush, SetOptions()...), a new enough snapshot is released (`VersionStorageInfo::UpdateOldestSnapshot()`) or when a compaction is picked and compaction score has to be re-calculated.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11701

Test Plan:
* Add two unit tests to test when bottommost_file_compaction_delay > 0.
* Ran crash test with the new option.

Reviewed By: jaykorean, ajkr

Differential Revision: D48331564

Pulled By: cbi42

fbshipit-source-id: c584f3dc5f6354fce3ed65f4c6366dc450b15ba8
2023-08-16 17:45:44 -07:00
Jay Huh b63018fb59 Wide Column Ingestion in CrashTest (#11697)
Summary:
`PutEntity` is now supported in SST file writer (https://github.com/facebook/rocksdb/issues/11688). This PR enables ingestion of wide column data in the stress/crash tests.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11697

Test Plan:
```
python3 tools/db_crashtest.py blackbox --simple --duration=300 --ingest_external_file_one_in=2 --use_put_entity_one_in=2 --max_key=1048576 -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 --interval=10 -value_size_mult=33 -column_families=1 -reopen=0 --key_len_percent_dist="1,30,69"
```

Reviewed By: ltamasi

Differential Revision: D48370719

Pulled By: jaykorean

fbshipit-source-id: 5855d3112b37b2fb300d05e6df110d899855d77d
2023-08-15 16:13:13 -07:00
Andrew Kryczka a09c141dde In TestIterateAgainstExpected(), verify iterator moves in expected direction (#11698)
Summary:
It's a bit repetitive in order to give reasonably informative error messages.

I also removed total_order_seek in cases where it's not needed, just to make sure a case that shouldn't matter really doesn't.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11698

Test Plan:
run it -

```
$ DEBUG_LEVEL=0 TEST_TMPDIR=/dev/shm python3 tools/db_crashtest.py blackbox --max_key=100000 --duration=86400 --interval=10 --write_buffer_size=524288 --target_file_size_base=524288 --max_bytes_for_level_base=2097152 --compression_type=none --blob_compression_type=none --writepercent=50 -iterpercent=45 -readpercent=0 -prefixpercent=0 --prefix_size=0 --verify_iterator_with_expected_state_one_in=10 --test_batches_snapshots=0 -enable_compaction_filter=0
```

Reviewed By: cbi42

Differential Revision: D48285036

Pulled By: ajkr

fbshipit-source-id: 51b147bd7c8011740629ae2fd8114d3d48ce7137
2023-08-14 14:57:28 -07:00
Peter Dillinger ef6f025563 Placeholder for AutoHyperClockCache, more (#11692)
Summary:
* The plan is for AutoHyperClockCache to be selected when HyperClockCacheOptions::estimated_entry_charge == 0, and in that case to use a new configuration option min_avg_entry_charge for determining an extreme case maximum size for the hash table. For the placeholder, a hack is in place in HyperClockCacheOptions::MakeSharedCache() to make the unit tests happy despite the new options not really making sense with the current implementation.
* Mostly updating and refactoring tests to test both the current HCC (internal name FixedHyperClockCache) and a placeholder for the new version (internal name AutoHyperClockCache).
* Simplify some existing tests not to depend directly on cache type.
* Type-parameterize the shard-level unit tests, which unfortunately requires more syntax like `this->` in places for disambiguation.
* Added means of choosing auto_hyper_clock_cache to cache_bench, db_bench, and db_stress, including add to crash test.
* Add another templated class BaseHyperClockCache to reduce future copy-paste
* Added ReportProblems support to cache_bench
* Added a DEBUG-level diagnostic to ReportProblems for the variance in load factor throughout the table, which will become more of a concern with linear hashing to be used in the Auto implementation. Example with current Fixed HCC:
```
2023/08/10-13:41:41.602450 6ac36 [DEBUG] [che/clock_cache.cc:1507] Slot occupancy stats: Overall 49% (129008/262144), Min/Max/Window = 39%/60%/500, MaxRun{Pos/Neg} = 18/17
```

In other words, with overall occupancy of 49%, the lowest across any 500 contiguous cells is 39% and highest 60%. Longest run of occupied is 18 and longest run of unoccupied is 17. This seems consistent with random samples from a uniform distribution.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11692

Test Plan: Shouldn't be any meaningful changes yet to production code or to what is tested, but there is temporary redundancy in testing until the new implementation is plugged in.

Reviewed By: jowlyzhang

Differential Revision: D48247413

Pulled By: pdillinger

fbshipit-source-id: 11541f996d97af403c2e43c92fb67ff22dd0b5da
2023-08-11 16:27:38 -07:00
Hui Xiao 38ecfabed2 Remove comment about locking about TestIterateAgainstExpected (#11695)
Summary:
**Context/Summary**
After https://github.com/facebook/rocksdb/pull/11058, we no longer lock the key range to iterate in TestIterateAgainstExpected, except for working with timestamp feature.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11695

Test Plan: no code change

Reviewed By: ajkr

Differential Revision: D48276668

Pulled By: hx235

fbshipit-source-id: dc92a3708b2281dc737c0877fb755548bf03a9fc
2023-08-11 13:14:04 -07:00
Peter Dillinger a85eccc6d6 Adjust db_stress handling of TryAgain from optimistic txn (#11691)
Summary:
We're still getting some rare cases of 5x TryAgains in a row. Here I'm boosting the failure threshold to 10 in a row and adding more info in the output, to help us manually verify whether there's anything suspicous about the sequence of TryAgains, such as if Rollback failed to reset to new sequence numbers.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11691

Test Plan: By lowering the threshold to 2 and adjusting some other db_crashtest parameters, I was able to hit my new code and saw fresh sequence number on the subsequent TryAgain.

Reviewed By: cbi42

Differential Revision: D48236153

Pulled By: pdillinger

fbshipit-source-id: c0530e969ddcf8de7348e5cf7daf5d6d5dec24f4
2023-08-10 13:05:45 -07:00