db_stress_common.h 29 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823
  1. // Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
  2. // This source code is licensed under both the GPLv2 (found in the
  3. // COPYING file in the root directory) and Apache 2.0 License
  4. // (found in the LICENSE.Apache file in the root directory).
  5. //
  6. // Copyright (c) 2011 The LevelDB Authors. All rights reserved.
  7. // Use of this source code is governed by a BSD-style license that can be
  8. // found in the LICENSE file. See the AUTHORS file for names of contributors.
  9. //
  10. // The test uses an array to compare against values written to the database.
  11. // Keys written to the array are in 1:1 correspondence to the actual values in
  12. // the database according to the formula in the function GenerateValue.
  13. // Space is reserved in the array from 0 to FLAGS_max_key and values are
  14. // randomly written/deleted/read from those positions. During verification we
  15. // compare all the positions in the array. To shorten/elongate the running
  16. // time, you could change the settings: FLAGS_max_key, FLAGS_ops_per_thread,
  17. // (sometimes also FLAGS_threads).
  18. //
  19. // NOTE that if FLAGS_test_batches_snapshots is set, the test will have
  20. // different behavior. See comment of the flag for details.
  21. #ifdef GFLAGS
  22. #pragma once
  23. #include <fcntl.h>
  24. #include <stdio.h>
  25. #include <stdlib.h>
  26. #include <sys/types.h>
  27. #include <algorithm>
  28. #include <array>
  29. #include <chrono>
  30. #include <cinttypes>
  31. #include <exception>
  32. #include <queue>
  33. #include <thread>
  34. #include "db/db_impl/db_impl.h"
  35. #include "db/version_set.h"
  36. #include "db/wide/wide_columns_helper.h"
  37. #include "db_stress_tool/db_stress_env_wrapper.h"
  38. #include "db_stress_tool/db_stress_listener.h"
  39. #include "db_stress_tool/db_stress_shared_state.h"
  40. #include "db_stress_tool/db_stress_test_base.h"
  41. #include "logging/logging.h"
  42. #include "monitoring/histogram.h"
  43. #include "options/options_helper.h"
  44. #include "port/port.h"
  45. #include "rocksdb/advanced_options.h"
  46. #include "rocksdb/cache.h"
  47. #include "rocksdb/env.h"
  48. #include "rocksdb/slice.h"
  49. #include "rocksdb/slice_transform.h"
  50. #include "rocksdb/statistics.h"
  51. #include "rocksdb/utilities/backup_engine.h"
  52. #include "rocksdb/utilities/checkpoint.h"
  53. #include "rocksdb/utilities/db_ttl.h"
  54. #include "rocksdb/utilities/debug.h"
  55. #include "rocksdb/utilities/optimistic_transaction_db.h"
  56. #include "rocksdb/utilities/options_util.h"
  57. #include "rocksdb/utilities/transaction.h"
  58. #include "rocksdb/utilities/transaction_db.h"
  59. #include "rocksdb/write_batch.h"
  60. #include "test_util/testutil.h"
  61. #include "util/coding.h"
  62. #include "util/compression.h"
  63. #include "util/crc32c.h"
  64. #include "util/gflags_compat.h"
  65. #include "util/mutexlock.h"
  66. #include "util/random.h"
  67. #include "util/string_util.h"
  68. #include "utilities/blob_db/blob_db.h"
  69. #include "utilities/fault_injection_fs.h"
  70. #include "utilities/merge_operators.h"
  71. using GFLAGS_NAMESPACE::ParseCommandLineFlags;
  72. using GFLAGS_NAMESPACE::RegisterFlagValidator;
  73. using GFLAGS_NAMESPACE::SetUsageMessage;
  74. DECLARE_uint64(seed);
  75. DECLARE_bool(read_only);
  76. DECLARE_int64(max_key);
  77. DECLARE_double(hot_key_alpha);
  78. DECLARE_int32(max_key_len);
  79. DECLARE_string(key_len_percent_dist);
  80. DECLARE_int32(key_window_scale_factor);
  81. DECLARE_int32(column_families);
  82. DECLARE_string(options_file);
  83. DECLARE_int64(active_width);
  84. DECLARE_bool(test_batches_snapshots);
  85. DECLARE_bool(atomic_flush);
  86. DECLARE_int32(lock_wal_one_in);
  87. DECLARE_bool(test_cf_consistency);
  88. DECLARE_bool(test_multi_ops_txns);
  89. DECLARE_int32(threads);
  90. DECLARE_int32(ttl);
  91. DECLARE_int32(value_size_mult);
  92. DECLARE_int32(compaction_readahead_size);
  93. DECLARE_bool(enable_pipelined_write);
  94. DECLARE_bool(verify_before_write);
  95. DECLARE_bool(histogram);
  96. DECLARE_bool(destroy_db_initially);
  97. DECLARE_bool(verbose);
  98. DECLARE_bool(progress_reports);
  99. DECLARE_uint64(db_write_buffer_size);
  100. DECLARE_int32(write_buffer_size);
  101. DECLARE_int32(max_write_buffer_number);
  102. DECLARE_int32(min_write_buffer_number_to_merge);
  103. DECLARE_int64(max_write_buffer_size_to_maintain);
  104. DECLARE_bool(use_write_buffer_manager);
  105. DECLARE_double(memtable_prefix_bloom_size_ratio);
  106. DECLARE_bool(memtable_whole_key_filtering);
  107. DECLARE_int32(open_files);
  108. DECLARE_uint64(compressed_secondary_cache_size);
  109. DECLARE_int32(compressed_secondary_cache_numshardbits);
  110. DECLARE_int32(secondary_cache_update_interval);
  111. DECLARE_double(compressed_secondary_cache_ratio);
  112. DECLARE_int32(compaction_style);
  113. DECLARE_int32(compaction_pri);
  114. DECLARE_int32(num_levels);
  115. DECLARE_int32(level0_file_num_compaction_trigger);
  116. DECLARE_int32(level0_slowdown_writes_trigger);
  117. DECLARE_int32(level0_stop_writes_trigger);
  118. DECLARE_int32(block_size);
  119. DECLARE_int32(format_version);
  120. DECLARE_int32(index_block_restart_interval);
  121. DECLARE_int32(max_background_compactions);
  122. DECLARE_int32(num_bottom_pri_threads);
  123. DECLARE_int32(compaction_thread_pool_adjust_interval);
  124. DECLARE_int32(compaction_thread_pool_variations);
  125. DECLARE_int32(max_background_flushes);
  126. DECLARE_int32(universal_size_ratio);
  127. DECLARE_int32(universal_min_merge_width);
  128. DECLARE_int32(universal_max_merge_width);
  129. DECLARE_int32(universal_max_size_amplification_percent);
  130. DECLARE_int32(universal_max_read_amp);
  131. DECLARE_int32(clear_column_family_one_in);
  132. DECLARE_int32(get_live_files_apis_one_in);
  133. DECLARE_int32(get_all_column_family_metadata_one_in);
  134. DECLARE_int32(get_sorted_wal_files_one_in);
  135. DECLARE_int32(get_current_wal_file_one_in);
  136. DECLARE_int32(set_options_one_in);
  137. DECLARE_int32(set_in_place_one_in);
  138. DECLARE_int64(cache_size);
  139. DECLARE_int32(cache_numshardbits);
  140. DECLARE_bool(cache_index_and_filter_blocks);
  141. DECLARE_bool(charge_compression_dictionary_building_buffer);
  142. DECLARE_bool(charge_filter_construction);
  143. DECLARE_bool(charge_table_reader);
  144. DECLARE_bool(charge_file_metadata);
  145. DECLARE_bool(charge_blob_cache);
  146. DECLARE_bool(decouple_partitioned_filters);
  147. DECLARE_int32(top_level_index_pinning);
  148. DECLARE_int32(partition_pinning);
  149. DECLARE_int32(unpartitioned_pinning);
  150. DECLARE_string(cache_type);
  151. DECLARE_uint64(subcompactions);
  152. DECLARE_uint64(periodic_compaction_seconds);
  153. DECLARE_string(daily_offpeak_time_utc);
  154. DECLARE_uint64(compaction_ttl);
  155. DECLARE_bool(fifo_allow_compaction);
  156. DECLARE_bool(allow_concurrent_memtable_write);
  157. DECLARE_double(experimental_mempurge_threshold);
  158. DECLARE_bool(enable_write_thread_adaptive_yield);
  159. DECLARE_int32(reopen);
  160. DECLARE_double(bloom_bits);
  161. DECLARE_int32(bloom_before_level);
  162. DECLARE_bool(partition_filters);
  163. DECLARE_bool(optimize_filters_for_memory);
  164. DECLARE_bool(detect_filter_construct_corruption);
  165. DECLARE_string(sqfc_name);
  166. DECLARE_uint32(sqfc_version);
  167. DECLARE_bool(use_sqfc_for_range_queries);
  168. DECLARE_int32(index_type);
  169. DECLARE_int32(data_block_index_type);
  170. DECLARE_string(db);
  171. DECLARE_string(secondaries_base);
  172. DECLARE_bool(test_secondary);
  173. DECLARE_string(expected_values_dir);
  174. DECLARE_bool(verify_checksum);
  175. DECLARE_bool(mmap_read);
  176. DECLARE_bool(mmap_write);
  177. DECLARE_bool(use_direct_reads);
  178. DECLARE_bool(use_direct_io_for_flush_and_compaction);
  179. DECLARE_bool(mock_direct_io);
  180. DECLARE_bool(statistics);
  181. DECLARE_bool(sync);
  182. DECLARE_bool(use_fsync);
  183. DECLARE_uint64(stats_dump_period_sec);
  184. DECLARE_uint64(bytes_per_sync);
  185. DECLARE_uint64(wal_bytes_per_sync);
  186. DECLARE_int32(kill_random_test);
  187. DECLARE_string(kill_exclude_prefixes);
  188. DECLARE_uint64(recycle_log_file_num);
  189. DECLARE_int64(target_file_size_base);
  190. DECLARE_int32(target_file_size_multiplier);
  191. DECLARE_uint64(max_bytes_for_level_base);
  192. DECLARE_double(max_bytes_for_level_multiplier);
  193. DECLARE_uint64(rate_limiter_bytes_per_sec);
  194. DECLARE_bool(rate_limit_bg_reads);
  195. DECLARE_bool(rate_limit_user_ops);
  196. DECLARE_bool(rate_limit_auto_wal_flush);
  197. DECLARE_uint64(sst_file_manager_bytes_per_sec);
  198. DECLARE_uint64(sst_file_manager_bytes_per_truncate);
  199. DECLARE_int32(backup_one_in);
  200. DECLARE_uint64(backup_max_size);
  201. DECLARE_int32(checkpoint_one_in);
  202. DECLARE_int32(ingest_external_file_one_in);
  203. DECLARE_int32(ingest_external_file_width);
  204. DECLARE_int32(compact_files_one_in);
  205. DECLARE_int32(compact_range_one_in);
  206. DECLARE_int32(promote_l0_one_in);
  207. DECLARE_int32(mark_for_compaction_one_file_in);
  208. DECLARE_int32(flush_one_in);
  209. DECLARE_int32(key_may_exist_one_in);
  210. DECLARE_int32(reset_stats_one_in);
  211. DECLARE_int32(pause_background_one_in);
  212. DECLARE_int32(disable_file_deletions_one_in);
  213. DECLARE_int32(disable_manual_compaction_one_in);
  214. DECLARE_int32(compact_range_width);
  215. DECLARE_int32(acquire_snapshot_one_in);
  216. DECLARE_bool(compare_full_db_state_snapshot);
  217. DECLARE_uint64(snapshot_hold_ops);
  218. DECLARE_bool(long_running_snapshots);
  219. DECLARE_bool(use_multiget);
  220. DECLARE_bool(use_get_entity);
  221. DECLARE_bool(use_multi_get_entity);
  222. DECLARE_int32(readpercent);
  223. DECLARE_int32(prefixpercent);
  224. DECLARE_int32(writepercent);
  225. DECLARE_int32(delpercent);
  226. DECLARE_int32(delrangepercent);
  227. DECLARE_int32(nooverwritepercent);
  228. DECLARE_int32(iterpercent);
  229. DECLARE_uint64(num_iterations);
  230. DECLARE_int32(customopspercent);
  231. DECLARE_string(compression_type);
  232. DECLARE_string(bottommost_compression_type);
  233. DECLARE_int32(compression_max_dict_bytes);
  234. DECLARE_int32(compression_zstd_max_train_bytes);
  235. DECLARE_int32(compression_parallel_threads);
  236. DECLARE_uint64(compression_max_dict_buffer_bytes);
  237. DECLARE_bool(compression_use_zstd_dict_trainer);
  238. DECLARE_bool(compression_checksum);
  239. DECLARE_string(checksum_type);
  240. DECLARE_string(env_uri);
  241. DECLARE_string(fs_uri);
  242. DECLARE_uint64(ops_per_thread);
  243. DECLARE_uint64(log2_keys_per_lock);
  244. DECLARE_uint64(max_manifest_file_size);
  245. DECLARE_bool(in_place_update);
  246. DECLARE_string(memtablerep);
  247. DECLARE_int32(prefix_size);
  248. DECLARE_bool(use_merge);
  249. DECLARE_uint32(use_put_entity_one_in);
  250. DECLARE_bool(use_attribute_group);
  251. DECLARE_bool(use_multi_cf_iterator);
  252. DECLARE_bool(use_full_merge_v1);
  253. DECLARE_int32(sync_wal_one_in);
  254. DECLARE_bool(avoid_unnecessary_blocking_io);
  255. DECLARE_bool(write_dbid_to_manifest);
  256. DECLARE_bool(write_identity_file);
  257. DECLARE_bool(avoid_flush_during_recovery);
  258. DECLARE_uint64(max_write_batch_group_size_bytes);
  259. DECLARE_bool(level_compaction_dynamic_level_bytes);
  260. DECLARE_int32(verify_checksum_one_in);
  261. DECLARE_int32(verify_file_checksums_one_in);
  262. DECLARE_int32(verify_db_one_in);
  263. DECLARE_int32(continuous_verification_interval);
  264. DECLARE_int32(get_property_one_in);
  265. DECLARE_int32(get_properties_of_all_tables_one_in);
  266. DECLARE_string(file_checksum_impl);
  267. DECLARE_bool(verification_only);
  268. DECLARE_string(last_level_temperature);
  269. DECLARE_string(default_write_temperature);
  270. DECLARE_string(default_temperature);
  271. DECLARE_bool(paranoid_memory_checks);
  272. DECLARE_bool(memtable_veirfy_per_key_checksum_on_seek);
  273. // Options for transaction dbs.
  274. // Use TransactionDB (a.k.a. Pessimistic Transaction DB)
  275. // OR OptimisticTransactionDB
  276. DECLARE_bool(use_txn);
  277. // Options for TransactionDB (a.k.a. Pessimistic Transaction DB)
  278. DECLARE_uint64(txn_write_policy);
  279. DECLARE_bool(unordered_write);
  280. DECLARE_bool(use_per_key_point_lock_mgr);
  281. // Options for OptimisticTransactionDB
  282. DECLARE_bool(use_optimistic_txn);
  283. DECLARE_uint64(occ_validation_policy);
  284. DECLARE_bool(share_occ_lock_buckets);
  285. DECLARE_uint32(occ_lock_bucket_count);
  286. // Options for StackableDB-based BlobDB
  287. DECLARE_bool(use_blob_db);
  288. DECLARE_uint64(blob_db_min_blob_size);
  289. DECLARE_uint64(blob_db_bytes_per_sync);
  290. DECLARE_uint64(blob_db_file_size);
  291. DECLARE_bool(blob_db_enable_gc);
  292. DECLARE_double(blob_db_gc_cutoff);
  293. // Options for integrated BlobDB
  294. DECLARE_bool(allow_setting_blob_options_dynamically);
  295. DECLARE_bool(enable_blob_files);
  296. DECLARE_uint64(min_blob_size);
  297. DECLARE_uint64(blob_file_size);
  298. DECLARE_string(blob_compression_type);
  299. DECLARE_bool(enable_blob_garbage_collection);
  300. DECLARE_double(blob_garbage_collection_age_cutoff);
  301. DECLARE_double(blob_garbage_collection_force_threshold);
  302. DECLARE_uint64(blob_compaction_readahead_size);
  303. DECLARE_int32(blob_file_starting_level);
  304. DECLARE_bool(use_blob_cache);
  305. DECLARE_bool(use_shared_block_and_blob_cache);
  306. DECLARE_uint64(blob_cache_size);
  307. DECLARE_int32(blob_cache_numshardbits);
  308. DECLARE_int32(prepopulate_blob_cache);
  309. DECLARE_int32(approximate_size_one_in);
  310. DECLARE_bool(best_efforts_recovery);
  311. DECLARE_bool(skip_verifydb);
  312. DECLARE_bool(paranoid_file_checks);
  313. DECLARE_uint64(batch_protection_bytes_per_key);
  314. DECLARE_uint32(memtable_protection_bytes_per_key);
  315. DECLARE_uint32(block_protection_bytes_per_key);
  316. DECLARE_uint64(user_timestamp_size);
  317. DECLARE_bool(persist_user_defined_timestamps);
  318. DECLARE_string(secondary_cache_uri);
  319. DECLARE_int32(secondary_cache_fault_one_in);
  320. DECLARE_int32(prepopulate_block_cache);
  321. DECLARE_bool(two_write_queues);
  322. DECLARE_bool(use_only_the_last_commit_time_batch_for_recovery);
  323. DECLARE_uint64(wp_snapshot_cache_bits);
  324. DECLARE_uint64(wp_commit_cache_bits);
  325. DECLARE_bool(adaptive_readahead);
  326. DECLARE_bool(async_io);
  327. DECLARE_string(wal_compression);
  328. DECLARE_bool(verify_sst_unique_id_in_manifest);
  329. DECLARE_int32(create_timestamped_snapshot_one_in);
  330. DECLARE_bool(allow_data_in_errors);
  331. DECLARE_bool(enable_thread_tracking);
  332. DECLARE_uint32(memtable_max_range_deletions);
  333. DECLARE_uint32(bottommost_file_compaction_delay);
  334. // Tiered storage
  335. DECLARE_int64(preclude_last_level_data_seconds);
  336. DECLARE_int64(preserve_internal_time_seconds);
  337. DECLARE_uint32(use_timed_put_one_in);
  338. DECLARE_int32(verify_iterator_with_expected_state_one_in);
  339. DECLARE_bool(preserve_unverified_changes);
  340. DECLARE_uint64(readahead_size);
  341. DECLARE_uint64(initial_auto_readahead_size);
  342. DECLARE_uint64(max_auto_readahead_size);
  343. DECLARE_uint64(num_file_reads_for_auto_readahead);
  344. DECLARE_bool(auto_readahead_size);
  345. DECLARE_bool(allow_fallocate);
  346. DECLARE_int32(table_cache_numshardbits);
  347. DECLARE_bool(enable_write_thread_adaptive_yield);
  348. DECLARE_uint64(log_readahead_size);
  349. DECLARE_uint64(bgerror_resume_retry_interval);
  350. DECLARE_uint64(delete_obsolete_files_period_micros);
  351. DECLARE_uint64(max_log_file_size);
  352. DECLARE_uint64(log_file_time_to_roll);
  353. DECLARE_bool(use_adaptive_mutex);
  354. DECLARE_bool(advise_random_on_open);
  355. DECLARE_uint64(WAL_ttl_seconds);
  356. DECLARE_uint64(WAL_size_limit_MB);
  357. DECLARE_bool(strict_bytes_per_sync);
  358. DECLARE_bool(avoid_flush_during_shutdown);
  359. DECLARE_bool(fill_cache);
  360. DECLARE_bool(optimize_multiget_for_io);
  361. DECLARE_bool(memtable_insert_hint_per_batch);
  362. DECLARE_bool(dump_malloc_stats);
  363. DECLARE_uint64(stats_history_buffer_size);
  364. DECLARE_bool(skip_stats_update_on_db_open);
  365. DECLARE_bool(optimize_filters_for_hits);
  366. DECLARE_uint64(sample_for_compression);
  367. DECLARE_bool(report_bg_io_stats);
  368. DECLARE_bool(cache_index_and_filter_blocks_with_high_priority);
  369. DECLARE_bool(use_delta_encoding);
  370. DECLARE_bool(verify_compression);
  371. DECLARE_uint32(read_amp_bytes_per_bit);
  372. DECLARE_bool(enable_index_compression);
  373. DECLARE_uint32(index_shortening);
  374. DECLARE_uint32(metadata_charge_policy);
  375. DECLARE_bool(use_adaptive_mutex_lru);
  376. DECLARE_uint32(compress_format_version);
  377. DECLARE_uint64(manifest_preallocation_size);
  378. DECLARE_bool(enable_checksum_handoff);
  379. DECLARE_string(compression_manager);
  380. DECLARE_uint64(max_total_wal_size);
  381. DECLARE_double(high_pri_pool_ratio);
  382. DECLARE_double(low_pri_pool_ratio);
  383. DECLARE_uint64(soft_pending_compaction_bytes_limit);
  384. DECLARE_uint64(hard_pending_compaction_bytes_limit);
  385. DECLARE_uint64(max_sequential_skip_in_iterations);
  386. DECLARE_bool(enable_sst_partitioner_factory);
  387. DECLARE_bool(enable_do_not_compress_roles);
  388. DECLARE_bool(block_align);
  389. DECLARE_uint64(super_block_alignment_size);
  390. DECLARE_uint64(super_block_alignment_space_overhead_ratio);
  391. DECLARE_uint32(lowest_used_cache_tier);
  392. DECLARE_bool(enable_custom_split_merge);
  393. DECLARE_uint32(adm_policy);
  394. DECLARE_bool(enable_memtable_insert_with_hint_prefix_extractor);
  395. DECLARE_bool(check_multiget_consistency);
  396. DECLARE_bool(check_multiget_entity_consistency);
  397. DECLARE_bool(inplace_update_support);
  398. DECLARE_uint32(uncache_aggressiveness);
  399. DECLARE_int32(test_ingest_standalone_range_deletion_one_in);
  400. DECLARE_bool(allow_unprepared_value);
  401. DECLARE_string(file_temperature_age_thresholds);
  402. DECLARE_bool(allow_trivial_copy_when_change_temperature);
  403. DECLARE_uint32(commit_bypass_memtable_one_in);
  404. DECLARE_bool(track_and_verify_wals);
  405. DECLARE_int32(remote_compaction_worker_threads);
  406. DECLARE_int32(remote_compaction_worker_interval);
  407. DECLARE_bool(remote_compaction_failure_fall_back_to_local);
  408. DECLARE_int32(allow_resumption_one_in);
  409. DECLARE_bool(auto_refresh_iterator_with_snapshot);
  410. DECLARE_uint32(memtable_op_scan_flush_trigger);
  411. DECLARE_uint32(memtable_avg_op_scan_flush_trigger);
  412. DECLARE_uint32(ingest_wbwi_one_in);
  413. DECLARE_bool(universal_reduce_file_locking);
  414. DECLARE_bool(use_multiscan);
  415. DECLARE_bool(multiscan_use_async_io);
  416. // Compaction deletion trigger declarations for stress testing
  417. DECLARE_bool(enable_compaction_on_deletion_trigger);
  418. DECLARE_uint64(compaction_on_deletion_min_file_size);
  419. DECLARE_int32(compaction_on_deletion_trigger_count);
  420. DECLARE_int32(compaction_on_deletion_window_size);
  421. DECLARE_double(compaction_on_deletion_ratio);
  422. constexpr long KB = 1024;
  423. constexpr int kRandomValueMaxFactor = 3;
  424. constexpr int kValueMaxLen = 100;
  425. constexpr uint32_t kLargePrimeForCommonFactorSkew = 1872439133;
  426. // wrapped posix environment
  427. extern ROCKSDB_NAMESPACE::Env* db_stress_env;
  428. extern ROCKSDB_NAMESPACE::Env* db_stress_listener_env;
  429. extern std::shared_ptr<ROCKSDB_NAMESPACE::FaultInjectionTestFS> fault_fs_guard;
  430. extern std::shared_ptr<ROCKSDB_NAMESPACE::SecondaryCache>
  431. compressed_secondary_cache;
  432. extern std::shared_ptr<ROCKSDB_NAMESPACE::Cache> block_cache;
  433. extern enum ROCKSDB_NAMESPACE::CompressionType compression_type_e;
  434. extern enum ROCKSDB_NAMESPACE::CompressionType bottommost_compression_type_e;
  435. extern enum ROCKSDB_NAMESPACE::ChecksumType checksum_type_e;
  436. enum RepFactory { kSkipList, kHashSkipList, kVectorRep };
  437. inline enum RepFactory StringToRepFactory(const char* ctype) {
  438. assert(ctype);
  439. if (!strcasecmp(ctype, "skip_list"))
  440. return kSkipList;
  441. else if (!strcasecmp(ctype, "prefix_hash"))
  442. return kHashSkipList;
  443. else if (!strcasecmp(ctype, "vector"))
  444. return kVectorRep;
  445. fprintf(stdout, "Cannot parse memreptable %s\n", ctype);
  446. return kSkipList;
  447. }
  448. extern enum RepFactory FLAGS_rep_factory;
  449. namespace ROCKSDB_NAMESPACE {
  450. inline enum ROCKSDB_NAMESPACE::CompressionType StringToCompressionType(
  451. const char* ctype) {
  452. assert(ctype);
  453. ROCKSDB_NAMESPACE::CompressionType ret_compression_type;
  454. if (!strcasecmp(ctype, "disable")) {
  455. ret_compression_type = ROCKSDB_NAMESPACE::kDisableCompressionOption;
  456. } else if (!strcasecmp(ctype, "none")) {
  457. ret_compression_type = ROCKSDB_NAMESPACE::kNoCompression;
  458. } else if (!strcasecmp(ctype, "snappy")) {
  459. ret_compression_type = ROCKSDB_NAMESPACE::kSnappyCompression;
  460. } else if (!strcasecmp(ctype, "zlib")) {
  461. ret_compression_type = ROCKSDB_NAMESPACE::kZlibCompression;
  462. } else if (!strcasecmp(ctype, "bzip2")) {
  463. ret_compression_type = ROCKSDB_NAMESPACE::kBZip2Compression;
  464. } else if (!strcasecmp(ctype, "lz4")) {
  465. ret_compression_type = ROCKSDB_NAMESPACE::kLZ4Compression;
  466. } else if (!strcasecmp(ctype, "lz4hc")) {
  467. ret_compression_type = ROCKSDB_NAMESPACE::kLZ4HCCompression;
  468. } else if (!strcasecmp(ctype, "xpress")) {
  469. ret_compression_type = ROCKSDB_NAMESPACE::kXpressCompression;
  470. } else if (!strcasecmp(ctype, "zstd")) {
  471. ret_compression_type = ROCKSDB_NAMESPACE::kZSTD;
  472. } else {
  473. fprintf(stderr, "Cannot parse compression type '%s'\n", ctype);
  474. ret_compression_type =
  475. ROCKSDB_NAMESPACE::kSnappyCompression; // default value
  476. }
  477. if (ret_compression_type != ROCKSDB_NAMESPACE::kDisableCompressionOption &&
  478. !CompressionTypeSupported(ret_compression_type)) {
  479. // Use no compression will be more portable but considering this is
  480. // only a stress test and snappy is widely available. Use snappy here.
  481. ret_compression_type = ROCKSDB_NAMESPACE::kSnappyCompression;
  482. }
  483. return ret_compression_type;
  484. }
  485. inline enum ROCKSDB_NAMESPACE::ChecksumType StringToChecksumType(
  486. const char* ctype) {
  487. assert(ctype);
  488. auto iter = ROCKSDB_NAMESPACE::checksum_type_string_map.find(ctype);
  489. if (iter != ROCKSDB_NAMESPACE::checksum_type_string_map.end()) {
  490. return iter->second;
  491. }
  492. fprintf(stderr, "Cannot parse checksum type '%s'\n", ctype);
  493. return ROCKSDB_NAMESPACE::kCRC32c;
  494. }
  495. inline std::string ChecksumTypeToString(ROCKSDB_NAMESPACE::ChecksumType ctype) {
  496. auto iter = std::find_if(
  497. ROCKSDB_NAMESPACE::checksum_type_string_map.begin(),
  498. ROCKSDB_NAMESPACE::checksum_type_string_map.end(),
  499. [&](const std::pair<std::string, ROCKSDB_NAMESPACE::ChecksumType>&
  500. name_and_enum_val) { return name_and_enum_val.second == ctype; });
  501. assert(iter != ROCKSDB_NAMESPACE::checksum_type_string_map.end());
  502. return iter->first;
  503. }
  504. inline enum ROCKSDB_NAMESPACE::Temperature StringToTemperature(
  505. const char* ctype) {
  506. assert(ctype);
  507. auto iter = std::find_if(
  508. ROCKSDB_NAMESPACE::temperature_to_string.begin(),
  509. ROCKSDB_NAMESPACE::temperature_to_string.end(),
  510. [&](const std::pair<ROCKSDB_NAMESPACE::Temperature, std::string>&
  511. temp_and_string_val) {
  512. return ctype == temp_and_string_val.second;
  513. });
  514. assert(iter != ROCKSDB_NAMESPACE::temperature_to_string.end());
  515. return iter->first;
  516. }
  517. inline std::string TemperatureToString(
  518. ROCKSDB_NAMESPACE::Temperature temperature) {
  519. auto iter =
  520. ROCKSDB_NAMESPACE::OptionsHelper::temperature_to_string.find(temperature);
  521. assert(iter != ROCKSDB_NAMESPACE::OptionsHelper::temperature_to_string.end());
  522. return iter->second;
  523. }
  524. inline std::vector<std::string> SplitString(std::string src) {
  525. std::vector<std::string> ret;
  526. if (src.empty()) {
  527. return ret;
  528. }
  529. size_t pos = 0;
  530. size_t pos_comma;
  531. while ((pos_comma = src.find(',', pos)) != std::string::npos) {
  532. ret.push_back(src.substr(pos, pos_comma - pos));
  533. pos = pos_comma + 1;
  534. }
  535. ret.push_back(src.substr(pos, src.length()));
  536. return ret;
  537. }
  538. #ifdef _MSC_VER
  539. #pragma warning(push)
  540. // truncation of constant value on static_cast
  541. #pragma warning(disable : 4309)
  542. #endif
  543. inline bool GetNextPrefix(const ROCKSDB_NAMESPACE::Slice& src, std::string* v) {
  544. std::string ret = src.ToString();
  545. for (int i = static_cast<int>(ret.size()) - 1; i >= 0; i--) {
  546. if (ret[i] != static_cast<char>(255)) {
  547. ret[i] = ret[i] + 1;
  548. break;
  549. } else if (i != 0) {
  550. ret[i] = 0;
  551. } else {
  552. // all FF. No next prefix
  553. return false;
  554. }
  555. }
  556. *v = ret;
  557. return true;
  558. }
  559. #ifdef _MSC_VER
  560. #pragma warning(pop)
  561. #endif
  562. // Append `val` to `*key` in fixed-width big-endian format
  563. inline void AppendIntToString(uint64_t val, std::string* key) {
  564. // PutFixed64 uses little endian
  565. PutFixed64(key, val);
  566. // Reverse to get big endian
  567. char* int_data = &((*key)[key->size() - sizeof(uint64_t)]);
  568. for (size_t i = 0; i < sizeof(uint64_t) / 2; ++i) {
  569. std::swap(int_data[i], int_data[sizeof(uint64_t) - 1 - i]);
  570. }
  571. }
  572. // A struct for maintaining the parameters for generating variable length keys
  573. struct KeyGenContext {
  574. // Number of adjacent keys in one cycle of key lengths
  575. uint64_t window;
  576. // Number of keys of each possible length in a given window
  577. std::vector<uint64_t> weights;
  578. };
  579. extern KeyGenContext key_gen_ctx;
  580. // Generate a variable length key string from the given int64 val. The
  581. // order of the keys is preserved. The key could be anywhere from 8 to
  582. // max_key_len * 8 bytes.
  583. // The algorithm picks the length based on the
  584. // offset of the val within a configured window and the distribution of the
  585. // number of keys of various lengths in that window. For example, if x, y, x are
  586. // the weights assigned to each possible key length, the keys generated would be
  587. // - {0}...{x-1}
  588. // {(x-1),0}..{(x-1),(y-1)},{(x-1),(y-1),0}..{(x-1),(y-1),(z-1)} and so on.
  589. // Additionally, a trailer of 0-7 bytes could be appended.
  590. inline std::string Key(int64_t val) {
  591. uint64_t window = key_gen_ctx.window;
  592. size_t levels = key_gen_ctx.weights.size();
  593. std::string key;
  594. // Over-reserve and for now do not bother `shrink_to_fit()` since the key
  595. // strings are transient.
  596. key.reserve(FLAGS_max_key_len * 8);
  597. uint64_t window_idx = static_cast<uint64_t>(val) / window;
  598. uint64_t offset = static_cast<uint64_t>(val) % window;
  599. for (size_t level = 0; level < levels; ++level) {
  600. uint64_t weight = key_gen_ctx.weights[level];
  601. uint64_t pfx;
  602. if (level == 0) {
  603. pfx = window_idx * weight;
  604. } else {
  605. pfx = 0;
  606. }
  607. pfx += offset >= weight ? weight - 1 : offset;
  608. AppendIntToString(pfx, &key);
  609. if (offset < weight) {
  610. // Use the bottom 3 bits of offset as the number of trailing 'x's in the
  611. // key. If the next key is going to be of the next level, then skip the
  612. // trailer as it would break ordering. If the key length is already at
  613. // max, skip the trailer.
  614. if (offset < weight - 1 && level < levels - 1) {
  615. size_t trailer_len = offset & 0x7;
  616. key.append(trailer_len, 'x');
  617. }
  618. break;
  619. }
  620. offset -= weight;
  621. }
  622. return key;
  623. }
  624. // Given a string key, map it to an index into the expected values buffer
  625. inline bool GetIntVal(std::string big_endian_key, uint64_t* key_p) {
  626. size_t size_key = big_endian_key.size();
  627. std::vector<uint64_t> prefixes;
  628. assert(size_key <= key_gen_ctx.weights.size() * sizeof(uint64_t));
  629. std::string little_endian_key;
  630. little_endian_key.resize(size_key);
  631. for (size_t start = 0; start + sizeof(uint64_t) <= size_key;
  632. start += sizeof(uint64_t)) {
  633. size_t end = start + sizeof(uint64_t);
  634. for (size_t i = 0; i < sizeof(uint64_t); ++i) {
  635. little_endian_key[start + i] = big_endian_key[end - 1 - i];
  636. }
  637. Slice little_endian_slice =
  638. Slice(&little_endian_key[start], sizeof(uint64_t));
  639. uint64_t pfx;
  640. if (!GetFixed64(&little_endian_slice, &pfx)) {
  641. return false;
  642. }
  643. prefixes.emplace_back(pfx);
  644. }
  645. uint64_t key = 0;
  646. for (size_t i = 0; i < prefixes.size(); ++i) {
  647. uint64_t pfx = prefixes[i];
  648. key += (pfx / key_gen_ctx.weights[i]) * key_gen_ctx.window +
  649. pfx % key_gen_ctx.weights[i];
  650. if (i < prefixes.size() - 1) {
  651. // The encoding writes a `key_gen_ctx.weights[i] - 1` that counts for
  652. // `key_gen_ctx.weights[i]` when there are more prefixes to come. So we
  653. // need to add back the one here as we're at a non-last prefix.
  654. ++key;
  655. }
  656. }
  657. *key_p = key;
  658. return true;
  659. }
  660. // Given a string prefix, map it to the first corresponding index in the
  661. // expected values buffer.
  662. inline bool GetFirstIntValInPrefix(std::string big_endian_prefix,
  663. uint64_t* key_p) {
  664. size_t size_key = big_endian_prefix.size();
  665. // Pad with zeros to make it a multiple of 8. This function may be called
  666. // with a prefix, in which case we return the first index that falls
  667. // inside or outside that prefix, dependeing on whether the prefix is
  668. // the start of upper bound of a scan
  669. unsigned int pad = sizeof(uint64_t) - (size_key % sizeof(uint64_t));
  670. if (pad < sizeof(uint64_t)) {
  671. big_endian_prefix.append(pad, '\0');
  672. }
  673. return GetIntVal(std::move(big_endian_prefix), key_p);
  674. }
  675. inline uint64_t GetPrefixKeyCount(const std::string& prefix,
  676. const std::string& ub) {
  677. uint64_t start = 0;
  678. uint64_t end = 0;
  679. if (!GetFirstIntValInPrefix(prefix, &start) ||
  680. !GetFirstIntValInPrefix(ub, &end)) {
  681. return 0;
  682. }
  683. return end - start;
  684. }
  685. inline std::string StringToHex(const std::string& str) {
  686. std::string result = "0x";
  687. result.append(Slice(str).ToString(true));
  688. return result;
  689. }
  690. inline std::string WideColumnsToHex(const WideColumns& columns) {
  691. if (columns.empty()) {
  692. return std::string();
  693. }
  694. std::ostringstream oss;
  695. WideColumnsHelper::DumpWideColumns(columns, oss, true);
  696. return oss.str();
  697. }
  698. // Unified output format for double parameters
  699. inline std::string FormatDoubleParam(double param) {
  700. return std::to_string(param);
  701. }
  702. // Make sure that double parameter is a value we can reproduce by
  703. // re-inputting the value printed.
  704. inline void SanitizeDoubleParam(double* param) {
  705. *param = std::atof(FormatDoubleParam(*param).c_str());
  706. }
  707. void PoolSizeChangeThread(void* v);
  708. void DbVerificationThread(void* v);
  709. void RemoteCompactionWorkerThread(void* v);
  710. void CompressedCacheSetCapacityThread(void* v);
  711. void TimestampedSnapshotsThread(void* v);
  712. void PrintKeyValue(int cf, uint64_t key, const char* value, size_t sz);
  713. int64_t GenerateOneKey(ThreadState* thread, uint64_t iteration);
  714. std::vector<int64_t> GenerateNKeys(ThreadState* thread, int num_keys,
  715. uint64_t iteration);
  716. size_t GenerateValue(uint32_t rand, char* v, size_t max_sz);
  717. uint32_t GetValueBase(Slice s);
  718. WideColumns GenerateWideColumns(uint32_t value_base, const Slice& slice);
  719. WideColumns GenerateExpectedWideColumns(uint32_t value_base,
  720. const Slice& slice);
  721. bool VerifyWideColumns(const Slice& value, const WideColumns& columns);
  722. bool VerifyWideColumns(const WideColumns& columns);
  723. bool VerifyIteratorAttributeGroups(
  724. const IteratorAttributeGroups& attribute_groups);
  725. AttributeGroups GenerateAttributeGroups(
  726. const std::vector<ColumnFamilyHandle*>& cfhs, uint32_t value_base,
  727. const Slice& slice);
  728. StressTest* CreateCfConsistencyStressTest();
  729. StressTest* CreateBatchedOpsStressTest();
  730. StressTest* CreateNonBatchedOpsStressTest();
  731. StressTest* CreateMultiOpsTxnsStressTest();
  732. void CheckAndSetOptionsForMultiOpsTxnStressTest();
  733. void InitializeHotKeyGenerator(double alpha);
  734. int64_t GetOneHotKeyID(double rand_seed, int64_t max_key);
  735. std::string GetNowNanos();
  736. uint64_t GetWriteUnixTime(ThreadState* thread);
  737. std::shared_ptr<FileChecksumGenFactory> GetFileChecksumImpl(
  738. const std::string& name);
  739. Status DeleteFilesInDirectory(const std::string& dirname);
  740. Status SaveFilesInDirectory(const std::string& src_dirname,
  741. const std::string& dst_dirname);
  742. Status DestroyUnverifiedSubdir(const std::string& dirname);
  743. Status InitUnverifiedSubdir(const std::string& dirname);
  744. } // namespace ROCKSDB_NAMESPACE
  745. #endif // GFLAGS